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 2020/07/02 07:10:51 UTC

[GitHub] [ignite] alievmirza opened a new pull request #7984: Ignite 13190

alievmirza opened a new pull request #7984:
URL: https://github.com/apache/ignite/pull/7984


   Thank you for submitting the pull request to the Apache Ignite.
   
   In order to streamline the review of the contribution 
   we ask you to ensure the following steps have been taken:
   
   ### The Contribution Checklist
   - [ ] There is a single JIRA ticket related to the pull request. 
   - [ ] The web-link to the pull request is attached to the JIRA ticket.
   - [ ] The JIRA ticket has the _Patch Available_ state.
   - [ ] The pull request body describes changes that have been made. 
   The description explains _WHAT_ and _WHY_ was made instead of _HOW_.
   - [ ] The pull request title is treated as the final commit message. 
   The following pattern must be used: `IGNITE-XXXX Change summary` where `XXXX` - number of JIRA issue.
   - [ ] A reviewer has been mentioned through the JIRA comments 
   (see [the Maintainers list](https://cwiki.apache.org/confluence/display/IGNITE/How+to+Contribute#HowtoContribute-ReviewProcessandMaintainers)) 
   - [ ] The pull request has been checked by the Teamcity Bot and 
   the `green visa` attached to the JIRA ticket (see [TC.Bot: Check PR](https://mtcga.gridgain.com/prs.html))
   
   ### Notes
   - [How to Contribute](https://cwiki.apache.org/confluence/display/IGNITE/How+to+Contribute)
   - [Coding abbreviation rules](https://cwiki.apache.org/confluence/display/IGNITE/Abbreviation+Rules)
   - [Coding Guidelines](https://cwiki.apache.org/confluence/display/IGNITE/Coding+Guidelines)
   - [Apache Ignite Teamcity Bot](https://cwiki.apache.org/confluence/display/IGNITE/Apache+Ignite+Teamcity+Bot)
   
   If you need any help, please email dev@ignite.apache.org or ask anу advice on http://asf.slack.com _#ignite_ channel.
   


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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r530209832



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {

Review comment:
       Sure




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r530198110



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/TreeIterator.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.nio.ByteBuffer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.util.GridUnsafe;
+
+/** */
+public class TreeIterator {
+    /** Direct memory buffer with a size of one page. */
+    private final ByteBuffer pageBuf;
+
+    /** Offheap page size. */
+    private final int pageSize;
+
+    /** */
+    public TreeIterator(int size) {
+        pageSize = size;
+
+        pageBuf = ByteBuffer.allocateDirect(pageSize);
+    }
+
+    // Performance impact of constant closures allocation is not clear. So this method should be avoided in massive
+    // operations like tree leaves access.
+    /** */
+    public static <T> T access(
+        PageAccessType access,
+        PageMemoryEx pageMemory,
+        int grpId,
+        long pageId,
+        PageAccessor<T> accessor
+    ) throws IgniteCheckedException {
+        assert access != null;
+        long page = pageMemory.acquirePage(grpId, pageId);
+
+        try {
+            long pageAddr = access == PageAccessType.ACCESS_READ
+                    ? pageMemory.readLock(grpId, pageId, page)
+                    : pageMemory.writeLock(grpId, pageId, page);
+
+            try {
+                return accessor.access(pageAddr);
+            }
+            finally {
+                if (access == PageAccessType.ACCESS_READ)
+                    pageMemory.readUnlock(grpId, pageId, page);
+                else
+                    pageMemory.writeUnlock(grpId, pageId, page, null, true);
+            }
+        }
+        finally {
+            pageMemory.releasePage(grpId, pageId, page);
+        }
+    }
+
+    /** */
+    @SuppressWarnings("PublicInnerClass")
+    public enum PageAccessType {

Review comment:
       You know, "access(ACCESS_READ, partCtx.cachePageMemory, partCtx.grpId, partMetaPageId, ..." looks better then
   "access(false, partCtx.cachePageMemory, partCtx.grpId, partMetaPageId, "
   
   That was the reason. I can create constants like "boolean ACCESS_READ = false;", but strict typing works better here I think.




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

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r524324305



##########
File path: modules/core/src/main/java/org/apache/ignite/configuration/DefragmentationConfiguration.java
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.configuration;
+
+import java.io.Serializable;
+
+/**
+ *
+ */
+public final class DefragmentationConfiguration implements Serializable {

Review comment:
       I suggest minimizing changes. If the wider defragmentation configuration scope appears we can create a such a class configuration, but currently, I see no reasons.




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r532425789



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,861 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.lang.IgniteOutClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createPageStore(
+                                () -> defragmentedPartMappingFile(workDir, partId).toPath(),
+                                partCtx.mappingPagesAllocated,
+                                partCtx.mappingPageMemory
+                            );
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createPageStore(
+                            () -> defragmentedPartMappingFile(workDir, partId).toPath(),
+                            partCtx.mappingPagesAllocated,
+                            partCtx.mappingPageMemory
+                        );
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPageStore(
+                            () -> defragmentedPartTmpFile(workDir, partId).toPath(),
+                            partCtx.partPagesAllocated,
+                            partCtx.partPageMemory
+                        );
+
+                        copyPartitionData(partCtx, treeIter, offheap);
+
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() != null)
+                                return;
+
+                            PageStore oldPageStore = null;
+
+                            try {
+                                oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                            }
+                            catch (IgniteCheckedException ignore) {
+                            }
+
+                            if (log.isDebugEnabled()) {
+                                log.debug(S.toString(
+                                    "Partition defragmented",
+                                    "grpId", grpId, false,
+                                    "partId", partId, false,
+                                    "oldPages", oldPageStore.pages(), false,
+                                    "newPages", partCtx.partPagesAllocated.get() + 1, false,
+                                    "mappingPages", partCtx.mappingPagesAllocated.get() + 1, false,
+                                    "pageSize", pageSize, false,
+                                    "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                    "workDir", workDir, false
+                                ));
+                            }
+
+                            oldPageMem.invalidate(grpId, partId);
+
+                            partCtx.partPageMemory.invalidate(grpId, partId);
+
+                            DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                            pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                            renameTempPartitionFile(workDir, partId);
+                        };
+
+                        GridFutureAdapter<?> cpFut = defragmentationCheckpoint
+                            .forceCheckpoint("partition defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+
+                        cpFut.listen(cpLsnr);
+
+                        cmpFut.add((IgniteInternalFuture<Object>)cpFut);
+                    }
+
+                    // A bit too general for now, but I like it more then saving only the last checkpoint future.
+                    cmpFut.markInitialized().get();
+
+                    idxDfrgFut = new GridFinishedFuture<>();
+
+                    if (filePageStoreMgr.hasIndexStore(grpId)) {
+                        defragmentIndexPartition(oldGrpCtx, newGrpCtx, linkMapByPart);
+
+                        idxDfrgFut = defragmentationCheckpoint
+                            .forceCheckpoint("index defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+                    }
+
+                    idxDfrgFut.listen(fut -> {
+                        oldPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx partPageMem = (PageMemoryEx)partDataRegion.pageMemory();
+
+                        partPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().removePageStore(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx mappingPageMem = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+                        pageMgr = (DefragmentationPageReadWriteManager)mappingPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().clear(grpId);
+
+                        renameTempIndexFile(workDir);
+
+                        writeDefragmentationCompletionMarker(filePageStoreMgr.getPageStoreFileIoFactory(), workDir, log);
+
+                        batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+                    });
+                }
+
+                // I guess we should wait for it?
+                if (idxDfrgFut != null)
+                    idxDfrgFut.get();
+            }
+
+            mntcReg.unregisterMaintenanceTask(DEFRAGMENTATION_MNTC_TASK_NAME);
+
+            log.info("Defragmentation completed. All partitions are defragmented.");
+        }
+        finally {
+            defragmentationCheckpoint.stop(true);
+        }
+    }
+
+    /** */
+    public void createIndexPageStore(
+        int grpId,
+        File workDir,
+        FilePageStoreFactory pageStoreFactory,
+        DataRegion partRegion,
+        LongConsumer allocatedTracker
+    ) throws IgniteCheckedException {
+        // Index partition file has to be deleted before we begin, otherwise there's a chance of reading corrupted file.
+        // There is a time period when index is already defragmented but marker file is not created yet. If node is
+        // failed in that time window then index will be deframented once again. That's fine, situation is rare but code
+        // to fix that would add unnecessary complications.
+        U.delete(defragmentedIndexTmpFile(workDir));
+
+        PageStore idxPageStore;
+
+        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+        try {
+            idxPageStore = pageStoreFactory.createPageStore(
+                FLAG_IDX,
+                () -> defragmentedIndexTmpFile(workDir).toPath(),
+                allocatedTracker
+            );
+        }
+        finally {
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+        }
+
+        idxPageStore.sync();
+
+        PageMemoryEx partPageMem = (PageMemoryEx)partRegion.pageMemory();
+
+        DefragmentationPageReadWriteManager partMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+        partMgr.pageStoreMap().addPageStore(grpId, PageIdAllocator.INDEX_PARTITION, idxPageStore);
+    }
+
+    /** */
+    public enum PartStages {
+        START,
+        CP_LOCK,
+        ITERATE,
+        READ_ROW,
+        INSERT_ROW,
+        STORE_MAP,
+        STORE_PK,
+        STORE_PENDING,
+        METADATA
+    }
+
+    /**
+     * Defragmentate partition.
+     *
+     * @param partCtx
+     * @param treeIter
+     * @param offheap
+     * @throws IgniteCheckedException If failed.
+     */
+    private void copyPartitionData(
+        PartitionContext partCtx,
+        TreeIterator treeIter,
+        GridCacheOffheapManager offheap
+    ) throws IgniteCheckedException {
+        partCtx.createNewCacheDataStore(offheap);

Review comment:
       Ok




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

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r525080410



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,887 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database schared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;
+
+                                log.info(S.toString(
+                                    "Partition defragmented",
+                                    "grpId", grpId, false,
+                                    "partId", partId, false,
+                                    "oldPages", oldPageStore.pages(), false,
+                                    "newPages", partCtx.partPagesAllocated.get(), false,
+                                    "bytesSaved", (oldPageStore.pages() - partCtx.partPagesAllocated.get()) * pageSize, false,
+                                    "mappingPages", partCtx.mappingPagesAllocated.get(), false,
+                                    "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                    "workDir", workDir, false
+                                ));
+
+                                oldPageMem.invalidate(grpId, partId);
+
+                                partCtx.partPageMemory.invalidate(grpId, partId);
+
+                                DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                                pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                                try {
+                                    renameTempPartitionFile(workDir, partId);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    throw new IgniteException(e);
+                                }
+                            }
+                        };
+
+                        GridFutureAdapter<?> cpFut = defragmentationCheckpoint
+                            .forceCheckpoint("partition defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+
+                        cpFut.listen(cpLsnr);
+
+                        cmpFut.add((IgniteInternalFuture<Object>)cpFut);
+                    }
+
+                    // A bit too general for now, but I like it more then saving only the last checkpoint future.
+                    cmpFut.markInitialized().get();
+
+                    idxDfrgFut = new GridFinishedFuture<>();
+
+                    if (filePageStoreMgr.hasIndexStore(grpId)) {
+                        defragmentIndexPartition(oldGrpCtx, newGrpCtx, linkMapByPart);
+
+                        idxDfrgFut = defragmentationCheckpoint
+                            .forceCheckpoint("index defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+                    }
+
+                    idxDfrgFut.listen(fut -> {
+                        oldPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx partPageMem = (PageMemoryEx)partDataRegion.pageMemory();
+
+                        partPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().removePageStore(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx mappingPageMem = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+                        pageMgr = (DefragmentationPageReadWriteManager)mappingPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().clear(grpId);
+
+                        try {
+                            renameTempIndexFile(workDir);
+
+                            writeDefragmentationCompletionMarker(filePageStoreMgr.getPageStoreFileIoFactory(), workDir, log);
+
+                            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    });
+                }
+
+                // I guess we should wait for it?
+                if (idxDfrgFut != null)
+                    idxDfrgFut.get();
+            }
+
+            mntcReg.unregisterMaintenanceTask(DEFRAGMENTATION_MNTC_TASK_NAME);
+
+            log.info("Defragmentation completed. All partitions are defragmented.");

Review comment:
       I think we should file a new JIRA issue and implement the defragmentation JMX metrics outside of the current task. I suggest keeping this PR as simple as possible and remove such verbose logging.




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r524322120



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/DefragmentationFileUtils.java
##########
@@ -0,0 +1,400 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static java.nio.file.StandardCopyOption.ATOMIC_MOVE;
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static java.nio.file.StandardOpenOption.CREATE_NEW;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.FILE_SUFFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_TEMPLATE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.TMP_SUFFIX;
+
+/**
+ * Everything related to file management during defragmentation process.
+ */
+public class DefragmentationFileUtils {
+    /** Prefix for link mapping files. */
+    private static final String DFRG_LINK_MAPPING_FILE_PREFIX = PART_FILE_PREFIX + "map-";
+
+    /** Link mapping file template. */
+    private static final String DFRG_LINK_MAPPING_FILE_TEMPLATE = DFRG_LINK_MAPPING_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmentation complation marker file name. */
+    private static final String DFRG_COMPLETION_MARKER_FILE_NAME = "dfrg-completion-marker";
+
+    /** Name of defragmentated index partition file. */
+    private static final String DFRG_INDEX_FILE_NAME = INDEX_FILE_PREFIX + "-dfrg" + FILE_SUFFIX;
+
+    /** Name of defragmentated index partition temporary file. */
+    private static final String DFRG_INDEX_TMP_FILE_NAME = DFRG_INDEX_FILE_NAME + TMP_SUFFIX;
+
+    /** Prefix for defragmented partition files. */
+    private static final String DFRG_PARTITION_FILE_PREFIX = PART_FILE_PREFIX + "dfrg-";
+
+    /** Defragmented partition file template. */
+    private static final String DFRG_PARTITION_FILE_TEMPLATE = DFRG_PARTITION_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmented partition temp file template. */
+    private static final String DFRG_PARTITION_TMP_FILE_TEMPLATE = DFRG_PARTITION_FILE_TEMPLATE + TMP_SUFFIX;
+
+    /**
+     * Performs cleanup of work dir before initializing file page stores.
+     * Will finish batch renaming if defragmentation was completed or delete garbage if it wasn't.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     */
+    public static void beforeInitPageStores(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+        U.delete(defragmentationCompletionMarkerFile(workDir));
+
+        for (File file : workDir.listFiles()) {
+            String fileName = file.getName();
+
+            if (
+                fileName.startsWith(DFRG_PARTITION_FILE_PREFIX)
+                || fileName.startsWith(DFRG_INDEX_FILE_NAME)
+                || fileName.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)
+            )
+                U.delete(file);
+        }
+    }
+
+    /**
+     * Checks whether cache group defragmentation completed or not. Completes it if all that's left is renaming.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param log Logger to write messages.
+     * @return {@code true} if given cache group is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentationCompletionMarkerFile(File)
+     */
+    public static boolean skipAlreadyDefragmentedCacheGroup(File workDir, int grpId, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (completionMarkerFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented page group",
+                    "grpId", grpId, false,
+                    "markerFileName", completionMarkerFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+            return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * Checks whether partition has already been defragmented or not. Cleans corrupted data if previous failed
+     * defragmentation attempt was found.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param partId Partionion index to check.
+     * @param log Logger to write messages.
+     * @return {@code true} if given partition is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartMappingFile(File, int)
+     */
+    public static boolean skipAlreadyDefragmentedPartition(File workDir, int grpId, int partId, IgniteLogger log) throws IgniteCheckedException {
+        File defragmentedPartFile = defragmentedPartFile(workDir, partId);
+        File defragmentedPartMappingFile = defragmentedPartMappingFile(workDir, partId);
+
+        if (defragmentedPartFile.exists() && defragmentedPartMappingFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented partition",
+                    "grpId", grpId, false,
+                    "partId", partId, false,
+                    "partFileName", defragmentedPartFile.getName(), false,
+                    "mappingFileName", defragmentedPartMappingFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            return true;
+        }
+
+        File defragmentedPartTmpFile = defragmentedPartTmpFile(workDir, partId);
+
+        try {
+            Files.deleteIfExists(defragmentedPartTmpFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartMappingFile.toPath());
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+
+        return false;
+    }
+
+    /**
+     * Failure-tolerant batch rename of defragmented partition files.
+     *
+     * Deletes all link mapping files old partition and index files, renaming defragmentated files in the process. Can
+     * be run on the same folder multiple times if failed for some reason.
+     *
+     * Does something only if completion marker is present in the folder. This marker won't be deleted in the end.
+     * Deletion of the marker must be done outside of defragmentation mode to prevent cache groups to be defragmentated
+     * several times in case of failures.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#writeDefragmentationCompletionMarker(FileIOFactory, File, IgniteLogger)
+     */
+    public static void batchRenameDefragmentedCacheGroupPartitions(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (!completionMarkerFile.exists())
+            return;
+
+        try {
+            for (File mappingFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)))
+                Files.delete(mappingFile.toPath());
+
+            for (File partFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_PARTITION_FILE_PREFIX))) {
+                int partId = extractPartId(partFile.getName());
+
+                File oldPartFile = new File(workDir, String.format(PART_FILE_TEMPLATE, partId));
+
+                Files.move(partFile.toPath(), oldPartFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+
+            File idxFile = new File(workDir, DFRG_INDEX_FILE_NAME);
+
+            if (idxFile.exists()) {
+                File oldIdxFile = new File(workDir, INDEX_FILE_NAME);
+
+                Files.move(idxFile.toPath(), oldIdxFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Extracts partition number from file names like {@code part-dfrg-%d.bin}.
+     *
+     * @param dfrgPartFileName Defragmented partition file name.
+     * @return Partition index.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    private static int extractPartId(String dfrgPartFileName) {
+        assert dfrgPartFileName.startsWith(DFRG_PARTITION_FILE_PREFIX) : dfrgPartFileName;
+        assert dfrgPartFileName.endsWith(FILE_SUFFIX) : dfrgPartFileName;
+
+        String partIdStr = dfrgPartFileName.substring(
+            DFRG_PARTITION_FILE_PREFIX.length(),
+            dfrgPartFileName.length() - FILE_SUFFIX.length()
+        );
+
+        return Integer.parseInt(partIdStr);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin.tmp} in given folder. It will be used for storing defragmented index
+     * partition during the process.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexFile(File)
+     */
+    public static File defragmentedIndexTmpFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_TMP_FILE_NAME);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin} in given folder. It will be used for storing defragmented index
+     * partition when the process is over.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexTmpFile(File)
+     */
+    public static File defragmentedIndexFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_FILE_NAME);
+    }
+
+    /**
+     * Rename temporary index defragmenation file to a finalized one.
+     *
+     * @param workDir Cache group working directory.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexTmpFile(File)
+     * @see DefragmentationFileUtils#defragmentedIndexFile(File)
+     */
+    public static void renameTempIndexFile(File workDir) throws IgniteCheckedException {
+        File defragmentedIdxTmpFile = defragmentedIndexTmpFile(workDir);
+        File defragmentedIdxFile = defragmentedIndexFile(workDir);
+
+        try {
+            Files.move(defragmentedIdxTmpFile.toPath(), defragmentedIdxFile.toPath(), ATOMIC_MOVE);
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Return file named {@code part-dfrg-%d.bin.tmp} in given folder. It will be used for storing defragmented data
+     * partition during the process.
+     *
+     * @param workDir Cache group working directory.
+     * @param partId Parition index, will be substituted into file name.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    public static File defragmentedPartTmpFile(File workDir, int partId) {
+        return new File(workDir, String.format(DFRG_PARTITION_TMP_FILE_TEMPLATE, partId));
+    }
+
+    /**
+     * Return file named {@code part-dfrg-%d.bin} in given folder. It will be used for storing defragmented data
+     * partition when the process is over.
+     *
+     * @param workDir Cache group working directory.
+     * @param partId Parition index, will be substituted into file name.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     */
+    public static File defragmentedPartFile(File workDir, int partId) {
+        return new File(workDir, String.format(DFRG_PARTITION_FILE_TEMPLATE, partId));
+    }
+
+    /**
+     * Rename temporary partition defragmenation file to a finalized one.
+     *
+     * @param workDir Cache group working directory.
+     * @param partId Parition index.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    public static void renameTempPartitionFile(File workDir, int partId) throws IgniteCheckedException {
+        File defragmentedPartTmpFile = defragmentedPartTmpFile(workDir, partId);
+        File defragmentedPartFile = defragmentedPartFile(workDir, partId);
+
+        assert !defragmentedPartFile.exists() : defragmentedPartFile;

Review comment:
       The assertion says that file "dfrg-part-N.bin" doesn't exist, it can't be there with current code flow.
   Anyway, that's an interesting point, I don't expect user to mess with file system during the process.




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r525141119



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/TimeTracker.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.util.Arrays;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/** */
+public class TimeTracker<Stage extends Enum<Stage>> {

Review comment:
       Oh, I got it wrong again. We need it for debugging and finding hot spots.




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r532435114



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/DefragmentationPageReadWriteManager.java
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageReadWriteManagerImpl;
+
+/** */
+public class DefragmentationPageReadWriteManager extends PageReadWriteManagerImpl {

Review comment:
       No, DefragmentationPageReadWriteManager is instantiated later then we need page stores map. Why would I put this code inside of the manager? Reason #2 - we have two maps, not one.

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/defragmentation/IndexingDefragmentation.java
##########
@@ -0,0 +1,467 @@
+/*
+ * 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.processors.query.h2.defragmentation;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.LinkMap;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.TimeTracker;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIoResolver;
+import org.apache.ignite.internal.processors.cache.persistence.tree.util.InsertLast;
+import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccDataRow;
+import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+import org.apache.ignite.internal.processors.query.h2.database.H2Tree;
+import org.apache.ignite.internal.processors.query.h2.database.H2TreeIndex;
+import org.apache.ignite.internal.processors.query.h2.database.InlineIndexColumn;
+import org.apache.ignite.internal.processors.query.h2.database.inlinecolumn.AbstractInlineIndexColumn;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2ExtrasInnerIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2ExtrasLeafIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2InnerIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2LeafIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.H2RowLinkIO;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.processors.query.h2.opt.H2CacheRow;
+import org.apache.ignite.internal.processors.query.h2.opt.H2Row;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.h2.index.Index;
+import org.h2.value.Value;
+
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.INIT_TREE;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.ITERATE;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.READ_MAP;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.READ_ROW;
+
+/**
+ *
+ */
+public class IndexingDefragmentation {
+    /** Indexing. */
+    private final IgniteH2Indexing indexing;
+
+    /** Constructor. */
+    public IndexingDefragmentation(IgniteH2Indexing indexing) {
+        this.indexing = indexing;
+    }
+
+    /** */
+    public enum IndexStages {
+        START,
+        CP_LOCK,
+        INIT_TREE,
+        ITERATE,
+        READ_ROW,
+        READ_MAP,
+        INSERT_ROW
+    }
+
+    /**
+     * Defragment index partition.
+     *
+     * @param grpCtx Old group context.
+     * @param newCtx New group context.
+     * @param partPageMem Partition page memory.
+     * @param mappingByPartition Mapping page memory.
+     * @param cpLock Defragmentation checkpoint read lock.
+     * @param log Log.
+     *
+     * @throws IgniteCheckedException If failed.
+     */
+    public void defragment(
+        CacheGroupContext grpCtx,
+        CacheGroupContext newCtx,
+        PageMemoryEx partPageMem,
+        IntMap<LinkMap> mappingByPartition,
+        CheckpointTimeoutLock cpLock,
+        IgniteLogger log
+    ) throws IgniteCheckedException {
+        int pageSize = grpCtx.cacheObjectContext().kernalContext().grid().configuration().getDataStorageConfiguration().getPageSize();
+
+        TreeIterator treeIterator = new TreeIterator(pageSize);
+
+        PageMemoryEx oldCachePageMem = (PageMemoryEx)grpCtx.dataRegion().pageMemory();
+
+        PageMemory newCachePageMemory = partPageMem;
+
+        Collection<GridH2Table> tables = indexing.schemaManager().dataTables();
+
+        long cpLockThreshold = 150L;
+
+        TimeTracker<IndexStages> tracker = new TimeTracker<>(IndexStages.class);
+
+        cpLock.checkpointReadLock();
+        tracker.complete(CP_LOCK);
+
+        try {
+            AtomicLong lastCpLockTs = new AtomicLong(System.currentTimeMillis());
+
+            for (GridH2Table table : tables) {
+                GridCacheContext<?, ?> cctx = table.cacheContext();
+
+                if (cctx.groupId() != grpCtx.groupId())
+                    continue; // Not our index.
+
+                GridH2RowDescriptor rowDesc = table.rowDescriptor();
+
+                List<Index> indexes = table.getIndexes();
+                H2TreeIndex oldH2Idx = (H2TreeIndex)indexes.get(2);
+
+                int segments = oldH2Idx.segmentsCount();
+
+                H2Tree firstTree = oldH2Idx.treeForRead(0);
+
+                PageIoResolver pageIoRslvr = pageAddr -> {
+                    PageIO io = PageIoResolver.DEFAULT_PAGE_IO_RESOLVER.resolve(pageAddr);
+
+                    if (io instanceof BPlusMetaIO)
+                        return io;
+
+                    //noinspection unchecked,rawtypes,rawtypes
+                    return wrap((BPlusIO)io);
+                };
+
+                //TODO Create new proper GridCacheContext for it?

Review comment:
       > Please, remove it.
   
   Please show me how do I use TimeBag for the same scenario. I'm sure that I can't do that.
   
   > It will be simpler and better if you'll reuse try-finally approach for all these cases.
   
   Ok, I'm removing all 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.

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r532415872



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/defragmentation/IndexingDefragmentation.java
##########
@@ -0,0 +1,467 @@
+/*
+ * 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.processors.query.h2.defragmentation;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.LinkMap;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.TimeTracker;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIoResolver;
+import org.apache.ignite.internal.processors.cache.persistence.tree.util.InsertLast;
+import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccDataRow;
+import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+import org.apache.ignite.internal.processors.query.h2.database.H2Tree;
+import org.apache.ignite.internal.processors.query.h2.database.H2TreeIndex;
+import org.apache.ignite.internal.processors.query.h2.database.InlineIndexColumn;
+import org.apache.ignite.internal.processors.query.h2.database.inlinecolumn.AbstractInlineIndexColumn;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2ExtrasInnerIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2ExtrasLeafIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2InnerIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2LeafIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.H2RowLinkIO;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.processors.query.h2.opt.H2CacheRow;
+import org.apache.ignite.internal.processors.query.h2.opt.H2Row;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.h2.index.Index;
+import org.h2.value.Value;
+
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.INIT_TREE;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.ITERATE;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.READ_MAP;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.READ_ROW;
+
+/**
+ *
+ */
+public class IndexingDefragmentation {
+    /** Indexing. */
+    private final IgniteH2Indexing indexing;
+
+    /** Constructor. */
+    public IndexingDefragmentation(IgniteH2Indexing indexing) {
+        this.indexing = indexing;
+    }
+
+    /** */
+    public enum IndexStages {
+        START,
+        CP_LOCK,
+        INIT_TREE,
+        ITERATE,
+        READ_ROW,
+        READ_MAP,
+        INSERT_ROW
+    }
+
+    /**
+     * Defragment index partition.
+     *
+     * @param grpCtx Old group context.
+     * @param newCtx New group context.
+     * @param partPageMem Partition page memory.
+     * @param mappingByPartition Mapping page memory.
+     * @param cpLock Defragmentation checkpoint read lock.
+     * @param log Log.
+     *
+     * @throws IgniteCheckedException If failed.
+     */
+    public void defragment(
+        CacheGroupContext grpCtx,
+        CacheGroupContext newCtx,
+        PageMemoryEx partPageMem,
+        IntMap<LinkMap> mappingByPartition,
+        CheckpointTimeoutLock cpLock,
+        IgniteLogger log
+    ) throws IgniteCheckedException {
+        int pageSize = grpCtx.cacheObjectContext().kernalContext().grid().configuration().getDataStorageConfiguration().getPageSize();
+
+        TreeIterator treeIterator = new TreeIterator(pageSize);
+
+        PageMemoryEx oldCachePageMem = (PageMemoryEx)grpCtx.dataRegion().pageMemory();
+
+        PageMemory newCachePageMemory = partPageMem;
+
+        Collection<GridH2Table> tables = indexing.schemaManager().dataTables();
+
+        long cpLockThreshold = 150L;
+
+        TimeTracker<IndexStages> tracker = new TimeTracker<>(IndexStages.class);
+
+        cpLock.checkpointReadLock();
+        tracker.complete(CP_LOCK);
+
+        try {
+            AtomicLong lastCpLockTs = new AtomicLong(System.currentTimeMillis());
+
+            for (GridH2Table table : tables) {
+                GridCacheContext<?, ?> cctx = table.cacheContext();
+
+                if (cctx.groupId() != grpCtx.groupId())
+                    continue; // Not our index.
+
+                GridH2RowDescriptor rowDesc = table.rowDescriptor();
+
+                List<Index> indexes = table.getIndexes();
+                H2TreeIndex oldH2Idx = (H2TreeIndex)indexes.get(2);
+
+                int segments = oldH2Idx.segmentsCount();
+
+                H2Tree firstTree = oldH2Idx.treeForRead(0);
+
+                PageIoResolver pageIoRslvr = pageAddr -> {
+                    PageIO io = PageIoResolver.DEFAULT_PAGE_IO_RESOLVER.resolve(pageAddr);
+
+                    if (io instanceof BPlusMetaIO)
+                        return io;
+
+                    //noinspection unchecked,rawtypes,rawtypes
+                    return wrap((BPlusIO)io);
+                };
+
+                //TODO Create new proper GridCacheContext for it?

Review comment:
       > Please, remove it.
   
   Please show me how do I use TimeBag for the same scenario. I'm sure that I can't do that.
   
   > Let's move everything here from PageStoreMap here? The DefragmentationPageReadWriteManager will also implement the PageStoreCollection.
   
   No, DefragmentationPageReadWriteManager is instantiated later then we need page stores map. Why would I put this code inside of the manager? Reason #2 - we have two maps, not one.
   
   > It will be simpler and better if you'll reuse try-finally approach for all these cases.
   
   Ok, I'm removing all 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.

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r530996335



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsDefragmentationTest.java
##########
@@ -0,0 +1,539 @@
+/*
+ * 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.processors.cache.persistence;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.FileVisitResult;
+import java.nio.file.FileVisitor;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardCopyOption;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.Collections;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+import java.util.function.UnaryOperator;
+import java.util.stream.IntStream;
+import javax.cache.configuration.Factory;
+import javax.cache.expiry.Duration;
+import javax.cache.expiry.ExpiryPolicy;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.FailureHandler;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.maintenance.MaintenanceFileStore;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.util.lang.IgniteThrowableConsumer;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentationCompletionMarkerFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.maintenance.DefragmentationParameters.toStore;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
+/** */
+public class IgnitePdsDefragmentationTest extends GridCommonAbstractTest {
+    /** */
+    public static final String CACHE_2_NAME = "cache2";
+
+    /** */
+    public static final int PARTS = 5;
+
+    /** */
+    public static final int ADDED_KEYS_COUNT = 150;
+
+    /** */
+    protected static final String GRP_NAME = "group";
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        stopAllGrids(true);
+
+        cleanPersistenceDir();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids(true);
+
+        cleanPersistenceDir();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected FailureHandler getFailureHandler(String igniteInstanceName) {
+        return new StopNodeFailureHandler();
+    }
+
+    /** */
+    protected static class PolicyFactory implements Factory<ExpiryPolicy> {
+        /** Serial version uid. */
+        private static final long serialVersionUID = 0L;
+
+        /** {@inheritDoc} */
+        @Override public ExpiryPolicy create() {
+            return new ExpiryPolicy() {
+                @Override public Duration getExpiryForCreation() {
+                    return new Duration(TimeUnit.MILLISECONDS, 13000);
+                }
+
+                /** {@inheritDoc} */
+                @Override public Duration getExpiryForAccess() {
+                    return new Duration(TimeUnit.MILLISECONDS, 13000);
+                }
+
+                /** {@inheritDoc} */
+                @Override public Duration getExpiryForUpdate() {
+                    return new Duration(TimeUnit.MILLISECONDS, 13000);
+                }
+            };
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setConsistentId(igniteInstanceName);
+
+        DataStorageConfiguration dsCfg = new DataStorageConfiguration();
+        dsCfg.setWalSegmentSize(4 * 1024 * 1024);
+
+        dsCfg.setDefaultDataRegionConfiguration(
+            new DataRegionConfiguration()
+                .setInitialSize(100L * 1024 * 1024)
+                .setMaxSize(1024L * 1024 * 1024)
+                .setPersistenceEnabled(true)
+        );
+
+        cfg.setDataStorageConfiguration(dsCfg);
+
+        CacheConfiguration<?, ?> cache1Cfg = new CacheConfiguration<>(DEFAULT_CACHE_NAME)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setGroupName(GRP_NAME)
+            .setAffinity(new RendezvousAffinityFunction(false, PARTS));
+
+        CacheConfiguration<?, ?> cache2Cfg = new CacheConfiguration<>(CACHE_2_NAME)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setGroupName(GRP_NAME)
+            .setExpiryPolicyFactory(new PolicyFactory())
+            .setAffinity(new RendezvousAffinityFunction(false, PARTS));
+
+        cfg.setCacheConfiguration(cache1Cfg, cache2Cfg);
+
+        return cfg;
+    }
+
+    /**
+     * Basic test scenario. Does following steps:
+     *  - Start node;
+     *  - Fill cache;
+     *  - Remove part of data;
+     *  - Stop node;
+     *  - Start node in defragmentation mode;
+     *  - Stop node;
+     *  - Start node;
+     *  - Check that partitions became smaller;
+     *  - Check that cache is accessible and works just fine.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testEssentials() throws Exception {
+        IgniteEx ig = startGrid(0);
+
+        ig.cluster().state(ClusterState.ACTIVE);
+
+        fillCache(ig.cache(DEFAULT_CACHE_NAME));
+
+        forceCheckpoint(ig);
+
+        createMaintenanceRecord();
+
+        stopGrid(0);
+
+        File workDir = resolveCacheWorkDir(ig);
+
+        long[] oldPartLen = partitionSizes(workDir);
+
+        long oldIdxFileLen = new File(workDir, FilePageStoreManager.INDEX_FILE_NAME).length();
+
+        startGrid(0);
+
+        long[] newPartLen = partitionSizes(workDir);
+
+        for (int p = 0; p < PARTS; p++)
+            assertTrue(newPartLen[p] < oldPartLen[p]); //TODO Fails.
+
+        long newIdxFileLen = new File(workDir, FilePageStoreManager.INDEX_FILE_NAME).length();
+
+        assertTrue(newIdxFileLen <= oldIdxFileLen);
+
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+        assertTrue(completionMarkerFile.exists());
+
+        stopGrid(0);
+
+        IgniteEx ig0 = startGrid(0);
+
+        ig0.cluster().state(ClusterState.ACTIVE);
+
+        assertFalse(completionMarkerFile.exists());
+
+        validateCache(grid(0).cache(DEFAULT_CACHE_NAME));
+    }
+
+    /**
+     * @return Working directory for cache group {@link IgnitePdsDefragmentationTest#GRP_NAME}.
+     * @throws IgniteCheckedException If failed for some reason, like if it's a file instead of directory.
+     */
+    private File resolveCacheWorkDir(IgniteEx ig) throws IgniteCheckedException {
+        File dbWorkDir = U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false);
+
+        File nodeWorkDir = new File(dbWorkDir, U.maskForFileName(ig.name()));
+
+        return new File(nodeWorkDir, FilePageStoreManager.CACHE_GRP_DIR_PREFIX + GRP_NAME);
+    }
+
+    /**
+     * Force checkpoint and wait for it so all partitions will be in their final state after restart if no more data is
+     * uploaded.
+     *
+     * @param ig Ignite node.
+     * @throws IgniteCheckedException If checkpoint failed for some reason.
+     */
+    private void forceCheckpoint(IgniteEx ig) throws IgniteCheckedException {
+        ig.context().cache().context().database()
+            .forceCheckpoint("testDefrag")
+            .futureFor(CheckpointState.FINISHED)
+            .get();
+    }
+
+    /** */
+    protected void createMaintenanceRecord() throws IgniteCheckedException {
+        IgniteEx grid = grid(0);
+        MaintenanceRegistry mntcReg = grid.context().maintenanceRegistry();
+
+        mntcReg.registerMaintenanceTask(toStore(Collections.singletonList(groupIdForCache(grid, DEFAULT_CACHE_NAME))));
+    }
+
+    /**
+     * Returns array that contains sizes of partition files in gived working directories. Assumes that partitions
+     * {@code 0} to {@code PARTS - 1} exist in that dir.
+     *
+     * @param workDir Working directory.
+     * @return The array.
+     */
+    protected long[] partitionSizes(File workDir) {
+        return IntStream.range(0, PARTS)
+            .mapToObj(p -> new File(workDir, String.format(FilePageStoreManager.PART_FILE_TEMPLATE, p)))
+            .mapToLong(File::length)
+            .toArray();
+    }
+
+    /**
+     * Checks that plain node start after failed defragmentation will finish batch renaming.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testFailoverRestartWithoutDefragmentation() throws Exception {
+        testFailover(workDir -> {
+            try {
+                File mntcRecFile = new File(workDir.getParent(), MaintenanceFileStore.MAINTENANCE_FILE_NAME);
+
+                assertTrue(mntcRecFile.exists());
+
+                Files.delete(mntcRecFile.toPath());
+
+                startGrid(0);
+
+                validateLeftovers(workDir);
+            }
+            catch (Exception e) {
+                throw new IgniteCheckedException(e);
+            }
+            finally {
+                createMaintenanceRecord();
+
+                stopGrid(0);
+            }
+        });
+    }
+
+    /**
+     * Checks that second start in defragmentation mode will finish defragmentation if no completion marker was found.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testFailoverBasic() throws Exception {

Review comment:
       Done




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

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



[GitHub] [ignite] agoncharuk commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
agoncharuk commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r524319380



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,887 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database schared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;
+
+                                log.info(S.toString(
+                                    "Partition defragmented",
+                                    "grpId", grpId, false,
+                                    "partId", partId, false,
+                                    "oldPages", oldPageStore.pages(), false,
+                                    "newPages", partCtx.partPagesAllocated.get(), false,
+                                    "bytesSaved", (oldPageStore.pages() - partCtx.partPagesAllocated.get()) * pageSize, false,
+                                    "mappingPages", partCtx.mappingPagesAllocated.get(), false,
+                                    "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                    "workDir", workDir, false
+                                ));
+
+                                oldPageMem.invalidate(grpId, partId);
+
+                                partCtx.partPageMemory.invalidate(grpId, partId);
+
+                                DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                                pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                                try {
+                                    renameTempPartitionFile(workDir, partId);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    throw new IgniteException(e);
+                                }
+                            }
+                        };
+
+                        GridFutureAdapter<?> cpFut = defragmentationCheckpoint
+                            .forceCheckpoint("partition defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+
+                        cpFut.listen(cpLsnr);
+
+                        cmpFut.add((IgniteInternalFuture<Object>)cpFut);
+                    }
+
+                    // A bit too general for now, but I like it more then saving only the last checkpoint future.
+                    cmpFut.markInitialized().get();
+
+                    idxDfrgFut = new GridFinishedFuture<>();
+
+                    if (filePageStoreMgr.hasIndexStore(grpId)) {
+                        defragmentIndexPartition(oldGrpCtx, newGrpCtx, linkMapByPart);
+
+                        idxDfrgFut = defragmentationCheckpoint
+                            .forceCheckpoint("index defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+                    }
+
+                    idxDfrgFut.listen(fut -> {
+                        oldPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx partPageMem = (PageMemoryEx)partDataRegion.pageMemory();
+
+                        partPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().removePageStore(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx mappingPageMem = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+                        pageMgr = (DefragmentationPageReadWriteManager)mappingPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().clear(grpId);
+
+                        try {
+                            renameTempIndexFile(workDir);
+
+                            writeDefragmentationCompletionMarker(filePageStoreMgr.getPageStoreFileIoFactory(), workDir, log);
+
+                            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    });
+                }
+
+                // I guess we should wait for it?
+                if (idxDfrgFut != null)
+                    idxDfrgFut.get();
+            }
+
+            mntcReg.unregisterMaintenanceTask(DEFRAGMENTATION_MNTC_TASK_NAME);
+
+            log.info("Defragmentation completed. All partitions are defragmented.");

Review comment:
       We have very verbose statistics for each partition printed out separately. Instead, I would have a separate aggregated statistics object that is periodically printed during defragmentation, and full stats printed out per cache/cache group completion (something like "Defragmentation [cache='cacheA', processedSize=10Gb, defragmentedSize=2Gb, partitions=24(56), progress=39%]"). Detailed per-partition logging can be moved to 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.

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r526711227



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/DefragmentationFileUtils.java
##########
@@ -0,0 +1,400 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static java.nio.file.StandardCopyOption.ATOMIC_MOVE;
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static java.nio.file.StandardOpenOption.CREATE_NEW;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.FILE_SUFFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_TEMPLATE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.TMP_SUFFIX;
+
+/**
+ * Everything related to file management during defragmentation process.
+ */
+public class DefragmentationFileUtils {
+    /** Prefix for link mapping files. */
+    private static final String DFRG_LINK_MAPPING_FILE_PREFIX = PART_FILE_PREFIX + "map-";
+
+    /** Link mapping file template. */
+    private static final String DFRG_LINK_MAPPING_FILE_TEMPLATE = DFRG_LINK_MAPPING_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmentation complation marker file name. */
+    private static final String DFRG_COMPLETION_MARKER_FILE_NAME = "dfrg-completion-marker";
+
+    /** Name of defragmentated index partition file. */
+    private static final String DFRG_INDEX_FILE_NAME = INDEX_FILE_PREFIX + "-dfrg" + FILE_SUFFIX;
+
+    /** Name of defragmentated index partition temporary file. */
+    private static final String DFRG_INDEX_TMP_FILE_NAME = DFRG_INDEX_FILE_NAME + TMP_SUFFIX;
+
+    /** Prefix for defragmented partition files. */
+    private static final String DFRG_PARTITION_FILE_PREFIX = PART_FILE_PREFIX + "dfrg-";
+
+    /** Defragmented partition file template. */
+    private static final String DFRG_PARTITION_FILE_TEMPLATE = DFRG_PARTITION_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmented partition temp file template. */
+    private static final String DFRG_PARTITION_TMP_FILE_TEMPLATE = DFRG_PARTITION_FILE_TEMPLATE + TMP_SUFFIX;
+
+    /**
+     * Performs cleanup of work dir before initializing file page stores.
+     * Will finish batch renaming if defragmentation was completed or delete garbage if it wasn't.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     */
+    public static void beforeInitPageStores(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+        U.delete(defragmentationCompletionMarkerFile(workDir));
+
+        for (File file : workDir.listFiles()) {
+            String fileName = file.getName();
+
+            if (
+                fileName.startsWith(DFRG_PARTITION_FILE_PREFIX)
+                || fileName.startsWith(DFRG_INDEX_FILE_NAME)
+                || fileName.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)
+            )
+                U.delete(file);
+        }
+    }
+
+    /**
+     * Checks whether cache group defragmentation completed or not. Completes it if all that's left is renaming.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param log Logger to write messages.
+     * @return {@code true} if given cache group is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentationCompletionMarkerFile(File)
+     */
+    public static boolean skipAlreadyDefragmentedCacheGroup(File workDir, int grpId, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (completionMarkerFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented page group",
+                    "grpId", grpId, false,
+                    "markerFileName", completionMarkerFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+            return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * Checks whether partition has already been defragmented or not. Cleans corrupted data if previous failed
+     * defragmentation attempt was found.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param partId Partionion index to check.
+     * @param log Logger to write messages.
+     * @return {@code true} if given partition is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartMappingFile(File, int)
+     */
+    public static boolean skipAlreadyDefragmentedPartition(File workDir, int grpId, int partId, IgniteLogger log) throws IgniteCheckedException {
+        File defragmentedPartFile = defragmentedPartFile(workDir, partId);
+        File defragmentedPartMappingFile = defragmentedPartMappingFile(workDir, partId);
+
+        if (defragmentedPartFile.exists() && defragmentedPartMappingFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented partition",
+                    "grpId", grpId, false,
+                    "partId", partId, false,
+                    "partFileName", defragmentedPartFile.getName(), false,
+                    "mappingFileName", defragmentedPartMappingFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            return true;
+        }
+
+        File defragmentedPartTmpFile = defragmentedPartTmpFile(workDir, partId);
+
+        try {
+            Files.deleteIfExists(defragmentedPartTmpFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartMappingFile.toPath());
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+
+        return false;
+    }
+
+    /**
+     * Failure-tolerant batch rename of defragmented partition files.
+     *
+     * Deletes all link mapping files old partition and index files, renaming defragmentated files in the process. Can
+     * be run on the same folder multiple times if failed for some reason.
+     *
+     * Does something only if completion marker is present in the folder. This marker won't be deleted in the end.
+     * Deletion of the marker must be done outside of defragmentation mode to prevent cache groups to be defragmentated
+     * several times in case of failures.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#writeDefragmentationCompletionMarker(FileIOFactory, File, IgniteLogger)
+     */
+    public static void batchRenameDefragmentedCacheGroupPartitions(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (!completionMarkerFile.exists())
+            return;
+
+        try {
+            for (File mappingFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)))
+                Files.delete(mappingFile.toPath());
+
+            for (File partFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_PARTITION_FILE_PREFIX))) {
+                int partId = extractPartId(partFile.getName());
+
+                File oldPartFile = new File(workDir, String.format(PART_FILE_TEMPLATE, partId));
+
+                Files.move(partFile.toPath(), oldPartFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+
+            File idxFile = new File(workDir, DFRG_INDEX_FILE_NAME);
+
+            if (idxFile.exists()) {
+                File oldIdxFile = new File(workDir, INDEX_FILE_NAME);
+
+                Files.move(idxFile.toPath(), oldIdxFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Extracts partition number from file names like {@code part-dfrg-%d.bin}.
+     *
+     * @param dfrgPartFileName Defragmented partition file name.
+     * @return Partition index.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    private static int extractPartId(String dfrgPartFileName) {
+        assert dfrgPartFileName.startsWith(DFRG_PARTITION_FILE_PREFIX) : dfrgPartFileName;
+        assert dfrgPartFileName.endsWith(FILE_SUFFIX) : dfrgPartFileName;
+
+        String partIdStr = dfrgPartFileName.substring(
+            DFRG_PARTITION_FILE_PREFIX.length(),
+            dfrgPartFileName.length() - FILE_SUFFIX.length()
+        );
+
+        return Integer.parseInt(partIdStr);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin.tmp} in given folder. It will be used for storing defragmented index
+     * partition during the process.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexFile(File)
+     */
+    public static File defragmentedIndexTmpFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_TMP_FILE_NAME);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin} in given folder. It will be used for storing defragmented index
+     * partition when the process is over.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexTmpFile(File)
+     */
+    public static File defragmentedIndexFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_FILE_NAME);
+    }
+
+    /**
+     * Rename temporary index defragmenation file to a finalized one.
+     *
+     * @param workDir Cache group working directory.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexTmpFile(File)
+     * @see DefragmentationFileUtils#defragmentedIndexFile(File)
+     */
+    public static void renameTempIndexFile(File workDir) throws IgniteCheckedException {
+        File defragmentedIdxTmpFile = defragmentedIndexTmpFile(workDir);
+        File defragmentedIdxFile = defragmentedIndexFile(workDir);
+
+        try {
+            Files.move(defragmentedIdxTmpFile.toPath(), defragmentedIdxFile.toPath(), ATOMIC_MOVE);
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Return file named {@code part-dfrg-%d.bin.tmp} in given folder. It will be used for storing defragmented data
+     * partition during the process.
+     *
+     * @param workDir Cache group working directory.
+     * @param partId Parition index, will be substituted into file name.

Review comment:
       Done

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/DefragmentationFileUtils.java
##########
@@ -0,0 +1,400 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static java.nio.file.StandardCopyOption.ATOMIC_MOVE;
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static java.nio.file.StandardOpenOption.CREATE_NEW;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.FILE_SUFFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_TEMPLATE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.TMP_SUFFIX;
+
+/**
+ * Everything related to file management during defragmentation process.
+ */
+public class DefragmentationFileUtils {
+    /** Prefix for link mapping files. */
+    private static final String DFRG_LINK_MAPPING_FILE_PREFIX = PART_FILE_PREFIX + "map-";
+
+    /** Link mapping file template. */
+    private static final String DFRG_LINK_MAPPING_FILE_TEMPLATE = DFRG_LINK_MAPPING_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmentation complation marker file name. */
+    private static final String DFRG_COMPLETION_MARKER_FILE_NAME = "dfrg-completion-marker";
+
+    /** Name of defragmentated index partition file. */
+    private static final String DFRG_INDEX_FILE_NAME = INDEX_FILE_PREFIX + "-dfrg" + FILE_SUFFIX;
+
+    /** Name of defragmentated index partition temporary file. */
+    private static final String DFRG_INDEX_TMP_FILE_NAME = DFRG_INDEX_FILE_NAME + TMP_SUFFIX;
+
+    /** Prefix for defragmented partition files. */
+    private static final String DFRG_PARTITION_FILE_PREFIX = PART_FILE_PREFIX + "dfrg-";
+
+    /** Defragmented partition file template. */
+    private static final String DFRG_PARTITION_FILE_TEMPLATE = DFRG_PARTITION_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmented partition temp file template. */
+    private static final String DFRG_PARTITION_TMP_FILE_TEMPLATE = DFRG_PARTITION_FILE_TEMPLATE + TMP_SUFFIX;
+
+    /**
+     * Performs cleanup of work dir before initializing file page stores.
+     * Will finish batch renaming if defragmentation was completed or delete garbage if it wasn't.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     */
+    public static void beforeInitPageStores(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+        U.delete(defragmentationCompletionMarkerFile(workDir));
+
+        for (File file : workDir.listFiles()) {
+            String fileName = file.getName();
+
+            if (
+                fileName.startsWith(DFRG_PARTITION_FILE_PREFIX)
+                || fileName.startsWith(DFRG_INDEX_FILE_NAME)
+                || fileName.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)
+            )
+                U.delete(file);
+        }
+    }
+
+    /**
+     * Checks whether cache group defragmentation completed or not. Completes it if all that's left is renaming.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param log Logger to write messages.
+     * @return {@code true} if given cache group is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentationCompletionMarkerFile(File)
+     */
+    public static boolean skipAlreadyDefragmentedCacheGroup(File workDir, int grpId, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (completionMarkerFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented page group",
+                    "grpId", grpId, false,
+                    "markerFileName", completionMarkerFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+            return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * Checks whether partition has already been defragmented or not. Cleans corrupted data if previous failed
+     * defragmentation attempt was found.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param partId Partionion index to check.
+     * @param log Logger to write messages.
+     * @return {@code true} if given partition is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartMappingFile(File, int)
+     */
+    public static boolean skipAlreadyDefragmentedPartition(File workDir, int grpId, int partId, IgniteLogger log) throws IgniteCheckedException {
+        File defragmentedPartFile = defragmentedPartFile(workDir, partId);
+        File defragmentedPartMappingFile = defragmentedPartMappingFile(workDir, partId);
+
+        if (defragmentedPartFile.exists() && defragmentedPartMappingFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented partition",
+                    "grpId", grpId, false,
+                    "partId", partId, false,
+                    "partFileName", defragmentedPartFile.getName(), false,
+                    "mappingFileName", defragmentedPartMappingFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            return true;
+        }
+
+        File defragmentedPartTmpFile = defragmentedPartTmpFile(workDir, partId);
+
+        try {
+            Files.deleteIfExists(defragmentedPartTmpFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartMappingFile.toPath());
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+
+        return false;
+    }
+
+    /**
+     * Failure-tolerant batch rename of defragmented partition files.
+     *
+     * Deletes all link mapping files old partition and index files, renaming defragmentated files in the process. Can
+     * be run on the same folder multiple times if failed for some reason.
+     *
+     * Does something only if completion marker is present in the folder. This marker won't be deleted in the end.
+     * Deletion of the marker must be done outside of defragmentation mode to prevent cache groups to be defragmentated
+     * several times in case of failures.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#writeDefragmentationCompletionMarker(FileIOFactory, File, IgniteLogger)
+     */
+    public static void batchRenameDefragmentedCacheGroupPartitions(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (!completionMarkerFile.exists())
+            return;
+
+        try {
+            for (File mappingFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)))
+                Files.delete(mappingFile.toPath());
+
+            for (File partFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_PARTITION_FILE_PREFIX))) {
+                int partId = extractPartId(partFile.getName());
+
+                File oldPartFile = new File(workDir, String.format(PART_FILE_TEMPLATE, partId));
+
+                Files.move(partFile.toPath(), oldPartFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+
+            File idxFile = new File(workDir, DFRG_INDEX_FILE_NAME);
+
+            if (idxFile.exists()) {
+                File oldIdxFile = new File(workDir, INDEX_FILE_NAME);
+
+                Files.move(idxFile.toPath(), oldIdxFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Extracts partition number from file names like {@code part-dfrg-%d.bin}.
+     *
+     * @param dfrgPartFileName Defragmented partition file name.
+     * @return Partition index.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    private static int extractPartId(String dfrgPartFileName) {
+        assert dfrgPartFileName.startsWith(DFRG_PARTITION_FILE_PREFIX) : dfrgPartFileName;
+        assert dfrgPartFileName.endsWith(FILE_SUFFIX) : dfrgPartFileName;
+
+        String partIdStr = dfrgPartFileName.substring(
+            DFRG_PARTITION_FILE_PREFIX.length(),
+            dfrgPartFileName.length() - FILE_SUFFIX.length()
+        );
+
+        return Integer.parseInt(partIdStr);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin.tmp} in given folder. It will be used for storing defragmented index
+     * partition during the process.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexFile(File)
+     */
+    public static File defragmentedIndexTmpFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_TMP_FILE_NAME);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin} in given folder. It will be used for storing defragmented index
+     * partition when the process is over.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexTmpFile(File)
+     */
+    public static File defragmentedIndexFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_FILE_NAME);
+    }
+
+    /**
+     * Rename temporary index defragmenation file to a finalized one.
+     *
+     * @param workDir Cache group working directory.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexTmpFile(File)
+     * @see DefragmentationFileUtils#defragmentedIndexFile(File)
+     */
+    public static void renameTempIndexFile(File workDir) throws IgniteCheckedException {
+        File defragmentedIdxTmpFile = defragmentedIndexTmpFile(workDir);
+        File defragmentedIdxFile = defragmentedIndexFile(workDir);
+
+        try {
+            Files.move(defragmentedIdxTmpFile.toPath(), defragmentedIdxFile.toPath(), ATOMIC_MOVE);
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Return file named {@code part-dfrg-%d.bin.tmp} in given folder. It will be used for storing defragmented data
+     * partition during the process.
+     *
+     * @param workDir Cache group working directory.
+     * @param partId Parition index, will be substituted into file name.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    public static File defragmentedPartTmpFile(File workDir, int partId) {
+        return new File(workDir, String.format(DFRG_PARTITION_TMP_FILE_TEMPLATE, partId));
+    }
+
+    /**
+     * Return file named {@code part-dfrg-%d.bin} in given folder. It will be used for storing defragmented data
+     * partition when the process is over.
+     *
+     * @param workDir Cache group working directory.
+     * @param partId Parition index, will be substituted into file name.

Review comment:
       Done




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

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r529606712



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/TreeIterator.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.nio.ByteBuffer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.util.GridUnsafe;
+
+/** */
+public class TreeIterator {
+    /** Direct memory buffer with a size of one page. */
+    private final ByteBuffer pageBuf;
+
+    /** Offheap page size. */
+    private final int pageSize;
+
+    /** */
+    public TreeIterator(int size) {
+        pageSize = size;
+
+        pageBuf = ByteBuffer.allocateDirect(pageSize);
+    }
+
+    // Performance impact of constant closures allocation is not clear. So this method should be avoided in massive
+    // operations like tree leaves access.
+    /** */
+    public static <T> T access(
+        PageAccessType access,
+        PageMemoryEx pageMemory,
+        int grpId,
+        long pageId,
+        PageAccessor<T> accessor
+    ) throws IgniteCheckedException {
+        assert access != null;

Review comment:
       Should we add assert here `PageAccessType.ACCESS_READ || PageAccessType.ACCESS_WRITE`? In case of extending `PageAccessType` we will get an incorrect results




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

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r529620887



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/GridQueryIndexingDefragmentation.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.processors.cache.persistence.defragmentation;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.util.collection.IntMap;
+
+/**
+ *
+ */
+public interface GridQueryIndexingDefragmentation {

Review comment:
       Why we can't place the `startDefragmentation` method right at `GridQueryIndexing` and remove this interface?




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r530210629



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;

Review comment:
       That assertion is impossible to fail, I need it as a guide for IDE. I think I'll just 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.

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r531452934



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;
+
+                                if (log.isDebugEnabled()) {
+                                    log.debug(S.toString(
+                                        "Partition defragmented",
+                                        "grpId", grpId, false,
+                                        "partId", partId, false,
+                                        "oldPages", oldPageStore.pages(), false,
+                                        "newPages", partCtx.partPagesAllocated.get() + 1, false,
+                                        "mappingPages", partCtx.mappingPagesAllocated.get() + 1, false,
+                                        "pageSize", pageSize, false,
+                                        "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                        "workDir", workDir, false
+                                    ));
+                                }
+
+                                oldPageMem.invalidate(grpId, partId);
+
+                                partCtx.partPageMemory.invalidate(grpId, partId);
+
+                                DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                                pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                                try {
+                                    renameTempPartitionFile(workDir, partId);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    throw new IgniteException(e);
+                                }
+                            }
+                        };
+
+                        GridFutureAdapter<?> cpFut = defragmentationCheckpoint
+                            .forceCheckpoint("partition defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+
+                        cpFut.listen(cpLsnr);
+
+                        cmpFut.add((IgniteInternalFuture<Object>)cpFut);
+                    }
+
+                    // A bit too general for now, but I like it more then saving only the last checkpoint future.
+                    cmpFut.markInitialized().get();
+
+                    idxDfrgFut = new GridFinishedFuture<>();
+
+                    if (filePageStoreMgr.hasIndexStore(grpId)) {
+                        defragmentIndexPartition(oldGrpCtx, newGrpCtx, linkMapByPart);
+
+                        idxDfrgFut = defragmentationCheckpoint
+                            .forceCheckpoint("index defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+                    }
+
+                    idxDfrgFut.listen(fut -> {
+                        oldPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx partPageMem = (PageMemoryEx)partDataRegion.pageMemory();
+
+                        partPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().removePageStore(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx mappingPageMem = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+                        pageMgr = (DefragmentationPageReadWriteManager)mappingPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().clear(grpId);
+
+                        try {
+                            renameTempIndexFile(workDir);
+
+                            writeDefragmentationCompletionMarker(filePageStoreMgr.getPageStoreFileIoFactory(), workDir, log);
+
+                            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    });
+                }
+
+                // I guess we should wait for it?
+                if (idxDfrgFut != null)
+                    idxDfrgFut.get();
+            }
+
+            mntcReg.unregisterMaintenanceTask(DEFRAGMENTATION_MNTC_TASK_NAME);
+
+            log.info("Defragmentation completed. All partitions are defragmented.");
+        }
+        finally {
+            defragmentationCheckpoint.stop(true);
+        }
+    }
+
+    /** */
+    public void createIndexPageStore(
+        int grpId,
+        File workDir,
+        FilePageStoreFactory pageStoreFactory,
+        DataRegion partRegion,
+        LongConsumer allocatedTracker
+    ) throws IgniteCheckedException {
+        // Index partition file has to be deleted before we begin, otherwise there's a chance of reading corrupted file.
+        // There is a time period when index is already defragmented but marker file is not created yet. If node is
+        // failed in that time window then index will be deframented once again. That's fine, situation is rare but code
+        // to fix that would add unnecessary complications.
+        U.delete(defragmentedIndexTmpFile(workDir));
+
+        PageStore idxPageStore;
+
+        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+        try {
+            idxPageStore = pageStoreFactory.createPageStore(
+                FLAG_IDX,
+                () -> defragmentedIndexTmpFile(workDir).toPath(),
+                allocatedTracker
+            );
+        }
+        finally {
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+        }
+
+        idxPageStore.sync();
+
+        PageMemoryEx partPageMem = (PageMemoryEx)partRegion.pageMemory();
+
+        DefragmentationPageReadWriteManager partMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+        partMgr.pageStoreMap().addPageStore(grpId, PageIdAllocator.INDEX_PARTITION, idxPageStore);
+    }
+
+    /** */
+    public enum PartStages {
+        START,
+        CP_LOCK,
+        ITERATE,
+        READ_ROW,
+        INSERT_ROW,
+        STORE_MAP,
+        STORE_PK,
+        STORE_PENDING,
+        METADATA
+    }
+
+    /**
+     * Defragmentate partition.
+     *
+     * @param partCtx
+     * @param treeIter
+     * @throws IgniteCheckedException If failed.
+     */
+    private void copyPartitionData(
+        PartitionContext partCtx,
+        TreeIterator treeIter,
+        GridSpinBusyLock busyLock
+    ) throws IgniteCheckedException {
+        partCtx.createNewCacheDataStore(busyLock);
+
+        CacheDataTree tree = partCtx.oldCacheDataStore.tree();
+
+        CacheDataTree newTree = partCtx.newCacheDataStore.tree();
+        PendingEntriesTree newPendingTree = partCtx.newCacheDataStore.pendingTree();
+        AbstractFreeList<CacheDataRow> freeList = partCtx.newCacheDataStore.getCacheStoreFreeList();
+
+        long cpLockThreshold = 150L;
+
+        TimeTracker<PartStages> tracker = new TimeTracker<>(PartStages.class);
+
+        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+        tracker.complete(CP_LOCK);
+
+        try {
+            AtomicLong lastCpLockTs = new AtomicLong(System.currentTimeMillis());
+            AtomicInteger entriesProcessed = new AtomicInteger();
+
+            treeIter.iterate(tree, partCtx.cachePageMemory, (tree0, io, pageAddr, idx) -> {
+                tracker.complete(ITERATE);
+
+                if (System.currentTimeMillis() - lastCpLockTs.get() >= cpLockThreshold) {
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+                    tracker.complete(CP_LOCK);
+
+                    lastCpLockTs.set(System.currentTimeMillis());
+                }
+
+                AbstractDataLeafIO leafIo = (AbstractDataLeafIO)io;
+                CacheDataRow row = tree.getRow(io, pageAddr, idx);
+
+                tracker.complete(READ_ROW);
+
+                int cacheId = row.cacheId();
+
+                // Reuse row that we just read.
+                row.link(0);
+
+                // "insertDataRow" will corrupt page memory if we don't do this.
+                if (row instanceof DataRow && !partCtx.oldGrpCtx.storeCacheIdInDataPage())
+                    ((DataRow)row).cacheId(CU.UNDEFINED_CACHE_ID);
+
+                freeList.insertDataRow(row, IoStatisticsHolderNoOp.INSTANCE);
+
+                // Put it back.
+                if (row instanceof DataRow)
+                    ((DataRow)row).cacheId(cacheId);
+
+                tracker.complete(INSERT_ROW);
+
+                newTree.putx(row);
+
+                long newLink = row.link();
+
+                tracker.complete(STORE_MAP);
+
+                partCtx.linkMap.put(leafIo.getLink(pageAddr, idx), newLink);
+
+                tracker.complete(STORE_PK);
+
+                if (row.expireTime() != 0)
+                    newPendingTree.putx(new PendingRow(cacheId, row.expireTime(), newLink));
+
+                tracker.complete(STORE_PENDING);
+
+                entriesProcessed.incrementAndGet();
+
+                return true;
+            });
+
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+            tracker.complete(CP_LOCK);
+
+            freeList.saveMetadata(IoStatisticsHolderNoOp.INSTANCE);
+
+            copyCacheMetadata(partCtx);
+
+            tracker.complete(METADATA);
+        }
+        finally {
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+        }
+
+        if (log.isDebugEnabled()) {
+            log.debug(
+                "Partition defragmentation timings for cache group " + partCtx.grpId +
+                " and partition " + partCtx.partId + ": " + tracker.toString()
+            );
+        }
+    }
+
+    /** */
+    private void copyCacheMetadata(
+        PartitionContext partCtx
+    ) throws IgniteCheckedException {
+        // Same for all page memories. Why does it need to be in PageMemory?
+        long partMetaPageId = partCtx.cachePageMemory.partitionMetaPageId(partCtx.grpId, partCtx.partId);
+
+        access(ACCESS_READ, partCtx.cachePageMemory, partCtx.grpId, partMetaPageId, oldPartMetaPageAddr -> {
+            PagePartitionMetaIO oldPartMetaIo = PageIO.getPageIO(oldPartMetaPageAddr);
+
+            // Newer meta versions may contain new data that we don't copy during defragmentation.
+            assert Arrays.asList(1, 2, 3).contains(oldPartMetaIo.getVersion())
+                : "IO version " + oldPartMetaIo.getVersion() + " is not supported by current defragmentation algorithm." +
+                " Please implement copying of all data added in new version.";
+
+            access(ACCESS_WRITE, partCtx.partPageMemory, partCtx.grpId, partMetaPageId, newPartMetaPageAddr -> {
+                PagePartitionMetaIOV3 newPartMetaIo = PageIO.getPageIO(newPartMetaPageAddr);
+
+                // Copy partition state.
+                byte partState = oldPartMetaIo.getPartitionState(oldPartMetaPageAddr);
+                newPartMetaIo.setPartitionState(newPartMetaPageAddr, partState);
+
+                // Copy cache size for single cache group.
+                long size = oldPartMetaIo.getSize(oldPartMetaPageAddr);
+                newPartMetaIo.setSize(newPartMetaPageAddr, size);
+
+                // Copy update counter value.
+                long updateCntr = oldPartMetaIo.getUpdateCounter(oldPartMetaPageAddr);
+                newPartMetaIo.setUpdateCounter(newPartMetaPageAddr, updateCntr);
+
+                // Copy global remove Id.
+                long rmvId = oldPartMetaIo.getGlobalRemoveId(oldPartMetaPageAddr);
+                newPartMetaIo.setGlobalRemoveId(newPartMetaPageAddr, rmvId);
+
+                // Copy cache sizes for shared cache group.
+                long oldCountersPageId = oldPartMetaIo.getCountersPageId(oldPartMetaPageAddr);
+                if (oldCountersPageId != 0L) {
+                    Map<Integer, Long> sizes = GridCacheOffheapManager.readSharedGroupCacheSizes(
+                        partCtx.cachePageMemory,
+                        partCtx.grpId,
+                        oldCountersPageId
+                    );
+
+                    long newCountersPageId = GridCacheOffheapManager.writeSharedGroupCacheSizes(
+                        partCtx.partPageMemory,
+                        partCtx.grpId,
+                        0L,
+                        partCtx.partId,
+                        sizes
+                    );
+
+                    newPartMetaIo.setCountersPageId(newPartMetaPageAddr, newCountersPageId);
+                }
+
+                // Copy counter gaps.
+                long oldGapsLink = oldPartMetaIo.getGapsLink(oldPartMetaPageAddr);
+                if (oldGapsLink != 0L) {
+                    byte[] gapsBytes = partCtx.oldCacheDataStore.partStorage().readRow(oldGapsLink);
+
+                    SimpleDataRow gapsDataRow = new SimpleDataRow(partCtx.partId, gapsBytes);
+
+                    partCtx.newCacheDataStore.partStorage().insertDataRow(gapsDataRow, IoStatisticsHolderNoOp.INSTANCE);
+
+                    newPartMetaIo.setGapsLink(newPartMetaPageAddr, gapsDataRow.link());
+                }
+
+                // Encryption stuff.
+                newPartMetaIo.setEncryptedPageCount(newPartMetaPageAddr, 0);
+                newPartMetaIo.setEncryptedPageIndex(newPartMetaPageAddr, 0);
+
+                return null;
+            });
+
+            return null;
+        });
+    }
+
+    /**
+     * Defragmentate indexing partition.
+     *
+     * @param grpCtx
+     * @param mappingByPartition
+     *
+     * @throws IgniteCheckedException If failed.
+     */
+    private void defragmentIndexPartition(
+        CacheGroupContext grpCtx,
+        CacheGroupContext newCtx,
+        IntMap<LinkMap> mappingByPartition
+    ) throws IgniteCheckedException {
+        GridQueryProcessor query = grpCtx.caches().get(0).kernalContext().query();
+
+        if (!query.moduleEnabled())
+            return;
+
+        final GridQueryIndexing idx = query.getIndexing();
+
+        CheckpointTimeoutLock cpLock = defragmentationCheckpoint.checkpointTimeoutLock();
+
+        idx.defragmentator().defragment(
+            grpCtx,
+            newCtx,
+            (PageMemoryEx)partDataRegion.pageMemory(),
+            mappingByPartition,
+            cpLock,
+            log
+        );
+    }
+
+    /** */
+    @SuppressWarnings("PublicField")
+    private class PartitionContext {
+        /** */
+        public final File workDir;
+
+        /** */
+        public final int grpId;
+
+        /** */
+        public final int partId;
+
+        /** */
+        public final DataRegion cacheDataRegion;
+
+        /** */
+        public final PageMemoryEx cachePageMemory;
+
+        /** */
+        public final PageMemoryEx partPageMemory;
+
+        /** */
+        public final PageMemoryEx mappingPageMemory;
+
+        /** */
+        public final CacheGroupContext oldGrpCtx;
+
+        /** */
+        public final CacheGroupContext newGrpCtx;
+
+        /** */
+        public final CacheDataStore oldCacheDataStore;
+
+        /** */
+        private GridCacheDataStore newCacheDataStore;
+
+        /** */
+        public final FilePageStoreFactory pageStoreFactory;
+
+        /** */
+        public final AtomicLong partPagesAllocated = new AtomicLong();
+
+        /** */
+        public final AtomicLong mappingPagesAllocated = new AtomicLong();
+
+        /** */
+        private LinkMap linkMap;
+
+        /** */
+        public PartitionContext(
+            File workDir,
+            int grpId,
+            int partId,
+            DataRegion partDataRegion,
+            DataRegion mappingDataRegion,
+            CacheGroupContext oldGrpCtx,
+            CacheGroupContext newGrpCtx,
+            CacheDataStore oldCacheDataStore,
+            FilePageStoreFactory pageStoreFactory
+        ) {
+            this.workDir = workDir;
+            this.grpId = grpId;
+            this.partId = partId;
+            cacheDataRegion = oldGrpCtx.dataRegion();
+
+            cachePageMemory = (PageMemoryEx)cacheDataRegion.pageMemory();
+            partPageMemory = (PageMemoryEx)partDataRegion.pageMemory();
+            mappingPageMemory = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+            this.oldGrpCtx = oldGrpCtx;
+            this.newGrpCtx = newGrpCtx;
+            this.oldCacheDataStore = oldCacheDataStore;
+            this.pageStoreFactory = pageStoreFactory;
+        }
+
+        /** */
+        public PageStore createPartPageStore() throws IgniteCheckedException {
+            PageStore partPageStore;
+
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+            try {
+                partPageStore = pageStoreFactory.createPageStore(
+                    FLAG_DATA,
+                    () -> defragmentedPartTmpFile(workDir, partId).toPath(),
+                    partPagesAllocated::addAndGet
+                );
+            }
+            finally {
+                defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+            }
+
+            partPageStore.sync();
+
+            DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMemory.pageManager();
+
+            pageMgr.pageStoreMap().addPageStore(grpId, partId, partPageStore);
+
+            return partPageStore;
+        }
+
+        /** */
+        public PageStore createMappingPageStore() throws IgniteCheckedException {
+            PageStore mappingPageStore;
+
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+            try {
+                mappingPageStore = pageStoreFactory.createPageStore(
+                    FLAG_DATA,
+                    () -> defragmentedPartMappingFile(workDir, partId).toPath(),
+                    mappingPagesAllocated::addAndGet
+                );
+            }
+            finally {
+                defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+            }
+
+            mappingPageStore.sync();
+
+            DefragmentationPageReadWriteManager partMgr = (DefragmentationPageReadWriteManager)mappingPageMemory.pageManager();
+
+            partMgr.pageStoreMap().addPageStore(grpId, partId, mappingPageStore);
+
+            return mappingPageStore;
+        }
+
+        /** */
+        public LinkMap createLinkMapTree(boolean initNew) throws IgniteCheckedException {
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+            //TODO Store link in meta page and remove META_PAGE_IDX constant?
+            try {
+                long mappingMetaPageId = initNew
+                    ? mappingPageMemory.allocatePage(grpId, partId, FLAG_DATA)
+                    : PageIdUtils.pageId(partId, FLAG_DATA, LinkMap.META_PAGE_IDX);
+
+                assert PageIdUtils.pageIndex(mappingMetaPageId) == LinkMap.META_PAGE_IDX
+                    : PageIdUtils.toDetailString(mappingMetaPageId);
+
+                linkMap = new LinkMap(newGrpCtx, mappingPageMemory, mappingMetaPageId, initNew);
+            }
+            finally {
+                defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+            }
+
+            return linkMap;
+        }
+
+        /** */
+        public void createNewCacheDataStore(GridSpinBusyLock busyLock) {

Review comment:
       I agree with you, I'll create that method.
   EDIT: Done




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

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r531034827



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/TimeTracker.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.util.Arrays;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/** */
+public class TimeTracker<Stage extends Enum<Stage>> {

Review comment:
       Thas why any build-in aggregation machinery is a bad idea. For building reports, the raw data must be used.
   
   I have no doubts about the necessity of having things for debugging and measuring performance, finding hot spots but let's use more flexible and standard things like - [Tracing](https://issues.apache.org/jira/browse/IGNITE-13060) and/or [Cluster performance profiling tool](https://issues.apache.org/jira/browse/IGNITE-12666)
   
   Let's remove this thing from PR and file a new issue to implement it the right and more flexible way.




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

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r529603324



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {

Review comment:
       Why all this can't be moved to `ExecuteDefragmentationAction`?




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r526013873



##########
File path: modules/core/src/main/java/org/apache/ignite/configuration/DefragmentationConfiguration.java
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.configuration;
+
+import java.io.Serializable;
+
+/**
+ *
+ */
+public final class DefragmentationConfiguration implements Serializable {

Review comment:
       We'll try to derive region sizes from current configuration. Not exactly the same way as @agoncharuk described, but approach will be similar. I hope that it won't take much time.




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r530209150



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/maintenance/StopDefragmentationAction.java
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.processors.cache.persistence.defragmentation.maintenance;
+
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager;
+import org.apache.ignite.maintenance.MaintenanceAction;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Action which allows to stop the defragmentation at any time from maintenance mode processor.
+ */
+class StopDefragmentationAction implements MaintenanceAction<Boolean> {

Review comment:
       Ok




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r531457700



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;
+
+                                if (log.isDebugEnabled()) {
+                                    log.debug(S.toString(
+                                        "Partition defragmented",
+                                        "grpId", grpId, false,
+                                        "partId", partId, false,
+                                        "oldPages", oldPageStore.pages(), false,
+                                        "newPages", partCtx.partPagesAllocated.get() + 1, false,
+                                        "mappingPages", partCtx.mappingPagesAllocated.get() + 1, false,
+                                        "pageSize", pageSize, false,
+                                        "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                        "workDir", workDir, false
+                                    ));
+                                }
+
+                                oldPageMem.invalidate(grpId, partId);
+
+                                partCtx.partPageMemory.invalidate(grpId, partId);
+
+                                DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                                pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                                try {
+                                    renameTempPartitionFile(workDir, partId);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    throw new IgniteException(e);

Review comment:
       Ok, I'm cool with it, not a big deal




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

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r529583582



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?

Review comment:
       @ibessonov should we resolve all TODOs?




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

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r529616840



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/TreeIterator.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.nio.ByteBuffer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.util.GridUnsafe;
+
+/** */
+public class TreeIterator {
+    /** Direct memory buffer with a size of one page. */
+    private final ByteBuffer pageBuf;
+
+    /** Offheap page size. */
+    private final int pageSize;
+
+    /** */
+    public TreeIterator(int size) {
+        pageSize = size;
+
+        pageBuf = ByteBuffer.allocateDirect(pageSize);
+    }
+
+    // Performance impact of constant closures allocation is not clear. So this method should be avoided in massive

Review comment:
       Do you have estimations on performance impact? Should we use 'classic try-finally' approach instead?




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

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r529838222



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();

Review comment:
       Please, add test then the node stops concurrently in the middle of the defragmentation process.




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

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r532832167



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
##########
@@ -487,4 +493,25 @@ default long indexSize(String schemaName, String tblName, String idxName) throws
     default Map<String, Integer> secondaryIndexesInlineSize() {
         return Collections.emptyMap();
     }
+
+    /**
+     * Defragment index partition.
+     *
+     * @param grpCtx Old group context.
+     * @param newCtx New group context.
+     * @param partPageMem Partition page memory.
+     * @param mappingByPart Mapping page memory.
+     * @param cpLock Defragmentation checkpoint read lock.
+     * @param log Log.
+     *
+     * @throws IgniteCheckedException If failed.
+     */
+    void defragment(
+        CacheGroupContext grpCtx,
+        CacheGroupContext newCtx,
+        PageMemoryEx partPageMem,
+        IntMap<LinkMap> mappingByPart,
+        CheckpointTimeoutLock cpLock,

Review comment:
       Yes. Is it better calling the whole `defragment` method for indexing under `CachePartitionDefragmentationManager` and remove `cpLock` from the method parameters? 
   
   ```
           defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
           
           try {
               idx.defragment(
                   grpCtx,
                   newCtx,
                   (PageMemoryEx)partDataRegion.pageMemory(),
                   mappingByPartition
               );
           }
           finally {
               defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
           }
   ```




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r533392341



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
##########
@@ -487,4 +493,25 @@ default long indexSize(String schemaName, String tblName, String idxName) throws
     default Map<String, Integer> secondaryIndexesInlineSize() {
         return Collections.emptyMap();
     }
+
+    /**
+     * Defragment index partition.
+     *
+     * @param grpCtx Old group context.
+     * @param newCtx New group context.
+     * @param partPageMem Partition page memory.
+     * @param mappingByPart Mapping page memory.
+     * @param cpLock Defragmentation checkpoint read lock.
+     * @param log Log.
+     *
+     * @throws IgniteCheckedException If failed.
+     */
+    void defragment(
+        CacheGroupContext grpCtx,
+        CacheGroupContext newCtx,
+        PageMemoryEx partPageMem,
+        IntMap<LinkMap> mappingByPart,
+        CheckpointTimeoutLock cpLock,

Review comment:
       No, we don't want it to be under a single lock, it can overflow page memory. We have to release checkpoint lock and acquire it again sometimes.




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

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r529810622



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsDefragmentationTest.java
##########
@@ -0,0 +1,539 @@
+/*
+ * 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.processors.cache.persistence;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.FileVisitResult;
+import java.nio.file.FileVisitor;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardCopyOption;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.Collections;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+import java.util.function.UnaryOperator;
+import java.util.stream.IntStream;
+import javax.cache.configuration.Factory;
+import javax.cache.expiry.Duration;
+import javax.cache.expiry.ExpiryPolicy;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.FailureHandler;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.maintenance.MaintenanceFileStore;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.util.lang.IgniteThrowableConsumer;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentationCompletionMarkerFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.maintenance.DefragmentationParameters.toStore;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
+/** */
+public class IgnitePdsDefragmentationTest extends GridCommonAbstractTest {

Review comment:
       Please, add test if there is no space left on device in the middle of the process defragmentation.

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsDefragmentationTest.java
##########
@@ -0,0 +1,539 @@
+/*
+ * 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.processors.cache.persistence;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.FileVisitResult;
+import java.nio.file.FileVisitor;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardCopyOption;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.Collections;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+import java.util.function.UnaryOperator;
+import java.util.stream.IntStream;
+import javax.cache.configuration.Factory;
+import javax.cache.expiry.Duration;
+import javax.cache.expiry.ExpiryPolicy;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.FailureHandler;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.maintenance.MaintenanceFileStore;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.util.lang.IgniteThrowableConsumer;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentationCompletionMarkerFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.maintenance.DefragmentationParameters.toStore;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
+/** */
+public class IgnitePdsDefragmentationTest extends GridCommonAbstractTest {
+    /** */
+    public static final String CACHE_2_NAME = "cache2";
+
+    /** */
+    public static final int PARTS = 5;
+
+    /** */
+    public static final int ADDED_KEYS_COUNT = 150;
+
+    /** */
+    protected static final String GRP_NAME = "group";
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        stopAllGrids(true);
+
+        cleanPersistenceDir();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids(true);
+
+        cleanPersistenceDir();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected FailureHandler getFailureHandler(String igniteInstanceName) {
+        return new StopNodeFailureHandler();
+    }
+
+    /** */
+    protected static class PolicyFactory implements Factory<ExpiryPolicy> {
+        /** Serial version uid. */
+        private static final long serialVersionUID = 0L;
+
+        /** {@inheritDoc} */
+        @Override public ExpiryPolicy create() {
+            return new ExpiryPolicy() {
+                @Override public Duration getExpiryForCreation() {
+                    return new Duration(TimeUnit.MILLISECONDS, 13000);
+                }
+
+                /** {@inheritDoc} */
+                @Override public Duration getExpiryForAccess() {
+                    return new Duration(TimeUnit.MILLISECONDS, 13000);
+                }
+
+                /** {@inheritDoc} */
+                @Override public Duration getExpiryForUpdate() {
+                    return new Duration(TimeUnit.MILLISECONDS, 13000);
+                }
+            };
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setConsistentId(igniteInstanceName);
+
+        DataStorageConfiguration dsCfg = new DataStorageConfiguration();
+        dsCfg.setWalSegmentSize(4 * 1024 * 1024);
+
+        dsCfg.setDefaultDataRegionConfiguration(
+            new DataRegionConfiguration()
+                .setInitialSize(100L * 1024 * 1024)
+                .setMaxSize(1024L * 1024 * 1024)
+                .setPersistenceEnabled(true)
+        );
+
+        cfg.setDataStorageConfiguration(dsCfg);
+
+        CacheConfiguration<?, ?> cache1Cfg = new CacheConfiguration<>(DEFAULT_CACHE_NAME)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setGroupName(GRP_NAME)
+            .setAffinity(new RendezvousAffinityFunction(false, PARTS));
+
+        CacheConfiguration<?, ?> cache2Cfg = new CacheConfiguration<>(CACHE_2_NAME)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setGroupName(GRP_NAME)
+            .setExpiryPolicyFactory(new PolicyFactory())
+            .setAffinity(new RendezvousAffinityFunction(false, PARTS));
+
+        cfg.setCacheConfiguration(cache1Cfg, cache2Cfg);
+
+        return cfg;
+    }
+
+    /**
+     * Basic test scenario. Does following steps:
+     *  - Start node;
+     *  - Fill cache;
+     *  - Remove part of data;
+     *  - Stop node;
+     *  - Start node in defragmentation mode;
+     *  - Stop node;
+     *  - Start node;
+     *  - Check that partitions became smaller;
+     *  - Check that cache is accessible and works just fine.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testEssentials() throws Exception {

Review comment:
       I don't think this is a good naming for tests. `testDefragmentationEndsSuccessfully`?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();

Review comment:
       This is a dangerous practice. Sharing a busy lock may freeze the node. It must be private.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {

Review comment:
       Let's use this approach which removes unnecessary code identation.
   
   ```
   if (fut.error() != null)
       return;
   ```

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsDefragmentationTest.java
##########
@@ -0,0 +1,539 @@
+/*
+ * 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.processors.cache.persistence;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.FileVisitResult;
+import java.nio.file.FileVisitor;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardCopyOption;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.Collections;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+import java.util.function.UnaryOperator;
+import java.util.stream.IntStream;
+import javax.cache.configuration.Factory;
+import javax.cache.expiry.Duration;
+import javax.cache.expiry.ExpiryPolicy;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.FailureHandler;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.maintenance.MaintenanceFileStore;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.util.lang.IgniteThrowableConsumer;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentationCompletionMarkerFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.maintenance.DefragmentationParameters.toStore;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
+/** */
+public class IgnitePdsDefragmentationTest extends GridCommonAbstractTest {
+    /** */
+    public static final String CACHE_2_NAME = "cache2";
+
+    /** */
+    public static final int PARTS = 5;
+
+    /** */
+    public static final int ADDED_KEYS_COUNT = 150;
+
+    /** */
+    protected static final String GRP_NAME = "group";
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        stopAllGrids(true);
+
+        cleanPersistenceDir();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids(true);
+
+        cleanPersistenceDir();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected FailureHandler getFailureHandler(String igniteInstanceName) {
+        return new StopNodeFailureHandler();
+    }
+
+    /** */
+    protected static class PolicyFactory implements Factory<ExpiryPolicy> {
+        /** Serial version uid. */
+        private static final long serialVersionUID = 0L;
+
+        /** {@inheritDoc} */
+        @Override public ExpiryPolicy create() {
+            return new ExpiryPolicy() {
+                @Override public Duration getExpiryForCreation() {
+                    return new Duration(TimeUnit.MILLISECONDS, 13000);
+                }
+
+                /** {@inheritDoc} */
+                @Override public Duration getExpiryForAccess() {
+                    return new Duration(TimeUnit.MILLISECONDS, 13000);
+                }
+
+                /** {@inheritDoc} */
+                @Override public Duration getExpiryForUpdate() {
+                    return new Duration(TimeUnit.MILLISECONDS, 13000);
+                }
+            };
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setConsistentId(igniteInstanceName);
+
+        DataStorageConfiguration dsCfg = new DataStorageConfiguration();
+        dsCfg.setWalSegmentSize(4 * 1024 * 1024);
+
+        dsCfg.setDefaultDataRegionConfiguration(
+            new DataRegionConfiguration()
+                .setInitialSize(100L * 1024 * 1024)
+                .setMaxSize(1024L * 1024 * 1024)
+                .setPersistenceEnabled(true)
+        );
+
+        cfg.setDataStorageConfiguration(dsCfg);
+
+        CacheConfiguration<?, ?> cache1Cfg = new CacheConfiguration<>(DEFAULT_CACHE_NAME)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setGroupName(GRP_NAME)
+            .setAffinity(new RendezvousAffinityFunction(false, PARTS));
+
+        CacheConfiguration<?, ?> cache2Cfg = new CacheConfiguration<>(CACHE_2_NAME)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setGroupName(GRP_NAME)
+            .setExpiryPolicyFactory(new PolicyFactory())
+            .setAffinity(new RendezvousAffinityFunction(false, PARTS));
+
+        cfg.setCacheConfiguration(cache1Cfg, cache2Cfg);
+
+        return cfg;
+    }
+
+    /**
+     * Basic test scenario. Does following steps:
+     *  - Start node;
+     *  - Fill cache;
+     *  - Remove part of data;
+     *  - Stop node;
+     *  - Start node in defragmentation mode;
+     *  - Stop node;
+     *  - Start node;
+     *  - Check that partitions became smaller;
+     *  - Check that cache is accessible and works just fine.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testEssentials() throws Exception {
+        IgniteEx ig = startGrid(0);
+
+        ig.cluster().state(ClusterState.ACTIVE);
+
+        fillCache(ig.cache(DEFAULT_CACHE_NAME));
+
+        forceCheckpoint(ig);
+
+        createMaintenanceRecord();
+
+        stopGrid(0);
+
+        File workDir = resolveCacheWorkDir(ig);
+
+        long[] oldPartLen = partitionSizes(workDir);
+
+        long oldIdxFileLen = new File(workDir, FilePageStoreManager.INDEX_FILE_NAME).length();
+
+        startGrid(0);
+
+        long[] newPartLen = partitionSizes(workDir);
+
+        for (int p = 0; p < PARTS; p++)
+            assertTrue(newPartLen[p] < oldPartLen[p]); //TODO Fails.
+
+        long newIdxFileLen = new File(workDir, FilePageStoreManager.INDEX_FILE_NAME).length();
+
+        assertTrue(newIdxFileLen <= oldIdxFileLen);
+
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+        assertTrue(completionMarkerFile.exists());
+
+        stopGrid(0);
+
+        IgniteEx ig0 = startGrid(0);
+
+        ig0.cluster().state(ClusterState.ACTIVE);
+
+        assertFalse(completionMarkerFile.exists());
+
+        validateCache(grid(0).cache(DEFAULT_CACHE_NAME));

Review comment:
       Check that there are not temporary defragmentation files left after the process completes successfully.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/maintenance/StopDefragmentationAction.java
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.processors.cache.persistence.defragmentation.maintenance;
+
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager;
+import org.apache.ignite.maintenance.MaintenanceAction;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Action which allows to stop the defragmentation at any time from maintenance mode processor.
+ */
+class StopDefragmentationAction implements MaintenanceAction<Boolean> {

Review comment:
       Let's remove it since it not used.

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsDefragmentationTest.java
##########
@@ -0,0 +1,539 @@
+/*
+ * 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.processors.cache.persistence;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.FileVisitResult;
+import java.nio.file.FileVisitor;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardCopyOption;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.Collections;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+import java.util.function.UnaryOperator;
+import java.util.stream.IntStream;
+import javax.cache.configuration.Factory;
+import javax.cache.expiry.Duration;
+import javax.cache.expiry.ExpiryPolicy;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.FailureHandler;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.maintenance.MaintenanceFileStore;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.util.lang.IgniteThrowableConsumer;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentationCompletionMarkerFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.maintenance.DefragmentationParameters.toStore;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
+/** */
+public class IgnitePdsDefragmentationTest extends GridCommonAbstractTest {
+    /** */
+    public static final String CACHE_2_NAME = "cache2";
+
+    /** */
+    public static final int PARTS = 5;
+
+    /** */
+    public static final int ADDED_KEYS_COUNT = 150;
+
+    /** */
+    protected static final String GRP_NAME = "group";
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        stopAllGrids(true);
+
+        cleanPersistenceDir();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids(true);
+
+        cleanPersistenceDir();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected FailureHandler getFailureHandler(String igniteInstanceName) {
+        return new StopNodeFailureHandler();
+    }
+
+    /** */
+    protected static class PolicyFactory implements Factory<ExpiryPolicy> {
+        /** Serial version uid. */
+        private static final long serialVersionUID = 0L;
+
+        /** {@inheritDoc} */
+        @Override public ExpiryPolicy create() {
+            return new ExpiryPolicy() {
+                @Override public Duration getExpiryForCreation() {
+                    return new Duration(TimeUnit.MILLISECONDS, 13000);
+                }
+
+                /** {@inheritDoc} */
+                @Override public Duration getExpiryForAccess() {
+                    return new Duration(TimeUnit.MILLISECONDS, 13000);
+                }
+
+                /** {@inheritDoc} */
+                @Override public Duration getExpiryForUpdate() {
+                    return new Duration(TimeUnit.MILLISECONDS, 13000);
+                }
+            };
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setConsistentId(igniteInstanceName);
+
+        DataStorageConfiguration dsCfg = new DataStorageConfiguration();
+        dsCfg.setWalSegmentSize(4 * 1024 * 1024);
+
+        dsCfg.setDefaultDataRegionConfiguration(
+            new DataRegionConfiguration()
+                .setInitialSize(100L * 1024 * 1024)
+                .setMaxSize(1024L * 1024 * 1024)
+                .setPersistenceEnabled(true)
+        );
+
+        cfg.setDataStorageConfiguration(dsCfg);
+
+        CacheConfiguration<?, ?> cache1Cfg = new CacheConfiguration<>(DEFAULT_CACHE_NAME)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setGroupName(GRP_NAME)
+            .setAffinity(new RendezvousAffinityFunction(false, PARTS));
+
+        CacheConfiguration<?, ?> cache2Cfg = new CacheConfiguration<>(CACHE_2_NAME)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setGroupName(GRP_NAME)
+            .setExpiryPolicyFactory(new PolicyFactory())
+            .setAffinity(new RendezvousAffinityFunction(false, PARTS));
+
+        cfg.setCacheConfiguration(cache1Cfg, cache2Cfg);
+
+        return cfg;
+    }
+
+    /**
+     * Basic test scenario. Does following steps:
+     *  - Start node;
+     *  - Fill cache;
+     *  - Remove part of data;
+     *  - Stop node;
+     *  - Start node in defragmentation mode;
+     *  - Stop node;
+     *  - Start node;
+     *  - Check that partitions became smaller;
+     *  - Check that cache is accessible and works just fine.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testEssentials() throws Exception {
+        IgniteEx ig = startGrid(0);
+
+        ig.cluster().state(ClusterState.ACTIVE);
+
+        fillCache(ig.cache(DEFAULT_CACHE_NAME));
+
+        forceCheckpoint(ig);
+
+        createMaintenanceRecord();
+
+        stopGrid(0);
+
+        File workDir = resolveCacheWorkDir(ig);
+
+        long[] oldPartLen = partitionSizes(workDir);
+
+        long oldIdxFileLen = new File(workDir, FilePageStoreManager.INDEX_FILE_NAME).length();
+
+        startGrid(0);
+
+        long[] newPartLen = partitionSizes(workDir);
+
+        for (int p = 0; p < PARTS; p++)
+            assertTrue(newPartLen[p] < oldPartLen[p]); //TODO Fails.
+
+        long newIdxFileLen = new File(workDir, FilePageStoreManager.INDEX_FILE_NAME).length();
+
+        assertTrue(newIdxFileLen <= oldIdxFileLen);
+
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+        assertTrue(completionMarkerFile.exists());
+
+        stopGrid(0);
+
+        IgniteEx ig0 = startGrid(0);
+
+        ig0.cluster().state(ClusterState.ACTIVE);
+
+        assertFalse(completionMarkerFile.exists());
+
+        validateCache(grid(0).cache(DEFAULT_CACHE_NAME));
+    }
+
+    /**
+     * @return Working directory for cache group {@link IgnitePdsDefragmentationTest#GRP_NAME}.
+     * @throws IgniteCheckedException If failed for some reason, like if it's a file instead of directory.
+     */
+    private File resolveCacheWorkDir(IgniteEx ig) throws IgniteCheckedException {
+        File dbWorkDir = U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false);
+
+        File nodeWorkDir = new File(dbWorkDir, U.maskForFileName(ig.name()));
+
+        return new File(nodeWorkDir, FilePageStoreManager.CACHE_GRP_DIR_PREFIX + GRP_NAME);
+    }
+
+    /**
+     * Force checkpoint and wait for it so all partitions will be in their final state after restart if no more data is
+     * uploaded.
+     *
+     * @param ig Ignite node.
+     * @throws IgniteCheckedException If checkpoint failed for some reason.
+     */
+    private void forceCheckpoint(IgniteEx ig) throws IgniteCheckedException {
+        ig.context().cache().context().database()
+            .forceCheckpoint("testDefrag")
+            .futureFor(CheckpointState.FINISHED)
+            .get();
+    }
+
+    /** */
+    protected void createMaintenanceRecord() throws IgniteCheckedException {
+        IgniteEx grid = grid(0);
+        MaintenanceRegistry mntcReg = grid.context().maintenanceRegistry();
+
+        mntcReg.registerMaintenanceTask(toStore(Collections.singletonList(groupIdForCache(grid, DEFAULT_CACHE_NAME))));
+    }
+
+    /**
+     * Returns array that contains sizes of partition files in gived working directories. Assumes that partitions
+     * {@code 0} to {@code PARTS - 1} exist in that dir.
+     *
+     * @param workDir Working directory.
+     * @return The array.
+     */
+    protected long[] partitionSizes(File workDir) {
+        return IntStream.range(0, PARTS)
+            .mapToObj(p -> new File(workDir, String.format(FilePageStoreManager.PART_FILE_TEMPLATE, p)))
+            .mapToLong(File::length)
+            .toArray();
+    }
+
+    /**
+     * Checks that plain node start after failed defragmentation will finish batch renaming.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testFailoverRestartWithoutDefragmentation() throws Exception {
+        testFailover(workDir -> {
+            try {
+                File mntcRecFile = new File(workDir.getParent(), MaintenanceFileStore.MAINTENANCE_FILE_NAME);
+
+                assertTrue(mntcRecFile.exists());
+
+                Files.delete(mntcRecFile.toPath());
+
+                startGrid(0);
+
+                validateLeftovers(workDir);
+            }
+            catch (Exception e) {
+                throw new IgniteCheckedException(e);
+            }
+            finally {
+                createMaintenanceRecord();
+
+                stopGrid(0);
+            }
+        });
+    }
+
+    /**
+     * Checks that second start in defragmentation mode will finish defragmentation if no completion marker was found.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testFailoverBasic() throws Exception {

Review comment:
       `testDefragmentationFinishedIfMarkerCleared`?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;
+
+                                if (log.isDebugEnabled()) {
+                                    log.debug(S.toString(
+                                        "Partition defragmented",
+                                        "grpId", grpId, false,
+                                        "partId", partId, false,
+                                        "oldPages", oldPageStore.pages(), false,
+                                        "newPages", partCtx.partPagesAllocated.get() + 1, false,
+                                        "mappingPages", partCtx.mappingPagesAllocated.get() + 1, false,
+                                        "pageSize", pageSize, false,
+                                        "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                        "workDir", workDir, false
+                                    ));
+                                }
+
+                                oldPageMem.invalidate(grpId, partId);
+
+                                partCtx.partPageMemory.invalidate(grpId, partId);
+
+                                DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                                pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                                try {
+                                    renameTempPartitionFile(workDir, partId);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    throw new IgniteException(e);

Review comment:
       What would happen with this exception? You will fail the node or it will be ignored? Will it be logged?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;
+
+                                if (log.isDebugEnabled()) {
+                                    log.debug(S.toString(
+                                        "Partition defragmented",
+                                        "grpId", grpId, false,
+                                        "partId", partId, false,
+                                        "oldPages", oldPageStore.pages(), false,
+                                        "newPages", partCtx.partPagesAllocated.get() + 1, false,
+                                        "mappingPages", partCtx.mappingPagesAllocated.get() + 1, false,
+                                        "pageSize", pageSize, false,
+                                        "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                        "workDir", workDir, false
+                                    ));
+                                }
+
+                                oldPageMem.invalidate(grpId, partId);
+
+                                partCtx.partPageMemory.invalidate(grpId, partId);
+
+                                DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                                pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                                try {
+                                    renameTempPartitionFile(workDir, partId);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    throw new IgniteException(e);
+                                }
+                            }
+                        };
+
+                        GridFutureAdapter<?> cpFut = defragmentationCheckpoint
+                            .forceCheckpoint("partition defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+
+                        cpFut.listen(cpLsnr);
+
+                        cmpFut.add((IgniteInternalFuture<Object>)cpFut);
+                    }
+
+                    // A bit too general for now, but I like it more then saving only the last checkpoint future.
+                    cmpFut.markInitialized().get();
+
+                    idxDfrgFut = new GridFinishedFuture<>();
+
+                    if (filePageStoreMgr.hasIndexStore(grpId)) {
+                        defragmentIndexPartition(oldGrpCtx, newGrpCtx, linkMapByPart);
+
+                        idxDfrgFut = defragmentationCheckpoint
+                            .forceCheckpoint("index defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+                    }
+
+                    idxDfrgFut.listen(fut -> {
+                        oldPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx partPageMem = (PageMemoryEx)partDataRegion.pageMemory();
+
+                        partPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().removePageStore(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx mappingPageMem = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+                        pageMgr = (DefragmentationPageReadWriteManager)mappingPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().clear(grpId);
+
+                        try {
+                            renameTempIndexFile(workDir);
+
+                            writeDefragmentationCompletionMarker(filePageStoreMgr.getPageStoreFileIoFactory(), workDir, log);
+
+                            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);

Review comment:
       Same, will the exception be ignored?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;
+
+                                if (log.isDebugEnabled()) {
+                                    log.debug(S.toString(
+                                        "Partition defragmented",
+                                        "grpId", grpId, false,
+                                        "partId", partId, false,
+                                        "oldPages", oldPageStore.pages(), false,
+                                        "newPages", partCtx.partPagesAllocated.get() + 1, false,
+                                        "mappingPages", partCtx.mappingPagesAllocated.get() + 1, false,
+                                        "pageSize", pageSize, false,
+                                        "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                        "workDir", workDir, false
+                                    ));
+                                }
+
+                                oldPageMem.invalidate(grpId, partId);
+
+                                partCtx.partPageMemory.invalidate(grpId, partId);
+
+                                DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                                pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                                try {
+                                    renameTempPartitionFile(workDir, partId);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    throw new IgniteException(e);
+                                }
+                            }
+                        };
+
+                        GridFutureAdapter<?> cpFut = defragmentationCheckpoint
+                            .forceCheckpoint("partition defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+
+                        cpFut.listen(cpLsnr);
+
+                        cmpFut.add((IgniteInternalFuture<Object>)cpFut);
+                    }
+
+                    // A bit too general for now, but I like it more then saving only the last checkpoint future.
+                    cmpFut.markInitialized().get();
+
+                    idxDfrgFut = new GridFinishedFuture<>();
+
+                    if (filePageStoreMgr.hasIndexStore(grpId)) {
+                        defragmentIndexPartition(oldGrpCtx, newGrpCtx, linkMapByPart);
+
+                        idxDfrgFut = defragmentationCheckpoint
+                            .forceCheckpoint("index defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+                    }
+
+                    idxDfrgFut.listen(fut -> {
+                        oldPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx partPageMem = (PageMemoryEx)partDataRegion.pageMemory();
+
+                        partPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().removePageStore(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx mappingPageMem = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+                        pageMgr = (DefragmentationPageReadWriteManager)mappingPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().clear(grpId);
+
+                        try {
+                            renameTempIndexFile(workDir);
+
+                            writeDefragmentationCompletionMarker(filePageStoreMgr.getPageStoreFileIoFactory(), workDir, log);

Review comment:
       Please, add failover test if `writeDefragmentationCompletionMarker` throw an IOException (e.g. no space left). Will the process be finished?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;

Review comment:
       You will fail the node with an assertion error if `getStore(grpId, partId)` throws an exception an it will be ignored.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;
+
+                                if (log.isDebugEnabled()) {
+                                    log.debug(S.toString(
+                                        "Partition defragmented",
+                                        "grpId", grpId, false,
+                                        "partId", partId, false,
+                                        "oldPages", oldPageStore.pages(), false,
+                                        "newPages", partCtx.partPagesAllocated.get() + 1, false,
+                                        "mappingPages", partCtx.mappingPagesAllocated.get() + 1, false,
+                                        "pageSize", pageSize, false,
+                                        "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                        "workDir", workDir, false
+                                    ));
+                                }
+
+                                oldPageMem.invalidate(grpId, partId);
+
+                                partCtx.partPageMemory.invalidate(grpId, partId);
+
+                                DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                                pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                                try {
+                                    renameTempPartitionFile(workDir, partId);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    throw new IgniteException(e);

Review comment:
       You throw `IgniteCheckedException` inside `renameTempPartitionFile` than catch it here and rethrow `IgniteException`. 
   Is it better to throw `IgniteException` right from renameTempPartitionFile method? Why we need checked exception here?




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

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r530967180



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;
+
+                                if (log.isDebugEnabled()) {
+                                    log.debug(S.toString(
+                                        "Partition defragmented",
+                                        "grpId", grpId, false,
+                                        "partId", partId, false,
+                                        "oldPages", oldPageStore.pages(), false,
+                                        "newPages", partCtx.partPagesAllocated.get() + 1, false,
+                                        "mappingPages", partCtx.mappingPagesAllocated.get() + 1, false,
+                                        "pageSize", pageSize, false,
+                                        "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                        "workDir", workDir, false
+                                    ));
+                                }
+
+                                oldPageMem.invalidate(grpId, partId);
+
+                                partCtx.partPageMemory.invalidate(grpId, partId);
+
+                                DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                                pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                                try {
+                                    renameTempPartitionFile(workDir, partId);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    throw new IgniteException(e);

Review comment:
       I don't think we need it here, let's throw directly `IgniteException`? This will simplify the 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.

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r529800369



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,896 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database schared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;
+
+                                if (log.isDebugEnabled()) {
+                                    log.debug(S.toString(
+                                        "Partition defragmented",
+                                        "grpId", grpId, false,
+                                        "partId", partId, false,
+                                        "oldPages", oldPageStore.pages(), false,
+                                        "newPages", partCtx.partPagesAllocated.get() + 1, false,
+                                        "mappingPages", partCtx.mappingPagesAllocated.get() + 1, false,
+                                        "pageSize", pageSize, false,
+                                        "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                        "workDir", workDir, false
+                                    ));
+                                }
+
+                                oldPageMem.invalidate(grpId, partId);
+
+                                partCtx.partPageMemory.invalidate(grpId, partId);
+
+                                DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                                pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                                try {
+                                    renameTempPartitionFile(workDir, partId);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    throw new IgniteException(e);
+                                }
+                            }
+                        };
+
+                        GridFutureAdapter<?> cpFut = defragmentationCheckpoint
+                            .forceCheckpoint("partition defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+
+                        cpFut.listen(cpLsnr);
+
+                        cmpFut.add((IgniteInternalFuture<Object>)cpFut);
+                    }
+
+                    // A bit too general for now, but I like it more then saving only the last checkpoint future.
+                    cmpFut.markInitialized().get();
+
+                    idxDfrgFut = new GridFinishedFuture<>();
+
+                    if (filePageStoreMgr.hasIndexStore(grpId)) {
+                        defragmentIndexPartition(oldGrpCtx, newGrpCtx, linkMapByPart);
+
+                        idxDfrgFut = defragmentationCheckpoint
+                            .forceCheckpoint("index defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+                    }
+
+                    idxDfrgFut.listen(fut -> {
+                        oldPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx partPageMem = (PageMemoryEx)partDataRegion.pageMemory();
+
+                        partPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().removePageStore(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx mappingPageMem = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+                        pageMgr = (DefragmentationPageReadWriteManager)mappingPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().clear(grpId);
+
+                        try {
+                            renameTempIndexFile(workDir);
+
+                            writeDefragmentationCompletionMarker(filePageStoreMgr.getPageStoreFileIoFactory(), workDir, log);
+
+                            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    });
+                }
+
+                // I guess we should wait for it?
+                if (idxDfrgFut != null)
+                    idxDfrgFut.get();
+            }
+
+            mntcReg.unregisterMaintenanceTask(DEFRAGMENTATION_MNTC_TASK_NAME);
+
+            log.info("Defragmentation completed. All partitions are defragmented.");
+        }
+        finally {
+            defragmentationCheckpoint.stop(true);
+        }
+    }
+
+    /** */
+    public void createIndexPageStore(
+        int grpId,
+        File workDir,
+        FilePageStoreFactory pageStoreFactory,
+        DataRegion partRegion,
+        LongConsumer allocatedTracker
+    ) throws IgniteCheckedException {
+        // Index partition file has to be deleted before we begin, otherwise there's a chance of reading corrupted file.
+        // There is a time period when index is already defragmented but marker file is not created yet. If node is
+        // failed in that time window then index will be deframented once again. That's fine, situation is rare but code
+        // to fix that would add unnecessary complications.
+        U.delete(defragmentedIndexTmpFile(workDir));
+
+        PageStore idxPageStore;
+
+        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+        try {
+            idxPageStore = pageStoreFactory.createPageStore(
+                FLAG_IDX,
+                () -> defragmentedIndexTmpFile(workDir).toPath(),
+                allocatedTracker
+            );
+        }
+        finally {
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+        }
+
+        idxPageStore.sync();
+
+        PageMemoryEx partPageMem = (PageMemoryEx)partRegion.pageMemory();
+
+        DefragmentationPageReadWriteManager partMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+        partMgr.pageStoreMap().addPageStore(grpId, PageIdAllocator.INDEX_PARTITION, idxPageStore);
+    }
+
+    /**
+     * Cancel the process of defragmentation.
+     */
+    public void cancel(){

Review comment:
       Ok, let's remove this stubs and all related things from this PR.




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r526713644



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,896 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database schared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;
+
+                                if (log.isDebugEnabled()) {
+                                    log.debug(S.toString(
+                                        "Partition defragmented",
+                                        "grpId", grpId, false,
+                                        "partId", partId, false,
+                                        "oldPages", oldPageStore.pages(), false,
+                                        "newPages", partCtx.partPagesAllocated.get() + 1, false,
+                                        "mappingPages", partCtx.mappingPagesAllocated.get() + 1, false,
+                                        "pageSize", pageSize, false,
+                                        "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                        "workDir", workDir, false
+                                    ));
+                                }
+
+                                oldPageMem.invalidate(grpId, partId);
+
+                                partCtx.partPageMemory.invalidate(grpId, partId);
+
+                                DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                                pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                                try {
+                                    renameTempPartitionFile(workDir, partId);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    throw new IgniteException(e);
+                                }
+                            }
+                        };
+
+                        GridFutureAdapter<?> cpFut = defragmentationCheckpoint
+                            .forceCheckpoint("partition defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+
+                        cpFut.listen(cpLsnr);
+
+                        cmpFut.add((IgniteInternalFuture<Object>)cpFut);
+                    }
+
+                    // A bit too general for now, but I like it more then saving only the last checkpoint future.
+                    cmpFut.markInitialized().get();
+
+                    idxDfrgFut = new GridFinishedFuture<>();
+
+                    if (filePageStoreMgr.hasIndexStore(grpId)) {
+                        defragmentIndexPartition(oldGrpCtx, newGrpCtx, linkMapByPart);
+
+                        idxDfrgFut = defragmentationCheckpoint
+                            .forceCheckpoint("index defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+                    }
+
+                    idxDfrgFut.listen(fut -> {
+                        oldPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx partPageMem = (PageMemoryEx)partDataRegion.pageMemory();
+
+                        partPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().removePageStore(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx mappingPageMem = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+                        pageMgr = (DefragmentationPageReadWriteManager)mappingPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().clear(grpId);
+
+                        try {
+                            renameTempIndexFile(workDir);
+
+                            writeDefragmentationCompletionMarker(filePageStoreMgr.getPageStoreFileIoFactory(), workDir, log);
+
+                            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    });
+                }
+
+                // I guess we should wait for it?
+                if (idxDfrgFut != null)
+                    idxDfrgFut.get();
+            }
+
+            mntcReg.unregisterMaintenanceTask(DEFRAGMENTATION_MNTC_TASK_NAME);
+
+            log.info("Defragmentation completed. All partitions are defragmented.");
+        }
+        finally {
+            defragmentationCheckpoint.stop(true);
+        }
+    }
+
+    /** */
+    public void createIndexPageStore(
+        int grpId,
+        File workDir,
+        FilePageStoreFactory pageStoreFactory,
+        DataRegion partRegion,
+        LongConsumer allocatedTracker
+    ) throws IgniteCheckedException {
+        // Index partition file has to be deleted before we begin, otherwise there's a chance of reading corrupted file.
+        // There is a time period when index is already defragmented but marker file is not created yet. If node is
+        // failed in that time window then index will be deframented once again. That's fine, situation is rare but code
+        // to fix that would add unnecessary complications.
+        U.delete(defragmentedIndexTmpFile(workDir));
+
+        PageStore idxPageStore;
+
+        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+        try {
+            idxPageStore = pageStoreFactory.createPageStore(
+                FLAG_IDX,
+                () -> defragmentedIndexTmpFile(workDir).toPath(),
+                allocatedTracker
+            );
+        }
+        finally {
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+        }
+
+        idxPageStore.sync();
+
+        PageMemoryEx partPageMem = (PageMemoryEx)partRegion.pageMemory();
+
+        DefragmentationPageReadWriteManager partMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+        partMgr.pageStoreMap().addPageStore(grpId, PageIdAllocator.INDEX_PARTITION, idxPageStore);
+    }
+
+    /**
+     * Cancel the process of defragmentation.
+     */
+    public void cancel(){

Review comment:
       We will implement it later along with "schedule" command. This particular issue doesn't contain any public API to trigger defragmentation.
   https://issues.apache.org/jira/browse/IGNITE-13697 is pretty much completed as well. We don't need additional 1500 lines of code here :)




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r524014723



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/defragmentation/IndexingDefragmentation.java
##########
@@ -0,0 +1,454 @@
+/*
+ * 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.processors.query.h2.defragmentation;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.GridQueryIndexingDefragmentation;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.LinkMap;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.TimeTracker;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIoResolver;
+import org.apache.ignite.internal.processors.cache.persistence.tree.util.InsertLast;
+import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccDataRow;
+import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+import org.apache.ignite.internal.processors.query.h2.database.H2Tree;
+import org.apache.ignite.internal.processors.query.h2.database.H2TreeIndex;
+import org.apache.ignite.internal.processors.query.h2.database.InlineIndexColumn;
+import org.apache.ignite.internal.processors.query.h2.database.inlinecolumn.AbstractInlineIndexColumn;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2ExtrasInnerIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2ExtrasLeafIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2InnerIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2LeafIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.H2RowLinkIO;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.processors.query.h2.opt.H2CacheRow;
+import org.apache.ignite.internal.processors.query.h2.opt.H2Row;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.h2.index.Index;
+import org.h2.value.Value;
+
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.INIT_TREE;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.ITERATE;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.READ_MAP;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.READ_ROW;
+
+/**
+ *
+ */
+public class IndexingDefragmentation implements GridQueryIndexingDefragmentation {
+    /** Indexing. */
+    private final IgniteH2Indexing indexing;
+
+    /** Constructor. */
+    public IndexingDefragmentation(IgniteH2Indexing indexing) {
+        this.indexing = indexing;
+    }
+
+    /** */
+    public enum IndexStages {
+        START,
+        CP_LOCK,
+        INIT_TREE,
+        ITERATE,
+        READ_ROW,
+        READ_MAP,
+        INSERT_ROW
+    }
+
+    /** {@inheritDoc} */
+    @Override public void defragmentate(
+        CacheGroupContext grpCtx,
+        CacheGroupContext newCtx,
+        PageMemoryEx partPageMem,
+        IntMap<LinkMap> mappingByPartition,
+        CheckpointTimeoutLock cpLock,
+        IgniteLogger log
+    ) throws IgniteCheckedException {
+        int pageSize = grpCtx.cacheObjectContext().kernalContext().grid().configuration().getDataStorageConfiguration().getPageSize();
+
+        TreeIterator treeIterator = new TreeIterator(pageSize);
+
+        PageMemoryEx oldCachePageMem = (PageMemoryEx)grpCtx.dataRegion().pageMemory();
+
+        PageMemory newCachePageMemory = partPageMem;
+
+        Collection<GridH2Table> tables = indexing.schemaManager().dataTables();
+
+        long cpLockThreshold = 250L;
+
+        TimeTracker<IndexStages> tracker = new TimeTracker<>(IndexStages.class);
+
+        cpLock.checkpointReadLock();
+        tracker.complete(CP_LOCK);
+
+        try {
+            AtomicLong lastCpLockTs = new AtomicLong(System.currentTimeMillis());
+
+            for (GridH2Table table : tables) {
+                GridCacheContext<?, ?> cctx = table.cacheContext();
+
+                if (cctx.groupId() != grpCtx.groupId())
+                    continue; // Not our index.
+
+                GridH2RowDescriptor rowDesc = table.rowDescriptor();
+
+                List<Index> indexes = table.getIndexes();
+                H2TreeIndex oldH2Idx = (H2TreeIndex)indexes.get(2);
+
+                int segments = oldH2Idx.segmentsCount();
+
+                H2Tree firstTree = oldH2Idx.treeForRead(0);
+
+                PageIoResolver pageIoRslvr = pageAddr -> {
+                    PageIO io = PageIoResolver.DEFAULT_PAGE_IO_RESOLVER.resolve(pageAddr);
+
+                    if (io instanceof BPlusMetaIO)
+                        return io;
+
+                    //noinspection unchecked,rawtypes,rawtypes
+                    return wrap((BPlusIO)io);
+                };
+
+                //TODO Create new proper GridCacheContext for it?
+                H2TreeIndex newIdx = H2TreeIndex.createIndex(
+                    cctx,
+                    null,
+                    table,
+                    oldH2Idx.getName(),
+                    firstTree.getPk(),
+                    firstTree.getAffinityKey(),
+                    Arrays.asList(firstTree.cols()),
+                    Arrays.asList(firstTree.cols()),
+                    oldH2Idx.inlineSize(),
+                    segments,
+                    newCachePageMemory,
+                    newCtx.offheap(),
+                    pageIoRslvr,
+                    log
+                );
+
+                tracker.complete(INIT_TREE);
+
+                for (int i = 0; i < segments; i++) {
+                    H2Tree tree = oldH2Idx.treeForRead(i);
+
+                    treeIterator.iterate(tree, oldCachePageMem, (theTree, io, pageAddr, idx) -> {
+                        tracker.complete(ITERATE);
+
+                        if (System.currentTimeMillis() - lastCpLockTs.get() >= cpLockThreshold) {
+                            cpLock.checkpointReadUnlock();
+
+                            cpLock.checkpointReadLock();
+                            tracker.complete(CP_LOCK);
+
+                            lastCpLockTs.set(System.currentTimeMillis());
+                        }
+
+                        assert 1 == io.getVersion();

Review comment:
       Add message here.




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r524322515



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/defragmentation/IndexingDefragmentation.java
##########
@@ -0,0 +1,454 @@
+/*
+ * 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.processors.query.h2.defragmentation;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.GridQueryIndexingDefragmentation;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.LinkMap;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.TimeTracker;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIoResolver;
+import org.apache.ignite.internal.processors.cache.persistence.tree.util.InsertLast;
+import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccDataRow;
+import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+import org.apache.ignite.internal.processors.query.h2.database.H2Tree;
+import org.apache.ignite.internal.processors.query.h2.database.H2TreeIndex;
+import org.apache.ignite.internal.processors.query.h2.database.InlineIndexColumn;
+import org.apache.ignite.internal.processors.query.h2.database.inlinecolumn.AbstractInlineIndexColumn;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2ExtrasInnerIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2ExtrasLeafIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2InnerIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2LeafIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.H2RowLinkIO;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.processors.query.h2.opt.H2CacheRow;
+import org.apache.ignite.internal.processors.query.h2.opt.H2Row;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.h2.index.Index;
+import org.h2.value.Value;
+
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.INIT_TREE;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.ITERATE;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.READ_MAP;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.READ_ROW;
+
+/**
+ *
+ */
+public class IndexingDefragmentation implements GridQueryIndexingDefragmentation {
+    /** Indexing. */
+    private final IgniteH2Indexing indexing;
+
+    /** Constructor. */
+    public IndexingDefragmentation(IgniteH2Indexing indexing) {
+        this.indexing = indexing;
+    }
+
+    /** */
+    public enum IndexStages {
+        START,
+        CP_LOCK,
+        INIT_TREE,
+        ITERATE,
+        READ_ROW,
+        READ_MAP,
+        INSERT_ROW
+    }
+
+    /** {@inheritDoc} */
+    @Override public void defragmentate(
+        CacheGroupContext grpCtx,
+        CacheGroupContext newCtx,
+        PageMemoryEx partPageMem,
+        IntMap<LinkMap> mappingByPartition,
+        CheckpointTimeoutLock cpLock,
+        IgniteLogger log
+    ) throws IgniteCheckedException {
+        int pageSize = grpCtx.cacheObjectContext().kernalContext().grid().configuration().getDataStorageConfiguration().getPageSize();
+
+        TreeIterator treeIterator = new TreeIterator(pageSize);
+
+        PageMemoryEx oldCachePageMem = (PageMemoryEx)grpCtx.dataRegion().pageMemory();
+
+        PageMemory newCachePageMemory = partPageMem;
+
+        Collection<GridH2Table> tables = indexing.schemaManager().dataTables();
+
+        long cpLockThreshold = 250L;
+
+        TimeTracker<IndexStages> tracker = new TimeTracker<>(IndexStages.class);
+
+        cpLock.checkpointReadLock();
+        tracker.complete(CP_LOCK);
+
+        try {
+            AtomicLong lastCpLockTs = new AtomicLong(System.currentTimeMillis());
+
+            for (GridH2Table table : tables) {
+                GridCacheContext<?, ?> cctx = table.cacheContext();
+
+                if (cctx.groupId() != grpCtx.groupId())
+                    continue; // Not our index.
+
+                GridH2RowDescriptor rowDesc = table.rowDescriptor();
+
+                List<Index> indexes = table.getIndexes();
+                H2TreeIndex oldH2Idx = (H2TreeIndex)indexes.get(2);
+
+                int segments = oldH2Idx.segmentsCount();
+
+                H2Tree firstTree = oldH2Idx.treeForRead(0);
+
+                PageIoResolver pageIoRslvr = pageAddr -> {
+                    PageIO io = PageIoResolver.DEFAULT_PAGE_IO_RESOLVER.resolve(pageAddr);
+
+                    if (io instanceof BPlusMetaIO)
+                        return io;
+
+                    //noinspection unchecked,rawtypes,rawtypes
+                    return wrap((BPlusIO)io);
+                };
+
+                //TODO Create new proper GridCacheContext for it?
+                H2TreeIndex newIdx = H2TreeIndex.createIndex(
+                    cctx,
+                    null,
+                    table,
+                    oldH2Idx.getName(),
+                    firstTree.getPk(),
+                    firstTree.getAffinityKey(),
+                    Arrays.asList(firstTree.cols()),
+                    Arrays.asList(firstTree.cols()),
+                    oldH2Idx.inlineSize(),
+                    segments,
+                    newCachePageMemory,
+                    newCtx.offheap(),
+                    pageIoRslvr,
+                    log
+                );
+
+                tracker.complete(INIT_TREE);
+
+                for (int i = 0; i < segments; i++) {
+                    H2Tree tree = oldH2Idx.treeForRead(i);
+
+                    treeIterator.iterate(tree, oldCachePageMem, (theTree, io, pageAddr, idx) -> {
+                        tracker.complete(ITERATE);
+
+                        if (System.currentTimeMillis() - lastCpLockTs.get() >= cpLockThreshold) {
+                            cpLock.checkpointReadUnlock();
+
+                            cpLock.checkpointReadLock();
+                            tracker.complete(CP_LOCK);
+
+                            lastCpLockTs.set(System.currentTimeMillis());
+                        }
+
+                        assert 1 == io.getVersion();
+
+                        BPlusIO<H2Row> h2IO = wrap(io);
+
+                        H2Row row = theTree.getRow(h2IO, pageAddr, idx);
+
+                        tracker.complete(READ_ROW);
+
+                        if (row instanceof H2CacheRowWithIndex) {
+                            H2CacheRowWithIndex h2CacheRow = (H2CacheRowWithIndex)row;
+
+                            CacheDataRow cacheDataRow = h2CacheRow.getRow();
+
+                            int partition = cacheDataRow.partition();
+
+                            long link = h2CacheRow.link();
+
+                            LinkMap map = mappingByPartition.get(partition);
+
+                            long newLink = map.get(link);
+
+                            tracker.complete(READ_MAP);
+
+                            H2CacheRowWithIndex newRow = H2CacheRowWithIndex.create(
+                                rowDesc,
+                                newLink,
+                                h2CacheRow,
+                                ((H2RowLinkIO)io).storeMvccInfo()
+                            );
+
+                            newIdx.putx(newRow);
+
+                            tracker.complete(INSERT_ROW);
+                        }
+
+                        return true;
+                    });
+                }
+            }
+        }
+        finally {
+            cpLock.checkpointReadUnlock();
+        }
+
+        System.out.println(tracker.toString());

Review comment:
       Yes, I'll get rid of it. It needs to be replaced with a log 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.

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



[GitHub] [ignite] agoncharuk commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
agoncharuk commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r524315198



##########
File path: modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsIndexingDefragmentationTest.java
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.processors.cache.persistence;
+
+import java.io.File;
+import java.util.Collections;
+import java.util.Objects;
+import java.util.function.Function;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.visor.verify.ValidateIndexesClosure;
+import org.apache.ignite.internal.visor.verify.VisorValidateIndexesJobResult;
+import org.apache.ignite.testframework.junits.WithSystemProperty;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
+/**
+ * Defragmentation tests with enabled ignite-indexing.
+ */
+public class IgnitePdsIndexingDefragmentationTest extends IgnitePdsDefragmentationTest {
+    /** Use MVCC in tests. */
+    private static final String USE_MVCC = "USE_MVCC";
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setConsistentId(igniteInstanceName);
+
+        DataStorageConfiguration dsCfg = new DataStorageConfiguration();
+        dsCfg.setWalSegmentSize(4 * 1024 * 1024);
+
+        dsCfg.setDefaultDataRegionConfiguration(
+            new DataRegionConfiguration()
+                .setInitialSize(100L * 1024 * 1024)
+                .setMaxSize(1024L * 1024 * 1024)
+                .setPersistenceEnabled(true)
+        );
+
+        cfg.setDataStorageConfiguration(dsCfg);
+
+        CacheConfiguration<?, ?> cache1Cfg = new CacheConfiguration<>(DEFAULT_CACHE_NAME)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setGroupName(GRP_NAME)
+            .setIndexedTypes(
+                ObjKey.class, byte[].class,
+                Integer.class, byte[].class
+            )
+            .setAffinity(new RendezvousAffinityFunction(false, PARTS));
+
+        CacheConfiguration<?, ?> cache2Cfg = new CacheConfiguration<>(CACHE_2_NAME)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setGroupName(GRP_NAME)
+            .setIndexedTypes(
+                ObjKey.class, byte[].class,
+                Integer.class, byte[].class

Review comment:
       Need to add more tests for actual complex indexes: multi-column, various inline sizes.




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

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r525054661



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/DefragmentationFileUtils.java
##########
@@ -0,0 +1,400 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static java.nio.file.StandardCopyOption.ATOMIC_MOVE;
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static java.nio.file.StandardOpenOption.CREATE_NEW;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.FILE_SUFFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_TEMPLATE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.TMP_SUFFIX;
+
+/**
+ * Everything related to file management during defragmentation process.
+ */
+public class DefragmentationFileUtils {
+    /** Prefix for link mapping files. */
+    private static final String DFRG_LINK_MAPPING_FILE_PREFIX = PART_FILE_PREFIX + "map-";
+
+    /** Link mapping file template. */
+    private static final String DFRG_LINK_MAPPING_FILE_TEMPLATE = DFRG_LINK_MAPPING_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmentation complation marker file name. */
+    private static final String DFRG_COMPLETION_MARKER_FILE_NAME = "dfrg-completion-marker";
+
+    /** Name of defragmentated index partition file. */
+    private static final String DFRG_INDEX_FILE_NAME = INDEX_FILE_PREFIX + "-dfrg" + FILE_SUFFIX;
+
+    /** Name of defragmentated index partition temporary file. */
+    private static final String DFRG_INDEX_TMP_FILE_NAME = DFRG_INDEX_FILE_NAME + TMP_SUFFIX;
+
+    /** Prefix for defragmented partition files. */
+    private static final String DFRG_PARTITION_FILE_PREFIX = PART_FILE_PREFIX + "dfrg-";
+
+    /** Defragmented partition file template. */
+    private static final String DFRG_PARTITION_FILE_TEMPLATE = DFRG_PARTITION_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmented partition temp file template. */
+    private static final String DFRG_PARTITION_TMP_FILE_TEMPLATE = DFRG_PARTITION_FILE_TEMPLATE + TMP_SUFFIX;
+
+    /**
+     * Performs cleanup of work dir before initializing file page stores.
+     * Will finish batch renaming if defragmentation was completed or delete garbage if it wasn't.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     */
+    public static void beforeInitPageStores(File workDir, IgniteLogger log) throws IgniteCheckedException {

Review comment:
       This method used only once, I suggest to inline it.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/DefragmentationFileUtils.java
##########
@@ -0,0 +1,400 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static java.nio.file.StandardCopyOption.ATOMIC_MOVE;
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static java.nio.file.StandardOpenOption.CREATE_NEW;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.FILE_SUFFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_TEMPLATE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.TMP_SUFFIX;
+
+/**
+ * Everything related to file management during defragmentation process.
+ */
+public class DefragmentationFileUtils {
+    /** Prefix for link mapping files. */
+    private static final String DFRG_LINK_MAPPING_FILE_PREFIX = PART_FILE_PREFIX + "map-";
+
+    /** Link mapping file template. */
+    private static final String DFRG_LINK_MAPPING_FILE_TEMPLATE = DFRG_LINK_MAPPING_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmentation complation marker file name. */
+    private static final String DFRG_COMPLETION_MARKER_FILE_NAME = "dfrg-completion-marker";
+
+    /** Name of defragmentated index partition file. */
+    private static final String DFRG_INDEX_FILE_NAME = INDEX_FILE_PREFIX + "-dfrg" + FILE_SUFFIX;
+
+    /** Name of defragmentated index partition temporary file. */
+    private static final String DFRG_INDEX_TMP_FILE_NAME = DFRG_INDEX_FILE_NAME + TMP_SUFFIX;
+
+    /** Prefix for defragmented partition files. */
+    private static final String DFRG_PARTITION_FILE_PREFIX = PART_FILE_PREFIX + "dfrg-";
+
+    /** Defragmented partition file template. */
+    private static final String DFRG_PARTITION_FILE_TEMPLATE = DFRG_PARTITION_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmented partition temp file template. */
+    private static final String DFRG_PARTITION_TMP_FILE_TEMPLATE = DFRG_PARTITION_FILE_TEMPLATE + TMP_SUFFIX;
+
+    /**
+     * Performs cleanup of work dir before initializing file page stores.
+     * Will finish batch renaming if defragmentation was completed or delete garbage if it wasn't.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     */
+    public static void beforeInitPageStores(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+        U.delete(defragmentationCompletionMarkerFile(workDir));
+
+        for (File file : workDir.listFiles()) {
+            String fileName = file.getName();
+
+            if (
+                fileName.startsWith(DFRG_PARTITION_FILE_PREFIX)
+                || fileName.startsWith(DFRG_INDEX_FILE_NAME)
+                || fileName.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)
+            )
+                U.delete(file);
+        }
+    }
+
+    /**
+     * Checks whether cache group defragmentation completed or not. Completes it if all that's left is renaming.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param log Logger to write messages.
+     * @return {@code true} if given cache group is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentationCompletionMarkerFile(File)
+     */
+    public static boolean skipAlreadyDefragmentedCacheGroup(File workDir, int grpId, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (completionMarkerFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented page group",
+                    "grpId", grpId, false,
+                    "markerFileName", completionMarkerFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+            return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * Checks whether partition has already been defragmented or not. Cleans corrupted data if previous failed
+     * defragmentation attempt was found.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param partId Partionion index to check.
+     * @param log Logger to write messages.
+     * @return {@code true} if given partition is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartMappingFile(File, int)
+     */
+    public static boolean skipAlreadyDefragmentedPartition(File workDir, int grpId, int partId, IgniteLogger log) throws IgniteCheckedException {
+        File defragmentedPartFile = defragmentedPartFile(workDir, partId);
+        File defragmentedPartMappingFile = defragmentedPartMappingFile(workDir, partId);
+
+        if (defragmentedPartFile.exists() && defragmentedPartMappingFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented partition",
+                    "grpId", grpId, false,
+                    "partId", partId, false,
+                    "partFileName", defragmentedPartFile.getName(), false,
+                    "mappingFileName", defragmentedPartMappingFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            return true;
+        }
+
+        File defragmentedPartTmpFile = defragmentedPartTmpFile(workDir, partId);
+
+        try {
+            Files.deleteIfExists(defragmentedPartTmpFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartMappingFile.toPath());
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+
+        return false;
+    }
+
+    /**
+     * Failure-tolerant batch rename of defragmented partition files.
+     *
+     * Deletes all link mapping files old partition and index files, renaming defragmentated files in the process. Can
+     * be run on the same folder multiple times if failed for some reason.
+     *
+     * Does something only if completion marker is present in the folder. This marker won't be deleted in the end.
+     * Deletion of the marker must be done outside of defragmentation mode to prevent cache groups to be defragmentated
+     * several times in case of failures.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#writeDefragmentationCompletionMarker(FileIOFactory, File, IgniteLogger)
+     */
+    public static void batchRenameDefragmentedCacheGroupPartitions(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (!completionMarkerFile.exists())
+            return;
+
+        try {
+            for (File mappingFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)))
+                Files.delete(mappingFile.toPath());
+
+            for (File partFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_PARTITION_FILE_PREFIX))) {
+                int partId = extractPartId(partFile.getName());
+
+                File oldPartFile = new File(workDir, String.format(PART_FILE_TEMPLATE, partId));
+
+                Files.move(partFile.toPath(), oldPartFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+
+            File idxFile = new File(workDir, DFRG_INDEX_FILE_NAME);
+
+            if (idxFile.exists()) {
+                File oldIdxFile = new File(workDir, INDEX_FILE_NAME);
+
+                Files.move(idxFile.toPath(), oldIdxFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Extracts partition number from file names like {@code part-dfrg-%d.bin}.
+     *
+     * @param dfrgPartFileName Defragmented partition file name.
+     * @return Partition index.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    private static int extractPartId(String dfrgPartFileName) {

Review comment:
       This method used only once, I suggest to inline it.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/DefragmentationFileUtils.java
##########
@@ -0,0 +1,400 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static java.nio.file.StandardCopyOption.ATOMIC_MOVE;
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static java.nio.file.StandardOpenOption.CREATE_NEW;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.FILE_SUFFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_TEMPLATE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.TMP_SUFFIX;
+
+/**
+ * Everything related to file management during defragmentation process.
+ */
+public class DefragmentationFileUtils {
+    /** Prefix for link mapping files. */
+    private static final String DFRG_LINK_MAPPING_FILE_PREFIX = PART_FILE_PREFIX + "map-";
+
+    /** Link mapping file template. */
+    private static final String DFRG_LINK_MAPPING_FILE_TEMPLATE = DFRG_LINK_MAPPING_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmentation complation marker file name. */
+    private static final String DFRG_COMPLETION_MARKER_FILE_NAME = "dfrg-completion-marker";
+
+    /** Name of defragmentated index partition file. */
+    private static final String DFRG_INDEX_FILE_NAME = INDEX_FILE_PREFIX + "-dfrg" + FILE_SUFFIX;
+
+    /** Name of defragmentated index partition temporary file. */
+    private static final String DFRG_INDEX_TMP_FILE_NAME = DFRG_INDEX_FILE_NAME + TMP_SUFFIX;
+
+    /** Prefix for defragmented partition files. */
+    private static final String DFRG_PARTITION_FILE_PREFIX = PART_FILE_PREFIX + "dfrg-";
+
+    /** Defragmented partition file template. */
+    private static final String DFRG_PARTITION_FILE_TEMPLATE = DFRG_PARTITION_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmented partition temp file template. */
+    private static final String DFRG_PARTITION_TMP_FILE_TEMPLATE = DFRG_PARTITION_FILE_TEMPLATE + TMP_SUFFIX;
+
+    /**
+     * Performs cleanup of work dir before initializing file page stores.
+     * Will finish batch renaming if defragmentation was completed or delete garbage if it wasn't.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     */
+    public static void beforeInitPageStores(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+        U.delete(defragmentationCompletionMarkerFile(workDir));
+
+        for (File file : workDir.listFiles()) {
+            String fileName = file.getName();
+
+            if (
+                fileName.startsWith(DFRG_PARTITION_FILE_PREFIX)
+                || fileName.startsWith(DFRG_INDEX_FILE_NAME)
+                || fileName.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)
+            )
+                U.delete(file);
+        }
+    }
+
+    /**
+     * Checks whether cache group defragmentation completed or not. Completes it if all that's left is renaming.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param log Logger to write messages.
+     * @return {@code true} if given cache group is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentationCompletionMarkerFile(File)
+     */
+    public static boolean skipAlreadyDefragmentedCacheGroup(File workDir, int grpId, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (completionMarkerFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented page group",
+                    "grpId", grpId, false,
+                    "markerFileName", completionMarkerFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+            return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * Checks whether partition has already been defragmented or not. Cleans corrupted data if previous failed
+     * defragmentation attempt was found.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param partId Partionion index to check.
+     * @param log Logger to write messages.
+     * @return {@code true} if given partition is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartMappingFile(File, int)
+     */
+    public static boolean skipAlreadyDefragmentedPartition(File workDir, int grpId, int partId, IgniteLogger log) throws IgniteCheckedException {
+        File defragmentedPartFile = defragmentedPartFile(workDir, partId);
+        File defragmentedPartMappingFile = defragmentedPartMappingFile(workDir, partId);
+
+        if (defragmentedPartFile.exists() && defragmentedPartMappingFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented partition",
+                    "grpId", grpId, false,
+                    "partId", partId, false,
+                    "partFileName", defragmentedPartFile.getName(), false,
+                    "mappingFileName", defragmentedPartMappingFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            return true;
+        }
+
+        File defragmentedPartTmpFile = defragmentedPartTmpFile(workDir, partId);
+
+        try {
+            Files.deleteIfExists(defragmentedPartTmpFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartMappingFile.toPath());
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+
+        return false;
+    }
+
+    /**
+     * Failure-tolerant batch rename of defragmented partition files.
+     *
+     * Deletes all link mapping files old partition and index files, renaming defragmentated files in the process. Can
+     * be run on the same folder multiple times if failed for some reason.
+     *
+     * Does something only if completion marker is present in the folder. This marker won't be deleted in the end.
+     * Deletion of the marker must be done outside of defragmentation mode to prevent cache groups to be defragmentated
+     * several times in case of failures.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#writeDefragmentationCompletionMarker(FileIOFactory, File, IgniteLogger)
+     */
+    public static void batchRenameDefragmentedCacheGroupPartitions(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (!completionMarkerFile.exists())
+            return;
+
+        try {
+            for (File mappingFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)))
+                Files.delete(mappingFile.toPath());
+
+            for (File partFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_PARTITION_FILE_PREFIX))) {
+                int partId = extractPartId(partFile.getName());
+
+                File oldPartFile = new File(workDir, String.format(PART_FILE_TEMPLATE, partId));
+
+                Files.move(partFile.toPath(), oldPartFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+
+            File idxFile = new File(workDir, DFRG_INDEX_FILE_NAME);
+
+            if (idxFile.exists()) {
+                File oldIdxFile = new File(workDir, INDEX_FILE_NAME);
+
+                Files.move(idxFile.toPath(), oldIdxFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Extracts partition number from file names like {@code part-dfrg-%d.bin}.
+     *
+     * @param dfrgPartFileName Defragmented partition file name.
+     * @return Partition index.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    private static int extractPartId(String dfrgPartFileName) {
+        assert dfrgPartFileName.startsWith(DFRG_PARTITION_FILE_PREFIX) : dfrgPartFileName;
+        assert dfrgPartFileName.endsWith(FILE_SUFFIX) : dfrgPartFileName;
+
+        String partIdStr = dfrgPartFileName.substring(
+            DFRG_PARTITION_FILE_PREFIX.length(),
+            dfrgPartFileName.length() - FILE_SUFFIX.length()
+        );
+
+        return Integer.parseInt(partIdStr);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin.tmp} in given folder. It will be used for storing defragmented index
+     * partition during the process.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexFile(File)
+     */
+    public static File defragmentedIndexTmpFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_TMP_FILE_NAME);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin} in given folder. It will be used for storing defragmented index
+     * partition when the process is over.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexTmpFile(File)
+     */
+    public static File defragmentedIndexFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_FILE_NAME);
+    }
+
+    /**
+     * Rename temporary index defragmenation file to a finalized one.
+     *
+     * @param workDir Cache group working directory.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexTmpFile(File)
+     * @see DefragmentationFileUtils#defragmentedIndexFile(File)
+     */
+    public static void renameTempIndexFile(File workDir) throws IgniteCheckedException {
+        File defragmentedIdxTmpFile = defragmentedIndexTmpFile(workDir);
+        File defragmentedIdxFile = defragmentedIndexFile(workDir);
+
+        try {
+            Files.move(defragmentedIdxTmpFile.toPath(), defragmentedIdxFile.toPath(), ATOMIC_MOVE);
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Return file named {@code part-dfrg-%d.bin.tmp} in given folder. It will be used for storing defragmented data
+     * partition during the process.
+     *
+     * @param workDir Cache group working directory.
+     * @param partId Parition index, will be substituted into file name.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    public static File defragmentedPartTmpFile(File workDir, int partId) {
+        return new File(workDir, String.format(DFRG_PARTITION_TMP_FILE_TEMPLATE, partId));
+    }
+
+    /**
+     * Return file named {@code part-dfrg-%d.bin} in given folder. It will be used for storing defragmented data
+     * partition when the process is over.
+     *
+     * @param workDir Cache group working directory.
+     * @param partId Parition index, will be substituted into file name.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     */
+    public static File defragmentedPartFile(File workDir, int partId) {
+        return new File(workDir, String.format(DFRG_PARTITION_FILE_TEMPLATE, partId));
+    }
+
+    /**
+     * Rename temporary partition defragmenation file to a finalized one.

Review comment:
       defragmenation  -> defragmentation

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/DefragmentationFileUtils.java
##########
@@ -0,0 +1,400 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static java.nio.file.StandardCopyOption.ATOMIC_MOVE;
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static java.nio.file.StandardOpenOption.CREATE_NEW;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.FILE_SUFFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_TEMPLATE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.TMP_SUFFIX;
+
+/**
+ * Everything related to file management during defragmentation process.
+ */
+public class DefragmentationFileUtils {
+    /** Prefix for link mapping files. */
+    private static final String DFRG_LINK_MAPPING_FILE_PREFIX = PART_FILE_PREFIX + "map-";
+
+    /** Link mapping file template. */
+    private static final String DFRG_LINK_MAPPING_FILE_TEMPLATE = DFRG_LINK_MAPPING_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmentation complation marker file name. */
+    private static final String DFRG_COMPLETION_MARKER_FILE_NAME = "dfrg-completion-marker";
+
+    /** Name of defragmentated index partition file. */
+    private static final String DFRG_INDEX_FILE_NAME = INDEX_FILE_PREFIX + "-dfrg" + FILE_SUFFIX;
+
+    /** Name of defragmentated index partition temporary file. */
+    private static final String DFRG_INDEX_TMP_FILE_NAME = DFRG_INDEX_FILE_NAME + TMP_SUFFIX;
+
+    /** Prefix for defragmented partition files. */
+    private static final String DFRG_PARTITION_FILE_PREFIX = PART_FILE_PREFIX + "dfrg-";
+
+    /** Defragmented partition file template. */
+    private static final String DFRG_PARTITION_FILE_TEMPLATE = DFRG_PARTITION_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmented partition temp file template. */
+    private static final String DFRG_PARTITION_TMP_FILE_TEMPLATE = DFRG_PARTITION_FILE_TEMPLATE + TMP_SUFFIX;
+
+    /**
+     * Performs cleanup of work dir before initializing file page stores.
+     * Will finish batch renaming if defragmentation was completed or delete garbage if it wasn't.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     */
+    public static void beforeInitPageStores(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+        U.delete(defragmentationCompletionMarkerFile(workDir));
+
+        for (File file : workDir.listFiles()) {
+            String fileName = file.getName();
+
+            if (
+                fileName.startsWith(DFRG_PARTITION_FILE_PREFIX)
+                || fileName.startsWith(DFRG_INDEX_FILE_NAME)
+                || fileName.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)
+            )
+                U.delete(file);
+        }
+    }
+
+    /**
+     * Checks whether cache group defragmentation completed or not. Completes it if all that's left is renaming.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param log Logger to write messages.
+     * @return {@code true} if given cache group is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentationCompletionMarkerFile(File)
+     */
+    public static boolean skipAlreadyDefragmentedCacheGroup(File workDir, int grpId, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (completionMarkerFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented page group",
+                    "grpId", grpId, false,
+                    "markerFileName", completionMarkerFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+            return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * Checks whether partition has already been defragmented or not. Cleans corrupted data if previous failed
+     * defragmentation attempt was found.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param partId Partionion index to check.
+     * @param log Logger to write messages.
+     * @return {@code true} if given partition is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartMappingFile(File, int)
+     */
+    public static boolean skipAlreadyDefragmentedPartition(File workDir, int grpId, int partId, IgniteLogger log) throws IgniteCheckedException {
+        File defragmentedPartFile = defragmentedPartFile(workDir, partId);
+        File defragmentedPartMappingFile = defragmentedPartMappingFile(workDir, partId);
+
+        if (defragmentedPartFile.exists() && defragmentedPartMappingFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented partition",
+                    "grpId", grpId, false,
+                    "partId", partId, false,
+                    "partFileName", defragmentedPartFile.getName(), false,
+                    "mappingFileName", defragmentedPartMappingFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            return true;
+        }
+
+        File defragmentedPartTmpFile = defragmentedPartTmpFile(workDir, partId);
+
+        try {
+            Files.deleteIfExists(defragmentedPartTmpFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartMappingFile.toPath());
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+
+        return false;
+    }
+
+    /**
+     * Failure-tolerant batch rename of defragmented partition files.
+     *
+     * Deletes all link mapping files old partition and index files, renaming defragmentated files in the process. Can
+     * be run on the same folder multiple times if failed for some reason.
+     *
+     * Does something only if completion marker is present in the folder. This marker won't be deleted in the end.
+     * Deletion of the marker must be done outside of defragmentation mode to prevent cache groups to be defragmentated
+     * several times in case of failures.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#writeDefragmentationCompletionMarker(FileIOFactory, File, IgniteLogger)
+     */
+    public static void batchRenameDefragmentedCacheGroupPartitions(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (!completionMarkerFile.exists())
+            return;
+
+        try {
+            for (File mappingFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)))
+                Files.delete(mappingFile.toPath());
+
+            for (File partFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_PARTITION_FILE_PREFIX))) {
+                int partId = extractPartId(partFile.getName());
+
+                File oldPartFile = new File(workDir, String.format(PART_FILE_TEMPLATE, partId));
+
+                Files.move(partFile.toPath(), oldPartFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+
+            File idxFile = new File(workDir, DFRG_INDEX_FILE_NAME);
+
+            if (idxFile.exists()) {
+                File oldIdxFile = new File(workDir, INDEX_FILE_NAME);
+
+                Files.move(idxFile.toPath(), oldIdxFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Extracts partition number from file names like {@code part-dfrg-%d.bin}.
+     *
+     * @param dfrgPartFileName Defragmented partition file name.
+     * @return Partition index.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    private static int extractPartId(String dfrgPartFileName) {
+        assert dfrgPartFileName.startsWith(DFRG_PARTITION_FILE_PREFIX) : dfrgPartFileName;
+        assert dfrgPartFileName.endsWith(FILE_SUFFIX) : dfrgPartFileName;
+
+        String partIdStr = dfrgPartFileName.substring(
+            DFRG_PARTITION_FILE_PREFIX.length(),
+            dfrgPartFileName.length() - FILE_SUFFIX.length()
+        );
+
+        return Integer.parseInt(partIdStr);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin.tmp} in given folder. It will be used for storing defragmented index
+     * partition during the process.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexFile(File)
+     */
+    public static File defragmentedIndexTmpFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_TMP_FILE_NAME);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin} in given folder. It will be used for storing defragmented index
+     * partition when the process is over.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexTmpFile(File)
+     */
+    public static File defragmentedIndexFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_FILE_NAME);
+    }
+
+    /**
+     * Rename temporary index defragmenation file to a finalized one.
+     *
+     * @param workDir Cache group working directory.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexTmpFile(File)
+     * @see DefragmentationFileUtils#defragmentedIndexFile(File)
+     */
+    public static void renameTempIndexFile(File workDir) throws IgniteCheckedException {
+        File defragmentedIdxTmpFile = defragmentedIndexTmpFile(workDir);
+        File defragmentedIdxFile = defragmentedIndexFile(workDir);
+
+        try {
+            Files.move(defragmentedIdxTmpFile.toPath(), defragmentedIdxFile.toPath(), ATOMIC_MOVE);
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Return file named {@code part-dfrg-%d.bin.tmp} in given folder. It will be used for storing defragmented data
+     * partition during the process.
+     *
+     * @param workDir Cache group working directory.
+     * @param partId Parition index, will be substituted into file name.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    public static File defragmentedPartTmpFile(File workDir, int partId) {
+        return new File(workDir, String.format(DFRG_PARTITION_TMP_FILE_TEMPLATE, partId));
+    }
+
+    /**
+     * Return file named {@code part-dfrg-%d.bin} in given folder. It will be used for storing defragmented data
+     * partition when the process is over.
+     *
+     * @param workDir Cache group working directory.
+     * @param partId Parition index, will be substituted into file name.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     */
+    public static File defragmentedPartFile(File workDir, int partId) {
+        return new File(workDir, String.format(DFRG_PARTITION_FILE_TEMPLATE, partId));
+    }
+
+    /**
+     * Rename temporary partition defragmenation file to a finalized one.
+     *
+     * @param workDir Cache group working directory.
+     * @param partId Parition index.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    public static void renameTempPartitionFile(File workDir, int partId) throws IgniteCheckedException {
+        File defragmentedPartTmpFile = defragmentedPartTmpFile(workDir, partId);
+        File defragmentedPartFile = defragmentedPartFile(workDir, partId);
+
+        assert !defragmentedPartFile.exists() : defragmentedPartFile;
+
+        try {
+            Files.move(defragmentedPartTmpFile.toPath(), defragmentedPartFile.toPath(), ATOMIC_MOVE);
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Return file named {@code part-map-%d.bin} in given folder. It will be used for storing defragmention links
+     * mapping for given partition during and after defragmentation process. No temporary counterpart is required here.
+     *
+     * @param workDir Cache group working directory.
+     * @param partId Parition index, will be substituted into file name.
+     * @return File.
+     *
+     * @see LinkMap
+     */
+    public static File defragmentedPartMappingFile(File workDir, int partId) {
+        return new File(workDir, String.format(DFRG_LINK_MAPPING_FILE_TEMPLATE, partId));
+    }
+
+    /**
+     * Return defragmentation completion marker file. This file can only be created when all partitions and index are
+     * defragmented and renamed from their original {@code *.tmp} versions. Presence of this file signals that no data
+     * will be lost if original partitions are deleted and batch rename process can be safely initiated.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#writeDefragmentationCompletionMarker(FileIOFactory, File, IgniteLogger)
+     * @see DefragmentationFileUtils#batchRenameDefragmentedCacheGroupPartitions(File, IgniteLogger)
+     */
+    public static File defragmentationCompletionMarkerFile(File workDir) {
+        return new File(workDir, DFRG_COMPLETION_MARKER_FILE_NAME);
+    }
+
+    /**
+     * Creates empty completion marker file in given directory.
+     *
+     * @param ioFactory File IO factory.
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentationCompletionMarkerFile(File)
+     */
+    public static void writeDefragmentationCompletionMarker(
+        FileIOFactory ioFactory,
+        File workDir,
+        IgniteLogger log
+    ) throws IgniteCheckedException {
+        File completionMarker = defragmentationCompletionMarkerFile(workDir);
+
+        try (FileIO io = ioFactory.create(completionMarker, CREATE_NEW, WRITE)) {
+            io.force(true);
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /** */
+    private static void handleIoException(IOException e) throws IgniteCheckedException {

Review comment:
       Why do we need this method at all?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/DefragmentationFileUtils.java
##########
@@ -0,0 +1,400 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static java.nio.file.StandardCopyOption.ATOMIC_MOVE;
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static java.nio.file.StandardOpenOption.CREATE_NEW;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.FILE_SUFFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_TEMPLATE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.TMP_SUFFIX;
+
+/**
+ * Everything related to file management during defragmentation process.
+ */
+public class DefragmentationFileUtils {
+    /** Prefix for link mapping files. */
+    private static final String DFRG_LINK_MAPPING_FILE_PREFIX = PART_FILE_PREFIX + "map-";
+
+    /** Link mapping file template. */
+    private static final String DFRG_LINK_MAPPING_FILE_TEMPLATE = DFRG_LINK_MAPPING_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmentation complation marker file name. */
+    private static final String DFRG_COMPLETION_MARKER_FILE_NAME = "dfrg-completion-marker";
+
+    /** Name of defragmentated index partition file. */
+    private static final String DFRG_INDEX_FILE_NAME = INDEX_FILE_PREFIX + "-dfrg" + FILE_SUFFIX;
+
+    /** Name of defragmentated index partition temporary file. */
+    private static final String DFRG_INDEX_TMP_FILE_NAME = DFRG_INDEX_FILE_NAME + TMP_SUFFIX;
+
+    /** Prefix for defragmented partition files. */
+    private static final String DFRG_PARTITION_FILE_PREFIX = PART_FILE_PREFIX + "dfrg-";
+
+    /** Defragmented partition file template. */
+    private static final String DFRG_PARTITION_FILE_TEMPLATE = DFRG_PARTITION_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmented partition temp file template. */
+    private static final String DFRG_PARTITION_TMP_FILE_TEMPLATE = DFRG_PARTITION_FILE_TEMPLATE + TMP_SUFFIX;
+
+    /**
+     * Performs cleanup of work dir before initializing file page stores.
+     * Will finish batch renaming if defragmentation was completed or delete garbage if it wasn't.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     */
+    public static void beforeInitPageStores(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+        U.delete(defragmentationCompletionMarkerFile(workDir));
+
+        for (File file : workDir.listFiles()) {
+            String fileName = file.getName();
+
+            if (
+                fileName.startsWith(DFRG_PARTITION_FILE_PREFIX)
+                || fileName.startsWith(DFRG_INDEX_FILE_NAME)
+                || fileName.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)
+            )
+                U.delete(file);
+        }
+    }
+
+    /**
+     * Checks whether cache group defragmentation completed or not. Completes it if all that's left is renaming.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param log Logger to write messages.
+     * @return {@code true} if given cache group is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentationCompletionMarkerFile(File)
+     */
+    public static boolean skipAlreadyDefragmentedCacheGroup(File workDir, int grpId, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (completionMarkerFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented page group",
+                    "grpId", grpId, false,
+                    "markerFileName", completionMarkerFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+            return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * Checks whether partition has already been defragmented or not. Cleans corrupted data if previous failed
+     * defragmentation attempt was found.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param partId Partionion index to check.
+     * @param log Logger to write messages.
+     * @return {@code true} if given partition is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartMappingFile(File, int)
+     */
+    public static boolean skipAlreadyDefragmentedPartition(File workDir, int grpId, int partId, IgniteLogger log) throws IgniteCheckedException {
+        File defragmentedPartFile = defragmentedPartFile(workDir, partId);
+        File defragmentedPartMappingFile = defragmentedPartMappingFile(workDir, partId);
+
+        if (defragmentedPartFile.exists() && defragmentedPartMappingFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented partition",
+                    "grpId", grpId, false,
+                    "partId", partId, false,
+                    "partFileName", defragmentedPartFile.getName(), false,
+                    "mappingFileName", defragmentedPartMappingFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            return true;
+        }
+
+        File defragmentedPartTmpFile = defragmentedPartTmpFile(workDir, partId);
+
+        try {
+            Files.deleteIfExists(defragmentedPartTmpFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartMappingFile.toPath());
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+
+        return false;
+    }
+
+    /**
+     * Failure-tolerant batch rename of defragmented partition files.
+     *
+     * Deletes all link mapping files old partition and index files, renaming defragmentated files in the process. Can
+     * be run on the same folder multiple times if failed for some reason.
+     *
+     * Does something only if completion marker is present in the folder. This marker won't be deleted in the end.
+     * Deletion of the marker must be done outside of defragmentation mode to prevent cache groups to be defragmentated
+     * several times in case of failures.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#writeDefragmentationCompletionMarker(FileIOFactory, File, IgniteLogger)
+     */
+    public static void batchRenameDefragmentedCacheGroupPartitions(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (!completionMarkerFile.exists())
+            return;
+
+        try {
+            for (File mappingFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)))
+                Files.delete(mappingFile.toPath());
+
+            for (File partFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_PARTITION_FILE_PREFIX))) {
+                int partId = extractPartId(partFile.getName());
+
+                File oldPartFile = new File(workDir, String.format(PART_FILE_TEMPLATE, partId));
+
+                Files.move(partFile.toPath(), oldPartFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+
+            File idxFile = new File(workDir, DFRG_INDEX_FILE_NAME);
+
+            if (idxFile.exists()) {
+                File oldIdxFile = new File(workDir, INDEX_FILE_NAME);
+
+                Files.move(idxFile.toPath(), oldIdxFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Extracts partition number from file names like {@code part-dfrg-%d.bin}.
+     *
+     * @param dfrgPartFileName Defragmented partition file name.
+     * @return Partition index.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    private static int extractPartId(String dfrgPartFileName) {
+        assert dfrgPartFileName.startsWith(DFRG_PARTITION_FILE_PREFIX) : dfrgPartFileName;
+        assert dfrgPartFileName.endsWith(FILE_SUFFIX) : dfrgPartFileName;
+
+        String partIdStr = dfrgPartFileName.substring(
+            DFRG_PARTITION_FILE_PREFIX.length(),
+            dfrgPartFileName.length() - FILE_SUFFIX.length()
+        );
+
+        return Integer.parseInt(partIdStr);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin.tmp} in given folder. It will be used for storing defragmented index
+     * partition during the process.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexFile(File)
+     */
+    public static File defragmentedIndexTmpFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_TMP_FILE_NAME);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin} in given folder. It will be used for storing defragmented index
+     * partition when the process is over.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexTmpFile(File)
+     */
+    public static File defragmentedIndexFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_FILE_NAME);
+    }
+
+    /**
+     * Rename temporary index defragmenation file to a finalized one.
+     *
+     * @param workDir Cache group working directory.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexTmpFile(File)
+     * @see DefragmentationFileUtils#defragmentedIndexFile(File)
+     */
+    public static void renameTempIndexFile(File workDir) throws IgniteCheckedException {
+        File defragmentedIdxTmpFile = defragmentedIndexTmpFile(workDir);
+        File defragmentedIdxFile = defragmentedIndexFile(workDir);
+
+        try {
+            Files.move(defragmentedIdxTmpFile.toPath(), defragmentedIdxFile.toPath(), ATOMIC_MOVE);
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Return file named {@code part-dfrg-%d.bin.tmp} in given folder. It will be used for storing defragmented data
+     * partition during the process.
+     *
+     * @param workDir Cache group working directory.
+     * @param partId Parition index, will be substituted into file name.

Review comment:
       Parition -> Partition

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/DefragmentationFileUtils.java
##########
@@ -0,0 +1,400 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static java.nio.file.StandardCopyOption.ATOMIC_MOVE;
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static java.nio.file.StandardOpenOption.CREATE_NEW;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.FILE_SUFFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_TEMPLATE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.TMP_SUFFIX;
+
+/**
+ * Everything related to file management during defragmentation process.
+ */
+public class DefragmentationFileUtils {
+    /** Prefix for link mapping files. */
+    private static final String DFRG_LINK_MAPPING_FILE_PREFIX = PART_FILE_PREFIX + "map-";
+
+    /** Link mapping file template. */
+    private static final String DFRG_LINK_MAPPING_FILE_TEMPLATE = DFRG_LINK_MAPPING_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmentation complation marker file name. */
+    private static final String DFRG_COMPLETION_MARKER_FILE_NAME = "dfrg-completion-marker";
+
+    /** Name of defragmentated index partition file. */
+    private static final String DFRG_INDEX_FILE_NAME = INDEX_FILE_PREFIX + "-dfrg" + FILE_SUFFIX;
+
+    /** Name of defragmentated index partition temporary file. */
+    private static final String DFRG_INDEX_TMP_FILE_NAME = DFRG_INDEX_FILE_NAME + TMP_SUFFIX;
+
+    /** Prefix for defragmented partition files. */
+    private static final String DFRG_PARTITION_FILE_PREFIX = PART_FILE_PREFIX + "dfrg-";
+
+    /** Defragmented partition file template. */
+    private static final String DFRG_PARTITION_FILE_TEMPLATE = DFRG_PARTITION_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmented partition temp file template. */
+    private static final String DFRG_PARTITION_TMP_FILE_TEMPLATE = DFRG_PARTITION_FILE_TEMPLATE + TMP_SUFFIX;
+
+    /**
+     * Performs cleanup of work dir before initializing file page stores.
+     * Will finish batch renaming if defragmentation was completed or delete garbage if it wasn't.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     */
+    public static void beforeInitPageStores(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+        U.delete(defragmentationCompletionMarkerFile(workDir));
+
+        for (File file : workDir.listFiles()) {
+            String fileName = file.getName();
+
+            if (
+                fileName.startsWith(DFRG_PARTITION_FILE_PREFIX)
+                || fileName.startsWith(DFRG_INDEX_FILE_NAME)
+                || fileName.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)
+            )
+                U.delete(file);
+        }
+    }
+
+    /**
+     * Checks whether cache group defragmentation completed or not. Completes it if all that's left is renaming.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param log Logger to write messages.
+     * @return {@code true} if given cache group is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentationCompletionMarkerFile(File)
+     */
+    public static boolean skipAlreadyDefragmentedCacheGroup(File workDir, int grpId, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (completionMarkerFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented page group",
+                    "grpId", grpId, false,
+                    "markerFileName", completionMarkerFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+            return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * Checks whether partition has already been defragmented or not. Cleans corrupted data if previous failed
+     * defragmentation attempt was found.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param partId Partionion index to check.
+     * @param log Logger to write messages.
+     * @return {@code true} if given partition is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartMappingFile(File, int)
+     */
+    public static boolean skipAlreadyDefragmentedPartition(File workDir, int grpId, int partId, IgniteLogger log) throws IgniteCheckedException {
+        File defragmentedPartFile = defragmentedPartFile(workDir, partId);
+        File defragmentedPartMappingFile = defragmentedPartMappingFile(workDir, partId);
+
+        if (defragmentedPartFile.exists() && defragmentedPartMappingFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented partition",
+                    "grpId", grpId, false,
+                    "partId", partId, false,
+                    "partFileName", defragmentedPartFile.getName(), false,
+                    "mappingFileName", defragmentedPartMappingFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            return true;
+        }
+
+        File defragmentedPartTmpFile = defragmentedPartTmpFile(workDir, partId);
+
+        try {
+            Files.deleteIfExists(defragmentedPartTmpFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartMappingFile.toPath());
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+
+        return false;
+    }
+
+    /**
+     * Failure-tolerant batch rename of defragmented partition files.
+     *
+     * Deletes all link mapping files old partition and index files, renaming defragmentated files in the process. Can
+     * be run on the same folder multiple times if failed for some reason.
+     *
+     * Does something only if completion marker is present in the folder. This marker won't be deleted in the end.
+     * Deletion of the marker must be done outside of defragmentation mode to prevent cache groups to be defragmentated
+     * several times in case of failures.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#writeDefragmentationCompletionMarker(FileIOFactory, File, IgniteLogger)
+     */
+    public static void batchRenameDefragmentedCacheGroupPartitions(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (!completionMarkerFile.exists())
+            return;
+
+        try {
+            for (File mappingFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)))
+                Files.delete(mappingFile.toPath());
+
+            for (File partFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_PARTITION_FILE_PREFIX))) {
+                int partId = extractPartId(partFile.getName());
+
+                File oldPartFile = new File(workDir, String.format(PART_FILE_TEMPLATE, partId));
+
+                Files.move(partFile.toPath(), oldPartFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+
+            File idxFile = new File(workDir, DFRG_INDEX_FILE_NAME);
+
+            if (idxFile.exists()) {
+                File oldIdxFile = new File(workDir, INDEX_FILE_NAME);
+
+                Files.move(idxFile.toPath(), oldIdxFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Extracts partition number from file names like {@code part-dfrg-%d.bin}.
+     *
+     * @param dfrgPartFileName Defragmented partition file name.
+     * @return Partition index.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    private static int extractPartId(String dfrgPartFileName) {
+        assert dfrgPartFileName.startsWith(DFRG_PARTITION_FILE_PREFIX) : dfrgPartFileName;
+        assert dfrgPartFileName.endsWith(FILE_SUFFIX) : dfrgPartFileName;
+
+        String partIdStr = dfrgPartFileName.substring(
+            DFRG_PARTITION_FILE_PREFIX.length(),
+            dfrgPartFileName.length() - FILE_SUFFIX.length()
+        );
+
+        return Integer.parseInt(partIdStr);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin.tmp} in given folder. It will be used for storing defragmented index
+     * partition during the process.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexFile(File)
+     */
+    public static File defragmentedIndexTmpFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_TMP_FILE_NAME);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin} in given folder. It will be used for storing defragmented index
+     * partition when the process is over.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexTmpFile(File)
+     */
+    public static File defragmentedIndexFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_FILE_NAME);
+    }
+
+    /**
+     * Rename temporary index defragmenation file to a finalized one.

Review comment:
       defragmenation -> defragmentation

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/DefragmentationFileUtils.java
##########
@@ -0,0 +1,400 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static java.nio.file.StandardCopyOption.ATOMIC_MOVE;
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static java.nio.file.StandardOpenOption.CREATE_NEW;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.FILE_SUFFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_TEMPLATE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.TMP_SUFFIX;
+
+/**
+ * Everything related to file management during defragmentation process.
+ */
+public class DefragmentationFileUtils {
+    /** Prefix for link mapping files. */
+    private static final String DFRG_LINK_MAPPING_FILE_PREFIX = PART_FILE_PREFIX + "map-";
+
+    /** Link mapping file template. */
+    private static final String DFRG_LINK_MAPPING_FILE_TEMPLATE = DFRG_LINK_MAPPING_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmentation complation marker file name. */
+    private static final String DFRG_COMPLETION_MARKER_FILE_NAME = "dfrg-completion-marker";
+
+    /** Name of defragmentated index partition file. */
+    private static final String DFRG_INDEX_FILE_NAME = INDEX_FILE_PREFIX + "-dfrg" + FILE_SUFFIX;
+
+    /** Name of defragmentated index partition temporary file. */
+    private static final String DFRG_INDEX_TMP_FILE_NAME = DFRG_INDEX_FILE_NAME + TMP_SUFFIX;
+
+    /** Prefix for defragmented partition files. */
+    private static final String DFRG_PARTITION_FILE_PREFIX = PART_FILE_PREFIX + "dfrg-";
+
+    /** Defragmented partition file template. */
+    private static final String DFRG_PARTITION_FILE_TEMPLATE = DFRG_PARTITION_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmented partition temp file template. */
+    private static final String DFRG_PARTITION_TMP_FILE_TEMPLATE = DFRG_PARTITION_FILE_TEMPLATE + TMP_SUFFIX;
+
+    /**
+     * Performs cleanup of work dir before initializing file page stores.
+     * Will finish batch renaming if defragmentation was completed or delete garbage if it wasn't.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     */
+    public static void beforeInitPageStores(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+        U.delete(defragmentationCompletionMarkerFile(workDir));
+
+        for (File file : workDir.listFiles()) {
+            String fileName = file.getName();
+
+            if (
+                fileName.startsWith(DFRG_PARTITION_FILE_PREFIX)
+                || fileName.startsWith(DFRG_INDEX_FILE_NAME)
+                || fileName.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)
+            )
+                U.delete(file);
+        }
+    }
+
+    /**
+     * Checks whether cache group defragmentation completed or not. Completes it if all that's left is renaming.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param log Logger to write messages.
+     * @return {@code true} if given cache group is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentationCompletionMarkerFile(File)
+     */
+    public static boolean skipAlreadyDefragmentedCacheGroup(File workDir, int grpId, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (completionMarkerFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented page group",
+                    "grpId", grpId, false,
+                    "markerFileName", completionMarkerFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+            return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * Checks whether partition has already been defragmented or not. Cleans corrupted data if previous failed
+     * defragmentation attempt was found.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param partId Partionion index to check.
+     * @param log Logger to write messages.
+     * @return {@code true} if given partition is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartMappingFile(File, int)
+     */
+    public static boolean skipAlreadyDefragmentedPartition(File workDir, int grpId, int partId, IgniteLogger log) throws IgniteCheckedException {
+        File defragmentedPartFile = defragmentedPartFile(workDir, partId);
+        File defragmentedPartMappingFile = defragmentedPartMappingFile(workDir, partId);
+
+        if (defragmentedPartFile.exists() && defragmentedPartMappingFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented partition",
+                    "grpId", grpId, false,
+                    "partId", partId, false,
+                    "partFileName", defragmentedPartFile.getName(), false,
+                    "mappingFileName", defragmentedPartMappingFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            return true;
+        }
+
+        File defragmentedPartTmpFile = defragmentedPartTmpFile(workDir, partId);
+
+        try {
+            Files.deleteIfExists(defragmentedPartTmpFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartMappingFile.toPath());
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+
+        return false;
+    }
+
+    /**
+     * Failure-tolerant batch rename of defragmented partition files.
+     *
+     * Deletes all link mapping files old partition and index files, renaming defragmentated files in the process. Can
+     * be run on the same folder multiple times if failed for some reason.
+     *
+     * Does something only if completion marker is present in the folder. This marker won't be deleted in the end.
+     * Deletion of the marker must be done outside of defragmentation mode to prevent cache groups to be defragmentated
+     * several times in case of failures.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#writeDefragmentationCompletionMarker(FileIOFactory, File, IgniteLogger)
+     */
+    public static void batchRenameDefragmentedCacheGroupPartitions(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (!completionMarkerFile.exists())
+            return;
+
+        try {
+            for (File mappingFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)))
+                Files.delete(mappingFile.toPath());
+
+            for (File partFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_PARTITION_FILE_PREFIX))) {
+                int partId = extractPartId(partFile.getName());
+
+                File oldPartFile = new File(workDir, String.format(PART_FILE_TEMPLATE, partId));
+
+                Files.move(partFile.toPath(), oldPartFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+
+            File idxFile = new File(workDir, DFRG_INDEX_FILE_NAME);
+
+            if (idxFile.exists()) {
+                File oldIdxFile = new File(workDir, INDEX_FILE_NAME);
+
+                Files.move(idxFile.toPath(), oldIdxFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Extracts partition number from file names like {@code part-dfrg-%d.bin}.
+     *
+     * @param dfrgPartFileName Defragmented partition file name.
+     * @return Partition index.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    private static int extractPartId(String dfrgPartFileName) {
+        assert dfrgPartFileName.startsWith(DFRG_PARTITION_FILE_PREFIX) : dfrgPartFileName;
+        assert dfrgPartFileName.endsWith(FILE_SUFFIX) : dfrgPartFileName;
+
+        String partIdStr = dfrgPartFileName.substring(
+            DFRG_PARTITION_FILE_PREFIX.length(),
+            dfrgPartFileName.length() - FILE_SUFFIX.length()
+        );
+
+        return Integer.parseInt(partIdStr);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin.tmp} in given folder. It will be used for storing defragmented index
+     * partition during the process.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexFile(File)
+     */
+    public static File defragmentedIndexTmpFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_TMP_FILE_NAME);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin} in given folder. It will be used for storing defragmented index
+     * partition when the process is over.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexTmpFile(File)
+     */
+    public static File defragmentedIndexFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_FILE_NAME);
+    }
+
+    /**
+     * Rename temporary index defragmenation file to a finalized one.
+     *
+     * @param workDir Cache group working directory.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexTmpFile(File)
+     * @see DefragmentationFileUtils#defragmentedIndexFile(File)
+     */
+    public static void renameTempIndexFile(File workDir) throws IgniteCheckedException {
+        File defragmentedIdxTmpFile = defragmentedIndexTmpFile(workDir);
+        File defragmentedIdxFile = defragmentedIndexFile(workDir);
+
+        try {
+            Files.move(defragmentedIdxTmpFile.toPath(), defragmentedIdxFile.toPath(), ATOMIC_MOVE);
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Return file named {@code part-dfrg-%d.bin.tmp} in given folder. It will be used for storing defragmented data
+     * partition during the process.
+     *
+     * @param workDir Cache group working directory.
+     * @param partId Parition index, will be substituted into file name.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    public static File defragmentedPartTmpFile(File workDir, int partId) {
+        return new File(workDir, String.format(DFRG_PARTITION_TMP_FILE_TEMPLATE, partId));
+    }
+
+    /**
+     * Return file named {@code part-dfrg-%d.bin} in given folder. It will be used for storing defragmented data
+     * partition when the process is over.
+     *
+     * @param workDir Cache group working directory.
+     * @param partId Parition index, will be substituted into file name.

Review comment:
       Parition -> Partition

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/DefragmentationFileUtils.java
##########
@@ -0,0 +1,400 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static java.nio.file.StandardCopyOption.ATOMIC_MOVE;
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static java.nio.file.StandardOpenOption.CREATE_NEW;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.FILE_SUFFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_TEMPLATE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.TMP_SUFFIX;
+
+/**
+ * Everything related to file management during defragmentation process.
+ */
+public class DefragmentationFileUtils {
+    /** Prefix for link mapping files. */
+    private static final String DFRG_LINK_MAPPING_FILE_PREFIX = PART_FILE_PREFIX + "map-";
+
+    /** Link mapping file template. */
+    private static final String DFRG_LINK_MAPPING_FILE_TEMPLATE = DFRG_LINK_MAPPING_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmentation complation marker file name. */
+    private static final String DFRG_COMPLETION_MARKER_FILE_NAME = "dfrg-completion-marker";
+
+    /** Name of defragmentated index partition file. */
+    private static final String DFRG_INDEX_FILE_NAME = INDEX_FILE_PREFIX + "-dfrg" + FILE_SUFFIX;
+
+    /** Name of defragmentated index partition temporary file. */
+    private static final String DFRG_INDEX_TMP_FILE_NAME = DFRG_INDEX_FILE_NAME + TMP_SUFFIX;
+
+    /** Prefix for defragmented partition files. */
+    private static final String DFRG_PARTITION_FILE_PREFIX = PART_FILE_PREFIX + "dfrg-";
+
+    /** Defragmented partition file template. */
+    private static final String DFRG_PARTITION_FILE_TEMPLATE = DFRG_PARTITION_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmented partition temp file template. */
+    private static final String DFRG_PARTITION_TMP_FILE_TEMPLATE = DFRG_PARTITION_FILE_TEMPLATE + TMP_SUFFIX;
+
+    /**
+     * Performs cleanup of work dir before initializing file page stores.
+     * Will finish batch renaming if defragmentation was completed or delete garbage if it wasn't.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     */
+    public static void beforeInitPageStores(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+        U.delete(defragmentationCompletionMarkerFile(workDir));
+
+        for (File file : workDir.listFiles()) {
+            String fileName = file.getName();
+
+            if (
+                fileName.startsWith(DFRG_PARTITION_FILE_PREFIX)
+                || fileName.startsWith(DFRG_INDEX_FILE_NAME)
+                || fileName.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)
+            )
+                U.delete(file);
+        }
+    }
+
+    /**
+     * Checks whether cache group defragmentation completed or not. Completes it if all that's left is renaming.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param log Logger to write messages.
+     * @return {@code true} if given cache group is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentationCompletionMarkerFile(File)
+     */
+    public static boolean skipAlreadyDefragmentedCacheGroup(File workDir, int grpId, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (completionMarkerFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented page group",
+                    "grpId", grpId, false,
+                    "markerFileName", completionMarkerFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+            return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * Checks whether partition has already been defragmented or not. Cleans corrupted data if previous failed
+     * defragmentation attempt was found.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param partId Partionion index to check.
+     * @param log Logger to write messages.
+     * @return {@code true} if given partition is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartMappingFile(File, int)
+     */
+    public static boolean skipAlreadyDefragmentedPartition(File workDir, int grpId, int partId, IgniteLogger log) throws IgniteCheckedException {
+        File defragmentedPartFile = defragmentedPartFile(workDir, partId);
+        File defragmentedPartMappingFile = defragmentedPartMappingFile(workDir, partId);
+
+        if (defragmentedPartFile.exists() && defragmentedPartMappingFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented partition",
+                    "grpId", grpId, false,
+                    "partId", partId, false,
+                    "partFileName", defragmentedPartFile.getName(), false,
+                    "mappingFileName", defragmentedPartMappingFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            return true;
+        }
+
+        File defragmentedPartTmpFile = defragmentedPartTmpFile(workDir, partId);
+
+        try {
+            Files.deleteIfExists(defragmentedPartTmpFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartMappingFile.toPath());
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+
+        return false;
+    }
+
+    /**
+     * Failure-tolerant batch rename of defragmented partition files.
+     *
+     * Deletes all link mapping files old partition and index files, renaming defragmentated files in the process. Can
+     * be run on the same folder multiple times if failed for some reason.
+     *
+     * Does something only if completion marker is present in the folder. This marker won't be deleted in the end.
+     * Deletion of the marker must be done outside of defragmentation mode to prevent cache groups to be defragmentated
+     * several times in case of failures.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#writeDefragmentationCompletionMarker(FileIOFactory, File, IgniteLogger)
+     */
+    public static void batchRenameDefragmentedCacheGroupPartitions(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (!completionMarkerFile.exists())
+            return;
+
+        try {
+            for (File mappingFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)))
+                Files.delete(mappingFile.toPath());
+
+            for (File partFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_PARTITION_FILE_PREFIX))) {
+                int partId = extractPartId(partFile.getName());
+
+                File oldPartFile = new File(workDir, String.format(PART_FILE_TEMPLATE, partId));
+
+                Files.move(partFile.toPath(), oldPartFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+
+            File idxFile = new File(workDir, DFRG_INDEX_FILE_NAME);
+
+            if (idxFile.exists()) {
+                File oldIdxFile = new File(workDir, INDEX_FILE_NAME);
+
+                Files.move(idxFile.toPath(), oldIdxFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Extracts partition number from file names like {@code part-dfrg-%d.bin}.
+     *
+     * @param dfrgPartFileName Defragmented partition file name.
+     * @return Partition index.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    private static int extractPartId(String dfrgPartFileName) {
+        assert dfrgPartFileName.startsWith(DFRG_PARTITION_FILE_PREFIX) : dfrgPartFileName;
+        assert dfrgPartFileName.endsWith(FILE_SUFFIX) : dfrgPartFileName;
+
+        String partIdStr = dfrgPartFileName.substring(
+            DFRG_PARTITION_FILE_PREFIX.length(),
+            dfrgPartFileName.length() - FILE_SUFFIX.length()
+        );
+
+        return Integer.parseInt(partIdStr);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin.tmp} in given folder. It will be used for storing defragmented index
+     * partition during the process.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexFile(File)
+     */
+    public static File defragmentedIndexTmpFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_TMP_FILE_NAME);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin} in given folder. It will be used for storing defragmented index
+     * partition when the process is over.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexTmpFile(File)
+     */
+    public static File defragmentedIndexFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_FILE_NAME);
+    }
+
+    /**
+     * Rename temporary index defragmenation file to a finalized one.
+     *
+     * @param workDir Cache group working directory.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexTmpFile(File)
+     * @see DefragmentationFileUtils#defragmentedIndexFile(File)
+     */
+    public static void renameTempIndexFile(File workDir) throws IgniteCheckedException {
+        File defragmentedIdxTmpFile = defragmentedIndexTmpFile(workDir);
+        File defragmentedIdxFile = defragmentedIndexFile(workDir);
+
+        try {
+            Files.move(defragmentedIdxTmpFile.toPath(), defragmentedIdxFile.toPath(), ATOMIC_MOVE);
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Return file named {@code part-dfrg-%d.bin.tmp} in given folder. It will be used for storing defragmented data
+     * partition during the process.
+     *
+     * @param workDir Cache group working directory.
+     * @param partId Parition index, will be substituted into file name.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    public static File defragmentedPartTmpFile(File workDir, int partId) {
+        return new File(workDir, String.format(DFRG_PARTITION_TMP_FILE_TEMPLATE, partId));
+    }
+
+    /**
+     * Return file named {@code part-dfrg-%d.bin} in given folder. It will be used for storing defragmented data
+     * partition when the process is over.
+     *
+     * @param workDir Cache group working directory.
+     * @param partId Parition index, will be substituted into file name.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     */
+    public static File defragmentedPartFile(File workDir, int partId) {
+        return new File(workDir, String.format(DFRG_PARTITION_FILE_TEMPLATE, partId));
+    }
+
+    /**
+     * Rename temporary partition defragmenation file to a finalized one.
+     *
+     * @param workDir Cache group working directory.
+     * @param partId Parition index.

Review comment:
       Parition -> Partition

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/DefragmentationFileUtils.java
##########
@@ -0,0 +1,400 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static java.nio.file.StandardCopyOption.ATOMIC_MOVE;
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static java.nio.file.StandardOpenOption.CREATE_NEW;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.FILE_SUFFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_TEMPLATE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.TMP_SUFFIX;
+
+/**
+ * Everything related to file management during defragmentation process.
+ */
+public class DefragmentationFileUtils {
+    /** Prefix for link mapping files. */
+    private static final String DFRG_LINK_MAPPING_FILE_PREFIX = PART_FILE_PREFIX + "map-";
+
+    /** Link mapping file template. */
+    private static final String DFRG_LINK_MAPPING_FILE_TEMPLATE = DFRG_LINK_MAPPING_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmentation complation marker file name. */
+    private static final String DFRG_COMPLETION_MARKER_FILE_NAME = "dfrg-completion-marker";
+
+    /** Name of defragmentated index partition file. */
+    private static final String DFRG_INDEX_FILE_NAME = INDEX_FILE_PREFIX + "-dfrg" + FILE_SUFFIX;
+
+    /** Name of defragmentated index partition temporary file. */
+    private static final String DFRG_INDEX_TMP_FILE_NAME = DFRG_INDEX_FILE_NAME + TMP_SUFFIX;
+
+    /** Prefix for defragmented partition files. */
+    private static final String DFRG_PARTITION_FILE_PREFIX = PART_FILE_PREFIX + "dfrg-";
+
+    /** Defragmented partition file template. */
+    private static final String DFRG_PARTITION_FILE_TEMPLATE = DFRG_PARTITION_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmented partition temp file template. */
+    private static final String DFRG_PARTITION_TMP_FILE_TEMPLATE = DFRG_PARTITION_FILE_TEMPLATE + TMP_SUFFIX;
+
+    /**
+     * Performs cleanup of work dir before initializing file page stores.
+     * Will finish batch renaming if defragmentation was completed or delete garbage if it wasn't.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     */
+    public static void beforeInitPageStores(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+        U.delete(defragmentationCompletionMarkerFile(workDir));
+
+        for (File file : workDir.listFiles()) {
+            String fileName = file.getName();
+
+            if (
+                fileName.startsWith(DFRG_PARTITION_FILE_PREFIX)
+                || fileName.startsWith(DFRG_INDEX_FILE_NAME)
+                || fileName.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)
+            )
+                U.delete(file);
+        }
+    }
+
+    /**
+     * Checks whether cache group defragmentation completed or not. Completes it if all that's left is renaming.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param log Logger to write messages.
+     * @return {@code true} if given cache group is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentationCompletionMarkerFile(File)
+     */
+    public static boolean skipAlreadyDefragmentedCacheGroup(File workDir, int grpId, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (completionMarkerFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented page group",
+                    "grpId", grpId, false,
+                    "markerFileName", completionMarkerFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+            return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * Checks whether partition has already been defragmented or not. Cleans corrupted data if previous failed
+     * defragmentation attempt was found.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param partId Partionion index to check.

Review comment:
       Partionion -> Partition

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,896 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database schared manager. */

Review comment:
       schared -> shared

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,896 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database schared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;
+
+                                if (log.isDebugEnabled()) {
+                                    log.debug(S.toString(
+                                        "Partition defragmented",
+                                        "grpId", grpId, false,
+                                        "partId", partId, false,
+                                        "oldPages", oldPageStore.pages(), false,
+                                        "newPages", partCtx.partPagesAllocated.get() + 1, false,
+                                        "mappingPages", partCtx.mappingPagesAllocated.get() + 1, false,
+                                        "pageSize", pageSize, false,
+                                        "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                        "workDir", workDir, false
+                                    ));
+                                }
+
+                                oldPageMem.invalidate(grpId, partId);
+
+                                partCtx.partPageMemory.invalidate(grpId, partId);
+
+                                DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                                pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                                try {
+                                    renameTempPartitionFile(workDir, partId);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    throw new IgniteException(e);
+                                }
+                            }
+                        };
+
+                        GridFutureAdapter<?> cpFut = defragmentationCheckpoint
+                            .forceCheckpoint("partition defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+
+                        cpFut.listen(cpLsnr);
+
+                        cmpFut.add((IgniteInternalFuture<Object>)cpFut);
+                    }
+
+                    // A bit too general for now, but I like it more then saving only the last checkpoint future.
+                    cmpFut.markInitialized().get();
+
+                    idxDfrgFut = new GridFinishedFuture<>();
+
+                    if (filePageStoreMgr.hasIndexStore(grpId)) {
+                        defragmentIndexPartition(oldGrpCtx, newGrpCtx, linkMapByPart);
+
+                        idxDfrgFut = defragmentationCheckpoint
+                            .forceCheckpoint("index defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+                    }
+
+                    idxDfrgFut.listen(fut -> {
+                        oldPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx partPageMem = (PageMemoryEx)partDataRegion.pageMemory();
+
+                        partPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().removePageStore(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx mappingPageMem = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+                        pageMgr = (DefragmentationPageReadWriteManager)mappingPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().clear(grpId);
+
+                        try {
+                            renameTempIndexFile(workDir);
+
+                            writeDefragmentationCompletionMarker(filePageStoreMgr.getPageStoreFileIoFactory(), workDir, log);
+
+                            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    });
+                }
+
+                // I guess we should wait for it?
+                if (idxDfrgFut != null)
+                    idxDfrgFut.get();
+            }
+
+            mntcReg.unregisterMaintenanceTask(DEFRAGMENTATION_MNTC_TASK_NAME);
+
+            log.info("Defragmentation completed. All partitions are defragmented.");
+        }
+        finally {
+            defragmentationCheckpoint.stop(true);
+        }
+    }
+
+    /** */
+    public void createIndexPageStore(
+        int grpId,
+        File workDir,
+        FilePageStoreFactory pageStoreFactory,
+        DataRegion partRegion,
+        LongConsumer allocatedTracker
+    ) throws IgniteCheckedException {
+        // Index partition file has to be deleted before we begin, otherwise there's a chance of reading corrupted file.
+        // There is a time period when index is already defragmented but marker file is not created yet. If node is
+        // failed in that time window then index will be deframented once again. That's fine, situation is rare but code
+        // to fix that would add unnecessary complications.
+        U.delete(defragmentedIndexTmpFile(workDir));
+
+        PageStore idxPageStore;
+
+        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+        try {
+            idxPageStore = pageStoreFactory.createPageStore(
+                FLAG_IDX,
+                () -> defragmentedIndexTmpFile(workDir).toPath(),
+                allocatedTracker
+            );
+        }
+        finally {
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+        }
+
+        idxPageStore.sync();
+
+        PageMemoryEx partPageMem = (PageMemoryEx)partRegion.pageMemory();
+
+        DefragmentationPageReadWriteManager partMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+        partMgr.pageStoreMap().addPageStore(grpId, PageIdAllocator.INDEX_PARTITION, idxPageStore);
+    }
+
+    /**
+     * Cancel the process of defragmentation.
+     */
+    public void cancel(){

Review comment:
       Will we implement 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.

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r530200177



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/GridQueryIndexingDefragmentation.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.processors.cache.persistence.defragmentation;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.util.collection.IntMap;
+
+/**
+ *
+ */
+public interface GridQueryIndexingDefragmentation {

Review comment:
       What if we need more methods? GridQueryIndexing is a mess already, this is just an attempt to use code decomposition.




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

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



[GitHub] [ignite] agoncharuk commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
agoncharuk commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r524285281



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/DefragmentationFileUtils.java
##########
@@ -0,0 +1,400 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static java.nio.file.StandardCopyOption.ATOMIC_MOVE;
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static java.nio.file.StandardOpenOption.CREATE_NEW;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.FILE_SUFFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_TEMPLATE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.TMP_SUFFIX;
+
+/**
+ * Everything related to file management during defragmentation process.
+ */
+public class DefragmentationFileUtils {
+    /** Prefix for link mapping files. */
+    private static final String DFRG_LINK_MAPPING_FILE_PREFIX = PART_FILE_PREFIX + "map-";
+
+    /** Link mapping file template. */
+    private static final String DFRG_LINK_MAPPING_FILE_TEMPLATE = DFRG_LINK_MAPPING_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmentation complation marker file name. */
+    private static final String DFRG_COMPLETION_MARKER_FILE_NAME = "dfrg-completion-marker";
+
+    /** Name of defragmentated index partition file. */
+    private static final String DFRG_INDEX_FILE_NAME = INDEX_FILE_PREFIX + "-dfrg" + FILE_SUFFIX;
+
+    /** Name of defragmentated index partition temporary file. */
+    private static final String DFRG_INDEX_TMP_FILE_NAME = DFRG_INDEX_FILE_NAME + TMP_SUFFIX;
+
+    /** Prefix for defragmented partition files. */
+    private static final String DFRG_PARTITION_FILE_PREFIX = PART_FILE_PREFIX + "dfrg-";
+
+    /** Defragmented partition file template. */
+    private static final String DFRG_PARTITION_FILE_TEMPLATE = DFRG_PARTITION_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmented partition temp file template. */
+    private static final String DFRG_PARTITION_TMP_FILE_TEMPLATE = DFRG_PARTITION_FILE_TEMPLATE + TMP_SUFFIX;
+
+    /**
+     * Performs cleanup of work dir before initializing file page stores.
+     * Will finish batch renaming if defragmentation was completed or delete garbage if it wasn't.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     */
+    public static void beforeInitPageStores(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+        U.delete(defragmentationCompletionMarkerFile(workDir));
+
+        for (File file : workDir.listFiles()) {
+            String fileName = file.getName();
+
+            if (
+                fileName.startsWith(DFRG_PARTITION_FILE_PREFIX)
+                || fileName.startsWith(DFRG_INDEX_FILE_NAME)
+                || fileName.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)
+            )
+                U.delete(file);
+        }
+    }
+
+    /**
+     * Checks whether cache group defragmentation completed or not. Completes it if all that's left is renaming.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param log Logger to write messages.
+     * @return {@code true} if given cache group is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentationCompletionMarkerFile(File)
+     */
+    public static boolean skipAlreadyDefragmentedCacheGroup(File workDir, int grpId, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (completionMarkerFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented page group",
+                    "grpId", grpId, false,
+                    "markerFileName", completionMarkerFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+            return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * Checks whether partition has already been defragmented or not. Cleans corrupted data if previous failed
+     * defragmentation attempt was found.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param partId Partionion index to check.
+     * @param log Logger to write messages.
+     * @return {@code true} if given partition is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartMappingFile(File, int)
+     */
+    public static boolean skipAlreadyDefragmentedPartition(File workDir, int grpId, int partId, IgniteLogger log) throws IgniteCheckedException {
+        File defragmentedPartFile = defragmentedPartFile(workDir, partId);
+        File defragmentedPartMappingFile = defragmentedPartMappingFile(workDir, partId);
+
+        if (defragmentedPartFile.exists() && defragmentedPartMappingFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented partition",
+                    "grpId", grpId, false,
+                    "partId", partId, false,
+                    "partFileName", defragmentedPartFile.getName(), false,
+                    "mappingFileName", defragmentedPartMappingFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            return true;
+        }
+
+        File defragmentedPartTmpFile = defragmentedPartTmpFile(workDir, partId);
+
+        try {
+            Files.deleteIfExists(defragmentedPartTmpFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartMappingFile.toPath());
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+
+        return false;
+    }
+
+    /**
+     * Failure-tolerant batch rename of defragmented partition files.
+     *
+     * Deletes all link mapping files old partition and index files, renaming defragmentated files in the process. Can
+     * be run on the same folder multiple times if failed for some reason.
+     *
+     * Does something only if completion marker is present in the folder. This marker won't be deleted in the end.
+     * Deletion of the marker must be done outside of defragmentation mode to prevent cache groups to be defragmentated
+     * several times in case of failures.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#writeDefragmentationCompletionMarker(FileIOFactory, File, IgniteLogger)
+     */
+    public static void batchRenameDefragmentedCacheGroupPartitions(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (!completionMarkerFile.exists())
+            return;
+
+        try {
+            for (File mappingFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)))
+                Files.delete(mappingFile.toPath());
+
+            for (File partFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_PARTITION_FILE_PREFIX))) {
+                int partId = extractPartId(partFile.getName());
+
+                File oldPartFile = new File(workDir, String.format(PART_FILE_TEMPLATE, partId));
+
+                Files.move(partFile.toPath(), oldPartFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+
+            File idxFile = new File(workDir, DFRG_INDEX_FILE_NAME);
+
+            if (idxFile.exists()) {
+                File oldIdxFile = new File(workDir, INDEX_FILE_NAME);
+
+                Files.move(idxFile.toPath(), oldIdxFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Extracts partition number from file names like {@code part-dfrg-%d.bin}.
+     *
+     * @param dfrgPartFileName Defragmented partition file name.
+     * @return Partition index.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    private static int extractPartId(String dfrgPartFileName) {
+        assert dfrgPartFileName.startsWith(DFRG_PARTITION_FILE_PREFIX) : dfrgPartFileName;
+        assert dfrgPartFileName.endsWith(FILE_SUFFIX) : dfrgPartFileName;
+
+        String partIdStr = dfrgPartFileName.substring(
+            DFRG_PARTITION_FILE_PREFIX.length(),
+            dfrgPartFileName.length() - FILE_SUFFIX.length()
+        );
+
+        return Integer.parseInt(partIdStr);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin.tmp} in given folder. It will be used for storing defragmented index
+     * partition during the process.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexFile(File)
+     */
+    public static File defragmentedIndexTmpFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_TMP_FILE_NAME);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin} in given folder. It will be used for storing defragmented index
+     * partition when the process is over.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexTmpFile(File)
+     */
+    public static File defragmentedIndexFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_FILE_NAME);
+    }
+
+    /**
+     * Rename temporary index defragmenation file to a finalized one.
+     *
+     * @param workDir Cache group working directory.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexTmpFile(File)
+     * @see DefragmentationFileUtils#defragmentedIndexFile(File)
+     */
+    public static void renameTempIndexFile(File workDir) throws IgniteCheckedException {
+        File defragmentedIdxTmpFile = defragmentedIndexTmpFile(workDir);
+        File defragmentedIdxFile = defragmentedIndexFile(workDir);
+
+        try {
+            Files.move(defragmentedIdxTmpFile.toPath(), defragmentedIdxFile.toPath(), ATOMIC_MOVE);
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Return file named {@code part-dfrg-%d.bin.tmp} in given folder. It will be used for storing defragmented data
+     * partition during the process.
+     *
+     * @param workDir Cache group working directory.
+     * @param partId Parition index, will be substituted into file name.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    public static File defragmentedPartTmpFile(File workDir, int partId) {
+        return new File(workDir, String.format(DFRG_PARTITION_TMP_FILE_TEMPLATE, partId));
+    }
+
+    /**
+     * Return file named {@code part-dfrg-%d.bin} in given folder. It will be used for storing defragmented data
+     * partition when the process is over.
+     *
+     * @param workDir Cache group working directory.
+     * @param partId Parition index, will be substituted into file name.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     */
+    public static File defragmentedPartFile(File workDir, int partId) {
+        return new File(workDir, String.format(DFRG_PARTITION_FILE_TEMPLATE, partId));
+    }
+
+    /**
+     * Rename temporary partition defragmenation file to a finalized one.
+     *
+     * @param workDir Cache group working directory.
+     * @param partId Parition index.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    public static void renameTempPartitionFile(File workDir, int partId) throws IgniteCheckedException {
+        File defragmentedPartTmpFile = defragmentedPartTmpFile(workDir, partId);
+        File defragmentedPartFile = defragmentedPartFile(workDir, partId);
+
+        assert !defragmentedPartFile.exists() : defragmentedPartFile;

Review comment:
       There may be a race if a user deletes this file manually - should we handle such cases as regular IO exceptions rather than assertions?




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

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r525124359



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,896 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database schared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;
+
+                                if (log.isDebugEnabled()) {
+                                    log.debug(S.toString(
+                                        "Partition defragmented",
+                                        "grpId", grpId, false,
+                                        "partId", partId, false,
+                                        "oldPages", oldPageStore.pages(), false,
+                                        "newPages", partCtx.partPagesAllocated.get() + 1, false,
+                                        "mappingPages", partCtx.mappingPagesAllocated.get() + 1, false,
+                                        "pageSize", pageSize, false,
+                                        "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                        "workDir", workDir, false
+                                    ));
+                                }
+
+                                oldPageMem.invalidate(grpId, partId);
+
+                                partCtx.partPageMemory.invalidate(grpId, partId);
+
+                                DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                                pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                                try {
+                                    renameTempPartitionFile(workDir, partId);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    throw new IgniteException(e);
+                                }
+                            }
+                        };
+
+                        GridFutureAdapter<?> cpFut = defragmentationCheckpoint
+                            .forceCheckpoint("partition defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+
+                        cpFut.listen(cpLsnr);
+
+                        cmpFut.add((IgniteInternalFuture<Object>)cpFut);
+                    }
+
+                    // A bit too general for now, but I like it more then saving only the last checkpoint future.
+                    cmpFut.markInitialized().get();
+
+                    idxDfrgFut = new GridFinishedFuture<>();
+
+                    if (filePageStoreMgr.hasIndexStore(grpId)) {
+                        defragmentIndexPartition(oldGrpCtx, newGrpCtx, linkMapByPart);
+
+                        idxDfrgFut = defragmentationCheckpoint
+                            .forceCheckpoint("index defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+                    }
+
+                    idxDfrgFut.listen(fut -> {
+                        oldPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx partPageMem = (PageMemoryEx)partDataRegion.pageMemory();
+
+                        partPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().removePageStore(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx mappingPageMem = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+                        pageMgr = (DefragmentationPageReadWriteManager)mappingPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().clear(grpId);
+
+                        try {
+                            renameTempIndexFile(workDir);
+
+                            writeDefragmentationCompletionMarker(filePageStoreMgr.getPageStoreFileIoFactory(), workDir, log);
+
+                            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    });
+                }
+
+                // I guess we should wait for it?
+                if (idxDfrgFut != null)
+                    idxDfrgFut.get();
+            }
+
+            mntcReg.unregisterMaintenanceTask(DEFRAGMENTATION_MNTC_TASK_NAME);
+
+            log.info("Defragmentation completed. All partitions are defragmented.");
+        }
+        finally {
+            defragmentationCheckpoint.stop(true);
+        }
+    }
+
+    /** */
+    public void createIndexPageStore(
+        int grpId,
+        File workDir,
+        FilePageStoreFactory pageStoreFactory,
+        DataRegion partRegion,
+        LongConsumer allocatedTracker
+    ) throws IgniteCheckedException {
+        // Index partition file has to be deleted before we begin, otherwise there's a chance of reading corrupted file.
+        // There is a time period when index is already defragmented but marker file is not created yet. If node is
+        // failed in that time window then index will be deframented once again. That's fine, situation is rare but code
+        // to fix that would add unnecessary complications.
+        U.delete(defragmentedIndexTmpFile(workDir));
+
+        PageStore idxPageStore;
+
+        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+        try {
+            idxPageStore = pageStoreFactory.createPageStore(
+                FLAG_IDX,
+                () -> defragmentedIndexTmpFile(workDir).toPath(),
+                allocatedTracker
+            );
+        }
+        finally {
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+        }
+
+        idxPageStore.sync();
+
+        PageMemoryEx partPageMem = (PageMemoryEx)partRegion.pageMemory();
+
+        DefragmentationPageReadWriteManager partMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+        partMgr.pageStoreMap().addPageStore(grpId, PageIdAllocator.INDEX_PARTITION, idxPageStore);
+    }
+
+    /**
+     * Cancel the process of defragmentation.
+     */
+    public void cancel(){

Review comment:
       The master branch is the release ready branch, so having defragmentation procedure for users without 'cancel' operation is not a good practice and is not friendly. 
   
   I think we must implement the cancel procedure.




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r524327899



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java
##########
@@ -1678,6 +1678,51 @@ public static String toString(String str,
         }
     }
 
+    /**
+     * Produces uniformed output of string with context properties
+     *
+     * @param str Output prefix or {@code null} if empty.
+     * @param triplets Triplets {@code {name, value, sencitivity}}.
+     * @return String presentation.
+     */
+    public static String toString(String str, Object... triplets) {
+        assert triplets.length % 3 == 0;

Review comment:
       Ok, I'll 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.

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r530207540



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
##########
@@ -447,6 +472,32 @@ private DataRegionConfiguration createMetastoreDataRegionConfig(DataStorageConfi
         return cfg;
     }
 
+    /** */
+    private DataRegionConfiguration createDefragmentationDataRegionConfig(long regionSize) {

Review comment:
       They will be deallocated when node stops, just like all the other regions. And I don't think I understand the first question.
   The thing is that we either create two new data regions dynamically or we deeply refactor and change our page memory classes, which would take many months and add inevitable performance degradation in everyday scenarios(

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsDefragmentationTest.java
##########
@@ -0,0 +1,539 @@
+/*
+ * 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.processors.cache.persistence;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.FileVisitResult;
+import java.nio.file.FileVisitor;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardCopyOption;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.Collections;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+import java.util.function.UnaryOperator;
+import java.util.stream.IntStream;
+import javax.cache.configuration.Factory;
+import javax.cache.expiry.Duration;
+import javax.cache.expiry.ExpiryPolicy;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.FailureHandler;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.maintenance.MaintenanceFileStore;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.util.lang.IgniteThrowableConsumer;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentationCompletionMarkerFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.maintenance.DefragmentationParameters.toStore;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
+/** */
+public class IgnitePdsDefragmentationTest extends GridCommonAbstractTest {
+    /** */
+    public static final String CACHE_2_NAME = "cache2";
+
+    /** */
+    public static final int PARTS = 5;
+
+    /** */
+    public static final int ADDED_KEYS_COUNT = 150;
+
+    /** */
+    protected static final String GRP_NAME = "group";
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        stopAllGrids(true);
+
+        cleanPersistenceDir();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids(true);
+
+        cleanPersistenceDir();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected FailureHandler getFailureHandler(String igniteInstanceName) {
+        return new StopNodeFailureHandler();
+    }
+
+    /** */
+    protected static class PolicyFactory implements Factory<ExpiryPolicy> {
+        /** Serial version uid. */
+        private static final long serialVersionUID = 0L;
+
+        /** {@inheritDoc} */
+        @Override public ExpiryPolicy create() {
+            return new ExpiryPolicy() {
+                @Override public Duration getExpiryForCreation() {
+                    return new Duration(TimeUnit.MILLISECONDS, 13000);
+                }
+
+                /** {@inheritDoc} */
+                @Override public Duration getExpiryForAccess() {
+                    return new Duration(TimeUnit.MILLISECONDS, 13000);
+                }
+
+                /** {@inheritDoc} */
+                @Override public Duration getExpiryForUpdate() {
+                    return new Duration(TimeUnit.MILLISECONDS, 13000);
+                }
+            };
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setConsistentId(igniteInstanceName);
+
+        DataStorageConfiguration dsCfg = new DataStorageConfiguration();
+        dsCfg.setWalSegmentSize(4 * 1024 * 1024);
+
+        dsCfg.setDefaultDataRegionConfiguration(
+            new DataRegionConfiguration()
+                .setInitialSize(100L * 1024 * 1024)
+                .setMaxSize(1024L * 1024 * 1024)
+                .setPersistenceEnabled(true)
+        );
+
+        cfg.setDataStorageConfiguration(dsCfg);
+
+        CacheConfiguration<?, ?> cache1Cfg = new CacheConfiguration<>(DEFAULT_CACHE_NAME)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setGroupName(GRP_NAME)
+            .setAffinity(new RendezvousAffinityFunction(false, PARTS));
+
+        CacheConfiguration<?, ?> cache2Cfg = new CacheConfiguration<>(CACHE_2_NAME)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setGroupName(GRP_NAME)
+            .setExpiryPolicyFactory(new PolicyFactory())
+            .setAffinity(new RendezvousAffinityFunction(false, PARTS));
+
+        cfg.setCacheConfiguration(cache1Cfg, cache2Cfg);
+
+        return cfg;
+    }
+
+    /**
+     * Basic test scenario. Does following steps:
+     *  - Start node;
+     *  - Fill cache;
+     *  - Remove part of data;
+     *  - Stop node;
+     *  - Start node in defragmentation mode;
+     *  - Stop node;
+     *  - Start node;
+     *  - Check that partitions became smaller;
+     *  - Check that cache is accessible and works just fine.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testEssentials() throws Exception {

Review comment:
       You're right))




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

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r531513801



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
##########
@@ -1357,8 +1357,19 @@ private void saveIndexReencryptionStatus(int grpId) throws IgniteCheckedExceptio
     }
 
     /** */
-    public GridSpinBusyLock busyLock() {
-        return busyLock;
+    public GridCacheDataStore createGridCacheDataStore(

Review comment:
       Please, reuse it also for `createCacheDataStore0`

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/TreeIterator.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.nio.ByteBuffer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.util.GridUnsafe;
+
+/** */
+public class TreeIterator {
+    /** Direct memory buffer with a size of one page. */
+    private final ByteBuffer pageBuf;
+
+    /** Offheap page size. */
+    private final int pageSize;
+
+    /** */
+    public TreeIterator(int size) {
+        pageSize = size;
+
+        pageBuf = ByteBuffer.allocateDirect(pageSize);
+    }
+
+    // Performance impact of constant closures allocation is not clear. So this method should be avoided in massive
+    // operations like tree leaves access.
+    /** */
+    public static <T> T access(
+        PageAccessType access,
+        PageMemoryEx pageMemory,
+        int grpId,
+        long pageId,
+        PageAccessor<T> accessor
+    ) throws IgniteCheckedException {
+        assert access == PageAccessType.ACCESS_READ || access == PageAccessType.ACCESS_WRITE;
+        long page = pageMemory.acquirePage(grpId, pageId);
+
+        try {
+            long pageAddr = access == PageAccessType.ACCESS_READ
+                    ? pageMemory.readLock(grpId, pageId, page)
+                    : pageMemory.writeLock(grpId, pageId, page);
+
+            try {
+                return accessor.access(pageAddr);
+            }
+            finally {
+                if (access == PageAccessType.ACCESS_READ)
+                    pageMemory.readUnlock(grpId, pageId, page);
+                else
+                    pageMemory.writeUnlock(grpId, pageId, page, null, true);
+            }
+        }
+        finally {
+            pageMemory.releasePage(grpId, pageId, page);
+        }
+    }
+
+    /** */
+    @SuppressWarnings("PublicInnerClass")
+    public enum PageAccessType {
+        /** Read access. */
+        ACCESS_READ,
+
+        /** Write access. */
+        ACCESS_WRITE;
+    }
+
+    /** */
+    @SuppressWarnings("PublicInnerClass")
+    @FunctionalInterface
+    public interface PageAccessor<T> {
+        /** */
+        public T access(long pageAddr) throws IgniteCheckedException;
+    }
+
+    /** */
+    // TODO Prefetch future pages?

Review comment:
       Let's fix this.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,861 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.lang.IgniteOutClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.

Review comment:
       Let's fix this.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/TimeTracker.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.util.Arrays;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/** */
+public class TimeTracker<Stage extends Enum<Stage>> {

Review comment:
       Please, remove it.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/TreeIterator.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.nio.ByteBuffer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.util.GridUnsafe;
+
+/** */
+public class TreeIterator {
+    /** Direct memory buffer with a size of one page. */
+    private final ByteBuffer pageBuf;
+
+    /** Offheap page size. */
+    private final int pageSize;
+
+    /** */
+    public TreeIterator(int size) {
+        pageSize = size;
+
+        pageBuf = ByteBuffer.allocateDirect(pageSize);
+    }
+
+    // Performance impact of constant closures allocation is not clear. So this method should be avoided in massive
+    // operations like tree leaves access.
+    /** */
+    public static <T> T access(
+        PageAccessType access,
+        PageMemoryEx pageMemory,
+        int grpId,
+        long pageId,
+        PageAccessor<T> accessor
+    ) throws IgniteCheckedException {
+        assert access != null;
+        long page = pageMemory.acquirePage(grpId, pageId);
+
+        try {
+            long pageAddr = access == PageAccessType.ACCESS_READ
+                    ? pageMemory.readLock(grpId, pageId, page)
+                    : pageMemory.writeLock(grpId, pageId, page);
+
+            try {
+                return accessor.access(pageAddr);
+            }
+            finally {
+                if (access == PageAccessType.ACCESS_READ)
+                    pageMemory.readUnlock(grpId, pageId, page);
+                else
+                    pageMemory.writeUnlock(grpId, pageId, page, null, true);
+            }
+        }
+        finally {
+            pageMemory.releasePage(grpId, pageId, page);
+        }
+    }
+
+    /** */
+    @SuppressWarnings("PublicInnerClass")
+    public enum PageAccessType {

Review comment:
       The `ACCESS_WRITE` used only once, so you can paste try-finally block directly into the code and eliminate any lambda creation penalties for the page writes. As for the `ACCESS_READ` you can rename it to `readPageAddr` and use it where it really needs.
   
   PageAccessType is completely useless and it never be extended. 
   PageAccessor -> PageReader
   
   But I see no real reasons for generalization such operations as page addresses write/reads. The other parts of code never go this way. It will be simpler and better if you'll reuse try-finally approach for all these cases.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,861 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.lang.IgniteOutClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createPageStore(
+                                () -> defragmentedPartMappingFile(workDir, partId).toPath(),
+                                partCtx.mappingPagesAllocated,
+                                partCtx.mappingPageMemory
+                            );
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createPageStore(
+                            () -> defragmentedPartMappingFile(workDir, partId).toPath(),
+                            partCtx.mappingPagesAllocated,
+                            partCtx.mappingPageMemory
+                        );
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPageStore(
+                            () -> defragmentedPartTmpFile(workDir, partId).toPath(),
+                            partCtx.partPagesAllocated,
+                            partCtx.partPageMemory
+                        );
+
+                        copyPartitionData(partCtx, treeIter, offheap);
+
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() != null)
+                                return;
+
+                            PageStore oldPageStore = null;
+
+                            try {
+                                oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                            }
+                            catch (IgniteCheckedException ignore) {
+                            }
+
+                            if (log.isDebugEnabled()) {
+                                log.debug(S.toString(
+                                    "Partition defragmented",
+                                    "grpId", grpId, false,
+                                    "partId", partId, false,
+                                    "oldPages", oldPageStore.pages(), false,
+                                    "newPages", partCtx.partPagesAllocated.get() + 1, false,
+                                    "mappingPages", partCtx.mappingPagesAllocated.get() + 1, false,
+                                    "pageSize", pageSize, false,
+                                    "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                    "workDir", workDir, false
+                                ));
+                            }
+
+                            oldPageMem.invalidate(grpId, partId);
+
+                            partCtx.partPageMemory.invalidate(grpId, partId);
+
+                            DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                            pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                            renameTempPartitionFile(workDir, partId);
+                        };
+
+                        GridFutureAdapter<?> cpFut = defragmentationCheckpoint
+                            .forceCheckpoint("partition defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+
+                        cpFut.listen(cpLsnr);
+
+                        cmpFut.add((IgniteInternalFuture<Object>)cpFut);
+                    }
+
+                    // A bit too general for now, but I like it more then saving only the last checkpoint future.
+                    cmpFut.markInitialized().get();
+
+                    idxDfrgFut = new GridFinishedFuture<>();
+
+                    if (filePageStoreMgr.hasIndexStore(grpId)) {
+                        defragmentIndexPartition(oldGrpCtx, newGrpCtx, linkMapByPart);
+
+                        idxDfrgFut = defragmentationCheckpoint
+                            .forceCheckpoint("index defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+                    }
+
+                    idxDfrgFut.listen(fut -> {
+                        oldPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx partPageMem = (PageMemoryEx)partDataRegion.pageMemory();
+
+                        partPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().removePageStore(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx mappingPageMem = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+                        pageMgr = (DefragmentationPageReadWriteManager)mappingPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().clear(grpId);
+
+                        renameTempIndexFile(workDir);
+
+                        writeDefragmentationCompletionMarker(filePageStoreMgr.getPageStoreFileIoFactory(), workDir, log);
+
+                        batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+                    });
+                }
+
+                // I guess we should wait for it?
+                if (idxDfrgFut != null)
+                    idxDfrgFut.get();
+            }
+
+            mntcReg.unregisterMaintenanceTask(DEFRAGMENTATION_MNTC_TASK_NAME);
+
+            log.info("Defragmentation completed. All partitions are defragmented.");
+        }
+        finally {
+            defragmentationCheckpoint.stop(true);
+        }
+    }
+
+    /** */
+    public void createIndexPageStore(
+        int grpId,
+        File workDir,
+        FilePageStoreFactory pageStoreFactory,
+        DataRegion partRegion,
+        LongConsumer allocatedTracker
+    ) throws IgniteCheckedException {
+        // Index partition file has to be deleted before we begin, otherwise there's a chance of reading corrupted file.
+        // There is a time period when index is already defragmented but marker file is not created yet. If node is
+        // failed in that time window then index will be deframented once again. That's fine, situation is rare but code
+        // to fix that would add unnecessary complications.
+        U.delete(defragmentedIndexTmpFile(workDir));
+
+        PageStore idxPageStore;
+
+        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+        try {
+            idxPageStore = pageStoreFactory.createPageStore(
+                FLAG_IDX,
+                () -> defragmentedIndexTmpFile(workDir).toPath(),
+                allocatedTracker
+            );
+        }
+        finally {
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+        }
+
+        idxPageStore.sync();
+
+        PageMemoryEx partPageMem = (PageMemoryEx)partRegion.pageMemory();
+
+        DefragmentationPageReadWriteManager partMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+        partMgr.pageStoreMap().addPageStore(grpId, PageIdAllocator.INDEX_PARTITION, idxPageStore);
+    }
+
+    /** */
+    public enum PartStages {
+        START,
+        CP_LOCK,
+        ITERATE,
+        READ_ROW,
+        INSERT_ROW,
+        STORE_MAP,
+        STORE_PK,
+        STORE_PENDING,
+        METADATA
+    }
+
+    /**
+     * Defragmentate partition.
+     *
+     * @param partCtx
+     * @param treeIter
+     * @param offheap
+     * @throws IgniteCheckedException If failed.
+     */
+    private void copyPartitionData(
+        PartitionContext partCtx,
+        TreeIterator treeIter,
+        GridCacheOffheapManager offheap
+    ) throws IgniteCheckedException {
+        partCtx.createNewCacheDataStore(offheap);

Review comment:
       Let's move it outside of this method.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/DefragmentationPageReadWriteManager.java
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageReadWriteManagerImpl;
+
+/** */
+public class DefragmentationPageReadWriteManager extends PageReadWriteManagerImpl {

Review comment:
       Let's move everything here from `PageStoreMap` here? The DefragmentationPageReadWriteManager will also implement the `PageStoreCollection`.

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/defragmentation/IndexingDefragmentation.java
##########
@@ -0,0 +1,467 @@
+/*
+ * 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.processors.query.h2.defragmentation;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.LinkMap;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.TimeTracker;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIoResolver;
+import org.apache.ignite.internal.processors.cache.persistence.tree.util.InsertLast;
+import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccDataRow;
+import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+import org.apache.ignite.internal.processors.query.h2.database.H2Tree;
+import org.apache.ignite.internal.processors.query.h2.database.H2TreeIndex;
+import org.apache.ignite.internal.processors.query.h2.database.InlineIndexColumn;
+import org.apache.ignite.internal.processors.query.h2.database.inlinecolumn.AbstractInlineIndexColumn;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2ExtrasInnerIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2ExtrasLeafIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2InnerIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2LeafIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.H2RowLinkIO;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.processors.query.h2.opt.H2CacheRow;
+import org.apache.ignite.internal.processors.query.h2.opt.H2Row;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.h2.index.Index;
+import org.h2.value.Value;
+
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.INIT_TREE;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.ITERATE;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.READ_MAP;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.READ_ROW;
+
+/**
+ *
+ */
+public class IndexingDefragmentation {
+    /** Indexing. */
+    private final IgniteH2Indexing indexing;
+
+    /** Constructor. */
+    public IndexingDefragmentation(IgniteH2Indexing indexing) {
+        this.indexing = indexing;
+    }
+
+    /** */
+    public enum IndexStages {
+        START,
+        CP_LOCK,
+        INIT_TREE,
+        ITERATE,
+        READ_ROW,
+        READ_MAP,
+        INSERT_ROW
+    }
+
+    /**
+     * Defragment index partition.
+     *
+     * @param grpCtx Old group context.
+     * @param newCtx New group context.
+     * @param partPageMem Partition page memory.
+     * @param mappingByPartition Mapping page memory.
+     * @param cpLock Defragmentation checkpoint read lock.
+     * @param log Log.
+     *
+     * @throws IgniteCheckedException If failed.
+     */
+    public void defragment(
+        CacheGroupContext grpCtx,
+        CacheGroupContext newCtx,
+        PageMemoryEx partPageMem,
+        IntMap<LinkMap> mappingByPartition,
+        CheckpointTimeoutLock cpLock,
+        IgniteLogger log
+    ) throws IgniteCheckedException {
+        int pageSize = grpCtx.cacheObjectContext().kernalContext().grid().configuration().getDataStorageConfiguration().getPageSize();
+
+        TreeIterator treeIterator = new TreeIterator(pageSize);
+
+        PageMemoryEx oldCachePageMem = (PageMemoryEx)grpCtx.dataRegion().pageMemory();
+
+        PageMemory newCachePageMemory = partPageMem;
+
+        Collection<GridH2Table> tables = indexing.schemaManager().dataTables();
+
+        long cpLockThreshold = 150L;
+
+        TimeTracker<IndexStages> tracker = new TimeTracker<>(IndexStages.class);
+
+        cpLock.checkpointReadLock();
+        tracker.complete(CP_LOCK);
+
+        try {
+            AtomicLong lastCpLockTs = new AtomicLong(System.currentTimeMillis());
+
+            for (GridH2Table table : tables) {
+                GridCacheContext<?, ?> cctx = table.cacheContext();
+
+                if (cctx.groupId() != grpCtx.groupId())
+                    continue; // Not our index.
+
+                GridH2RowDescriptor rowDesc = table.rowDescriptor();
+
+                List<Index> indexes = table.getIndexes();
+                H2TreeIndex oldH2Idx = (H2TreeIndex)indexes.get(2);
+
+                int segments = oldH2Idx.segmentsCount();
+
+                H2Tree firstTree = oldH2Idx.treeForRead(0);
+
+                PageIoResolver pageIoRslvr = pageAddr -> {
+                    PageIO io = PageIoResolver.DEFAULT_PAGE_IO_RESOLVER.resolve(pageAddr);
+
+                    if (io instanceof BPlusMetaIO)
+                        return io;
+
+                    //noinspection unchecked,rawtypes,rawtypes
+                    return wrap((BPlusIO)io);
+                };
+
+                //TODO Create new proper GridCacheContext for it?

Review comment:
       Let's fix it.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
##########
@@ -487,4 +493,25 @@ default long indexSize(String schemaName, String tblName, String idxName) throws
     default Map<String, Integer> secondaryIndexesInlineSize() {
         return Collections.emptyMap();
     }
+
+    /**
+     * Defragment index partition.
+     *
+     * @param grpCtx Old group context.
+     * @param newCtx New group context.
+     * @param partPageMem Partition page memory.
+     * @param mappingByPart Mapping page memory.
+     * @param cpLock Defragmentation checkpoint read lock.
+     * @param log Log.
+     *
+     * @throws IgniteCheckedException If failed.
+     */
+    void defragment(
+        CacheGroupContext grpCtx,
+        CacheGroupContext newCtx,
+        PageMemoryEx partPageMem,
+        IntMap<LinkMap> mappingByPart,
+        CheckpointTimeoutLock cpLock,

Review comment:
       `cpLock` and `log` should be part of API. You can obtain it from the `context` in implementation.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,861 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.lang.IgniteOutClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.

Review comment:
       Let's 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.

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r529641717



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;
+
+                                if (log.isDebugEnabled()) {
+                                    log.debug(S.toString(
+                                        "Partition defragmented",
+                                        "grpId", grpId, false,
+                                        "partId", partId, false,
+                                        "oldPages", oldPageStore.pages(), false,
+                                        "newPages", partCtx.partPagesAllocated.get() + 1, false,
+                                        "mappingPages", partCtx.mappingPagesAllocated.get() + 1, false,
+                                        "pageSize", pageSize, false,
+                                        "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                        "workDir", workDir, false
+                                    ));
+                                }
+
+                                oldPageMem.invalidate(grpId, partId);
+
+                                partCtx.partPageMemory.invalidate(grpId, partId);
+
+                                DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                                pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                                try {
+                                    renameTempPartitionFile(workDir, partId);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    throw new IgniteException(e);
+                                }
+                            }
+                        };
+
+                        GridFutureAdapter<?> cpFut = defragmentationCheckpoint
+                            .forceCheckpoint("partition defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+
+                        cpFut.listen(cpLsnr);
+
+                        cmpFut.add((IgniteInternalFuture<Object>)cpFut);
+                    }
+
+                    // A bit too general for now, but I like it more then saving only the last checkpoint future.
+                    cmpFut.markInitialized().get();
+
+                    idxDfrgFut = new GridFinishedFuture<>();
+
+                    if (filePageStoreMgr.hasIndexStore(grpId)) {
+                        defragmentIndexPartition(oldGrpCtx, newGrpCtx, linkMapByPart);
+
+                        idxDfrgFut = defragmentationCheckpoint
+                            .forceCheckpoint("index defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+                    }
+
+                    idxDfrgFut.listen(fut -> {
+                        oldPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx partPageMem = (PageMemoryEx)partDataRegion.pageMemory();
+
+                        partPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().removePageStore(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx mappingPageMem = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+                        pageMgr = (DefragmentationPageReadWriteManager)mappingPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().clear(grpId);
+
+                        try {
+                            renameTempIndexFile(workDir);
+
+                            writeDefragmentationCompletionMarker(filePageStoreMgr.getPageStoreFileIoFactory(), workDir, log);
+
+                            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    });
+                }
+
+                // I guess we should wait for it?
+                if (idxDfrgFut != null)
+                    idxDfrgFut.get();
+            }
+
+            mntcReg.unregisterMaintenanceTask(DEFRAGMENTATION_MNTC_TASK_NAME);
+
+            log.info("Defragmentation completed. All partitions are defragmented.");
+        }
+        finally {
+            defragmentationCheckpoint.stop(true);
+        }
+    }
+
+    /** */
+    public void createIndexPageStore(
+        int grpId,
+        File workDir,
+        FilePageStoreFactory pageStoreFactory,
+        DataRegion partRegion,
+        LongConsumer allocatedTracker
+    ) throws IgniteCheckedException {
+        // Index partition file has to be deleted before we begin, otherwise there's a chance of reading corrupted file.
+        // There is a time period when index is already defragmented but marker file is not created yet. If node is
+        // failed in that time window then index will be deframented once again. That's fine, situation is rare but code
+        // to fix that would add unnecessary complications.
+        U.delete(defragmentedIndexTmpFile(workDir));
+
+        PageStore idxPageStore;
+
+        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+        try {
+            idxPageStore = pageStoreFactory.createPageStore(
+                FLAG_IDX,
+                () -> defragmentedIndexTmpFile(workDir).toPath(),
+                allocatedTracker
+            );
+        }
+        finally {
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+        }
+
+        idxPageStore.sync();
+
+        PageMemoryEx partPageMem = (PageMemoryEx)partRegion.pageMemory();
+
+        DefragmentationPageReadWriteManager partMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+        partMgr.pageStoreMap().addPageStore(grpId, PageIdAllocator.INDEX_PARTITION, idxPageStore);
+    }
+
+    /** */
+    public enum PartStages {
+        START,
+        CP_LOCK,
+        ITERATE,
+        READ_ROW,
+        INSERT_ROW,
+        STORE_MAP,
+        STORE_PK,
+        STORE_PENDING,
+        METADATA
+    }
+
+    /**
+     * Defragmentate partition.
+     *
+     * @param partCtx
+     * @param treeIter
+     * @throws IgniteCheckedException If failed.
+     */
+    private void copyPartitionData(
+        PartitionContext partCtx,
+        TreeIterator treeIter,
+        GridSpinBusyLock busyLock
+    ) throws IgniteCheckedException {
+        partCtx.createNewCacheDataStore(busyLock);
+
+        CacheDataTree tree = partCtx.oldCacheDataStore.tree();
+
+        CacheDataTree newTree = partCtx.newCacheDataStore.tree();
+        PendingEntriesTree newPendingTree = partCtx.newCacheDataStore.pendingTree();
+        AbstractFreeList<CacheDataRow> freeList = partCtx.newCacheDataStore.getCacheStoreFreeList();
+
+        long cpLockThreshold = 150L;
+
+        TimeTracker<PartStages> tracker = new TimeTracker<>(PartStages.class);
+
+        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+        tracker.complete(CP_LOCK);
+
+        try {
+            AtomicLong lastCpLockTs = new AtomicLong(System.currentTimeMillis());
+            AtomicInteger entriesProcessed = new AtomicInteger();
+
+            treeIter.iterate(tree, partCtx.cachePageMemory, (tree0, io, pageAddr, idx) -> {
+                tracker.complete(ITERATE);
+
+                if (System.currentTimeMillis() - lastCpLockTs.get() >= cpLockThreshold) {
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+                    tracker.complete(CP_LOCK);
+
+                    lastCpLockTs.set(System.currentTimeMillis());
+                }
+
+                AbstractDataLeafIO leafIo = (AbstractDataLeafIO)io;
+                CacheDataRow row = tree.getRow(io, pageAddr, idx);
+
+                tracker.complete(READ_ROW);
+
+                int cacheId = row.cacheId();
+
+                // Reuse row that we just read.
+                row.link(0);
+
+                // "insertDataRow" will corrupt page memory if we don't do this.
+                if (row instanceof DataRow && !partCtx.oldGrpCtx.storeCacheIdInDataPage())
+                    ((DataRow)row).cacheId(CU.UNDEFINED_CACHE_ID);
+
+                freeList.insertDataRow(row, IoStatisticsHolderNoOp.INSTANCE);
+
+                // Put it back.
+                if (row instanceof DataRow)
+                    ((DataRow)row).cacheId(cacheId);
+
+                tracker.complete(INSERT_ROW);
+
+                newTree.putx(row);
+
+                long newLink = row.link();
+
+                tracker.complete(STORE_MAP);
+
+                partCtx.linkMap.put(leafIo.getLink(pageAddr, idx), newLink);
+
+                tracker.complete(STORE_PK);
+
+                if (row.expireTime() != 0)
+                    newPendingTree.putx(new PendingRow(cacheId, row.expireTime(), newLink));
+
+                tracker.complete(STORE_PENDING);
+
+                entriesProcessed.incrementAndGet();
+
+                return true;
+            });
+
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+            tracker.complete(CP_LOCK);
+
+            freeList.saveMetadata(IoStatisticsHolderNoOp.INSTANCE);
+
+            copyCacheMetadata(partCtx);
+
+            tracker.complete(METADATA);
+        }
+        finally {
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+        }
+
+        if (log.isDebugEnabled()) {
+            log.debug(
+                "Partition defragmentation timings for cache group " + partCtx.grpId +
+                " and partition " + partCtx.partId + ": " + tracker.toString()
+            );
+        }
+    }
+
+    /** */
+    private void copyCacheMetadata(
+        PartitionContext partCtx
+    ) throws IgniteCheckedException {
+        // Same for all page memories. Why does it need to be in PageMemory?
+        long partMetaPageId = partCtx.cachePageMemory.partitionMetaPageId(partCtx.grpId, partCtx.partId);
+
+        access(ACCESS_READ, partCtx.cachePageMemory, partCtx.grpId, partMetaPageId, oldPartMetaPageAddr -> {
+            PagePartitionMetaIO oldPartMetaIo = PageIO.getPageIO(oldPartMetaPageAddr);
+
+            // Newer meta versions may contain new data that we don't copy during defragmentation.
+            assert Arrays.asList(1, 2, 3).contains(oldPartMetaIo.getVersion())
+                : "IO version " + oldPartMetaIo.getVersion() + " is not supported by current defragmentation algorithm." +
+                " Please implement copying of all data added in new version.";
+
+            access(ACCESS_WRITE, partCtx.partPageMemory, partCtx.grpId, partMetaPageId, newPartMetaPageAddr -> {
+                PagePartitionMetaIOV3 newPartMetaIo = PageIO.getPageIO(newPartMetaPageAddr);
+
+                // Copy partition state.
+                byte partState = oldPartMetaIo.getPartitionState(oldPartMetaPageAddr);
+                newPartMetaIo.setPartitionState(newPartMetaPageAddr, partState);
+
+                // Copy cache size for single cache group.
+                long size = oldPartMetaIo.getSize(oldPartMetaPageAddr);
+                newPartMetaIo.setSize(newPartMetaPageAddr, size);
+
+                // Copy update counter value.
+                long updateCntr = oldPartMetaIo.getUpdateCounter(oldPartMetaPageAddr);
+                newPartMetaIo.setUpdateCounter(newPartMetaPageAddr, updateCntr);
+
+                // Copy global remove Id.
+                long rmvId = oldPartMetaIo.getGlobalRemoveId(oldPartMetaPageAddr);
+                newPartMetaIo.setGlobalRemoveId(newPartMetaPageAddr, rmvId);
+
+                // Copy cache sizes for shared cache group.
+                long oldCountersPageId = oldPartMetaIo.getCountersPageId(oldPartMetaPageAddr);
+                if (oldCountersPageId != 0L) {
+                    Map<Integer, Long> sizes = GridCacheOffheapManager.readSharedGroupCacheSizes(
+                        partCtx.cachePageMemory,
+                        partCtx.grpId,
+                        oldCountersPageId
+                    );
+
+                    long newCountersPageId = GridCacheOffheapManager.writeSharedGroupCacheSizes(
+                        partCtx.partPageMemory,
+                        partCtx.grpId,
+                        0L,
+                        partCtx.partId,
+                        sizes
+                    );
+
+                    newPartMetaIo.setCountersPageId(newPartMetaPageAddr, newCountersPageId);
+                }
+
+                // Copy counter gaps.
+                long oldGapsLink = oldPartMetaIo.getGapsLink(oldPartMetaPageAddr);
+                if (oldGapsLink != 0L) {
+                    byte[] gapsBytes = partCtx.oldCacheDataStore.partStorage().readRow(oldGapsLink);
+
+                    SimpleDataRow gapsDataRow = new SimpleDataRow(partCtx.partId, gapsBytes);
+
+                    partCtx.newCacheDataStore.partStorage().insertDataRow(gapsDataRow, IoStatisticsHolderNoOp.INSTANCE);
+
+                    newPartMetaIo.setGapsLink(newPartMetaPageAddr, gapsDataRow.link());
+                }
+
+                // Encryption stuff.
+                newPartMetaIo.setEncryptedPageCount(newPartMetaPageAddr, 0);
+                newPartMetaIo.setEncryptedPageIndex(newPartMetaPageAddr, 0);
+
+                return null;
+            });
+
+            return null;
+        });
+    }
+
+    /**
+     * Defragmentate indexing partition.
+     *
+     * @param grpCtx
+     * @param mappingByPartition
+     *
+     * @throws IgniteCheckedException If failed.
+     */
+    private void defragmentIndexPartition(
+        CacheGroupContext grpCtx,
+        CacheGroupContext newCtx,
+        IntMap<LinkMap> mappingByPartition
+    ) throws IgniteCheckedException {
+        GridQueryProcessor query = grpCtx.caches().get(0).kernalContext().query();
+
+        if (!query.moduleEnabled())
+            return;
+
+        final GridQueryIndexing idx = query.getIndexing();
+
+        CheckpointTimeoutLock cpLock = defragmentationCheckpoint.checkpointTimeoutLock();
+
+        idx.defragmentator().defragment(
+            grpCtx,
+            newCtx,
+            (PageMemoryEx)partDataRegion.pageMemory(),
+            mappingByPartition,
+            cpLock,
+            log
+        );
+    }
+
+    /** */
+    @SuppressWarnings("PublicField")
+    private class PartitionContext {
+        /** */
+        public final File workDir;
+
+        /** */
+        public final int grpId;
+
+        /** */
+        public final int partId;
+
+        /** */
+        public final DataRegion cacheDataRegion;
+
+        /** */
+        public final PageMemoryEx cachePageMemory;
+
+        /** */
+        public final PageMemoryEx partPageMemory;
+
+        /** */
+        public final PageMemoryEx mappingPageMemory;
+
+        /** */
+        public final CacheGroupContext oldGrpCtx;
+
+        /** */
+        public final CacheGroupContext newGrpCtx;
+
+        /** */
+        public final CacheDataStore oldCacheDataStore;
+
+        /** */
+        private GridCacheDataStore newCacheDataStore;
+
+        /** */
+        public final FilePageStoreFactory pageStoreFactory;
+
+        /** */
+        public final AtomicLong partPagesAllocated = new AtomicLong();
+
+        /** */
+        public final AtomicLong mappingPagesAllocated = new AtomicLong();
+
+        /** */
+        private LinkMap linkMap;
+
+        /** */
+        public PartitionContext(
+            File workDir,
+            int grpId,
+            int partId,
+            DataRegion partDataRegion,
+            DataRegion mappingDataRegion,
+            CacheGroupContext oldGrpCtx,
+            CacheGroupContext newGrpCtx,
+            CacheDataStore oldCacheDataStore,
+            FilePageStoreFactory pageStoreFactory
+        ) {
+            this.workDir = workDir;
+            this.grpId = grpId;
+            this.partId = partId;
+            cacheDataRegion = oldGrpCtx.dataRegion();
+
+            cachePageMemory = (PageMemoryEx)cacheDataRegion.pageMemory();
+            partPageMemory = (PageMemoryEx)partDataRegion.pageMemory();
+            mappingPageMemory = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+            this.oldGrpCtx = oldGrpCtx;
+            this.newGrpCtx = newGrpCtx;
+            this.oldCacheDataStore = oldCacheDataStore;
+            this.pageStoreFactory = pageStoreFactory;
+        }
+
+        /** */
+        public PageStore createPartPageStore() throws IgniteCheckedException {
+            PageStore partPageStore;
+
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+            try {
+                partPageStore = pageStoreFactory.createPageStore(
+                    FLAG_DATA,
+                    () -> defragmentedPartTmpFile(workDir, partId).toPath(),
+                    partPagesAllocated::addAndGet
+                );
+            }
+            finally {
+                defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+            }
+
+            partPageStore.sync();
+
+            DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMemory.pageManager();
+
+            pageMgr.pageStoreMap().addPageStore(grpId, partId, partPageStore);
+
+            return partPageStore;
+        }
+
+        /** */
+        public PageStore createMappingPageStore() throws IgniteCheckedException {
+            PageStore mappingPageStore;
+
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+            try {
+                mappingPageStore = pageStoreFactory.createPageStore(
+                    FLAG_DATA,
+                    () -> defragmentedPartMappingFile(workDir, partId).toPath(),
+                    mappingPagesAllocated::addAndGet
+                );
+            }
+            finally {
+                defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+            }
+
+            mappingPageStore.sync();
+
+            DefragmentationPageReadWriteManager partMgr = (DefragmentationPageReadWriteManager)mappingPageMemory.pageManager();
+
+            partMgr.pageStoreMap().addPageStore(grpId, partId, mappingPageStore);
+
+            return mappingPageStore;
+        }
+
+        /** */
+        public LinkMap createLinkMapTree(boolean initNew) throws IgniteCheckedException {
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+            //TODO Store link in meta page and remove META_PAGE_IDX constant?
+            try {
+                long mappingMetaPageId = initNew
+                    ? mappingPageMemory.allocatePage(grpId, partId, FLAG_DATA)
+                    : PageIdUtils.pageId(partId, FLAG_DATA, LinkMap.META_PAGE_IDX);
+
+                assert PageIdUtils.pageIndex(mappingMetaPageId) == LinkMap.META_PAGE_IDX
+                    : PageIdUtils.toDetailString(mappingMetaPageId);
+
+                linkMap = new LinkMap(newGrpCtx, mappingPageMemory, mappingMetaPageId, initNew);
+            }
+            finally {
+                defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+            }
+
+            return linkMap;
+        }
+
+        /** */
+        public void createNewCacheDataStore(GridSpinBusyLock busyLock) {

Review comment:
       From my understanding 'context' means 'sharing state/references'. For instance, I haven't seen any such fabric methods inside `IgniteKernalContext`. I think we should choose the right place for such actions - OffheapManager seems to be the right place.




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

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



[GitHub] [ignite] agoncharuk commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
agoncharuk commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r524330324



##########
File path: modules/core/src/main/java/org/apache/ignite/configuration/DefragmentationConfiguration.java
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.configuration;
+
+import java.io.Serializable;
+
+/**
+ *
+ */
+public final class DefragmentationConfiguration implements Serializable {
+    /** Serial version uid. */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     *
+     */
+    public static final long DFLT_MAPPIN_REGION_SIZE = 256L * 1024 * 1024;
+
+    /**
+     *
+     */
+    private long regionSize = DataStorageConfiguration.DFLT_DATA_REGION_MAX_SIZE;
+
+    /**
+     *
+     */
+    private long mappingRegionSize = DFLT_MAPPIN_REGION_SIZE;

Review comment:
       Can we calculate defragmentation regions automatically? For example, the mapping region size can be calculated as max(size of the cache) * C (we store a link->link mapping, so the region size can be estimated pretty much accurately). 
   For the defragmentation region size, we can calculate DS = Sum(all data region sizes in the node configuration). Then, assuming defragmentation runs sequentially per cache/cache group, we can split the DS proportionally between the original cache data region, the defragmentation data region, and mapping data region.
   
   If this is something viable, I would move these configuration parameters to system properties so that user does not have to configure anything at all.




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r530216036



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsDefragmentationTest.java
##########
@@ -0,0 +1,539 @@
+/*
+ * 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.processors.cache.persistence;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.FileVisitResult;
+import java.nio.file.FileVisitor;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardCopyOption;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.Collections;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+import java.util.function.UnaryOperator;
+import java.util.stream.IntStream;
+import javax.cache.configuration.Factory;
+import javax.cache.expiry.Duration;
+import javax.cache.expiry.ExpiryPolicy;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.FailureHandler;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.maintenance.MaintenanceFileStore;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.util.lang.IgniteThrowableConsumer;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentationCompletionMarkerFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.maintenance.DefragmentationParameters.toStore;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
+/** */
+public class IgnitePdsDefragmentationTest extends GridCommonAbstractTest {

Review comment:
       This is the same as regular failover tests that throw arbitrary IOException objects in the middle of the operation.




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r527618647



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/DefragmentationFileUtils.java
##########
@@ -0,0 +1,400 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static java.nio.file.StandardCopyOption.ATOMIC_MOVE;
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static java.nio.file.StandardOpenOption.CREATE_NEW;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.FILE_SUFFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_TEMPLATE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.TMP_SUFFIX;
+
+/**
+ * Everything related to file management during defragmentation process.
+ */
+public class DefragmentationFileUtils {
+    /** Prefix for link mapping files. */
+    private static final String DFRG_LINK_MAPPING_FILE_PREFIX = PART_FILE_PREFIX + "map-";
+
+    /** Link mapping file template. */
+    private static final String DFRG_LINK_MAPPING_FILE_TEMPLATE = DFRG_LINK_MAPPING_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmentation complation marker file name. */
+    private static final String DFRG_COMPLETION_MARKER_FILE_NAME = "dfrg-completion-marker";
+
+    /** Name of defragmentated index partition file. */
+    private static final String DFRG_INDEX_FILE_NAME = INDEX_FILE_PREFIX + "-dfrg" + FILE_SUFFIX;
+
+    /** Name of defragmentated index partition temporary file. */
+    private static final String DFRG_INDEX_TMP_FILE_NAME = DFRG_INDEX_FILE_NAME + TMP_SUFFIX;
+
+    /** Prefix for defragmented partition files. */
+    private static final String DFRG_PARTITION_FILE_PREFIX = PART_FILE_PREFIX + "dfrg-";
+
+    /** Defragmented partition file template. */
+    private static final String DFRG_PARTITION_FILE_TEMPLATE = DFRG_PARTITION_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmented partition temp file template. */
+    private static final String DFRG_PARTITION_TMP_FILE_TEMPLATE = DFRG_PARTITION_FILE_TEMPLATE + TMP_SUFFIX;
+
+    /**
+     * Performs cleanup of work dir before initializing file page stores.
+     * Will finish batch renaming if defragmentation was completed or delete garbage if it wasn't.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     */
+    public static void beforeInitPageStores(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+        U.delete(defragmentationCompletionMarkerFile(workDir));
+
+        for (File file : workDir.listFiles()) {
+            String fileName = file.getName();
+
+            if (
+                fileName.startsWith(DFRG_PARTITION_FILE_PREFIX)
+                || fileName.startsWith(DFRG_INDEX_FILE_NAME)
+                || fileName.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)
+            )
+                U.delete(file);
+        }
+    }
+
+    /**
+     * Checks whether cache group defragmentation completed or not. Completes it if all that's left is renaming.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param log Logger to write messages.
+     * @return {@code true} if given cache group is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentationCompletionMarkerFile(File)
+     */
+    public static boolean skipAlreadyDefragmentedCacheGroup(File workDir, int grpId, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (completionMarkerFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented page group",
+                    "grpId", grpId, false,
+                    "markerFileName", completionMarkerFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+            return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * Checks whether partition has already been defragmented or not. Cleans corrupted data if previous failed
+     * defragmentation attempt was found.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param partId Partionion index to check.
+     * @param log Logger to write messages.
+     * @return {@code true} if given partition is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartMappingFile(File, int)
+     */
+    public static boolean skipAlreadyDefragmentedPartition(File workDir, int grpId, int partId, IgniteLogger log) throws IgniteCheckedException {
+        File defragmentedPartFile = defragmentedPartFile(workDir, partId);
+        File defragmentedPartMappingFile = defragmentedPartMappingFile(workDir, partId);
+
+        if (defragmentedPartFile.exists() && defragmentedPartMappingFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented partition",
+                    "grpId", grpId, false,
+                    "partId", partId, false,
+                    "partFileName", defragmentedPartFile.getName(), false,
+                    "mappingFileName", defragmentedPartMappingFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            return true;
+        }
+
+        File defragmentedPartTmpFile = defragmentedPartTmpFile(workDir, partId);
+
+        try {
+            Files.deleteIfExists(defragmentedPartTmpFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartMappingFile.toPath());
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+
+        return false;
+    }
+
+    /**
+     * Failure-tolerant batch rename of defragmented partition files.
+     *
+     * Deletes all link mapping files old partition and index files, renaming defragmentated files in the process. Can
+     * be run on the same folder multiple times if failed for some reason.
+     *
+     * Does something only if completion marker is present in the folder. This marker won't be deleted in the end.
+     * Deletion of the marker must be done outside of defragmentation mode to prevent cache groups to be defragmentated
+     * several times in case of failures.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#writeDefragmentationCompletionMarker(FileIOFactory, File, IgniteLogger)
+     */
+    public static void batchRenameDefragmentedCacheGroupPartitions(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (!completionMarkerFile.exists())
+            return;
+
+        try {
+            for (File mappingFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)))
+                Files.delete(mappingFile.toPath());
+
+            for (File partFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_PARTITION_FILE_PREFIX))) {
+                int partId = extractPartId(partFile.getName());
+
+                File oldPartFile = new File(workDir, String.format(PART_FILE_TEMPLATE, partId));
+
+                Files.move(partFile.toPath(), oldPartFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+
+            File idxFile = new File(workDir, DFRG_INDEX_FILE_NAME);
+
+            if (idxFile.exists()) {
+                File oldIdxFile = new File(workDir, INDEX_FILE_NAME);
+
+                Files.move(idxFile.toPath(), oldIdxFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Extracts partition number from file names like {@code part-dfrg-%d.bin}.
+     *
+     * @param dfrgPartFileName Defragmented partition file name.
+     * @return Partition index.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    private static int extractPartId(String dfrgPartFileName) {
+        assert dfrgPartFileName.startsWith(DFRG_PARTITION_FILE_PREFIX) : dfrgPartFileName;
+        assert dfrgPartFileName.endsWith(FILE_SUFFIX) : dfrgPartFileName;
+
+        String partIdStr = dfrgPartFileName.substring(
+            DFRG_PARTITION_FILE_PREFIX.length(),
+            dfrgPartFileName.length() - FILE_SUFFIX.length()
+        );
+
+        return Integer.parseInt(partIdStr);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin.tmp} in given folder. It will be used for storing defragmented index
+     * partition during the process.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexFile(File)
+     */
+    public static File defragmentedIndexTmpFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_TMP_FILE_NAME);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin} in given folder. It will be used for storing defragmented index
+     * partition when the process is over.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexTmpFile(File)
+     */
+    public static File defragmentedIndexFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_FILE_NAME);
+    }
+
+    /**
+     * Rename temporary index defragmenation file to a finalized one.
+     *
+     * @param workDir Cache group working directory.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexTmpFile(File)
+     * @see DefragmentationFileUtils#defragmentedIndexFile(File)
+     */
+    public static void renameTempIndexFile(File workDir) throws IgniteCheckedException {
+        File defragmentedIdxTmpFile = defragmentedIndexTmpFile(workDir);
+        File defragmentedIdxFile = defragmentedIndexFile(workDir);
+
+        try {
+            Files.move(defragmentedIdxTmpFile.toPath(), defragmentedIdxFile.toPath(), ATOMIC_MOVE);
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Return file named {@code part-dfrg-%d.bin.tmp} in given folder. It will be used for storing defragmented data
+     * partition during the process.
+     *
+     * @param workDir Cache group working directory.
+     * @param partId Parition index, will be substituted into file name.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    public static File defragmentedPartTmpFile(File workDir, int partId) {
+        return new File(workDir, String.format(DFRG_PARTITION_TMP_FILE_TEMPLATE, partId));
+    }
+
+    /**
+     * Return file named {@code part-dfrg-%d.bin} in given folder. It will be used for storing defragmented data
+     * partition when the process is over.
+     *
+     * @param workDir Cache group working directory.
+     * @param partId Parition index, will be substituted into file name.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     */
+    public static File defragmentedPartFile(File workDir, int partId) {
+        return new File(workDir, String.format(DFRG_PARTITION_FILE_TEMPLATE, partId));
+    }
+
+    /**
+     * Rename temporary partition defragmenation file to a finalized one.

Review comment:
       Done




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r524946751



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,887 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database schared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;
+
+                                log.info(S.toString(
+                                    "Partition defragmented",
+                                    "grpId", grpId, false,
+                                    "partId", partId, false,
+                                    "oldPages", oldPageStore.pages(), false,
+                                    "newPages", partCtx.partPagesAllocated.get(), false,
+                                    "bytesSaved", (oldPageStore.pages() - partCtx.partPagesAllocated.get()) * pageSize, false,
+                                    "mappingPages", partCtx.mappingPagesAllocated.get(), false,
+                                    "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                    "workDir", workDir, false
+                                ));
+
+                                oldPageMem.invalidate(grpId, partId);
+
+                                partCtx.partPageMemory.invalidate(grpId, partId);
+
+                                DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                                pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                                try {
+                                    renameTempPartitionFile(workDir, partId);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    throw new IgniteException(e);
+                                }
+                            }
+                        };
+
+                        GridFutureAdapter<?> cpFut = defragmentationCheckpoint
+                            .forceCheckpoint("partition defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+
+                        cpFut.listen(cpLsnr);
+
+                        cmpFut.add((IgniteInternalFuture<Object>)cpFut);
+                    }
+
+                    // A bit too general for now, but I like it more then saving only the last checkpoint future.
+                    cmpFut.markInitialized().get();
+
+                    idxDfrgFut = new GridFinishedFuture<>();
+
+                    if (filePageStoreMgr.hasIndexStore(grpId)) {
+                        defragmentIndexPartition(oldGrpCtx, newGrpCtx, linkMapByPart);
+
+                        idxDfrgFut = defragmentationCheckpoint
+                            .forceCheckpoint("index defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+                    }
+
+                    idxDfrgFut.listen(fut -> {
+                        oldPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx partPageMem = (PageMemoryEx)partDataRegion.pageMemory();
+
+                        partPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().removePageStore(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx mappingPageMem = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+                        pageMgr = (DefragmentationPageReadWriteManager)mappingPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().clear(grpId);
+
+                        try {
+                            renameTempIndexFile(workDir);
+
+                            writeDefragmentationCompletionMarker(filePageStoreMgr.getPageStoreFileIoFactory(), workDir, log);
+
+                            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    });
+                }
+
+                // I guess we should wait for it?
+                if (idxDfrgFut != null)
+                    idxDfrgFut.get();
+            }
+
+            mntcReg.unregisterMaintenanceTask(DEFRAGMENTATION_MNTC_TASK_NAME);
+
+            log.info("Defragmentation completed. All partitions are defragmented.");
+        }
+        finally {
+            defragmentationCheckpoint.stop(true);
+        }
+    }
+
+    /** */
+    public void createIndexPageStore(
+        int grpId,
+        File workDir,
+        FilePageStoreFactory pageStoreFactory,
+        DataRegion partRegion,
+        LongConsumer allocatedTracker
+    ) throws IgniteCheckedException {
+        // Index partition file has to be deleted before we begin, otherwise there's a chance of reading corrupted file.
+        // There is a time period when index is already defragmented but marker file is not created yet. If node is
+        // failed in that time window then index will be deframented once again. That's fine, situation is rare but code
+        // to fix that would add unnecessary complications.
+        U.delete(defragmentedIndexTmpFile(workDir));
+
+        PageStore idxPageStore;
+
+        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+        try {
+            idxPageStore = pageStoreFactory.createPageStore(
+                FLAG_IDX,
+                () -> defragmentedIndexTmpFile(workDir).toPath(),
+                allocatedTracker
+            );
+        }
+        finally {
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+        }
+
+        idxPageStore.sync();
+
+        PageMemoryEx partPageMem = (PageMemoryEx)partRegion.pageMemory();
+
+        DefragmentationPageReadWriteManager partMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+        partMgr.pageStoreMap().addPageStore(grpId, PageIdAllocator.INDEX_PARTITION, idxPageStore);
+    }
+
+    /**
+     * Cancel the process of defragmentation.
+     */
+    public void cancel(){
+
+    }
+
+    /** */
+    public enum PartStages {
+        START,
+        CP_LOCK,
+        ITERATE,
+        READ_ROW,
+        INSERT_ROW,
+        STORE_MAP,
+        STORE_PK,
+        STORE_PENDING,
+        METADATA
+    }
+
+    /**
+     * Defragmentate partition.
+     *
+     * @param partCtx
+     * @param treeIter
+     * @throws IgniteCheckedException If failed.
+     */
+    private void copyPartitionData(
+        PartitionContext partCtx,
+        TreeIterator treeIter,
+        GridSpinBusyLock busyLock
+    ) throws IgniteCheckedException {
+        partCtx.createNewCacheDataStore(busyLock);
+
+        CacheDataTree tree = partCtx.oldCacheDataStore.tree();
+
+        CacheDataTree newTree = partCtx.newCacheDataStore.tree();
+        PendingEntriesTree newPendingTree = partCtx.newCacheDataStore.pendingTree();
+        AbstractFreeList<CacheDataRow> freeList = partCtx.newCacheDataStore.getCacheStoreFreeList();
+
+        long cpLockThreshold = 250L;
+
+        TimeTracker<PartStages> tracker = new TimeTracker<>(PartStages.class);
+
+        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+        tracker.complete(CP_LOCK);
+
+        try {
+            AtomicLong lastCpLockTs = new AtomicLong(System.currentTimeMillis());
+            AtomicInteger entriesProcessed = new AtomicInteger();
+
+            treeIter.iterate(tree, partCtx.cachePageMemory, (tree0, io, pageAddr, idx) -> {
+                tracker.complete(ITERATE);
+
+                if (System.currentTimeMillis() - lastCpLockTs.get() >= cpLockThreshold) {
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+                    tracker.complete(CP_LOCK);
+
+                    lastCpLockTs.set(System.currentTimeMillis());
+                }
+
+                AbstractDataLeafIO leafIo = (AbstractDataLeafIO)io;
+                CacheDataRow row = tree.getRow(io, pageAddr, idx);
+
+                tracker.complete(READ_ROW);
+
+                int cacheId = row.cacheId();
+
+                // Reuse row that we just read.
+                row.link(0);
+
+                // "insertDataRow" will corrupt page memory if we don't do this.
+                if (row instanceof DataRow && !partCtx.oldGrpCtx.storeCacheIdInDataPage())
+                    ((DataRow)row).cacheId(CU.UNDEFINED_CACHE_ID);
+
+                freeList.insertDataRow(row, IoStatisticsHolderNoOp.INSTANCE);
+
+                // Put it back.
+                if (row instanceof DataRow)
+                    ((DataRow)row).cacheId(cacheId);
+
+                tracker.complete(INSERT_ROW);
+
+                newTree.putx(row);
+
+                long newLink = row.link();
+
+                tracker.complete(STORE_MAP);
+
+                partCtx.linkMap.put(leafIo.getLink(pageAddr, idx), newLink);
+
+                tracker.complete(STORE_PK);
+
+                if (row.expireTime() != 0)
+                    newPendingTree.putx(new PendingRow(cacheId, row.expireTime(), newLink));
+
+                tracker.complete(STORE_PENDING);
+
+                entriesProcessed.incrementAndGet();
+
+                return true;
+            });
+
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+            tracker.complete(CP_LOCK);
+
+            freeList.saveMetadata(IoStatisticsHolderNoOp.INSTANCE);
+
+            copyCacheMetadata(partCtx);
+
+            tracker.complete(METADATA);
+        }
+        finally {
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+        }
+
+        System.out.println(tracker.toString());
+    }
+
+    /** */
+    private void copyCacheMetadata(
+        PartitionContext partCtx
+    ) throws IgniteCheckedException {
+        // Same for all page memories. Why does it need to be in PageMemory?
+        long partMetaPageId = partCtx.cachePageMemory.partitionMetaPageId(partCtx.grpId, partCtx.partId);
+
+        access(ACCESS_READ, partCtx.cachePageMemory, partCtx.grpId, partMetaPageId, oldPartMetaPageAddr -> {
+            PagePartitionMetaIO oldPartMetaIo = PageIO.getPageIO(oldPartMetaPageAddr);
+
+            // Newer meta versions may contain new data that we don't copy during defragmentation.
+            assert Arrays.asList(1, 2, 3).contains(oldPartMetaIo.getVersion()) : oldPartMetaIo.getVersion();

Review comment:
       Done.

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/defragmentation/IndexingDefragmentation.java
##########
@@ -0,0 +1,454 @@
+/*
+ * 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.processors.query.h2.defragmentation;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.GridQueryIndexingDefragmentation;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.LinkMap;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.TimeTracker;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIoResolver;
+import org.apache.ignite.internal.processors.cache.persistence.tree.util.InsertLast;
+import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccDataRow;
+import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+import org.apache.ignite.internal.processors.query.h2.database.H2Tree;
+import org.apache.ignite.internal.processors.query.h2.database.H2TreeIndex;
+import org.apache.ignite.internal.processors.query.h2.database.InlineIndexColumn;
+import org.apache.ignite.internal.processors.query.h2.database.inlinecolumn.AbstractInlineIndexColumn;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2ExtrasInnerIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2ExtrasLeafIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2InnerIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2LeafIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.H2RowLinkIO;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.processors.query.h2.opt.H2CacheRow;
+import org.apache.ignite.internal.processors.query.h2.opt.H2Row;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.h2.index.Index;
+import org.h2.value.Value;
+
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.INIT_TREE;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.ITERATE;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.READ_MAP;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.READ_ROW;
+
+/**
+ *
+ */
+public class IndexingDefragmentation implements GridQueryIndexingDefragmentation {
+    /** Indexing. */
+    private final IgniteH2Indexing indexing;
+
+    /** Constructor. */
+    public IndexingDefragmentation(IgniteH2Indexing indexing) {
+        this.indexing = indexing;
+    }
+
+    /** */
+    public enum IndexStages {
+        START,
+        CP_LOCK,
+        INIT_TREE,
+        ITERATE,
+        READ_ROW,
+        READ_MAP,
+        INSERT_ROW
+    }
+
+    /** {@inheritDoc} */
+    @Override public void defragmentate(
+        CacheGroupContext grpCtx,
+        CacheGroupContext newCtx,
+        PageMemoryEx partPageMem,
+        IntMap<LinkMap> mappingByPartition,
+        CheckpointTimeoutLock cpLock,
+        IgniteLogger log
+    ) throws IgniteCheckedException {
+        int pageSize = grpCtx.cacheObjectContext().kernalContext().grid().configuration().getDataStorageConfiguration().getPageSize();
+
+        TreeIterator treeIterator = new TreeIterator(pageSize);
+
+        PageMemoryEx oldCachePageMem = (PageMemoryEx)grpCtx.dataRegion().pageMemory();
+
+        PageMemory newCachePageMemory = partPageMem;
+
+        Collection<GridH2Table> tables = indexing.schemaManager().dataTables();
+
+        long cpLockThreshold = 250L;
+
+        TimeTracker<IndexStages> tracker = new TimeTracker<>(IndexStages.class);
+
+        cpLock.checkpointReadLock();
+        tracker.complete(CP_LOCK);
+
+        try {
+            AtomicLong lastCpLockTs = new AtomicLong(System.currentTimeMillis());
+
+            for (GridH2Table table : tables) {
+                GridCacheContext<?, ?> cctx = table.cacheContext();
+
+                if (cctx.groupId() != grpCtx.groupId())
+                    continue; // Not our index.
+
+                GridH2RowDescriptor rowDesc = table.rowDescriptor();
+
+                List<Index> indexes = table.getIndexes();
+                H2TreeIndex oldH2Idx = (H2TreeIndex)indexes.get(2);
+
+                int segments = oldH2Idx.segmentsCount();
+
+                H2Tree firstTree = oldH2Idx.treeForRead(0);
+
+                PageIoResolver pageIoRslvr = pageAddr -> {
+                    PageIO io = PageIoResolver.DEFAULT_PAGE_IO_RESOLVER.resolve(pageAddr);
+
+                    if (io instanceof BPlusMetaIO)
+                        return io;
+
+                    //noinspection unchecked,rawtypes,rawtypes
+                    return wrap((BPlusIO)io);
+                };
+
+                //TODO Create new proper GridCacheContext for it?
+                H2TreeIndex newIdx = H2TreeIndex.createIndex(
+                    cctx,
+                    null,
+                    table,
+                    oldH2Idx.getName(),
+                    firstTree.getPk(),
+                    firstTree.getAffinityKey(),
+                    Arrays.asList(firstTree.cols()),
+                    Arrays.asList(firstTree.cols()),
+                    oldH2Idx.inlineSize(),
+                    segments,
+                    newCachePageMemory,
+                    newCtx.offheap(),
+                    pageIoRslvr,
+                    log
+                );
+
+                tracker.complete(INIT_TREE);
+
+                for (int i = 0; i < segments; i++) {
+                    H2Tree tree = oldH2Idx.treeForRead(i);
+
+                    treeIterator.iterate(tree, oldCachePageMem, (theTree, io, pageAddr, idx) -> {
+                        tracker.complete(ITERATE);
+
+                        if (System.currentTimeMillis() - lastCpLockTs.get() >= cpLockThreshold) {
+                            cpLock.checkpointReadUnlock();
+
+                            cpLock.checkpointReadLock();
+                            tracker.complete(CP_LOCK);
+
+                            lastCpLockTs.set(System.currentTimeMillis());
+                        }
+
+                        assert 1 == io.getVersion();

Review comment:
       Done.




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

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r524039648



##########
File path: modules/core/src/main/java/org/apache/ignite/configuration/DefragmentationConfiguration.java
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.configuration;
+
+import java.io.Serializable;
+
+/**
+ *
+ */
+public final class DefragmentationConfiguration implements Serializable {

Review comment:
       Do we need a dedicated `DefragmentationConfiguration`? Can we store all these properties in `DataStorageConfiguration`?




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r525113692



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/DefragmentationFileUtils.java
##########
@@ -0,0 +1,400 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static java.nio.file.StandardCopyOption.ATOMIC_MOVE;
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static java.nio.file.StandardOpenOption.CREATE_NEW;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.FILE_SUFFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_TEMPLATE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.TMP_SUFFIX;
+
+/**
+ * Everything related to file management during defragmentation process.
+ */
+public class DefragmentationFileUtils {
+    /** Prefix for link mapping files. */
+    private static final String DFRG_LINK_MAPPING_FILE_PREFIX = PART_FILE_PREFIX + "map-";
+
+    /** Link mapping file template. */
+    private static final String DFRG_LINK_MAPPING_FILE_TEMPLATE = DFRG_LINK_MAPPING_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmentation complation marker file name. */
+    private static final String DFRG_COMPLETION_MARKER_FILE_NAME = "dfrg-completion-marker";
+
+    /** Name of defragmentated index partition file. */
+    private static final String DFRG_INDEX_FILE_NAME = INDEX_FILE_PREFIX + "-dfrg" + FILE_SUFFIX;
+
+    /** Name of defragmentated index partition temporary file. */
+    private static final String DFRG_INDEX_TMP_FILE_NAME = DFRG_INDEX_FILE_NAME + TMP_SUFFIX;
+
+    /** Prefix for defragmented partition files. */
+    private static final String DFRG_PARTITION_FILE_PREFIX = PART_FILE_PREFIX + "dfrg-";
+
+    /** Defragmented partition file template. */
+    private static final String DFRG_PARTITION_FILE_TEMPLATE = DFRG_PARTITION_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmented partition temp file template. */
+    private static final String DFRG_PARTITION_TMP_FILE_TEMPLATE = DFRG_PARTITION_FILE_TEMPLATE + TMP_SUFFIX;
+
+    /**
+     * Performs cleanup of work dir before initializing file page stores.
+     * Will finish batch renaming if defragmentation was completed or delete garbage if it wasn't.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     */
+    public static void beforeInitPageStores(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+        U.delete(defragmentationCompletionMarkerFile(workDir));
+
+        for (File file : workDir.listFiles()) {
+            String fileName = file.getName();
+
+            if (
+                fileName.startsWith(DFRG_PARTITION_FILE_PREFIX)
+                || fileName.startsWith(DFRG_INDEX_FILE_NAME)
+                || fileName.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)
+            )
+                U.delete(file);
+        }
+    }
+
+    /**
+     * Checks whether cache group defragmentation completed or not. Completes it if all that's left is renaming.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param log Logger to write messages.
+     * @return {@code true} if given cache group is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentationCompletionMarkerFile(File)
+     */
+    public static boolean skipAlreadyDefragmentedCacheGroup(File workDir, int grpId, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (completionMarkerFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented page group",
+                    "grpId", grpId, false,
+                    "markerFileName", completionMarkerFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+            return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * Checks whether partition has already been defragmented or not. Cleans corrupted data if previous failed
+     * defragmentation attempt was found.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param partId Partionion index to check.
+     * @param log Logger to write messages.
+     * @return {@code true} if given partition is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartMappingFile(File, int)
+     */
+    public static boolean skipAlreadyDefragmentedPartition(File workDir, int grpId, int partId, IgniteLogger log) throws IgniteCheckedException {
+        File defragmentedPartFile = defragmentedPartFile(workDir, partId);
+        File defragmentedPartMappingFile = defragmentedPartMappingFile(workDir, partId);
+
+        if (defragmentedPartFile.exists() && defragmentedPartMappingFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented partition",
+                    "grpId", grpId, false,
+                    "partId", partId, false,
+                    "partFileName", defragmentedPartFile.getName(), false,
+                    "mappingFileName", defragmentedPartMappingFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            return true;
+        }
+
+        File defragmentedPartTmpFile = defragmentedPartTmpFile(workDir, partId);
+
+        try {
+            Files.deleteIfExists(defragmentedPartTmpFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartMappingFile.toPath());
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+
+        return false;
+    }
+
+    /**
+     * Failure-tolerant batch rename of defragmented partition files.
+     *
+     * Deletes all link mapping files old partition and index files, renaming defragmentated files in the process. Can
+     * be run on the same folder multiple times if failed for some reason.
+     *
+     * Does something only if completion marker is present in the folder. This marker won't be deleted in the end.
+     * Deletion of the marker must be done outside of defragmentation mode to prevent cache groups to be defragmentated
+     * several times in case of failures.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#writeDefragmentationCompletionMarker(FileIOFactory, File, IgniteLogger)
+     */
+    public static void batchRenameDefragmentedCacheGroupPartitions(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (!completionMarkerFile.exists())
+            return;
+
+        try {
+            for (File mappingFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)))
+                Files.delete(mappingFile.toPath());
+
+            for (File partFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_PARTITION_FILE_PREFIX))) {
+                int partId = extractPartId(partFile.getName());
+
+                File oldPartFile = new File(workDir, String.format(PART_FILE_TEMPLATE, partId));
+
+                Files.move(partFile.toPath(), oldPartFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+
+            File idxFile = new File(workDir, DFRG_INDEX_FILE_NAME);
+
+            if (idxFile.exists()) {
+                File oldIdxFile = new File(workDir, INDEX_FILE_NAME);
+
+                Files.move(idxFile.toPath(), oldIdxFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Extracts partition number from file names like {@code part-dfrg-%d.bin}.
+     *
+     * @param dfrgPartFileName Defragmented partition file name.
+     * @return Partition index.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    private static int extractPartId(String dfrgPartFileName) {
+        assert dfrgPartFileName.startsWith(DFRG_PARTITION_FILE_PREFIX) : dfrgPartFileName;
+        assert dfrgPartFileName.endsWith(FILE_SUFFIX) : dfrgPartFileName;
+
+        String partIdStr = dfrgPartFileName.substring(
+            DFRG_PARTITION_FILE_PREFIX.length(),
+            dfrgPartFileName.length() - FILE_SUFFIX.length()
+        );
+
+        return Integer.parseInt(partIdStr);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin.tmp} in given folder. It will be used for storing defragmented index
+     * partition during the process.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexFile(File)
+     */
+    public static File defragmentedIndexTmpFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_TMP_FILE_NAME);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin} in given folder. It will be used for storing defragmented index
+     * partition when the process is over.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexTmpFile(File)
+     */
+    public static File defragmentedIndexFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_FILE_NAME);
+    }
+
+    /**
+     * Rename temporary index defragmenation file to a finalized one.
+     *
+     * @param workDir Cache group working directory.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexTmpFile(File)
+     * @see DefragmentationFileUtils#defragmentedIndexFile(File)
+     */
+    public static void renameTempIndexFile(File workDir) throws IgniteCheckedException {
+        File defragmentedIdxTmpFile = defragmentedIndexTmpFile(workDir);
+        File defragmentedIdxFile = defragmentedIndexFile(workDir);
+
+        try {
+            Files.move(defragmentedIdxTmpFile.toPath(), defragmentedIdxFile.toPath(), ATOMIC_MOVE);
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Return file named {@code part-dfrg-%d.bin.tmp} in given folder. It will be used for storing defragmented data
+     * partition during the process.
+     *
+     * @param workDir Cache group working directory.
+     * @param partId Parition index, will be substituted into file name.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    public static File defragmentedPartTmpFile(File workDir, int partId) {
+        return new File(workDir, String.format(DFRG_PARTITION_TMP_FILE_TEMPLATE, partId));
+    }
+
+    /**
+     * Return file named {@code part-dfrg-%d.bin} in given folder. It will be used for storing defragmented data
+     * partition when the process is over.
+     *
+     * @param workDir Cache group working directory.
+     * @param partId Parition index, will be substituted into file name.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     */
+    public static File defragmentedPartFile(File workDir, int partId) {
+        return new File(workDir, String.format(DFRG_PARTITION_FILE_TEMPLATE, partId));
+    }
+
+    /**
+     * Rename temporary partition defragmenation file to a finalized one.
+     *
+     * @param workDir Cache group working directory.
+     * @param partId Parition index.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    public static void renameTempPartitionFile(File workDir, int partId) throws IgniteCheckedException {
+        File defragmentedPartTmpFile = defragmentedPartTmpFile(workDir, partId);
+        File defragmentedPartFile = defragmentedPartFile(workDir, partId);
+
+        assert !defragmentedPartFile.exists() : defragmentedPartFile;
+
+        try {
+            Files.move(defragmentedPartTmpFile.toPath(), defragmentedPartFile.toPath(), ATOMIC_MOVE);
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Return file named {@code part-map-%d.bin} in given folder. It will be used for storing defragmention links
+     * mapping for given partition during and after defragmentation process. No temporary counterpart is required here.
+     *
+     * @param workDir Cache group working directory.
+     * @param partId Parition index, will be substituted into file name.
+     * @return File.
+     *
+     * @see LinkMap
+     */
+    public static File defragmentedPartMappingFile(File workDir, int partId) {
+        return new File(workDir, String.format(DFRG_LINK_MAPPING_FILE_TEMPLATE, partId));
+    }
+
+    /**
+     * Return defragmentation completion marker file. This file can only be created when all partitions and index are
+     * defragmented and renamed from their original {@code *.tmp} versions. Presence of this file signals that no data
+     * will be lost if original partitions are deleted and batch rename process can be safely initiated.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#writeDefragmentationCompletionMarker(FileIOFactory, File, IgniteLogger)
+     * @see DefragmentationFileUtils#batchRenameDefragmentedCacheGroupPartitions(File, IgniteLogger)
+     */
+    public static File defragmentationCompletionMarkerFile(File workDir) {
+        return new File(workDir, DFRG_COMPLETION_MARKER_FILE_NAME);
+    }
+
+    /**
+     * Creates empty completion marker file in given directory.
+     *
+     * @param ioFactory File IO factory.
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentationCompletionMarkerFile(File)
+     */
+    public static void writeDefragmentationCompletionMarker(
+        FileIOFactory ioFactory,
+        File workDir,
+        IgniteLogger log
+    ) throws IgniteCheckedException {
+        File completionMarker = defragmentationCompletionMarkerFile(workDir);
+
+        try (FileIO io = ioFactory.create(completionMarker, CREATE_NEW, WRITE)) {
+            io.force(true);
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /** */
+    private static void handleIoException(IOException e) throws IgniteCheckedException {

Review comment:
       Agree, it needs to 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.

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r525116463



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/TimeTracker.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.util.Arrays;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/** */
+public class TimeTracker<Stage extends Enum<Stage>> {

Review comment:
       It kinda is, but it's not. Here we accumulate time of the same stages over many iterations. For example, we can have 1000 checkpoint read lock acquirings and 100000 cache data entries readings. Technically, each of those are a separate "stage", but what we need is only the sum of timings.




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

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r531034827



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/TimeTracker.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.util.Arrays;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/** */
+public class TimeTracker<Stage extends Enum<Stage>> {

Review comment:
       Thas why any build-in aggregation machinery is a bad idea. For building reports, the raw data must be used.
   
   I have no doubts about the necessity of having things for debugging and measuring performance, finding hot spots but let's use more flexible and standard things like - [Tracing](https://issues.apache.org/jira/browse/IGNITE-13060) and/or [Cluster performance profiling tool](https://issues.apache.org/jira/browse/IGNITE-12666)
   
   Let's remove this thing from PR and file a new issue to implement it the right and more flexible way without a rush.




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r531443625



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/GridQueryIndexingDefragmentation.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.processors.cache.persistence.defragmentation;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.util.collection.IntMap;
+
+/**
+ *
+ */
+public interface GridQueryIndexingDefragmentation {

Review comment:
       Ok, done




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

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r530966407



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
##########
@@ -447,6 +472,32 @@ private DataRegionConfiguration createMetastoreDataRegionConfig(DataStorageConfi
         return cfg;
     }
 
+    /** */
+    private DataRegionConfiguration createDefragmentationDataRegionConfig(long regionSize) {

Review comment:
       Is it true that the defragmentation region allocated at the node start will not allow the node to allocate the full amount of available memory until node restart? The node may live months without restart with shrunk memory for technical needs, right?




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r531452934



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;
+
+                                if (log.isDebugEnabled()) {
+                                    log.debug(S.toString(
+                                        "Partition defragmented",
+                                        "grpId", grpId, false,
+                                        "partId", partId, false,
+                                        "oldPages", oldPageStore.pages(), false,
+                                        "newPages", partCtx.partPagesAllocated.get() + 1, false,
+                                        "mappingPages", partCtx.mappingPagesAllocated.get() + 1, false,
+                                        "pageSize", pageSize, false,
+                                        "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                        "workDir", workDir, false
+                                    ));
+                                }
+
+                                oldPageMem.invalidate(grpId, partId);
+
+                                partCtx.partPageMemory.invalidate(grpId, partId);
+
+                                DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                                pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                                try {
+                                    renameTempPartitionFile(workDir, partId);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    throw new IgniteException(e);
+                                }
+                            }
+                        };
+
+                        GridFutureAdapter<?> cpFut = defragmentationCheckpoint
+                            .forceCheckpoint("partition defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+
+                        cpFut.listen(cpLsnr);
+
+                        cmpFut.add((IgniteInternalFuture<Object>)cpFut);
+                    }
+
+                    // A bit too general for now, but I like it more then saving only the last checkpoint future.
+                    cmpFut.markInitialized().get();
+
+                    idxDfrgFut = new GridFinishedFuture<>();
+
+                    if (filePageStoreMgr.hasIndexStore(grpId)) {
+                        defragmentIndexPartition(oldGrpCtx, newGrpCtx, linkMapByPart);
+
+                        idxDfrgFut = defragmentationCheckpoint
+                            .forceCheckpoint("index defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+                    }
+
+                    idxDfrgFut.listen(fut -> {
+                        oldPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx partPageMem = (PageMemoryEx)partDataRegion.pageMemory();
+
+                        partPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().removePageStore(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx mappingPageMem = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+                        pageMgr = (DefragmentationPageReadWriteManager)mappingPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().clear(grpId);
+
+                        try {
+                            renameTempIndexFile(workDir);
+
+                            writeDefragmentationCompletionMarker(filePageStoreMgr.getPageStoreFileIoFactory(), workDir, log);
+
+                            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    });
+                }
+
+                // I guess we should wait for it?
+                if (idxDfrgFut != null)
+                    idxDfrgFut.get();
+            }
+
+            mntcReg.unregisterMaintenanceTask(DEFRAGMENTATION_MNTC_TASK_NAME);
+
+            log.info("Defragmentation completed. All partitions are defragmented.");
+        }
+        finally {
+            defragmentationCheckpoint.stop(true);
+        }
+    }
+
+    /** */
+    public void createIndexPageStore(
+        int grpId,
+        File workDir,
+        FilePageStoreFactory pageStoreFactory,
+        DataRegion partRegion,
+        LongConsumer allocatedTracker
+    ) throws IgniteCheckedException {
+        // Index partition file has to be deleted before we begin, otherwise there's a chance of reading corrupted file.
+        // There is a time period when index is already defragmented but marker file is not created yet. If node is
+        // failed in that time window then index will be deframented once again. That's fine, situation is rare but code
+        // to fix that would add unnecessary complications.
+        U.delete(defragmentedIndexTmpFile(workDir));
+
+        PageStore idxPageStore;
+
+        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+        try {
+            idxPageStore = pageStoreFactory.createPageStore(
+                FLAG_IDX,
+                () -> defragmentedIndexTmpFile(workDir).toPath(),
+                allocatedTracker
+            );
+        }
+        finally {
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+        }
+
+        idxPageStore.sync();
+
+        PageMemoryEx partPageMem = (PageMemoryEx)partRegion.pageMemory();
+
+        DefragmentationPageReadWriteManager partMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+        partMgr.pageStoreMap().addPageStore(grpId, PageIdAllocator.INDEX_PARTITION, idxPageStore);
+    }
+
+    /** */
+    public enum PartStages {
+        START,
+        CP_LOCK,
+        ITERATE,
+        READ_ROW,
+        INSERT_ROW,
+        STORE_MAP,
+        STORE_PK,
+        STORE_PENDING,
+        METADATA
+    }
+
+    /**
+     * Defragmentate partition.
+     *
+     * @param partCtx
+     * @param treeIter
+     * @throws IgniteCheckedException If failed.
+     */
+    private void copyPartitionData(
+        PartitionContext partCtx,
+        TreeIterator treeIter,
+        GridSpinBusyLock busyLock
+    ) throws IgniteCheckedException {
+        partCtx.createNewCacheDataStore(busyLock);
+
+        CacheDataTree tree = partCtx.oldCacheDataStore.tree();
+
+        CacheDataTree newTree = partCtx.newCacheDataStore.tree();
+        PendingEntriesTree newPendingTree = partCtx.newCacheDataStore.pendingTree();
+        AbstractFreeList<CacheDataRow> freeList = partCtx.newCacheDataStore.getCacheStoreFreeList();
+
+        long cpLockThreshold = 150L;
+
+        TimeTracker<PartStages> tracker = new TimeTracker<>(PartStages.class);
+
+        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+        tracker.complete(CP_LOCK);
+
+        try {
+            AtomicLong lastCpLockTs = new AtomicLong(System.currentTimeMillis());
+            AtomicInteger entriesProcessed = new AtomicInteger();
+
+            treeIter.iterate(tree, partCtx.cachePageMemory, (tree0, io, pageAddr, idx) -> {
+                tracker.complete(ITERATE);
+
+                if (System.currentTimeMillis() - lastCpLockTs.get() >= cpLockThreshold) {
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+                    tracker.complete(CP_LOCK);
+
+                    lastCpLockTs.set(System.currentTimeMillis());
+                }
+
+                AbstractDataLeafIO leafIo = (AbstractDataLeafIO)io;
+                CacheDataRow row = tree.getRow(io, pageAddr, idx);
+
+                tracker.complete(READ_ROW);
+
+                int cacheId = row.cacheId();
+
+                // Reuse row that we just read.
+                row.link(0);
+
+                // "insertDataRow" will corrupt page memory if we don't do this.
+                if (row instanceof DataRow && !partCtx.oldGrpCtx.storeCacheIdInDataPage())
+                    ((DataRow)row).cacheId(CU.UNDEFINED_CACHE_ID);
+
+                freeList.insertDataRow(row, IoStatisticsHolderNoOp.INSTANCE);
+
+                // Put it back.
+                if (row instanceof DataRow)
+                    ((DataRow)row).cacheId(cacheId);
+
+                tracker.complete(INSERT_ROW);
+
+                newTree.putx(row);
+
+                long newLink = row.link();
+
+                tracker.complete(STORE_MAP);
+
+                partCtx.linkMap.put(leafIo.getLink(pageAddr, idx), newLink);
+
+                tracker.complete(STORE_PK);
+
+                if (row.expireTime() != 0)
+                    newPendingTree.putx(new PendingRow(cacheId, row.expireTime(), newLink));
+
+                tracker.complete(STORE_PENDING);
+
+                entriesProcessed.incrementAndGet();
+
+                return true;
+            });
+
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+            tracker.complete(CP_LOCK);
+
+            freeList.saveMetadata(IoStatisticsHolderNoOp.INSTANCE);
+
+            copyCacheMetadata(partCtx);
+
+            tracker.complete(METADATA);
+        }
+        finally {
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+        }
+
+        if (log.isDebugEnabled()) {
+            log.debug(
+                "Partition defragmentation timings for cache group " + partCtx.grpId +
+                " and partition " + partCtx.partId + ": " + tracker.toString()
+            );
+        }
+    }
+
+    /** */
+    private void copyCacheMetadata(
+        PartitionContext partCtx
+    ) throws IgniteCheckedException {
+        // Same for all page memories. Why does it need to be in PageMemory?
+        long partMetaPageId = partCtx.cachePageMemory.partitionMetaPageId(partCtx.grpId, partCtx.partId);
+
+        access(ACCESS_READ, partCtx.cachePageMemory, partCtx.grpId, partMetaPageId, oldPartMetaPageAddr -> {
+            PagePartitionMetaIO oldPartMetaIo = PageIO.getPageIO(oldPartMetaPageAddr);
+
+            // Newer meta versions may contain new data that we don't copy during defragmentation.
+            assert Arrays.asList(1, 2, 3).contains(oldPartMetaIo.getVersion())
+                : "IO version " + oldPartMetaIo.getVersion() + " is not supported by current defragmentation algorithm." +
+                " Please implement copying of all data added in new version.";
+
+            access(ACCESS_WRITE, partCtx.partPageMemory, partCtx.grpId, partMetaPageId, newPartMetaPageAddr -> {
+                PagePartitionMetaIOV3 newPartMetaIo = PageIO.getPageIO(newPartMetaPageAddr);
+
+                // Copy partition state.
+                byte partState = oldPartMetaIo.getPartitionState(oldPartMetaPageAddr);
+                newPartMetaIo.setPartitionState(newPartMetaPageAddr, partState);
+
+                // Copy cache size for single cache group.
+                long size = oldPartMetaIo.getSize(oldPartMetaPageAddr);
+                newPartMetaIo.setSize(newPartMetaPageAddr, size);
+
+                // Copy update counter value.
+                long updateCntr = oldPartMetaIo.getUpdateCounter(oldPartMetaPageAddr);
+                newPartMetaIo.setUpdateCounter(newPartMetaPageAddr, updateCntr);
+
+                // Copy global remove Id.
+                long rmvId = oldPartMetaIo.getGlobalRemoveId(oldPartMetaPageAddr);
+                newPartMetaIo.setGlobalRemoveId(newPartMetaPageAddr, rmvId);
+
+                // Copy cache sizes for shared cache group.
+                long oldCountersPageId = oldPartMetaIo.getCountersPageId(oldPartMetaPageAddr);
+                if (oldCountersPageId != 0L) {
+                    Map<Integer, Long> sizes = GridCacheOffheapManager.readSharedGroupCacheSizes(
+                        partCtx.cachePageMemory,
+                        partCtx.grpId,
+                        oldCountersPageId
+                    );
+
+                    long newCountersPageId = GridCacheOffheapManager.writeSharedGroupCacheSizes(
+                        partCtx.partPageMemory,
+                        partCtx.grpId,
+                        0L,
+                        partCtx.partId,
+                        sizes
+                    );
+
+                    newPartMetaIo.setCountersPageId(newPartMetaPageAddr, newCountersPageId);
+                }
+
+                // Copy counter gaps.
+                long oldGapsLink = oldPartMetaIo.getGapsLink(oldPartMetaPageAddr);
+                if (oldGapsLink != 0L) {
+                    byte[] gapsBytes = partCtx.oldCacheDataStore.partStorage().readRow(oldGapsLink);
+
+                    SimpleDataRow gapsDataRow = new SimpleDataRow(partCtx.partId, gapsBytes);
+
+                    partCtx.newCacheDataStore.partStorage().insertDataRow(gapsDataRow, IoStatisticsHolderNoOp.INSTANCE);
+
+                    newPartMetaIo.setGapsLink(newPartMetaPageAddr, gapsDataRow.link());
+                }
+
+                // Encryption stuff.
+                newPartMetaIo.setEncryptedPageCount(newPartMetaPageAddr, 0);
+                newPartMetaIo.setEncryptedPageIndex(newPartMetaPageAddr, 0);
+
+                return null;
+            });
+
+            return null;
+        });
+    }
+
+    /**
+     * Defragmentate indexing partition.
+     *
+     * @param grpCtx
+     * @param mappingByPartition
+     *
+     * @throws IgniteCheckedException If failed.
+     */
+    private void defragmentIndexPartition(
+        CacheGroupContext grpCtx,
+        CacheGroupContext newCtx,
+        IntMap<LinkMap> mappingByPartition
+    ) throws IgniteCheckedException {
+        GridQueryProcessor query = grpCtx.caches().get(0).kernalContext().query();
+
+        if (!query.moduleEnabled())
+            return;
+
+        final GridQueryIndexing idx = query.getIndexing();
+
+        CheckpointTimeoutLock cpLock = defragmentationCheckpoint.checkpointTimeoutLock();
+
+        idx.defragmentator().defragment(
+            grpCtx,
+            newCtx,
+            (PageMemoryEx)partDataRegion.pageMemory(),
+            mappingByPartition,
+            cpLock,
+            log
+        );
+    }
+
+    /** */
+    @SuppressWarnings("PublicField")
+    private class PartitionContext {
+        /** */
+        public final File workDir;
+
+        /** */
+        public final int grpId;
+
+        /** */
+        public final int partId;
+
+        /** */
+        public final DataRegion cacheDataRegion;
+
+        /** */
+        public final PageMemoryEx cachePageMemory;
+
+        /** */
+        public final PageMemoryEx partPageMemory;
+
+        /** */
+        public final PageMemoryEx mappingPageMemory;
+
+        /** */
+        public final CacheGroupContext oldGrpCtx;
+
+        /** */
+        public final CacheGroupContext newGrpCtx;
+
+        /** */
+        public final CacheDataStore oldCacheDataStore;
+
+        /** */
+        private GridCacheDataStore newCacheDataStore;
+
+        /** */
+        public final FilePageStoreFactory pageStoreFactory;
+
+        /** */
+        public final AtomicLong partPagesAllocated = new AtomicLong();
+
+        /** */
+        public final AtomicLong mappingPagesAllocated = new AtomicLong();
+
+        /** */
+        private LinkMap linkMap;
+
+        /** */
+        public PartitionContext(
+            File workDir,
+            int grpId,
+            int partId,
+            DataRegion partDataRegion,
+            DataRegion mappingDataRegion,
+            CacheGroupContext oldGrpCtx,
+            CacheGroupContext newGrpCtx,
+            CacheDataStore oldCacheDataStore,
+            FilePageStoreFactory pageStoreFactory
+        ) {
+            this.workDir = workDir;
+            this.grpId = grpId;
+            this.partId = partId;
+            cacheDataRegion = oldGrpCtx.dataRegion();
+
+            cachePageMemory = (PageMemoryEx)cacheDataRegion.pageMemory();
+            partPageMemory = (PageMemoryEx)partDataRegion.pageMemory();
+            mappingPageMemory = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+            this.oldGrpCtx = oldGrpCtx;
+            this.newGrpCtx = newGrpCtx;
+            this.oldCacheDataStore = oldCacheDataStore;
+            this.pageStoreFactory = pageStoreFactory;
+        }
+
+        /** */
+        public PageStore createPartPageStore() throws IgniteCheckedException {
+            PageStore partPageStore;
+
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+            try {
+                partPageStore = pageStoreFactory.createPageStore(
+                    FLAG_DATA,
+                    () -> defragmentedPartTmpFile(workDir, partId).toPath(),
+                    partPagesAllocated::addAndGet
+                );
+            }
+            finally {
+                defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+            }
+
+            partPageStore.sync();
+
+            DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMemory.pageManager();
+
+            pageMgr.pageStoreMap().addPageStore(grpId, partId, partPageStore);
+
+            return partPageStore;
+        }
+
+        /** */
+        public PageStore createMappingPageStore() throws IgniteCheckedException {
+            PageStore mappingPageStore;
+
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+            try {
+                mappingPageStore = pageStoreFactory.createPageStore(
+                    FLAG_DATA,
+                    () -> defragmentedPartMappingFile(workDir, partId).toPath(),
+                    mappingPagesAllocated::addAndGet
+                );
+            }
+            finally {
+                defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+            }
+
+            mappingPageStore.sync();
+
+            DefragmentationPageReadWriteManager partMgr = (DefragmentationPageReadWriteManager)mappingPageMemory.pageManager();
+
+            partMgr.pageStoreMap().addPageStore(grpId, partId, mappingPageStore);
+
+            return mappingPageStore;
+        }
+
+        /** */
+        public LinkMap createLinkMapTree(boolean initNew) throws IgniteCheckedException {
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+            //TODO Store link in meta page and remove META_PAGE_IDX constant?
+            try {
+                long mappingMetaPageId = initNew
+                    ? mappingPageMemory.allocatePage(grpId, partId, FLAG_DATA)
+                    : PageIdUtils.pageId(partId, FLAG_DATA, LinkMap.META_PAGE_IDX);
+
+                assert PageIdUtils.pageIndex(mappingMetaPageId) == LinkMap.META_PAGE_IDX
+                    : PageIdUtils.toDetailString(mappingMetaPageId);
+
+                linkMap = new LinkMap(newGrpCtx, mappingPageMemory, mappingMetaPageId, initNew);
+            }
+            finally {
+                defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+            }
+
+            return linkMap;
+        }
+
+        /** */
+        public void createNewCacheDataStore(GridSpinBusyLock busyLock) {

Review comment:
       I agree with you, I'll create that 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.

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r530964570



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/GridQueryIndexingDefragmentation.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.processors.cache.persistence.defragmentation;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.util.collection.IntMap;
+
+/**
+ *
+ */
+public interface GridQueryIndexingDefragmentation {

Review comment:
       I don't think we will ever need any methods for defragmentation since all our attention will be switched to Ignite 3.0. So let's simplify here as much as possible and remain only the single 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.

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r530968905



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();

Review comment:
       So the only choice to stop the node during logical recovery the kill process command, right?




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r523996817



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/lang/IgniteThrowableBiFunction.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.util.lang;
+
+import java.io.Serializable;
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ * Represents a bi function that accepts one argument and produces a result. Unlike {@link java.util.function.BiFunction}
+ * it is able to throw {@link IgniteCheckedException}.
+ *
+ * @param <T> The type of the first input to the function.
+ * @param <U> The type of the second input to the function.
+ * @param <R> The type of the result of the function.
+ */
+public interface IgniteThrowableBiFunction<T, U, R> extends Serializable {

Review comment:
       Exactly, seems like you found a dead 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.

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r530201129



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;
+
+                                if (log.isDebugEnabled()) {
+                                    log.debug(S.toString(
+                                        "Partition defragmented",
+                                        "grpId", grpId, false,
+                                        "partId", partId, false,
+                                        "oldPages", oldPageStore.pages(), false,
+                                        "newPages", partCtx.partPagesAllocated.get() + 1, false,
+                                        "mappingPages", partCtx.mappingPagesAllocated.get() + 1, false,
+                                        "pageSize", pageSize, false,
+                                        "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                        "workDir", workDir, false
+                                    ));
+                                }
+
+                                oldPageMem.invalidate(grpId, partId);
+
+                                partCtx.partPageMemory.invalidate(grpId, partId);
+
+                                DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                                pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                                try {
+                                    renameTempPartitionFile(workDir, partId);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    throw new IgniteException(e);
+                                }
+                            }
+                        };
+
+                        GridFutureAdapter<?> cpFut = defragmentationCheckpoint
+                            .forceCheckpoint("partition defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+
+                        cpFut.listen(cpLsnr);
+
+                        cmpFut.add((IgniteInternalFuture<Object>)cpFut);
+                    }
+
+                    // A bit too general for now, but I like it more then saving only the last checkpoint future.
+                    cmpFut.markInitialized().get();
+
+                    idxDfrgFut = new GridFinishedFuture<>();
+
+                    if (filePageStoreMgr.hasIndexStore(grpId)) {
+                        defragmentIndexPartition(oldGrpCtx, newGrpCtx, linkMapByPart);
+
+                        idxDfrgFut = defragmentationCheckpoint
+                            .forceCheckpoint("index defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+                    }
+
+                    idxDfrgFut.listen(fut -> {
+                        oldPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx partPageMem = (PageMemoryEx)partDataRegion.pageMemory();
+
+                        partPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().removePageStore(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx mappingPageMem = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+                        pageMgr = (DefragmentationPageReadWriteManager)mappingPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().clear(grpId);
+
+                        try {
+                            renameTempIndexFile(workDir);
+
+                            writeDefragmentationCompletionMarker(filePageStoreMgr.getPageStoreFileIoFactory(), workDir, log);
+
+                            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    });
+                }
+
+                // I guess we should wait for it?
+                if (idxDfrgFut != null)
+                    idxDfrgFut.get();
+            }
+
+            mntcReg.unregisterMaintenanceTask(DEFRAGMENTATION_MNTC_TASK_NAME);
+
+            log.info("Defragmentation completed. All partitions are defragmented.");
+        }
+        finally {
+            defragmentationCheckpoint.stop(true);
+        }
+    }
+
+    /** */
+    public void createIndexPageStore(
+        int grpId,
+        File workDir,
+        FilePageStoreFactory pageStoreFactory,
+        DataRegion partRegion,
+        LongConsumer allocatedTracker
+    ) throws IgniteCheckedException {
+        // Index partition file has to be deleted before we begin, otherwise there's a chance of reading corrupted file.
+        // There is a time period when index is already defragmented but marker file is not created yet. If node is
+        // failed in that time window then index will be deframented once again. That's fine, situation is rare but code
+        // to fix that would add unnecessary complications.
+        U.delete(defragmentedIndexTmpFile(workDir));
+
+        PageStore idxPageStore;
+
+        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+        try {
+            idxPageStore = pageStoreFactory.createPageStore(
+                FLAG_IDX,
+                () -> defragmentedIndexTmpFile(workDir).toPath(),
+                allocatedTracker
+            );
+        }
+        finally {
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+        }
+
+        idxPageStore.sync();
+
+        PageMemoryEx partPageMem = (PageMemoryEx)partRegion.pageMemory();
+
+        DefragmentationPageReadWriteManager partMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+        partMgr.pageStoreMap().addPageStore(grpId, PageIdAllocator.INDEX_PARTITION, idxPageStore);
+    }
+
+    /** */
+    public enum PartStages {
+        START,
+        CP_LOCK,
+        ITERATE,
+        READ_ROW,
+        INSERT_ROW,
+        STORE_MAP,
+        STORE_PK,
+        STORE_PENDING,
+        METADATA
+    }
+
+    /**
+     * Defragmentate partition.
+     *
+     * @param partCtx
+     * @param treeIter
+     * @throws IgniteCheckedException If failed.
+     */
+    private void copyPartitionData(
+        PartitionContext partCtx,
+        TreeIterator treeIter,
+        GridSpinBusyLock busyLock
+    ) throws IgniteCheckedException {
+        partCtx.createNewCacheDataStore(busyLock);
+
+        CacheDataTree tree = partCtx.oldCacheDataStore.tree();
+
+        CacheDataTree newTree = partCtx.newCacheDataStore.tree();
+        PendingEntriesTree newPendingTree = partCtx.newCacheDataStore.pendingTree();
+        AbstractFreeList<CacheDataRow> freeList = partCtx.newCacheDataStore.getCacheStoreFreeList();
+
+        long cpLockThreshold = 150L;
+
+        TimeTracker<PartStages> tracker = new TimeTracker<>(PartStages.class);
+
+        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+        tracker.complete(CP_LOCK);
+
+        try {
+            AtomicLong lastCpLockTs = new AtomicLong(System.currentTimeMillis());
+            AtomicInteger entriesProcessed = new AtomicInteger();
+
+            treeIter.iterate(tree, partCtx.cachePageMemory, (tree0, io, pageAddr, idx) -> {
+                tracker.complete(ITERATE);
+
+                if (System.currentTimeMillis() - lastCpLockTs.get() >= cpLockThreshold) {
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+                    tracker.complete(CP_LOCK);
+
+                    lastCpLockTs.set(System.currentTimeMillis());
+                }
+
+                AbstractDataLeafIO leafIo = (AbstractDataLeafIO)io;
+                CacheDataRow row = tree.getRow(io, pageAddr, idx);
+
+                tracker.complete(READ_ROW);
+
+                int cacheId = row.cacheId();
+
+                // Reuse row that we just read.
+                row.link(0);
+
+                // "insertDataRow" will corrupt page memory if we don't do this.
+                if (row instanceof DataRow && !partCtx.oldGrpCtx.storeCacheIdInDataPage())
+                    ((DataRow)row).cacheId(CU.UNDEFINED_CACHE_ID);
+
+                freeList.insertDataRow(row, IoStatisticsHolderNoOp.INSTANCE);
+
+                // Put it back.
+                if (row instanceof DataRow)
+                    ((DataRow)row).cacheId(cacheId);
+
+                tracker.complete(INSERT_ROW);
+
+                newTree.putx(row);
+
+                long newLink = row.link();
+
+                tracker.complete(STORE_MAP);
+
+                partCtx.linkMap.put(leafIo.getLink(pageAddr, idx), newLink);
+
+                tracker.complete(STORE_PK);
+
+                if (row.expireTime() != 0)
+                    newPendingTree.putx(new PendingRow(cacheId, row.expireTime(), newLink));
+
+                tracker.complete(STORE_PENDING);
+
+                entriesProcessed.incrementAndGet();
+
+                return true;
+            });
+
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+            tracker.complete(CP_LOCK);
+
+            freeList.saveMetadata(IoStatisticsHolderNoOp.INSTANCE);
+
+            copyCacheMetadata(partCtx);
+
+            tracker.complete(METADATA);
+        }
+        finally {
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+        }
+
+        if (log.isDebugEnabled()) {
+            log.debug(
+                "Partition defragmentation timings for cache group " + partCtx.grpId +
+                " and partition " + partCtx.partId + ": " + tracker.toString()
+            );
+        }
+    }
+
+    /** */
+    private void copyCacheMetadata(
+        PartitionContext partCtx
+    ) throws IgniteCheckedException {
+        // Same for all page memories. Why does it need to be in PageMemory?
+        long partMetaPageId = partCtx.cachePageMemory.partitionMetaPageId(partCtx.grpId, partCtx.partId);
+
+        access(ACCESS_READ, partCtx.cachePageMemory, partCtx.grpId, partMetaPageId, oldPartMetaPageAddr -> {
+            PagePartitionMetaIO oldPartMetaIo = PageIO.getPageIO(oldPartMetaPageAddr);
+
+            // Newer meta versions may contain new data that we don't copy during defragmentation.
+            assert Arrays.asList(1, 2, 3).contains(oldPartMetaIo.getVersion())
+                : "IO version " + oldPartMetaIo.getVersion() + " is not supported by current defragmentation algorithm." +
+                " Please implement copying of all data added in new version.";
+
+            access(ACCESS_WRITE, partCtx.partPageMemory, partCtx.grpId, partMetaPageId, newPartMetaPageAddr -> {
+                PagePartitionMetaIOV3 newPartMetaIo = PageIO.getPageIO(newPartMetaPageAddr);
+
+                // Copy partition state.
+                byte partState = oldPartMetaIo.getPartitionState(oldPartMetaPageAddr);
+                newPartMetaIo.setPartitionState(newPartMetaPageAddr, partState);
+
+                // Copy cache size for single cache group.
+                long size = oldPartMetaIo.getSize(oldPartMetaPageAddr);
+                newPartMetaIo.setSize(newPartMetaPageAddr, size);
+
+                // Copy update counter value.
+                long updateCntr = oldPartMetaIo.getUpdateCounter(oldPartMetaPageAddr);
+                newPartMetaIo.setUpdateCounter(newPartMetaPageAddr, updateCntr);
+
+                // Copy global remove Id.
+                long rmvId = oldPartMetaIo.getGlobalRemoveId(oldPartMetaPageAddr);
+                newPartMetaIo.setGlobalRemoveId(newPartMetaPageAddr, rmvId);
+
+                // Copy cache sizes for shared cache group.
+                long oldCountersPageId = oldPartMetaIo.getCountersPageId(oldPartMetaPageAddr);
+                if (oldCountersPageId != 0L) {
+                    Map<Integer, Long> sizes = GridCacheOffheapManager.readSharedGroupCacheSizes(
+                        partCtx.cachePageMemory,
+                        partCtx.grpId,
+                        oldCountersPageId
+                    );
+
+                    long newCountersPageId = GridCacheOffheapManager.writeSharedGroupCacheSizes(
+                        partCtx.partPageMemory,
+                        partCtx.grpId,
+                        0L,
+                        partCtx.partId,
+                        sizes
+                    );
+
+                    newPartMetaIo.setCountersPageId(newPartMetaPageAddr, newCountersPageId);
+                }
+
+                // Copy counter gaps.
+                long oldGapsLink = oldPartMetaIo.getGapsLink(oldPartMetaPageAddr);
+                if (oldGapsLink != 0L) {
+                    byte[] gapsBytes = partCtx.oldCacheDataStore.partStorage().readRow(oldGapsLink);
+
+                    SimpleDataRow gapsDataRow = new SimpleDataRow(partCtx.partId, gapsBytes);
+
+                    partCtx.newCacheDataStore.partStorage().insertDataRow(gapsDataRow, IoStatisticsHolderNoOp.INSTANCE);
+
+                    newPartMetaIo.setGapsLink(newPartMetaPageAddr, gapsDataRow.link());
+                }
+
+                // Encryption stuff.
+                newPartMetaIo.setEncryptedPageCount(newPartMetaPageAddr, 0);
+                newPartMetaIo.setEncryptedPageIndex(newPartMetaPageAddr, 0);
+
+                return null;
+            });
+
+            return null;
+        });
+    }
+
+    /**
+     * Defragmentate indexing partition.
+     *
+     * @param grpCtx
+     * @param mappingByPartition
+     *
+     * @throws IgniteCheckedException If failed.
+     */
+    private void defragmentIndexPartition(
+        CacheGroupContext grpCtx,
+        CacheGroupContext newCtx,
+        IntMap<LinkMap> mappingByPartition
+    ) throws IgniteCheckedException {
+        GridQueryProcessor query = grpCtx.caches().get(0).kernalContext().query();
+
+        if (!query.moduleEnabled())
+            return;
+
+        final GridQueryIndexing idx = query.getIndexing();
+
+        CheckpointTimeoutLock cpLock = defragmentationCheckpoint.checkpointTimeoutLock();
+
+        idx.defragmentator().defragment(
+            grpCtx,
+            newCtx,
+            (PageMemoryEx)partDataRegion.pageMemory(),
+            mappingByPartition,
+            cpLock,
+            log
+        );
+    }
+
+    /** */
+    @SuppressWarnings("PublicField")
+    private class PartitionContext {
+        /** */
+        public final File workDir;
+
+        /** */
+        public final int grpId;
+
+        /** */
+        public final int partId;
+
+        /** */
+        public final DataRegion cacheDataRegion;
+
+        /** */
+        public final PageMemoryEx cachePageMemory;
+
+        /** */
+        public final PageMemoryEx partPageMemory;
+
+        /** */
+        public final PageMemoryEx mappingPageMemory;
+
+        /** */
+        public final CacheGroupContext oldGrpCtx;
+
+        /** */
+        public final CacheGroupContext newGrpCtx;
+
+        /** */
+        public final CacheDataStore oldCacheDataStore;
+
+        /** */
+        private GridCacheDataStore newCacheDataStore;
+
+        /** */
+        public final FilePageStoreFactory pageStoreFactory;
+
+        /** */
+        public final AtomicLong partPagesAllocated = new AtomicLong();
+
+        /** */
+        public final AtomicLong mappingPagesAllocated = new AtomicLong();
+
+        /** */
+        private LinkMap linkMap;
+
+        /** */
+        public PartitionContext(
+            File workDir,
+            int grpId,
+            int partId,
+            DataRegion partDataRegion,
+            DataRegion mappingDataRegion,
+            CacheGroupContext oldGrpCtx,
+            CacheGroupContext newGrpCtx,
+            CacheDataStore oldCacheDataStore,
+            FilePageStoreFactory pageStoreFactory
+        ) {
+            this.workDir = workDir;
+            this.grpId = grpId;
+            this.partId = partId;
+            cacheDataRegion = oldGrpCtx.dataRegion();
+
+            cachePageMemory = (PageMemoryEx)cacheDataRegion.pageMemory();
+            partPageMemory = (PageMemoryEx)partDataRegion.pageMemory();
+            mappingPageMemory = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+            this.oldGrpCtx = oldGrpCtx;
+            this.newGrpCtx = newGrpCtx;
+            this.oldCacheDataStore = oldCacheDataStore;
+            this.pageStoreFactory = pageStoreFactory;
+        }
+
+        /** */
+        public PageStore createPartPageStore() throws IgniteCheckedException {
+            PageStore partPageStore;
+
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+            try {
+                partPageStore = pageStoreFactory.createPageStore(
+                    FLAG_DATA,
+                    () -> defragmentedPartTmpFile(workDir, partId).toPath(),
+                    partPagesAllocated::addAndGet
+                );
+            }
+            finally {
+                defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+            }
+
+            partPageStore.sync();
+
+            DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMemory.pageManager();
+
+            pageMgr.pageStoreMap().addPageStore(grpId, partId, partPageStore);
+
+            return partPageStore;
+        }
+
+        /** */
+        public PageStore createMappingPageStore() throws IgniteCheckedException {

Review comment:
       I'll join them




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r530215326



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();

Review comment:
       Sharing busy lock was the only available option. I'll add test later, semantics of node start procedure (during defragmentation) is changed in "control.sh API" issue, it would be better to introduce such test there.
   
   EDIT: I think I need to clarify what I mean by "semantics is changed".
   Right now defragmentation happens as a part of logical recovery, before node completed its "start" procedure. You can't stop the node while it's starting. In "control.sh API" issue I move defragmentation outside of starting routine and your scenario becomes possible. It's necessary because control.sh requires started and fully functional thin client.




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r526014091



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/defragmentation/IndexingDefragmentation.java
##########
@@ -0,0 +1,454 @@
+/*
+ * 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.processors.query.h2.defragmentation;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.GridQueryIndexingDefragmentation;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.LinkMap;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.TimeTracker;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIoResolver;
+import org.apache.ignite.internal.processors.cache.persistence.tree.util.InsertLast;
+import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccDataRow;
+import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+import org.apache.ignite.internal.processors.query.h2.database.H2Tree;
+import org.apache.ignite.internal.processors.query.h2.database.H2TreeIndex;
+import org.apache.ignite.internal.processors.query.h2.database.InlineIndexColumn;
+import org.apache.ignite.internal.processors.query.h2.database.inlinecolumn.AbstractInlineIndexColumn;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2ExtrasInnerIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2ExtrasLeafIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2InnerIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2LeafIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.H2RowLinkIO;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.processors.query.h2.opt.H2CacheRow;
+import org.apache.ignite.internal.processors.query.h2.opt.H2Row;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.h2.index.Index;
+import org.h2.value.Value;
+
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.INIT_TREE;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.ITERATE;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.READ_MAP;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.READ_ROW;
+
+/**
+ *
+ */
+public class IndexingDefragmentation implements GridQueryIndexingDefragmentation {
+    /** Indexing. */
+    private final IgniteH2Indexing indexing;
+
+    /** Constructor. */
+    public IndexingDefragmentation(IgniteH2Indexing indexing) {
+        this.indexing = indexing;
+    }
+
+    /** */
+    public enum IndexStages {
+        START,
+        CP_LOCK,
+        INIT_TREE,
+        ITERATE,
+        READ_ROW,
+        READ_MAP,
+        INSERT_ROW
+    }
+
+    /** {@inheritDoc} */
+    @Override public void defragmentate(
+        CacheGroupContext grpCtx,
+        CacheGroupContext newCtx,
+        PageMemoryEx partPageMem,
+        IntMap<LinkMap> mappingByPartition,
+        CheckpointTimeoutLock cpLock,
+        IgniteLogger log
+    ) throws IgniteCheckedException {
+        int pageSize = grpCtx.cacheObjectContext().kernalContext().grid().configuration().getDataStorageConfiguration().getPageSize();
+
+        TreeIterator treeIterator = new TreeIterator(pageSize);
+
+        PageMemoryEx oldCachePageMem = (PageMemoryEx)grpCtx.dataRegion().pageMemory();
+
+        PageMemory newCachePageMemory = partPageMem;
+
+        Collection<GridH2Table> tables = indexing.schemaManager().dataTables();
+
+        long cpLockThreshold = 250L;
+
+        TimeTracker<IndexStages> tracker = new TimeTracker<>(IndexStages.class);
+
+        cpLock.checkpointReadLock();
+        tracker.complete(CP_LOCK);
+
+        try {
+            AtomicLong lastCpLockTs = new AtomicLong(System.currentTimeMillis());
+
+            for (GridH2Table table : tables) {
+                GridCacheContext<?, ?> cctx = table.cacheContext();
+
+                if (cctx.groupId() != grpCtx.groupId())
+                    continue; // Not our index.
+
+                GridH2RowDescriptor rowDesc = table.rowDescriptor();
+
+                List<Index> indexes = table.getIndexes();
+                H2TreeIndex oldH2Idx = (H2TreeIndex)indexes.get(2);
+
+                int segments = oldH2Idx.segmentsCount();
+
+                H2Tree firstTree = oldH2Idx.treeForRead(0);
+
+                PageIoResolver pageIoRslvr = pageAddr -> {
+                    PageIO io = PageIoResolver.DEFAULT_PAGE_IO_RESOLVER.resolve(pageAddr);
+
+                    if (io instanceof BPlusMetaIO)
+                        return io;
+
+                    //noinspection unchecked,rawtypes,rawtypes
+                    return wrap((BPlusIO)io);
+                };
+
+                //TODO Create new proper GridCacheContext for it?
+                H2TreeIndex newIdx = H2TreeIndex.createIndex(
+                    cctx,
+                    null,
+                    table,
+                    oldH2Idx.getName(),
+                    firstTree.getPk(),
+                    firstTree.getAffinityKey(),
+                    Arrays.asList(firstTree.cols()),
+                    Arrays.asList(firstTree.cols()),
+                    oldH2Idx.inlineSize(),
+                    segments,
+                    newCachePageMemory,
+                    newCtx.offheap(),
+                    pageIoRslvr,
+                    log
+                );
+
+                tracker.complete(INIT_TREE);
+
+                for (int i = 0; i < segments; i++) {
+                    H2Tree tree = oldH2Idx.treeForRead(i);
+
+                    treeIterator.iterate(tree, oldCachePageMem, (theTree, io, pageAddr, idx) -> {
+                        tracker.complete(ITERATE);
+
+                        if (System.currentTimeMillis() - lastCpLockTs.get() >= cpLockThreshold) {
+                            cpLock.checkpointReadUnlock();
+
+                            cpLock.checkpointReadLock();
+                            tracker.complete(CP_LOCK);
+
+                            lastCpLockTs.set(System.currentTimeMillis());
+                        }
+
+                        assert 1 == io.getVersion();
+
+                        BPlusIO<H2Row> h2IO = wrap(io);
+
+                        H2Row row = theTree.getRow(h2IO, pageAddr, idx);
+
+                        tracker.complete(READ_ROW);
+
+                        if (row instanceof H2CacheRowWithIndex) {
+                            H2CacheRowWithIndex h2CacheRow = (H2CacheRowWithIndex)row;
+
+                            CacheDataRow cacheDataRow = h2CacheRow.getRow();
+
+                            int partition = cacheDataRow.partition();
+
+                            long link = h2CacheRow.link();
+
+                            LinkMap map = mappingByPartition.get(partition);
+
+                            long newLink = map.get(link);
+
+                            tracker.complete(READ_MAP);
+
+                            H2CacheRowWithIndex newRow = H2CacheRowWithIndex.create(
+                                rowDesc,
+                                newLink,
+                                h2CacheRow,
+                                ((H2RowLinkIO)io).storeMvccInfo()
+                            );
+
+                            newIdx.putx(newRow);
+
+                            tracker.complete(INSERT_ROW);
+                        }
+
+                        return true;
+                    });
+                }
+            }
+        }
+        finally {
+            cpLock.checkpointReadUnlock();
+        }
+
+        System.out.println(tracker.toString());

Review comment:
       Done.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java
##########
@@ -1678,6 +1678,51 @@ public static String toString(String str,
         }
     }
 
+    /**
+     * Produces uniformed output of string with context properties
+     *
+     * @param str Output prefix or {@code null} if empty.
+     * @param triplets Triplets {@code {name, value, sencitivity}}.
+     * @return String presentation.
+     */
+    public static String toString(String str, Object... triplets) {
+        assert triplets.length % 3 == 0;

Review comment:
       Done.




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r530192979



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?

Review comment:
       Some of them are obsolete and some are fixed in other PRs already. I will try to clean them here




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

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r522227175



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/lang/IgniteThrowableBiFunction.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.util.lang;
+
+import java.io.Serializable;
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ * Represents a bi function that accepts one argument and produces a result. Unlike {@link java.util.function.BiFunction}
+ * it is able to throw {@link IgniteCheckedException}.
+ *
+ * @param <T> The type of the first input to the function.
+ * @param <U> The type of the second input to the function.
+ * @param <R> The type of the result of the function.
+ */
+public interface IgniteThrowableBiFunction<T, U, R> extends Serializable {

Review comment:
       Is this you are looking for `ThrowableBiFunction`?




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r525111847



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/DefragmentationFileUtils.java
##########
@@ -0,0 +1,400 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static java.nio.file.StandardCopyOption.ATOMIC_MOVE;
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static java.nio.file.StandardOpenOption.CREATE_NEW;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.FILE_SUFFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_TEMPLATE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.TMP_SUFFIX;
+
+/**
+ * Everything related to file management during defragmentation process.
+ */
+public class DefragmentationFileUtils {
+    /** Prefix for link mapping files. */
+    private static final String DFRG_LINK_MAPPING_FILE_PREFIX = PART_FILE_PREFIX + "map-";
+
+    /** Link mapping file template. */
+    private static final String DFRG_LINK_MAPPING_FILE_TEMPLATE = DFRG_LINK_MAPPING_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmentation complation marker file name. */
+    private static final String DFRG_COMPLETION_MARKER_FILE_NAME = "dfrg-completion-marker";
+
+    /** Name of defragmentated index partition file. */
+    private static final String DFRG_INDEX_FILE_NAME = INDEX_FILE_PREFIX + "-dfrg" + FILE_SUFFIX;
+
+    /** Name of defragmentated index partition temporary file. */
+    private static final String DFRG_INDEX_TMP_FILE_NAME = DFRG_INDEX_FILE_NAME + TMP_SUFFIX;
+
+    /** Prefix for defragmented partition files. */
+    private static final String DFRG_PARTITION_FILE_PREFIX = PART_FILE_PREFIX + "dfrg-";
+
+    /** Defragmented partition file template. */
+    private static final String DFRG_PARTITION_FILE_TEMPLATE = DFRG_PARTITION_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmented partition temp file template. */
+    private static final String DFRG_PARTITION_TMP_FILE_TEMPLATE = DFRG_PARTITION_FILE_TEMPLATE + TMP_SUFFIX;
+
+    /**
+     * Performs cleanup of work dir before initializing file page stores.
+     * Will finish batch renaming if defragmentation was completed or delete garbage if it wasn't.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     */
+    public static void beforeInitPageStores(File workDir, IgniteLogger log) throws IgniteCheckedException {

Review comment:
       I don't think that we need to inline it - method contains a sensitive piece of logic that needs to be encapsulated in the same class as all other file util methods. It makes more sense this way.




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

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r529626498



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;
+
+                                if (log.isDebugEnabled()) {
+                                    log.debug(S.toString(
+                                        "Partition defragmented",
+                                        "grpId", grpId, false,
+                                        "partId", partId, false,
+                                        "oldPages", oldPageStore.pages(), false,
+                                        "newPages", partCtx.partPagesAllocated.get() + 1, false,
+                                        "mappingPages", partCtx.mappingPagesAllocated.get() + 1, false,
+                                        "pageSize", pageSize, false,
+                                        "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                        "workDir", workDir, false
+                                    ));
+                                }
+
+                                oldPageMem.invalidate(grpId, partId);
+
+                                partCtx.partPageMemory.invalidate(grpId, partId);
+
+                                DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                                pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                                try {
+                                    renameTempPartitionFile(workDir, partId);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    throw new IgniteException(e);
+                                }
+                            }
+                        };
+
+                        GridFutureAdapter<?> cpFut = defragmentationCheckpoint
+                            .forceCheckpoint("partition defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+
+                        cpFut.listen(cpLsnr);
+
+                        cmpFut.add((IgniteInternalFuture<Object>)cpFut);
+                    }
+
+                    // A bit too general for now, but I like it more then saving only the last checkpoint future.
+                    cmpFut.markInitialized().get();
+
+                    idxDfrgFut = new GridFinishedFuture<>();
+
+                    if (filePageStoreMgr.hasIndexStore(grpId)) {
+                        defragmentIndexPartition(oldGrpCtx, newGrpCtx, linkMapByPart);
+
+                        idxDfrgFut = defragmentationCheckpoint
+                            .forceCheckpoint("index defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+                    }
+
+                    idxDfrgFut.listen(fut -> {
+                        oldPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx partPageMem = (PageMemoryEx)partDataRegion.pageMemory();
+
+                        partPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().removePageStore(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx mappingPageMem = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+                        pageMgr = (DefragmentationPageReadWriteManager)mappingPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().clear(grpId);
+
+                        try {
+                            renameTempIndexFile(workDir);
+
+                            writeDefragmentationCompletionMarker(filePageStoreMgr.getPageStoreFileIoFactory(), workDir, log);
+
+                            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    });
+                }
+
+                // I guess we should wait for it?
+                if (idxDfrgFut != null)
+                    idxDfrgFut.get();
+            }
+
+            mntcReg.unregisterMaintenanceTask(DEFRAGMENTATION_MNTC_TASK_NAME);
+
+            log.info("Defragmentation completed. All partitions are defragmented.");
+        }
+        finally {
+            defragmentationCheckpoint.stop(true);
+        }
+    }
+
+    /** */
+    public void createIndexPageStore(
+        int grpId,
+        File workDir,
+        FilePageStoreFactory pageStoreFactory,
+        DataRegion partRegion,
+        LongConsumer allocatedTracker
+    ) throws IgniteCheckedException {
+        // Index partition file has to be deleted before we begin, otherwise there's a chance of reading corrupted file.
+        // There is a time period when index is already defragmented but marker file is not created yet. If node is
+        // failed in that time window then index will be deframented once again. That's fine, situation is rare but code
+        // to fix that would add unnecessary complications.
+        U.delete(defragmentedIndexTmpFile(workDir));
+
+        PageStore idxPageStore;
+
+        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+        try {
+            idxPageStore = pageStoreFactory.createPageStore(
+                FLAG_IDX,
+                () -> defragmentedIndexTmpFile(workDir).toPath(),
+                allocatedTracker
+            );
+        }
+        finally {
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+        }
+
+        idxPageStore.sync();
+
+        PageMemoryEx partPageMem = (PageMemoryEx)partRegion.pageMemory();
+
+        DefragmentationPageReadWriteManager partMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+        partMgr.pageStoreMap().addPageStore(grpId, PageIdAllocator.INDEX_PARTITION, idxPageStore);
+    }
+
+    /** */
+    public enum PartStages {
+        START,
+        CP_LOCK,
+        ITERATE,
+        READ_ROW,
+        INSERT_ROW,
+        STORE_MAP,
+        STORE_PK,
+        STORE_PENDING,
+        METADATA
+    }
+
+    /**
+     * Defragmentate partition.
+     *
+     * @param partCtx
+     * @param treeIter
+     * @throws IgniteCheckedException If failed.
+     */
+    private void copyPartitionData(
+        PartitionContext partCtx,
+        TreeIterator treeIter,
+        GridSpinBusyLock busyLock
+    ) throws IgniteCheckedException {
+        partCtx.createNewCacheDataStore(busyLock);
+
+        CacheDataTree tree = partCtx.oldCacheDataStore.tree();
+
+        CacheDataTree newTree = partCtx.newCacheDataStore.tree();
+        PendingEntriesTree newPendingTree = partCtx.newCacheDataStore.pendingTree();
+        AbstractFreeList<CacheDataRow> freeList = partCtx.newCacheDataStore.getCacheStoreFreeList();
+
+        long cpLockThreshold = 150L;
+
+        TimeTracker<PartStages> tracker = new TimeTracker<>(PartStages.class);
+
+        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+        tracker.complete(CP_LOCK);
+
+        try {
+            AtomicLong lastCpLockTs = new AtomicLong(System.currentTimeMillis());
+            AtomicInteger entriesProcessed = new AtomicInteger();
+
+            treeIter.iterate(tree, partCtx.cachePageMemory, (tree0, io, pageAddr, idx) -> {
+                tracker.complete(ITERATE);
+
+                if (System.currentTimeMillis() - lastCpLockTs.get() >= cpLockThreshold) {
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+                    tracker.complete(CP_LOCK);
+
+                    lastCpLockTs.set(System.currentTimeMillis());
+                }
+
+                AbstractDataLeafIO leafIo = (AbstractDataLeafIO)io;
+                CacheDataRow row = tree.getRow(io, pageAddr, idx);
+
+                tracker.complete(READ_ROW);
+
+                int cacheId = row.cacheId();
+
+                // Reuse row that we just read.
+                row.link(0);
+
+                // "insertDataRow" will corrupt page memory if we don't do this.
+                if (row instanceof DataRow && !partCtx.oldGrpCtx.storeCacheIdInDataPage())
+                    ((DataRow)row).cacheId(CU.UNDEFINED_CACHE_ID);
+
+                freeList.insertDataRow(row, IoStatisticsHolderNoOp.INSTANCE);
+
+                // Put it back.
+                if (row instanceof DataRow)
+                    ((DataRow)row).cacheId(cacheId);
+
+                tracker.complete(INSERT_ROW);
+
+                newTree.putx(row);
+
+                long newLink = row.link();
+
+                tracker.complete(STORE_MAP);
+
+                partCtx.linkMap.put(leafIo.getLink(pageAddr, idx), newLink);
+
+                tracker.complete(STORE_PK);
+
+                if (row.expireTime() != 0)
+                    newPendingTree.putx(new PendingRow(cacheId, row.expireTime(), newLink));
+
+                tracker.complete(STORE_PENDING);
+
+                entriesProcessed.incrementAndGet();
+
+                return true;
+            });
+
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+            tracker.complete(CP_LOCK);
+
+            freeList.saveMetadata(IoStatisticsHolderNoOp.INSTANCE);
+
+            copyCacheMetadata(partCtx);
+
+            tracker.complete(METADATA);
+        }
+        finally {
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+        }
+
+        if (log.isDebugEnabled()) {
+            log.debug(
+                "Partition defragmentation timings for cache group " + partCtx.grpId +
+                " and partition " + partCtx.partId + ": " + tracker.toString()
+            );
+        }
+    }
+
+    /** */
+    private void copyCacheMetadata(
+        PartitionContext partCtx
+    ) throws IgniteCheckedException {
+        // Same for all page memories. Why does it need to be in PageMemory?
+        long partMetaPageId = partCtx.cachePageMemory.partitionMetaPageId(partCtx.grpId, partCtx.partId);
+
+        access(ACCESS_READ, partCtx.cachePageMemory, partCtx.grpId, partMetaPageId, oldPartMetaPageAddr -> {
+            PagePartitionMetaIO oldPartMetaIo = PageIO.getPageIO(oldPartMetaPageAddr);
+
+            // Newer meta versions may contain new data that we don't copy during defragmentation.
+            assert Arrays.asList(1, 2, 3).contains(oldPartMetaIo.getVersion())
+                : "IO version " + oldPartMetaIo.getVersion() + " is not supported by current defragmentation algorithm." +
+                " Please implement copying of all data added in new version.";
+
+            access(ACCESS_WRITE, partCtx.partPageMemory, partCtx.grpId, partMetaPageId, newPartMetaPageAddr -> {
+                PagePartitionMetaIOV3 newPartMetaIo = PageIO.getPageIO(newPartMetaPageAddr);
+
+                // Copy partition state.
+                byte partState = oldPartMetaIo.getPartitionState(oldPartMetaPageAddr);
+                newPartMetaIo.setPartitionState(newPartMetaPageAddr, partState);
+
+                // Copy cache size for single cache group.
+                long size = oldPartMetaIo.getSize(oldPartMetaPageAddr);
+                newPartMetaIo.setSize(newPartMetaPageAddr, size);
+
+                // Copy update counter value.
+                long updateCntr = oldPartMetaIo.getUpdateCounter(oldPartMetaPageAddr);
+                newPartMetaIo.setUpdateCounter(newPartMetaPageAddr, updateCntr);
+
+                // Copy global remove Id.
+                long rmvId = oldPartMetaIo.getGlobalRemoveId(oldPartMetaPageAddr);
+                newPartMetaIo.setGlobalRemoveId(newPartMetaPageAddr, rmvId);
+
+                // Copy cache sizes for shared cache group.
+                long oldCountersPageId = oldPartMetaIo.getCountersPageId(oldPartMetaPageAddr);
+                if (oldCountersPageId != 0L) {
+                    Map<Integer, Long> sizes = GridCacheOffheapManager.readSharedGroupCacheSizes(
+                        partCtx.cachePageMemory,
+                        partCtx.grpId,
+                        oldCountersPageId
+                    );
+
+                    long newCountersPageId = GridCacheOffheapManager.writeSharedGroupCacheSizes(
+                        partCtx.partPageMemory,
+                        partCtx.grpId,
+                        0L,
+                        partCtx.partId,
+                        sizes
+                    );
+
+                    newPartMetaIo.setCountersPageId(newPartMetaPageAddr, newCountersPageId);
+                }
+
+                // Copy counter gaps.
+                long oldGapsLink = oldPartMetaIo.getGapsLink(oldPartMetaPageAddr);
+                if (oldGapsLink != 0L) {
+                    byte[] gapsBytes = partCtx.oldCacheDataStore.partStorage().readRow(oldGapsLink);
+
+                    SimpleDataRow gapsDataRow = new SimpleDataRow(partCtx.partId, gapsBytes);
+
+                    partCtx.newCacheDataStore.partStorage().insertDataRow(gapsDataRow, IoStatisticsHolderNoOp.INSTANCE);
+
+                    newPartMetaIo.setGapsLink(newPartMetaPageAddr, gapsDataRow.link());
+                }
+
+                // Encryption stuff.
+                newPartMetaIo.setEncryptedPageCount(newPartMetaPageAddr, 0);
+                newPartMetaIo.setEncryptedPageIndex(newPartMetaPageAddr, 0);
+
+                return null;
+            });
+
+            return null;
+        });
+    }
+
+    /**
+     * Defragmentate indexing partition.
+     *
+     * @param grpCtx
+     * @param mappingByPartition
+     *
+     * @throws IgniteCheckedException If failed.
+     */
+    private void defragmentIndexPartition(
+        CacheGroupContext grpCtx,
+        CacheGroupContext newCtx,
+        IntMap<LinkMap> mappingByPartition
+    ) throws IgniteCheckedException {
+        GridQueryProcessor query = grpCtx.caches().get(0).kernalContext().query();
+
+        if (!query.moduleEnabled())
+            return;
+
+        final GridQueryIndexing idx = query.getIndexing();
+
+        CheckpointTimeoutLock cpLock = defragmentationCheckpoint.checkpointTimeoutLock();
+
+        idx.defragmentator().defragment(
+            grpCtx,
+            newCtx,
+            (PageMemoryEx)partDataRegion.pageMemory(),
+            mappingByPartition,
+            cpLock,
+            log
+        );
+    }
+
+    /** */
+    @SuppressWarnings("PublicField")
+    private class PartitionContext {
+        /** */
+        public final File workDir;
+
+        /** */
+        public final int grpId;
+
+        /** */
+        public final int partId;
+
+        /** */
+        public final DataRegion cacheDataRegion;
+
+        /** */
+        public final PageMemoryEx cachePageMemory;
+
+        /** */
+        public final PageMemoryEx partPageMemory;
+
+        /** */
+        public final PageMemoryEx mappingPageMemory;
+
+        /** */
+        public final CacheGroupContext oldGrpCtx;
+
+        /** */
+        public final CacheGroupContext newGrpCtx;
+
+        /** */
+        public final CacheDataStore oldCacheDataStore;
+
+        /** */
+        private GridCacheDataStore newCacheDataStore;
+
+        /** */
+        public final FilePageStoreFactory pageStoreFactory;
+
+        /** */
+        public final AtomicLong partPagesAllocated = new AtomicLong();
+
+        /** */
+        public final AtomicLong mappingPagesAllocated = new AtomicLong();
+
+        /** */
+        private LinkMap linkMap;
+
+        /** */
+        public PartitionContext(
+            File workDir,
+            int grpId,
+            int partId,
+            DataRegion partDataRegion,
+            DataRegion mappingDataRegion,
+            CacheGroupContext oldGrpCtx,
+            CacheGroupContext newGrpCtx,
+            CacheDataStore oldCacheDataStore,
+            FilePageStoreFactory pageStoreFactory
+        ) {
+            this.workDir = workDir;
+            this.grpId = grpId;
+            this.partId = partId;
+            cacheDataRegion = oldGrpCtx.dataRegion();
+
+            cachePageMemory = (PageMemoryEx)cacheDataRegion.pageMemory();
+            partPageMemory = (PageMemoryEx)partDataRegion.pageMemory();
+            mappingPageMemory = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+            this.oldGrpCtx = oldGrpCtx;
+            this.newGrpCtx = newGrpCtx;
+            this.oldCacheDataStore = oldCacheDataStore;
+            this.pageStoreFactory = pageStoreFactory;
+        }
+
+        /** */
+        public PageStore createPartPageStore() throws IgniteCheckedException {
+            PageStore partPageStore;
+
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+            try {
+                partPageStore = pageStoreFactory.createPageStore(
+                    FLAG_DATA,
+                    () -> defragmentedPartTmpFile(workDir, partId).toPath(),
+                    partPagesAllocated::addAndGet
+                );
+            }
+            finally {
+                defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+            }
+
+            partPageStore.sync();
+
+            DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMemory.pageManager();
+
+            pageMgr.pageStoreMap().addPageStore(grpId, partId, partPageStore);
+
+            return partPageStore;
+        }
+
+        /** */
+        public PageStore createMappingPageStore() throws IgniteCheckedException {

Review comment:
       `createMappingPageStore`, `createPartPageStore` are almost the same. Is it good practice having such 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.

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r530204183



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;
+
+                                if (log.isDebugEnabled()) {
+                                    log.debug(S.toString(
+                                        "Partition defragmented",
+                                        "grpId", grpId, false,
+                                        "partId", partId, false,
+                                        "oldPages", oldPageStore.pages(), false,
+                                        "newPages", partCtx.partPagesAllocated.get() + 1, false,
+                                        "mappingPages", partCtx.mappingPagesAllocated.get() + 1, false,
+                                        "pageSize", pageSize, false,
+                                        "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                        "workDir", workDir, false
+                                    ));
+                                }
+
+                                oldPageMem.invalidate(grpId, partId);
+
+                                partCtx.partPageMemory.invalidate(grpId, partId);
+
+                                DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                                pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                                try {
+                                    renameTempPartitionFile(workDir, partId);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    throw new IgniteException(e);
+                                }
+                            }
+                        };
+
+                        GridFutureAdapter<?> cpFut = defragmentationCheckpoint
+                            .forceCheckpoint("partition defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+
+                        cpFut.listen(cpLsnr);
+
+                        cmpFut.add((IgniteInternalFuture<Object>)cpFut);
+                    }
+
+                    // A bit too general for now, but I like it more then saving only the last checkpoint future.
+                    cmpFut.markInitialized().get();
+
+                    idxDfrgFut = new GridFinishedFuture<>();
+
+                    if (filePageStoreMgr.hasIndexStore(grpId)) {
+                        defragmentIndexPartition(oldGrpCtx, newGrpCtx, linkMapByPart);
+
+                        idxDfrgFut = defragmentationCheckpoint
+                            .forceCheckpoint("index defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+                    }
+
+                    idxDfrgFut.listen(fut -> {
+                        oldPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx partPageMem = (PageMemoryEx)partDataRegion.pageMemory();
+
+                        partPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().removePageStore(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx mappingPageMem = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+                        pageMgr = (DefragmentationPageReadWriteManager)mappingPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().clear(grpId);
+
+                        try {
+                            renameTempIndexFile(workDir);
+
+                            writeDefragmentationCompletionMarker(filePageStoreMgr.getPageStoreFileIoFactory(), workDir, log);
+
+                            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    });
+                }
+
+                // I guess we should wait for it?
+                if (idxDfrgFut != null)
+                    idxDfrgFut.get();
+            }
+
+            mntcReg.unregisterMaintenanceTask(DEFRAGMENTATION_MNTC_TASK_NAME);
+
+            log.info("Defragmentation completed. All partitions are defragmented.");
+        }
+        finally {
+            defragmentationCheckpoint.stop(true);
+        }
+    }
+
+    /** */
+    public void createIndexPageStore(
+        int grpId,
+        File workDir,
+        FilePageStoreFactory pageStoreFactory,
+        DataRegion partRegion,
+        LongConsumer allocatedTracker
+    ) throws IgniteCheckedException {
+        // Index partition file has to be deleted before we begin, otherwise there's a chance of reading corrupted file.
+        // There is a time period when index is already defragmented but marker file is not created yet. If node is
+        // failed in that time window then index will be deframented once again. That's fine, situation is rare but code
+        // to fix that would add unnecessary complications.
+        U.delete(defragmentedIndexTmpFile(workDir));
+
+        PageStore idxPageStore;
+
+        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+        try {
+            idxPageStore = pageStoreFactory.createPageStore(
+                FLAG_IDX,
+                () -> defragmentedIndexTmpFile(workDir).toPath(),
+                allocatedTracker
+            );
+        }
+        finally {
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+        }
+
+        idxPageStore.sync();
+
+        PageMemoryEx partPageMem = (PageMemoryEx)partRegion.pageMemory();
+
+        DefragmentationPageReadWriteManager partMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+        partMgr.pageStoreMap().addPageStore(grpId, PageIdAllocator.INDEX_PARTITION, idxPageStore);
+    }
+
+    /** */
+    public enum PartStages {
+        START,
+        CP_LOCK,
+        ITERATE,
+        READ_ROW,
+        INSERT_ROW,
+        STORE_MAP,
+        STORE_PK,
+        STORE_PENDING,
+        METADATA
+    }
+
+    /**
+     * Defragmentate partition.
+     *
+     * @param partCtx
+     * @param treeIter
+     * @throws IgniteCheckedException If failed.
+     */
+    private void copyPartitionData(
+        PartitionContext partCtx,
+        TreeIterator treeIter,
+        GridSpinBusyLock busyLock
+    ) throws IgniteCheckedException {
+        partCtx.createNewCacheDataStore(busyLock);
+
+        CacheDataTree tree = partCtx.oldCacheDataStore.tree();
+
+        CacheDataTree newTree = partCtx.newCacheDataStore.tree();
+        PendingEntriesTree newPendingTree = partCtx.newCacheDataStore.pendingTree();
+        AbstractFreeList<CacheDataRow> freeList = partCtx.newCacheDataStore.getCacheStoreFreeList();
+
+        long cpLockThreshold = 150L;
+
+        TimeTracker<PartStages> tracker = new TimeTracker<>(PartStages.class);
+
+        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+        tracker.complete(CP_LOCK);
+
+        try {
+            AtomicLong lastCpLockTs = new AtomicLong(System.currentTimeMillis());
+            AtomicInteger entriesProcessed = new AtomicInteger();
+
+            treeIter.iterate(tree, partCtx.cachePageMemory, (tree0, io, pageAddr, idx) -> {
+                tracker.complete(ITERATE);
+
+                if (System.currentTimeMillis() - lastCpLockTs.get() >= cpLockThreshold) {
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+                    tracker.complete(CP_LOCK);
+
+                    lastCpLockTs.set(System.currentTimeMillis());
+                }
+
+                AbstractDataLeafIO leafIo = (AbstractDataLeafIO)io;
+                CacheDataRow row = tree.getRow(io, pageAddr, idx);
+
+                tracker.complete(READ_ROW);
+
+                int cacheId = row.cacheId();
+
+                // Reuse row that we just read.
+                row.link(0);
+
+                // "insertDataRow" will corrupt page memory if we don't do this.
+                if (row instanceof DataRow && !partCtx.oldGrpCtx.storeCacheIdInDataPage())
+                    ((DataRow)row).cacheId(CU.UNDEFINED_CACHE_ID);
+
+                freeList.insertDataRow(row, IoStatisticsHolderNoOp.INSTANCE);
+
+                // Put it back.
+                if (row instanceof DataRow)
+                    ((DataRow)row).cacheId(cacheId);
+
+                tracker.complete(INSERT_ROW);
+
+                newTree.putx(row);
+
+                long newLink = row.link();
+
+                tracker.complete(STORE_MAP);
+
+                partCtx.linkMap.put(leafIo.getLink(pageAddr, idx), newLink);
+
+                tracker.complete(STORE_PK);
+
+                if (row.expireTime() != 0)
+                    newPendingTree.putx(new PendingRow(cacheId, row.expireTime(), newLink));
+
+                tracker.complete(STORE_PENDING);
+
+                entriesProcessed.incrementAndGet();
+
+                return true;
+            });
+
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+            tracker.complete(CP_LOCK);
+
+            freeList.saveMetadata(IoStatisticsHolderNoOp.INSTANCE);
+
+            copyCacheMetadata(partCtx);
+
+            tracker.complete(METADATA);
+        }
+        finally {
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+        }
+
+        if (log.isDebugEnabled()) {
+            log.debug(
+                "Partition defragmentation timings for cache group " + partCtx.grpId +
+                " and partition " + partCtx.partId + ": " + tracker.toString()
+            );
+        }
+    }
+
+    /** */
+    private void copyCacheMetadata(
+        PartitionContext partCtx
+    ) throws IgniteCheckedException {
+        // Same for all page memories. Why does it need to be in PageMemory?
+        long partMetaPageId = partCtx.cachePageMemory.partitionMetaPageId(partCtx.grpId, partCtx.partId);
+
+        access(ACCESS_READ, partCtx.cachePageMemory, partCtx.grpId, partMetaPageId, oldPartMetaPageAddr -> {
+            PagePartitionMetaIO oldPartMetaIo = PageIO.getPageIO(oldPartMetaPageAddr);
+
+            // Newer meta versions may contain new data that we don't copy during defragmentation.
+            assert Arrays.asList(1, 2, 3).contains(oldPartMetaIo.getVersion())
+                : "IO version " + oldPartMetaIo.getVersion() + " is not supported by current defragmentation algorithm." +
+                " Please implement copying of all data added in new version.";
+
+            access(ACCESS_WRITE, partCtx.partPageMemory, partCtx.grpId, partMetaPageId, newPartMetaPageAddr -> {
+                PagePartitionMetaIOV3 newPartMetaIo = PageIO.getPageIO(newPartMetaPageAddr);
+
+                // Copy partition state.
+                byte partState = oldPartMetaIo.getPartitionState(oldPartMetaPageAddr);
+                newPartMetaIo.setPartitionState(newPartMetaPageAddr, partState);
+
+                // Copy cache size for single cache group.
+                long size = oldPartMetaIo.getSize(oldPartMetaPageAddr);
+                newPartMetaIo.setSize(newPartMetaPageAddr, size);
+
+                // Copy update counter value.
+                long updateCntr = oldPartMetaIo.getUpdateCounter(oldPartMetaPageAddr);
+                newPartMetaIo.setUpdateCounter(newPartMetaPageAddr, updateCntr);
+
+                // Copy global remove Id.
+                long rmvId = oldPartMetaIo.getGlobalRemoveId(oldPartMetaPageAddr);
+                newPartMetaIo.setGlobalRemoveId(newPartMetaPageAddr, rmvId);
+
+                // Copy cache sizes for shared cache group.
+                long oldCountersPageId = oldPartMetaIo.getCountersPageId(oldPartMetaPageAddr);
+                if (oldCountersPageId != 0L) {
+                    Map<Integer, Long> sizes = GridCacheOffheapManager.readSharedGroupCacheSizes(
+                        partCtx.cachePageMemory,
+                        partCtx.grpId,
+                        oldCountersPageId
+                    );
+
+                    long newCountersPageId = GridCacheOffheapManager.writeSharedGroupCacheSizes(
+                        partCtx.partPageMemory,
+                        partCtx.grpId,
+                        0L,
+                        partCtx.partId,
+                        sizes
+                    );
+
+                    newPartMetaIo.setCountersPageId(newPartMetaPageAddr, newCountersPageId);
+                }
+
+                // Copy counter gaps.
+                long oldGapsLink = oldPartMetaIo.getGapsLink(oldPartMetaPageAddr);
+                if (oldGapsLink != 0L) {
+                    byte[] gapsBytes = partCtx.oldCacheDataStore.partStorage().readRow(oldGapsLink);
+
+                    SimpleDataRow gapsDataRow = new SimpleDataRow(partCtx.partId, gapsBytes);
+
+                    partCtx.newCacheDataStore.partStorage().insertDataRow(gapsDataRow, IoStatisticsHolderNoOp.INSTANCE);
+
+                    newPartMetaIo.setGapsLink(newPartMetaPageAddr, gapsDataRow.link());
+                }
+
+                // Encryption stuff.
+                newPartMetaIo.setEncryptedPageCount(newPartMetaPageAddr, 0);
+                newPartMetaIo.setEncryptedPageIndex(newPartMetaPageAddr, 0);
+
+                return null;
+            });
+
+            return null;
+        });
+    }
+
+    /**
+     * Defragmentate indexing partition.
+     *
+     * @param grpCtx
+     * @param mappingByPartition
+     *
+     * @throws IgniteCheckedException If failed.
+     */
+    private void defragmentIndexPartition(
+        CacheGroupContext grpCtx,
+        CacheGroupContext newCtx,
+        IntMap<LinkMap> mappingByPartition
+    ) throws IgniteCheckedException {
+        GridQueryProcessor query = grpCtx.caches().get(0).kernalContext().query();
+
+        if (!query.moduleEnabled())
+            return;
+
+        final GridQueryIndexing idx = query.getIndexing();
+
+        CheckpointTimeoutLock cpLock = defragmentationCheckpoint.checkpointTimeoutLock();
+
+        idx.defragmentator().defragment(
+            grpCtx,
+            newCtx,
+            (PageMemoryEx)partDataRegion.pageMemory(),
+            mappingByPartition,
+            cpLock,
+            log
+        );
+    }
+
+    /** */
+    @SuppressWarnings("PublicField")
+    private class PartitionContext {
+        /** */
+        public final File workDir;
+
+        /** */
+        public final int grpId;
+
+        /** */
+        public final int partId;
+
+        /** */
+        public final DataRegion cacheDataRegion;
+
+        /** */
+        public final PageMemoryEx cachePageMemory;
+
+        /** */
+        public final PageMemoryEx partPageMemory;
+
+        /** */
+        public final PageMemoryEx mappingPageMemory;
+
+        /** */
+        public final CacheGroupContext oldGrpCtx;
+
+        /** */
+        public final CacheGroupContext newGrpCtx;
+
+        /** */
+        public final CacheDataStore oldCacheDataStore;
+
+        /** */
+        private GridCacheDataStore newCacheDataStore;
+
+        /** */
+        public final FilePageStoreFactory pageStoreFactory;
+
+        /** */
+        public final AtomicLong partPagesAllocated = new AtomicLong();
+
+        /** */
+        public final AtomicLong mappingPagesAllocated = new AtomicLong();
+
+        /** */
+        private LinkMap linkMap;
+
+        /** */
+        public PartitionContext(
+            File workDir,
+            int grpId,
+            int partId,
+            DataRegion partDataRegion,
+            DataRegion mappingDataRegion,
+            CacheGroupContext oldGrpCtx,
+            CacheGroupContext newGrpCtx,
+            CacheDataStore oldCacheDataStore,
+            FilePageStoreFactory pageStoreFactory
+        ) {
+            this.workDir = workDir;
+            this.grpId = grpId;
+            this.partId = partId;
+            cacheDataRegion = oldGrpCtx.dataRegion();
+
+            cachePageMemory = (PageMemoryEx)cacheDataRegion.pageMemory();
+            partPageMemory = (PageMemoryEx)partDataRegion.pageMemory();
+            mappingPageMemory = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+            this.oldGrpCtx = oldGrpCtx;
+            this.newGrpCtx = newGrpCtx;
+            this.oldCacheDataStore = oldCacheDataStore;
+            this.pageStoreFactory = pageStoreFactory;
+        }
+
+        /** */
+        public PageStore createPartPageStore() throws IgniteCheckedException {
+            PageStore partPageStore;
+
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+            try {
+                partPageStore = pageStoreFactory.createPageStore(
+                    FLAG_DATA,
+                    () -> defragmentedPartTmpFile(workDir, partId).toPath(),
+                    partPagesAllocated::addAndGet
+                );
+            }
+            finally {
+                defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+            }
+
+            partPageStore.sync();
+
+            DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMemory.pageManager();
+
+            pageMgr.pageStoreMap().addPageStore(grpId, partId, partPageStore);
+
+            return partPageStore;
+        }
+
+        /** */
+        public PageStore createMappingPageStore() throws IgniteCheckedException {
+            PageStore mappingPageStore;
+
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+            try {
+                mappingPageStore = pageStoreFactory.createPageStore(
+                    FLAG_DATA,
+                    () -> defragmentedPartMappingFile(workDir, partId).toPath(),
+                    mappingPagesAllocated::addAndGet
+                );
+            }
+            finally {
+                defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+            }
+
+            mappingPageStore.sync();
+
+            DefragmentationPageReadWriteManager partMgr = (DefragmentationPageReadWriteManager)mappingPageMemory.pageManager();
+
+            partMgr.pageStoreMap().addPageStore(grpId, partId, mappingPageStore);
+
+            return mappingPageStore;
+        }
+
+        /** */
+        public LinkMap createLinkMapTree(boolean initNew) throws IgniteCheckedException {
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+            //TODO Store link in meta page and remove META_PAGE_IDX constant?
+            try {
+                long mappingMetaPageId = initNew
+                    ? mappingPageMemory.allocatePage(grpId, partId, FLAG_DATA)
+                    : PageIdUtils.pageId(partId, FLAG_DATA, LinkMap.META_PAGE_IDX);
+
+                assert PageIdUtils.pageIndex(mappingMetaPageId) == LinkMap.META_PAGE_IDX
+                    : PageIdUtils.toDetailString(mappingMetaPageId);
+
+                linkMap = new LinkMap(newGrpCtx, mappingPageMemory, mappingMetaPageId, initNew);
+            }
+            finally {
+                defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+            }
+
+            return linkMap;
+        }
+
+        /** */
+        public void createNewCacheDataStore(GridSpinBusyLock busyLock) {

Review comment:
       I think we should have defragmentation code separately. We already have issues with classes responsibility scopes, I don't want to make it worse. I'd like to be as far from OffheapManager as possible.




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r530212433



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;
+
+                                if (log.isDebugEnabled()) {
+                                    log.debug(S.toString(
+                                        "Partition defragmented",
+                                        "grpId", grpId, false,
+                                        "partId", partId, false,
+                                        "oldPages", oldPageStore.pages(), false,
+                                        "newPages", partCtx.partPagesAllocated.get() + 1, false,
+                                        "mappingPages", partCtx.mappingPagesAllocated.get() + 1, false,
+                                        "pageSize", pageSize, false,
+                                        "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                        "workDir", workDir, false
+                                    ));
+                                }
+
+                                oldPageMem.invalidate(grpId, partId);
+
+                                partCtx.partPageMemory.invalidate(grpId, partId);
+
+                                DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                                pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                                try {
+                                    renameTempPartitionFile(workDir, partId);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    throw new IgniteException(e);

Review comment:
       Checked exceptions are a standard for Java, that's why I use them when I can.
   Exception will not be ignored, node will fail later because it couldn't complete checkpoint.




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

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r531225240



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();

Review comment:
       Why sharing busyLock is the only available option? You know this code much better than me, can you please describe the problem you faced.




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r530992918



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,887 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database schared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;
+
+                                log.info(S.toString(
+                                    "Partition defragmented",
+                                    "grpId", grpId, false,
+                                    "partId", partId, false,
+                                    "oldPages", oldPageStore.pages(), false,
+                                    "newPages", partCtx.partPagesAllocated.get(), false,
+                                    "bytesSaved", (oldPageStore.pages() - partCtx.partPagesAllocated.get()) * pageSize, false,
+                                    "mappingPages", partCtx.mappingPagesAllocated.get(), false,
+                                    "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                    "workDir", workDir, false
+                                ));
+
+                                oldPageMem.invalidate(grpId, partId);
+
+                                partCtx.partPageMemory.invalidate(grpId, partId);
+
+                                DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                                pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                                try {
+                                    renameTempPartitionFile(workDir, partId);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    throw new IgniteException(e);
+                                }
+                            }
+                        };
+
+                        GridFutureAdapter<?> cpFut = defragmentationCheckpoint
+                            .forceCheckpoint("partition defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+
+                        cpFut.listen(cpLsnr);
+
+                        cmpFut.add((IgniteInternalFuture<Object>)cpFut);
+                    }
+
+                    // A bit too general for now, but I like it more then saving only the last checkpoint future.
+                    cmpFut.markInitialized().get();
+
+                    idxDfrgFut = new GridFinishedFuture<>();
+
+                    if (filePageStoreMgr.hasIndexStore(grpId)) {
+                        defragmentIndexPartition(oldGrpCtx, newGrpCtx, linkMapByPart);
+
+                        idxDfrgFut = defragmentationCheckpoint
+                            .forceCheckpoint("index defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+                    }
+
+                    idxDfrgFut.listen(fut -> {
+                        oldPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx partPageMem = (PageMemoryEx)partDataRegion.pageMemory();
+
+                        partPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().removePageStore(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx mappingPageMem = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+                        pageMgr = (DefragmentationPageReadWriteManager)mappingPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().clear(grpId);
+
+                        try {
+                            renameTempIndexFile(workDir);
+
+                            writeDefragmentationCompletionMarker(filePageStoreMgr.getPageStoreFileIoFactory(), workDir, log);
+
+                            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    });
+                }
+
+                // I guess we should wait for it?
+                if (idxDfrgFut != null)
+                    idxDfrgFut.get();
+            }
+
+            mntcReg.unregisterMaintenanceTask(DEFRAGMENTATION_MNTC_TASK_NAME);
+
+            log.info("Defragmentation completed. All partitions are defragmented.");

Review comment:
       https://issues.apache.org/jira/browse/IGNITE-13743




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r532435528



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/DefragmentationPageReadWriteManager.java
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageReadWriteManagerImpl;
+
+/** */
+public class DefragmentationPageReadWriteManager extends PageReadWriteManagerImpl {

Review comment:
       > Please, remove it.
   
   Please show me how do I use TimeBag for the same scenario. I'm sure that I can't do that.
   
   > It will be simpler and better if you'll reuse try-finally approach for all these cases.
   
   Ok, I'm removing all this code.

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/defragmentation/IndexingDefragmentation.java
##########
@@ -0,0 +1,467 @@
+/*
+ * 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.processors.query.h2.defragmentation;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.LinkMap;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.TimeTracker;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIoResolver;
+import org.apache.ignite.internal.processors.cache.persistence.tree.util.InsertLast;
+import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccDataRow;
+import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+import org.apache.ignite.internal.processors.query.h2.database.H2Tree;
+import org.apache.ignite.internal.processors.query.h2.database.H2TreeIndex;
+import org.apache.ignite.internal.processors.query.h2.database.InlineIndexColumn;
+import org.apache.ignite.internal.processors.query.h2.database.inlinecolumn.AbstractInlineIndexColumn;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2ExtrasInnerIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2ExtrasLeafIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2InnerIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2LeafIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.H2RowLinkIO;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.processors.query.h2.opt.H2CacheRow;
+import org.apache.ignite.internal.processors.query.h2.opt.H2Row;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.h2.index.Index;
+import org.h2.value.Value;
+
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.INIT_TREE;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.ITERATE;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.READ_MAP;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.READ_ROW;
+
+/**
+ *
+ */
+public class IndexingDefragmentation {
+    /** Indexing. */
+    private final IgniteH2Indexing indexing;
+
+    /** Constructor. */
+    public IndexingDefragmentation(IgniteH2Indexing indexing) {
+        this.indexing = indexing;
+    }
+
+    /** */
+    public enum IndexStages {
+        START,
+        CP_LOCK,
+        INIT_TREE,
+        ITERATE,
+        READ_ROW,
+        READ_MAP,
+        INSERT_ROW
+    }
+
+    /**
+     * Defragment index partition.
+     *
+     * @param grpCtx Old group context.
+     * @param newCtx New group context.
+     * @param partPageMem Partition page memory.
+     * @param mappingByPartition Mapping page memory.
+     * @param cpLock Defragmentation checkpoint read lock.
+     * @param log Log.
+     *
+     * @throws IgniteCheckedException If failed.
+     */
+    public void defragment(
+        CacheGroupContext grpCtx,
+        CacheGroupContext newCtx,
+        PageMemoryEx partPageMem,
+        IntMap<LinkMap> mappingByPartition,
+        CheckpointTimeoutLock cpLock,
+        IgniteLogger log
+    ) throws IgniteCheckedException {
+        int pageSize = grpCtx.cacheObjectContext().kernalContext().grid().configuration().getDataStorageConfiguration().getPageSize();
+
+        TreeIterator treeIterator = new TreeIterator(pageSize);
+
+        PageMemoryEx oldCachePageMem = (PageMemoryEx)grpCtx.dataRegion().pageMemory();
+
+        PageMemory newCachePageMemory = partPageMem;
+
+        Collection<GridH2Table> tables = indexing.schemaManager().dataTables();
+
+        long cpLockThreshold = 150L;
+
+        TimeTracker<IndexStages> tracker = new TimeTracker<>(IndexStages.class);
+
+        cpLock.checkpointReadLock();
+        tracker.complete(CP_LOCK);
+
+        try {
+            AtomicLong lastCpLockTs = new AtomicLong(System.currentTimeMillis());
+
+            for (GridH2Table table : tables) {
+                GridCacheContext<?, ?> cctx = table.cacheContext();
+
+                if (cctx.groupId() != grpCtx.groupId())
+                    continue; // Not our index.
+
+                GridH2RowDescriptor rowDesc = table.rowDescriptor();
+
+                List<Index> indexes = table.getIndexes();
+                H2TreeIndex oldH2Idx = (H2TreeIndex)indexes.get(2);
+
+                int segments = oldH2Idx.segmentsCount();
+
+                H2Tree firstTree = oldH2Idx.treeForRead(0);
+
+                PageIoResolver pageIoRslvr = pageAddr -> {
+                    PageIO io = PageIoResolver.DEFAULT_PAGE_IO_RESOLVER.resolve(pageAddr);
+
+                    if (io instanceof BPlusMetaIO)
+                        return io;
+
+                    //noinspection unchecked,rawtypes,rawtypes
+                    return wrap((BPlusIO)io);
+                };
+
+                //TODO Create new proper GridCacheContext for it?

Review comment:
       Done




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r532412171



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,861 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.lang.IgniteOutClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.

Review comment:
       Removed, there are no races




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r531457700



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;
+
+                                if (log.isDebugEnabled()) {
+                                    log.debug(S.toString(
+                                        "Partition defragmented",
+                                        "grpId", grpId, false,
+                                        "partId", partId, false,
+                                        "oldPages", oldPageStore.pages(), false,
+                                        "newPages", partCtx.partPagesAllocated.get() + 1, false,
+                                        "mappingPages", partCtx.mappingPagesAllocated.get() + 1, false,
+                                        "pageSize", pageSize, false,
+                                        "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                        "workDir", workDir, false
+                                    ));
+                                }
+
+                                oldPageMem.invalidate(grpId, partId);
+
+                                partCtx.partPageMemory.invalidate(grpId, partId);
+
+                                DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                                pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                                try {
+                                    renameTempPartitionFile(workDir, partId);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    throw new IgniteException(e);

Review comment:
       Ok, I'm cool with it, not a big deal. It'll make other places more complicated though




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

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



[GitHub] [ignite] agoncharuk commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
agoncharuk commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r524315640



##########
File path: modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsIndexingDefragmentationTest.java
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.processors.cache.persistence;
+
+import java.io.File;
+import java.util.Collections;
+import java.util.Objects;
+import java.util.function.Function;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.visor.verify.ValidateIndexesClosure;
+import org.apache.ignite.internal.visor.verify.VisorValidateIndexesJobResult;
+import org.apache.ignite.testframework.junits.WithSystemProperty;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
+/**
+ * Defragmentation tests with enabled ignite-indexing.
+ */
+public class IgnitePdsIndexingDefragmentationTest extends IgnitePdsDefragmentationTest {
+    /** Use MVCC in tests. */
+    private static final String USE_MVCC = "USE_MVCC";
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setConsistentId(igniteInstanceName);
+
+        DataStorageConfiguration dsCfg = new DataStorageConfiguration();
+        dsCfg.setWalSegmentSize(4 * 1024 * 1024);
+
+        dsCfg.setDefaultDataRegionConfiguration(
+            new DataRegionConfiguration()
+                .setInitialSize(100L * 1024 * 1024)
+                .setMaxSize(1024L * 1024 * 1024)
+                .setPersistenceEnabled(true)
+        );
+
+        cfg.setDataStorageConfiguration(dsCfg);
+
+        CacheConfiguration<?, ?> cache1Cfg = new CacheConfiguration<>(DEFAULT_CACHE_NAME)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setGroupName(GRP_NAME)
+            .setIndexedTypes(
+                ObjKey.class, byte[].class,
+                Integer.class, byte[].class
+            )
+            .setAffinity(new RendezvousAffinityFunction(false, PARTS));
+
+        CacheConfiguration<?, ?> cache2Cfg = new CacheConfiguration<>(CACHE_2_NAME)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setGroupName(GRP_NAME)
+            .setIndexedTypes(
+                ObjKey.class, byte[].class,
+                Integer.class, byte[].class
+            )
+            .setAffinity(new RendezvousAffinityFunction(false, PARTS));
+
+        if (Boolean.TRUE.toString().equals(System.getProperty(USE_MVCC))) {
+            cache1Cfg.setAtomicityMode(TRANSACTIONAL_SNAPSHOT);
+            cache2Cfg.setAtomicityMode(TRANSACTIONAL_SNAPSHOT);
+        } else
+            cache2Cfg.setExpiryPolicyFactory(new PolicyFactory());
+
+        cfg.setCacheConfiguration(cache1Cfg, cache2Cfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        GridQueryProcessor.idxCls = null;
+    }
+
+    /**
+     * Fill cache, remove half of the entries, defragmentate PDS and check index.
+     *
+     * @param keyMapper Function that provides key based on the index of entry.
+     * @param <T> Type of cache key.
+     *
+     * @throws Exception If failed.
+     */
+    private <T> void test(Function<Integer, T> keyMapper) throws Exception {
+        IgniteEx ig = startGrid(0);
+
+        ig.cluster().state(ClusterState.ACTIVE);
+
+        fillCache(keyMapper, ig.cache(DEFAULT_CACHE_NAME));
+
+        forceCheckpoint(ig);
+
+        createMaintenanceRecord();
+
+        stopGrid(0);
+
+        File dbWorkDir = U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false);
+        File nodeWorkDir = new File(dbWorkDir, U.maskForFileName(ig.name()));
+        File workDir = new File(nodeWorkDir, FilePageStoreManager.CACHE_GRP_DIR_PREFIX + GRP_NAME);
+
+        long oldIdxFileLen = new File(workDir, FilePageStoreManager.INDEX_FILE_NAME).length();
+
+        startGrid(0);
+
+        long newIdxFileLen = new File(workDir, FilePageStoreManager.INDEX_FILE_NAME).length();
+
+        assertTrue(newIdxFileLen <= oldIdxFileLen);
+
+        File completionMarkerFile = DefragmentationFileUtils.defragmentationCompletionMarkerFile(workDir);
+        assertTrue(completionMarkerFile.exists());
+
+        stopGrid(0);
+
+        GridQueryProcessor.idxCls = CaptureRebuildGridQueryIndexing.class;
+
+        IgniteEx node = startGrid(0);
+
+        awaitPartitionMapExchange();
+
+        CaptureRebuildGridQueryIndexing indexing = (CaptureRebuildGridQueryIndexing) node.context().query().getIndexing();
+
+        assertFalse(indexing.didRebuildIndexes());
+
+        IgniteCache<Object, Object> cache = node.cache(DEFAULT_CACHE_NAME);
+
+        assertFalse(completionMarkerFile.exists());
+
+        validateIndexes(node);

Review comment:
       Let's also check that indexes are used when issuing an SQL query after defragmentation.




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

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r529606712



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/TreeIterator.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.nio.ByteBuffer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.util.GridUnsafe;
+
+/** */
+public class TreeIterator {
+    /** Direct memory buffer with a size of one page. */
+    private final ByteBuffer pageBuf;
+
+    /** Offheap page size. */
+    private final int pageSize;
+
+    /** */
+    public TreeIterator(int size) {
+        pageSize = size;
+
+        pageBuf = ByteBuffer.allocateDirect(pageSize);
+    }
+
+    // Performance impact of constant closures allocation is not clear. So this method should be avoided in massive
+    // operations like tree leaves access.
+    /** */
+    public static <T> T access(
+        PageAccessType access,
+        PageMemoryEx pageMemory,
+        int grpId,
+        long pageId,
+        PageAccessor<T> accessor
+    ) throws IgniteCheckedException {
+        assert access != null;

Review comment:
       Should we add assert here `PageAccessType.ACCESS_READ | PageAccessType.ACCESS_WRITE`? In case of extending `PageAccessType` we will get an incorrect results




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r530201129



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;
+
+                                if (log.isDebugEnabled()) {
+                                    log.debug(S.toString(
+                                        "Partition defragmented",
+                                        "grpId", grpId, false,
+                                        "partId", partId, false,
+                                        "oldPages", oldPageStore.pages(), false,
+                                        "newPages", partCtx.partPagesAllocated.get() + 1, false,
+                                        "mappingPages", partCtx.mappingPagesAllocated.get() + 1, false,
+                                        "pageSize", pageSize, false,
+                                        "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                        "workDir", workDir, false
+                                    ));
+                                }
+
+                                oldPageMem.invalidate(grpId, partId);
+
+                                partCtx.partPageMemory.invalidate(grpId, partId);
+
+                                DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                                pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                                try {
+                                    renameTempPartitionFile(workDir, partId);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    throw new IgniteException(e);
+                                }
+                            }
+                        };
+
+                        GridFutureAdapter<?> cpFut = defragmentationCheckpoint
+                            .forceCheckpoint("partition defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+
+                        cpFut.listen(cpLsnr);
+
+                        cmpFut.add((IgniteInternalFuture<Object>)cpFut);
+                    }
+
+                    // A bit too general for now, but I like it more then saving only the last checkpoint future.
+                    cmpFut.markInitialized().get();
+
+                    idxDfrgFut = new GridFinishedFuture<>();
+
+                    if (filePageStoreMgr.hasIndexStore(grpId)) {
+                        defragmentIndexPartition(oldGrpCtx, newGrpCtx, linkMapByPart);
+
+                        idxDfrgFut = defragmentationCheckpoint
+                            .forceCheckpoint("index defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+                    }
+
+                    idxDfrgFut.listen(fut -> {
+                        oldPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx partPageMem = (PageMemoryEx)partDataRegion.pageMemory();
+
+                        partPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().removePageStore(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx mappingPageMem = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+                        pageMgr = (DefragmentationPageReadWriteManager)mappingPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().clear(grpId);
+
+                        try {
+                            renameTempIndexFile(workDir);
+
+                            writeDefragmentationCompletionMarker(filePageStoreMgr.getPageStoreFileIoFactory(), workDir, log);
+
+                            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    });
+                }
+
+                // I guess we should wait for it?
+                if (idxDfrgFut != null)
+                    idxDfrgFut.get();
+            }
+
+            mntcReg.unregisterMaintenanceTask(DEFRAGMENTATION_MNTC_TASK_NAME);
+
+            log.info("Defragmentation completed. All partitions are defragmented.");
+        }
+        finally {
+            defragmentationCheckpoint.stop(true);
+        }
+    }
+
+    /** */
+    public void createIndexPageStore(
+        int grpId,
+        File workDir,
+        FilePageStoreFactory pageStoreFactory,
+        DataRegion partRegion,
+        LongConsumer allocatedTracker
+    ) throws IgniteCheckedException {
+        // Index partition file has to be deleted before we begin, otherwise there's a chance of reading corrupted file.
+        // There is a time period when index is already defragmented but marker file is not created yet. If node is
+        // failed in that time window then index will be deframented once again. That's fine, situation is rare but code
+        // to fix that would add unnecessary complications.
+        U.delete(defragmentedIndexTmpFile(workDir));
+
+        PageStore idxPageStore;
+
+        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+        try {
+            idxPageStore = pageStoreFactory.createPageStore(
+                FLAG_IDX,
+                () -> defragmentedIndexTmpFile(workDir).toPath(),
+                allocatedTracker
+            );
+        }
+        finally {
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+        }
+
+        idxPageStore.sync();
+
+        PageMemoryEx partPageMem = (PageMemoryEx)partRegion.pageMemory();
+
+        DefragmentationPageReadWriteManager partMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+        partMgr.pageStoreMap().addPageStore(grpId, PageIdAllocator.INDEX_PARTITION, idxPageStore);
+    }
+
+    /** */
+    public enum PartStages {
+        START,
+        CP_LOCK,
+        ITERATE,
+        READ_ROW,
+        INSERT_ROW,
+        STORE_MAP,
+        STORE_PK,
+        STORE_PENDING,
+        METADATA
+    }
+
+    /**
+     * Defragmentate partition.
+     *
+     * @param partCtx
+     * @param treeIter
+     * @throws IgniteCheckedException If failed.
+     */
+    private void copyPartitionData(
+        PartitionContext partCtx,
+        TreeIterator treeIter,
+        GridSpinBusyLock busyLock
+    ) throws IgniteCheckedException {
+        partCtx.createNewCacheDataStore(busyLock);
+
+        CacheDataTree tree = partCtx.oldCacheDataStore.tree();
+
+        CacheDataTree newTree = partCtx.newCacheDataStore.tree();
+        PendingEntriesTree newPendingTree = partCtx.newCacheDataStore.pendingTree();
+        AbstractFreeList<CacheDataRow> freeList = partCtx.newCacheDataStore.getCacheStoreFreeList();
+
+        long cpLockThreshold = 150L;
+
+        TimeTracker<PartStages> tracker = new TimeTracker<>(PartStages.class);
+
+        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+        tracker.complete(CP_LOCK);
+
+        try {
+            AtomicLong lastCpLockTs = new AtomicLong(System.currentTimeMillis());
+            AtomicInteger entriesProcessed = new AtomicInteger();
+
+            treeIter.iterate(tree, partCtx.cachePageMemory, (tree0, io, pageAddr, idx) -> {
+                tracker.complete(ITERATE);
+
+                if (System.currentTimeMillis() - lastCpLockTs.get() >= cpLockThreshold) {
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+                    tracker.complete(CP_LOCK);
+
+                    lastCpLockTs.set(System.currentTimeMillis());
+                }
+
+                AbstractDataLeafIO leafIo = (AbstractDataLeafIO)io;
+                CacheDataRow row = tree.getRow(io, pageAddr, idx);
+
+                tracker.complete(READ_ROW);
+
+                int cacheId = row.cacheId();
+
+                // Reuse row that we just read.
+                row.link(0);
+
+                // "insertDataRow" will corrupt page memory if we don't do this.
+                if (row instanceof DataRow && !partCtx.oldGrpCtx.storeCacheIdInDataPage())
+                    ((DataRow)row).cacheId(CU.UNDEFINED_CACHE_ID);
+
+                freeList.insertDataRow(row, IoStatisticsHolderNoOp.INSTANCE);
+
+                // Put it back.
+                if (row instanceof DataRow)
+                    ((DataRow)row).cacheId(cacheId);
+
+                tracker.complete(INSERT_ROW);
+
+                newTree.putx(row);
+
+                long newLink = row.link();
+
+                tracker.complete(STORE_MAP);
+
+                partCtx.linkMap.put(leafIo.getLink(pageAddr, idx), newLink);
+
+                tracker.complete(STORE_PK);
+
+                if (row.expireTime() != 0)
+                    newPendingTree.putx(new PendingRow(cacheId, row.expireTime(), newLink));
+
+                tracker.complete(STORE_PENDING);
+
+                entriesProcessed.incrementAndGet();
+
+                return true;
+            });
+
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+            tracker.complete(CP_LOCK);
+
+            freeList.saveMetadata(IoStatisticsHolderNoOp.INSTANCE);
+
+            copyCacheMetadata(partCtx);
+
+            tracker.complete(METADATA);
+        }
+        finally {
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+        }
+
+        if (log.isDebugEnabled()) {
+            log.debug(
+                "Partition defragmentation timings for cache group " + partCtx.grpId +
+                " and partition " + partCtx.partId + ": " + tracker.toString()
+            );
+        }
+    }
+
+    /** */
+    private void copyCacheMetadata(
+        PartitionContext partCtx
+    ) throws IgniteCheckedException {
+        // Same for all page memories. Why does it need to be in PageMemory?
+        long partMetaPageId = partCtx.cachePageMemory.partitionMetaPageId(partCtx.grpId, partCtx.partId);
+
+        access(ACCESS_READ, partCtx.cachePageMemory, partCtx.grpId, partMetaPageId, oldPartMetaPageAddr -> {
+            PagePartitionMetaIO oldPartMetaIo = PageIO.getPageIO(oldPartMetaPageAddr);
+
+            // Newer meta versions may contain new data that we don't copy during defragmentation.
+            assert Arrays.asList(1, 2, 3).contains(oldPartMetaIo.getVersion())
+                : "IO version " + oldPartMetaIo.getVersion() + " is not supported by current defragmentation algorithm." +
+                " Please implement copying of all data added in new version.";
+
+            access(ACCESS_WRITE, partCtx.partPageMemory, partCtx.grpId, partMetaPageId, newPartMetaPageAddr -> {
+                PagePartitionMetaIOV3 newPartMetaIo = PageIO.getPageIO(newPartMetaPageAddr);
+
+                // Copy partition state.
+                byte partState = oldPartMetaIo.getPartitionState(oldPartMetaPageAddr);
+                newPartMetaIo.setPartitionState(newPartMetaPageAddr, partState);
+
+                // Copy cache size for single cache group.
+                long size = oldPartMetaIo.getSize(oldPartMetaPageAddr);
+                newPartMetaIo.setSize(newPartMetaPageAddr, size);
+
+                // Copy update counter value.
+                long updateCntr = oldPartMetaIo.getUpdateCounter(oldPartMetaPageAddr);
+                newPartMetaIo.setUpdateCounter(newPartMetaPageAddr, updateCntr);
+
+                // Copy global remove Id.
+                long rmvId = oldPartMetaIo.getGlobalRemoveId(oldPartMetaPageAddr);
+                newPartMetaIo.setGlobalRemoveId(newPartMetaPageAddr, rmvId);
+
+                // Copy cache sizes for shared cache group.
+                long oldCountersPageId = oldPartMetaIo.getCountersPageId(oldPartMetaPageAddr);
+                if (oldCountersPageId != 0L) {
+                    Map<Integer, Long> sizes = GridCacheOffheapManager.readSharedGroupCacheSizes(
+                        partCtx.cachePageMemory,
+                        partCtx.grpId,
+                        oldCountersPageId
+                    );
+
+                    long newCountersPageId = GridCacheOffheapManager.writeSharedGroupCacheSizes(
+                        partCtx.partPageMemory,
+                        partCtx.grpId,
+                        0L,
+                        partCtx.partId,
+                        sizes
+                    );
+
+                    newPartMetaIo.setCountersPageId(newPartMetaPageAddr, newCountersPageId);
+                }
+
+                // Copy counter gaps.
+                long oldGapsLink = oldPartMetaIo.getGapsLink(oldPartMetaPageAddr);
+                if (oldGapsLink != 0L) {
+                    byte[] gapsBytes = partCtx.oldCacheDataStore.partStorage().readRow(oldGapsLink);
+
+                    SimpleDataRow gapsDataRow = new SimpleDataRow(partCtx.partId, gapsBytes);
+
+                    partCtx.newCacheDataStore.partStorage().insertDataRow(gapsDataRow, IoStatisticsHolderNoOp.INSTANCE);
+
+                    newPartMetaIo.setGapsLink(newPartMetaPageAddr, gapsDataRow.link());
+                }
+
+                // Encryption stuff.
+                newPartMetaIo.setEncryptedPageCount(newPartMetaPageAddr, 0);
+                newPartMetaIo.setEncryptedPageIndex(newPartMetaPageAddr, 0);
+
+                return null;
+            });
+
+            return null;
+        });
+    }
+
+    /**
+     * Defragmentate indexing partition.
+     *
+     * @param grpCtx
+     * @param mappingByPartition
+     *
+     * @throws IgniteCheckedException If failed.
+     */
+    private void defragmentIndexPartition(
+        CacheGroupContext grpCtx,
+        CacheGroupContext newCtx,
+        IntMap<LinkMap> mappingByPartition
+    ) throws IgniteCheckedException {
+        GridQueryProcessor query = grpCtx.caches().get(0).kernalContext().query();
+
+        if (!query.moduleEnabled())
+            return;
+
+        final GridQueryIndexing idx = query.getIndexing();
+
+        CheckpointTimeoutLock cpLock = defragmentationCheckpoint.checkpointTimeoutLock();
+
+        idx.defragmentator().defragment(
+            grpCtx,
+            newCtx,
+            (PageMemoryEx)partDataRegion.pageMemory(),
+            mappingByPartition,
+            cpLock,
+            log
+        );
+    }
+
+    /** */
+    @SuppressWarnings("PublicField")
+    private class PartitionContext {
+        /** */
+        public final File workDir;
+
+        /** */
+        public final int grpId;
+
+        /** */
+        public final int partId;
+
+        /** */
+        public final DataRegion cacheDataRegion;
+
+        /** */
+        public final PageMemoryEx cachePageMemory;
+
+        /** */
+        public final PageMemoryEx partPageMemory;
+
+        /** */
+        public final PageMemoryEx mappingPageMemory;
+
+        /** */
+        public final CacheGroupContext oldGrpCtx;
+
+        /** */
+        public final CacheGroupContext newGrpCtx;
+
+        /** */
+        public final CacheDataStore oldCacheDataStore;
+
+        /** */
+        private GridCacheDataStore newCacheDataStore;
+
+        /** */
+        public final FilePageStoreFactory pageStoreFactory;
+
+        /** */
+        public final AtomicLong partPagesAllocated = new AtomicLong();
+
+        /** */
+        public final AtomicLong mappingPagesAllocated = new AtomicLong();
+
+        /** */
+        private LinkMap linkMap;
+
+        /** */
+        public PartitionContext(
+            File workDir,
+            int grpId,
+            int partId,
+            DataRegion partDataRegion,
+            DataRegion mappingDataRegion,
+            CacheGroupContext oldGrpCtx,
+            CacheGroupContext newGrpCtx,
+            CacheDataStore oldCacheDataStore,
+            FilePageStoreFactory pageStoreFactory
+        ) {
+            this.workDir = workDir;
+            this.grpId = grpId;
+            this.partId = partId;
+            cacheDataRegion = oldGrpCtx.dataRegion();
+
+            cachePageMemory = (PageMemoryEx)cacheDataRegion.pageMemory();
+            partPageMemory = (PageMemoryEx)partDataRegion.pageMemory();
+            mappingPageMemory = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+            this.oldGrpCtx = oldGrpCtx;
+            this.newGrpCtx = newGrpCtx;
+            this.oldCacheDataStore = oldCacheDataStore;
+            this.pageStoreFactory = pageStoreFactory;
+        }
+
+        /** */
+        public PageStore createPartPageStore() throws IgniteCheckedException {
+            PageStore partPageStore;
+
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+            try {
+                partPageStore = pageStoreFactory.createPageStore(
+                    FLAG_DATA,
+                    () -> defragmentedPartTmpFile(workDir, partId).toPath(),
+                    partPagesAllocated::addAndGet
+                );
+            }
+            finally {
+                defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+            }
+
+            partPageStore.sync();
+
+            DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMemory.pageManager();
+
+            pageMgr.pageStoreMap().addPageStore(grpId, partId, partPageStore);
+
+            return partPageStore;
+        }
+
+        /** */
+        public PageStore createMappingPageStore() throws IgniteCheckedException {

Review comment:
       I'll merge them




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

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r525126152



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/TimeTracker.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.util.Arrays;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/** */
+public class TimeTracker<Stage extends Enum<Stage>> {

Review comment:
       Can you clarify why we need accumulated timings per each stage?




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r530205083



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/DefragmentationFileUtils.java
##########
@@ -0,0 +1,395 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static java.nio.file.StandardCopyOption.ATOMIC_MOVE;
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static java.nio.file.StandardOpenOption.CREATE_NEW;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.FILE_SUFFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_TEMPLATE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.TMP_SUFFIX;
+
+/**
+ * Everything related to file management during defragmentation process.
+ */
+public class DefragmentationFileUtils {
+    /** Prefix for link mapping files. */
+    private static final String DFRG_LINK_MAPPING_FILE_PREFIX = PART_FILE_PREFIX + "map-";
+
+    /** Link mapping file template. */
+    private static final String DFRG_LINK_MAPPING_FILE_TEMPLATE = DFRG_LINK_MAPPING_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmentation complation marker file name. */
+    private static final String DFRG_COMPLETION_MARKER_FILE_NAME = "dfrg-completion-marker";
+
+    /** Name of defragmentated index partition file. */
+    private static final String DFRG_INDEX_FILE_NAME = INDEX_FILE_PREFIX + "-dfrg" + FILE_SUFFIX;
+
+    /** Name of defragmentated index partition temporary file. */
+    private static final String DFRG_INDEX_TMP_FILE_NAME = DFRG_INDEX_FILE_NAME + TMP_SUFFIX;
+
+    /** Prefix for defragmented partition files. */
+    private static final String DFRG_PARTITION_FILE_PREFIX = PART_FILE_PREFIX + "dfrg-";
+
+    /** Defragmented partition file template. */
+    private static final String DFRG_PARTITION_FILE_TEMPLATE = DFRG_PARTITION_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmented partition temp file template. */
+    private static final String DFRG_PARTITION_TMP_FILE_TEMPLATE = DFRG_PARTITION_FILE_TEMPLATE + TMP_SUFFIX;
+
+    /**
+     * Performs cleanup of work dir before initializing file page stores.
+     * Will finish batch renaming if defragmentation was completed or delete garbage if it wasn't.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     */
+    public static void beforeInitPageStores(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+        U.delete(defragmentationCompletionMarkerFile(workDir));
+
+        for (File file : workDir.listFiles()) {
+            String fileName = file.getName();
+
+            if (
+                fileName.startsWith(DFRG_PARTITION_FILE_PREFIX)
+                || fileName.startsWith(DFRG_INDEX_FILE_NAME)
+                || fileName.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)
+            )
+                U.delete(file);
+        }
+    }
+
+    /**
+     * Checks whether cache group defragmentation completed or not. Completes it if all that's left is renaming.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param log Logger to write messages.
+     * @return {@code true} if given cache group is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentationCompletionMarkerFile(File)
+     */
+    public static boolean skipAlreadyDefragmentedCacheGroup(File workDir, int grpId, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (completionMarkerFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented page group",
+                    "grpId", grpId, false,
+                    "markerFileName", completionMarkerFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+            return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * Checks whether partition has already been defragmented or not. Cleans corrupted data if previous failed
+     * defragmentation attempt was found.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param partId Partition index to check.
+     * @param log Logger to write messages.
+     * @return {@code true} if given partition is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartMappingFile(File, int)
+     */
+    public static boolean skipAlreadyDefragmentedPartition(File workDir, int grpId, int partId, IgniteLogger log) throws IgniteCheckedException {
+        File defragmentedPartFile = defragmentedPartFile(workDir, partId);
+        File defragmentedPartMappingFile = defragmentedPartMappingFile(workDir, partId);
+
+        if (defragmentedPartFile.exists() && defragmentedPartMappingFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented partition",
+                    "grpId", grpId, false,
+                    "partId", partId, false,
+                    "partFileName", defragmentedPartFile.getName(), false,
+                    "mappingFileName", defragmentedPartMappingFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            return true;
+        }
+
+        File defragmentedPartTmpFile = defragmentedPartTmpFile(workDir, partId);
+
+        try {
+            Files.deleteIfExists(defragmentedPartTmpFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartMappingFile.toPath());
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException(e);
+        }
+
+        return false;
+    }
+
+    /**
+     * Failure-tolerant batch rename of defragmented partition files.
+     *
+     * Deletes all link mapping files old partition and index files, renaming defragmentated files in the process. Can
+     * be run on the same folder multiple times if failed for some reason.
+     *
+     * Does something only if completion marker is present in the folder. This marker won't be deleted in the end.
+     * Deletion of the marker must be done outside of defragmentation mode to prevent cache groups to be defragmentated
+     * several times in case of failures.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#writeDefragmentationCompletionMarker(FileIOFactory, File, IgniteLogger)
+     */
+    public static void batchRenameDefragmentedCacheGroupPartitions(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (!completionMarkerFile.exists())
+            return;
+
+        try {
+            for (File mappingFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)))
+                Files.delete(mappingFile.toPath());
+
+            for (File partFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_PARTITION_FILE_PREFIX))) {
+                int partId = extractPartId(partFile.getName());
+
+                File oldPartFile = new File(workDir, String.format(PART_FILE_TEMPLATE, partId));
+
+                Files.move(partFile.toPath(), oldPartFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+
+            File idxFile = new File(workDir, DFRG_INDEX_FILE_NAME);
+
+            if (idxFile.exists()) {
+                File oldIdxFile = new File(workDir, INDEX_FILE_NAME);
+
+                Files.move(idxFile.toPath(), oldIdxFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException(e);
+        }
+    }
+
+    /**
+     * Extracts partition number from file names like {@code part-dfrg-%d.bin}.
+     *
+     * @param dfrgPartFileName Defragmented partition file name.
+     * @return Partition index.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    private static int extractPartId(String dfrgPartFileName) {
+        assert dfrgPartFileName.startsWith(DFRG_PARTITION_FILE_PREFIX) : dfrgPartFileName;
+        assert dfrgPartFileName.endsWith(FILE_SUFFIX) : dfrgPartFileName;
+
+        String partIdStr = dfrgPartFileName.substring(
+            DFRG_PARTITION_FILE_PREFIX.length(),
+            dfrgPartFileName.length() - FILE_SUFFIX.length()
+        );
+
+        return Integer.parseInt(partIdStr);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin.tmp} in given folder. It will be used for storing defragmented index
+     * partition during the process.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexFile(File)
+     */
+    public static File defragmentedIndexTmpFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_TMP_FILE_NAME);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin} in given folder. It will be used for storing defragmented index
+     * partition when the process is over.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexTmpFile(File)
+     */
+    public static File defragmentedIndexFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_FILE_NAME);
+    }
+
+    /**
+     * Rename temporary index defragmentation file to a finalized one.
+     *
+     * @param workDir Cache group working directory.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexTmpFile(File)
+     * @see DefragmentationFileUtils#defragmentedIndexFile(File)
+     */
+    public static void renameTempIndexFile(File workDir) throws IgniteCheckedException {

Review comment:
       Because code looks better when chunks of code have names. Method that calls it is already about 250 lines, that's too much




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r530213242



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;
+
+                                if (log.isDebugEnabled()) {
+                                    log.debug(S.toString(
+                                        "Partition defragmented",
+                                        "grpId", grpId, false,
+                                        "partId", partId, false,
+                                        "oldPages", oldPageStore.pages(), false,
+                                        "newPages", partCtx.partPagesAllocated.get() + 1, false,
+                                        "mappingPages", partCtx.mappingPagesAllocated.get() + 1, false,
+                                        "pageSize", pageSize, false,
+                                        "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                        "workDir", workDir, false
+                                    ));
+                                }
+
+                                oldPageMem.invalidate(grpId, partId);
+
+                                partCtx.partPageMemory.invalidate(grpId, partId);
+
+                                DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                                pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                                try {
+                                    renameTempPartitionFile(workDir, partId);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    throw new IgniteException(e);
+                                }
+                            }
+                        };
+
+                        GridFutureAdapter<?> cpFut = defragmentationCheckpoint
+                            .forceCheckpoint("partition defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+
+                        cpFut.listen(cpLsnr);
+
+                        cmpFut.add((IgniteInternalFuture<Object>)cpFut);
+                    }
+
+                    // A bit too general for now, but I like it more then saving only the last checkpoint future.
+                    cmpFut.markInitialized().get();
+
+                    idxDfrgFut = new GridFinishedFuture<>();
+
+                    if (filePageStoreMgr.hasIndexStore(grpId)) {
+                        defragmentIndexPartition(oldGrpCtx, newGrpCtx, linkMapByPart);
+
+                        idxDfrgFut = defragmentationCheckpoint
+                            .forceCheckpoint("index defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+                    }
+
+                    idxDfrgFut.listen(fut -> {
+                        oldPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx partPageMem = (PageMemoryEx)partDataRegion.pageMemory();
+
+                        partPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().removePageStore(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx mappingPageMem = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+                        pageMgr = (DefragmentationPageReadWriteManager)mappingPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().clear(grpId);
+
+                        try {
+                            renameTempIndexFile(workDir);
+
+                            writeDefragmentationCompletionMarker(filePageStoreMgr.getPageStoreFileIoFactory(), workDir, log);
+
+                            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);

Review comment:
       No, of course it won't be ignored.




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

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r531224773



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;
+
+                                if (log.isDebugEnabled()) {
+                                    log.debug(S.toString(
+                                        "Partition defragmented",
+                                        "grpId", grpId, false,
+                                        "partId", partId, false,
+                                        "oldPages", oldPageStore.pages(), false,
+                                        "newPages", partCtx.partPagesAllocated.get() + 1, false,
+                                        "mappingPages", partCtx.mappingPagesAllocated.get() + 1, false,
+                                        "pageSize", pageSize, false,
+                                        "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                        "workDir", workDir, false
+                                    ));
+                                }
+
+                                oldPageMem.invalidate(grpId, partId);
+
+                                partCtx.partPageMemory.invalidate(grpId, partId);
+
+                                DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                                pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                                try {
+                                    renameTempPartitionFile(workDir, partId);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    throw new IgniteException(e);
+                                }
+                            }
+                        };
+
+                        GridFutureAdapter<?> cpFut = defragmentationCheckpoint
+                            .forceCheckpoint("partition defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+
+                        cpFut.listen(cpLsnr);
+
+                        cmpFut.add((IgniteInternalFuture<Object>)cpFut);
+                    }
+
+                    // A bit too general for now, but I like it more then saving only the last checkpoint future.
+                    cmpFut.markInitialized().get();
+
+                    idxDfrgFut = new GridFinishedFuture<>();
+
+                    if (filePageStoreMgr.hasIndexStore(grpId)) {
+                        defragmentIndexPartition(oldGrpCtx, newGrpCtx, linkMapByPart);
+
+                        idxDfrgFut = defragmentationCheckpoint
+                            .forceCheckpoint("index defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+                    }
+
+                    idxDfrgFut.listen(fut -> {
+                        oldPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx partPageMem = (PageMemoryEx)partDataRegion.pageMemory();
+
+                        partPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().removePageStore(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx mappingPageMem = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+                        pageMgr = (DefragmentationPageReadWriteManager)mappingPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().clear(grpId);
+
+                        try {
+                            renameTempIndexFile(workDir);
+
+                            writeDefragmentationCompletionMarker(filePageStoreMgr.getPageStoreFileIoFactory(), workDir, log);
+
+                            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    });
+                }
+
+                // I guess we should wait for it?
+                if (idxDfrgFut != null)
+                    idxDfrgFut.get();
+            }
+
+            mntcReg.unregisterMaintenanceTask(DEFRAGMENTATION_MNTC_TASK_NAME);
+
+            log.info("Defragmentation completed. All partitions are defragmented.");
+        }
+        finally {
+            defragmentationCheckpoint.stop(true);
+        }
+    }
+
+    /** */
+    public void createIndexPageStore(
+        int grpId,
+        File workDir,
+        FilePageStoreFactory pageStoreFactory,
+        DataRegion partRegion,
+        LongConsumer allocatedTracker
+    ) throws IgniteCheckedException {
+        // Index partition file has to be deleted before we begin, otherwise there's a chance of reading corrupted file.
+        // There is a time period when index is already defragmented but marker file is not created yet. If node is
+        // failed in that time window then index will be deframented once again. That's fine, situation is rare but code
+        // to fix that would add unnecessary complications.
+        U.delete(defragmentedIndexTmpFile(workDir));
+
+        PageStore idxPageStore;
+
+        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+        try {
+            idxPageStore = pageStoreFactory.createPageStore(
+                FLAG_IDX,
+                () -> defragmentedIndexTmpFile(workDir).toPath(),
+                allocatedTracker
+            );
+        }
+        finally {
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+        }
+
+        idxPageStore.sync();
+
+        PageMemoryEx partPageMem = (PageMemoryEx)partRegion.pageMemory();
+
+        DefragmentationPageReadWriteManager partMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+        partMgr.pageStoreMap().addPageStore(grpId, PageIdAllocator.INDEX_PARTITION, idxPageStore);
+    }
+
+    /** */
+    public enum PartStages {
+        START,
+        CP_LOCK,
+        ITERATE,
+        READ_ROW,
+        INSERT_ROW,
+        STORE_MAP,
+        STORE_PK,
+        STORE_PENDING,
+        METADATA
+    }
+
+    /**
+     * Defragmentate partition.
+     *
+     * @param partCtx
+     * @param treeIter
+     * @throws IgniteCheckedException If failed.
+     */
+    private void copyPartitionData(
+        PartitionContext partCtx,
+        TreeIterator treeIter,
+        GridSpinBusyLock busyLock
+    ) throws IgniteCheckedException {
+        partCtx.createNewCacheDataStore(busyLock);
+
+        CacheDataTree tree = partCtx.oldCacheDataStore.tree();
+
+        CacheDataTree newTree = partCtx.newCacheDataStore.tree();
+        PendingEntriesTree newPendingTree = partCtx.newCacheDataStore.pendingTree();
+        AbstractFreeList<CacheDataRow> freeList = partCtx.newCacheDataStore.getCacheStoreFreeList();
+
+        long cpLockThreshold = 150L;
+
+        TimeTracker<PartStages> tracker = new TimeTracker<>(PartStages.class);
+
+        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+        tracker.complete(CP_LOCK);
+
+        try {
+            AtomicLong lastCpLockTs = new AtomicLong(System.currentTimeMillis());
+            AtomicInteger entriesProcessed = new AtomicInteger();
+
+            treeIter.iterate(tree, partCtx.cachePageMemory, (tree0, io, pageAddr, idx) -> {
+                tracker.complete(ITERATE);
+
+                if (System.currentTimeMillis() - lastCpLockTs.get() >= cpLockThreshold) {
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+                    tracker.complete(CP_LOCK);
+
+                    lastCpLockTs.set(System.currentTimeMillis());
+                }
+
+                AbstractDataLeafIO leafIo = (AbstractDataLeafIO)io;
+                CacheDataRow row = tree.getRow(io, pageAddr, idx);
+
+                tracker.complete(READ_ROW);
+
+                int cacheId = row.cacheId();
+
+                // Reuse row that we just read.
+                row.link(0);
+
+                // "insertDataRow" will corrupt page memory if we don't do this.
+                if (row instanceof DataRow && !partCtx.oldGrpCtx.storeCacheIdInDataPage())
+                    ((DataRow)row).cacheId(CU.UNDEFINED_CACHE_ID);
+
+                freeList.insertDataRow(row, IoStatisticsHolderNoOp.INSTANCE);
+
+                // Put it back.
+                if (row instanceof DataRow)
+                    ((DataRow)row).cacheId(cacheId);
+
+                tracker.complete(INSERT_ROW);
+
+                newTree.putx(row);
+
+                long newLink = row.link();
+
+                tracker.complete(STORE_MAP);
+
+                partCtx.linkMap.put(leafIo.getLink(pageAddr, idx), newLink);
+
+                tracker.complete(STORE_PK);
+
+                if (row.expireTime() != 0)
+                    newPendingTree.putx(new PendingRow(cacheId, row.expireTime(), newLink));
+
+                tracker.complete(STORE_PENDING);
+
+                entriesProcessed.incrementAndGet();
+
+                return true;
+            });
+
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+            tracker.complete(CP_LOCK);
+
+            freeList.saveMetadata(IoStatisticsHolderNoOp.INSTANCE);
+
+            copyCacheMetadata(partCtx);
+
+            tracker.complete(METADATA);
+        }
+        finally {
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+        }
+
+        if (log.isDebugEnabled()) {
+            log.debug(
+                "Partition defragmentation timings for cache group " + partCtx.grpId +
+                " and partition " + partCtx.partId + ": " + tracker.toString()
+            );
+        }
+    }
+
+    /** */
+    private void copyCacheMetadata(
+        PartitionContext partCtx
+    ) throws IgniteCheckedException {
+        // Same for all page memories. Why does it need to be in PageMemory?
+        long partMetaPageId = partCtx.cachePageMemory.partitionMetaPageId(partCtx.grpId, partCtx.partId);
+
+        access(ACCESS_READ, partCtx.cachePageMemory, partCtx.grpId, partMetaPageId, oldPartMetaPageAddr -> {
+            PagePartitionMetaIO oldPartMetaIo = PageIO.getPageIO(oldPartMetaPageAddr);
+
+            // Newer meta versions may contain new data that we don't copy during defragmentation.
+            assert Arrays.asList(1, 2, 3).contains(oldPartMetaIo.getVersion())
+                : "IO version " + oldPartMetaIo.getVersion() + " is not supported by current defragmentation algorithm." +
+                " Please implement copying of all data added in new version.";
+
+            access(ACCESS_WRITE, partCtx.partPageMemory, partCtx.grpId, partMetaPageId, newPartMetaPageAddr -> {
+                PagePartitionMetaIOV3 newPartMetaIo = PageIO.getPageIO(newPartMetaPageAddr);
+
+                // Copy partition state.
+                byte partState = oldPartMetaIo.getPartitionState(oldPartMetaPageAddr);
+                newPartMetaIo.setPartitionState(newPartMetaPageAddr, partState);
+
+                // Copy cache size for single cache group.
+                long size = oldPartMetaIo.getSize(oldPartMetaPageAddr);
+                newPartMetaIo.setSize(newPartMetaPageAddr, size);
+
+                // Copy update counter value.
+                long updateCntr = oldPartMetaIo.getUpdateCounter(oldPartMetaPageAddr);
+                newPartMetaIo.setUpdateCounter(newPartMetaPageAddr, updateCntr);
+
+                // Copy global remove Id.
+                long rmvId = oldPartMetaIo.getGlobalRemoveId(oldPartMetaPageAddr);
+                newPartMetaIo.setGlobalRemoveId(newPartMetaPageAddr, rmvId);
+
+                // Copy cache sizes for shared cache group.
+                long oldCountersPageId = oldPartMetaIo.getCountersPageId(oldPartMetaPageAddr);
+                if (oldCountersPageId != 0L) {
+                    Map<Integer, Long> sizes = GridCacheOffheapManager.readSharedGroupCacheSizes(
+                        partCtx.cachePageMemory,
+                        partCtx.grpId,
+                        oldCountersPageId
+                    );
+
+                    long newCountersPageId = GridCacheOffheapManager.writeSharedGroupCacheSizes(
+                        partCtx.partPageMemory,
+                        partCtx.grpId,
+                        0L,
+                        partCtx.partId,
+                        sizes
+                    );
+
+                    newPartMetaIo.setCountersPageId(newPartMetaPageAddr, newCountersPageId);
+                }
+
+                // Copy counter gaps.
+                long oldGapsLink = oldPartMetaIo.getGapsLink(oldPartMetaPageAddr);
+                if (oldGapsLink != 0L) {
+                    byte[] gapsBytes = partCtx.oldCacheDataStore.partStorage().readRow(oldGapsLink);
+
+                    SimpleDataRow gapsDataRow = new SimpleDataRow(partCtx.partId, gapsBytes);
+
+                    partCtx.newCacheDataStore.partStorage().insertDataRow(gapsDataRow, IoStatisticsHolderNoOp.INSTANCE);
+
+                    newPartMetaIo.setGapsLink(newPartMetaPageAddr, gapsDataRow.link());
+                }
+
+                // Encryption stuff.
+                newPartMetaIo.setEncryptedPageCount(newPartMetaPageAddr, 0);
+                newPartMetaIo.setEncryptedPageIndex(newPartMetaPageAddr, 0);
+
+                return null;
+            });
+
+            return null;
+        });
+    }
+
+    /**
+     * Defragmentate indexing partition.
+     *
+     * @param grpCtx
+     * @param mappingByPartition
+     *
+     * @throws IgniteCheckedException If failed.
+     */
+    private void defragmentIndexPartition(
+        CacheGroupContext grpCtx,
+        CacheGroupContext newCtx,
+        IntMap<LinkMap> mappingByPartition
+    ) throws IgniteCheckedException {
+        GridQueryProcessor query = grpCtx.caches().get(0).kernalContext().query();
+
+        if (!query.moduleEnabled())
+            return;
+
+        final GridQueryIndexing idx = query.getIndexing();
+
+        CheckpointTimeoutLock cpLock = defragmentationCheckpoint.checkpointTimeoutLock();
+
+        idx.defragmentator().defragment(
+            grpCtx,
+            newCtx,
+            (PageMemoryEx)partDataRegion.pageMemory(),
+            mappingByPartition,
+            cpLock,
+            log
+        );
+    }
+
+    /** */
+    @SuppressWarnings("PublicField")
+    private class PartitionContext {
+        /** */
+        public final File workDir;
+
+        /** */
+        public final int grpId;
+
+        /** */
+        public final int partId;
+
+        /** */
+        public final DataRegion cacheDataRegion;
+
+        /** */
+        public final PageMemoryEx cachePageMemory;
+
+        /** */
+        public final PageMemoryEx partPageMemory;
+
+        /** */
+        public final PageMemoryEx mappingPageMemory;
+
+        /** */
+        public final CacheGroupContext oldGrpCtx;
+
+        /** */
+        public final CacheGroupContext newGrpCtx;
+
+        /** */
+        public final CacheDataStore oldCacheDataStore;
+
+        /** */
+        private GridCacheDataStore newCacheDataStore;
+
+        /** */
+        public final FilePageStoreFactory pageStoreFactory;
+
+        /** */
+        public final AtomicLong partPagesAllocated = new AtomicLong();
+
+        /** */
+        public final AtomicLong mappingPagesAllocated = new AtomicLong();
+
+        /** */
+        private LinkMap linkMap;
+
+        /** */
+        public PartitionContext(
+            File workDir,
+            int grpId,
+            int partId,
+            DataRegion partDataRegion,
+            DataRegion mappingDataRegion,
+            CacheGroupContext oldGrpCtx,
+            CacheGroupContext newGrpCtx,
+            CacheDataStore oldCacheDataStore,
+            FilePageStoreFactory pageStoreFactory
+        ) {
+            this.workDir = workDir;
+            this.grpId = grpId;
+            this.partId = partId;
+            cacheDataRegion = oldGrpCtx.dataRegion();
+
+            cachePageMemory = (PageMemoryEx)cacheDataRegion.pageMemory();
+            partPageMemory = (PageMemoryEx)partDataRegion.pageMemory();
+            mappingPageMemory = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+            this.oldGrpCtx = oldGrpCtx;
+            this.newGrpCtx = newGrpCtx;
+            this.oldCacheDataStore = oldCacheDataStore;
+            this.pageStoreFactory = pageStoreFactory;
+        }
+
+        /** */
+        public PageStore createPartPageStore() throws IgniteCheckedException {
+            PageStore partPageStore;
+
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+            try {
+                partPageStore = pageStoreFactory.createPageStore(
+                    FLAG_DATA,
+                    () -> defragmentedPartTmpFile(workDir, partId).toPath(),
+                    partPagesAllocated::addAndGet
+                );
+            }
+            finally {
+                defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+            }
+
+            partPageStore.sync();
+
+            DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMemory.pageManager();
+
+            pageMgr.pageStoreMap().addPageStore(grpId, partId, partPageStore);
+
+            return partPageStore;
+        }
+
+        /** */
+        public PageStore createMappingPageStore() throws IgniteCheckedException {
+            PageStore mappingPageStore;
+
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+            try {
+                mappingPageStore = pageStoreFactory.createPageStore(
+                    FLAG_DATA,
+                    () -> defragmentedPartMappingFile(workDir, partId).toPath(),
+                    mappingPagesAllocated::addAndGet
+                );
+            }
+            finally {
+                defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+            }
+
+            mappingPageStore.sync();
+
+            DefragmentationPageReadWriteManager partMgr = (DefragmentationPageReadWriteManager)mappingPageMemory.pageManager();
+
+            partMgr.pageStoreMap().addPageStore(grpId, partId, mappingPageStore);
+
+            return mappingPageStore;
+        }
+
+        /** */
+        public LinkMap createLinkMapTree(boolean initNew) throws IgniteCheckedException {
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+            //TODO Store link in meta page and remove META_PAGE_IDX constant?
+            try {
+                long mappingMetaPageId = initNew
+                    ? mappingPageMemory.allocatePage(grpId, partId, FLAG_DATA)
+                    : PageIdUtils.pageId(partId, FLAG_DATA, LinkMap.META_PAGE_IDX);
+
+                assert PageIdUtils.pageIndex(mappingMetaPageId) == LinkMap.META_PAGE_IDX
+                    : PageIdUtils.toDetailString(mappingMetaPageId);
+
+                linkMap = new LinkMap(newGrpCtx, mappingPageMemory, mappingMetaPageId, initNew);
+            }
+            finally {
+                defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+            }
+
+            return linkMap;
+        }
+
+        /** */
+        public void createNewCacheDataStore(GridSpinBusyLock busyLock) {

Review comment:
       I agree that defragmentation code must be as self-sufficient as possible. But here you are using the `busyLock` from `OffheapManager` as a method parameter and it seems there can be nothing worse than this.
   Let's move this factory method to the OffheapManager.




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r532411788



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
##########
@@ -1357,8 +1357,19 @@ private void saveIndexReencryptionStatus(int grpId) throws IgniteCheckedExceptio
     }
 
     /** */
-    public GridSpinBusyLock busyLock() {
-        return busyLock;
+    public GridCacheDataStore createGridCacheDataStore(

Review comment:
       Done




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r524009334



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,887 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database schared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;
+
+                                log.info(S.toString(
+                                    "Partition defragmented",
+                                    "grpId", grpId, false,
+                                    "partId", partId, false,
+                                    "oldPages", oldPageStore.pages(), false,
+                                    "newPages", partCtx.partPagesAllocated.get(), false,
+                                    "bytesSaved", (oldPageStore.pages() - partCtx.partPagesAllocated.get()) * pageSize, false,
+                                    "mappingPages", partCtx.mappingPagesAllocated.get(), false,
+                                    "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                    "workDir", workDir, false
+                                ));
+
+                                oldPageMem.invalidate(grpId, partId);
+
+                                partCtx.partPageMemory.invalidate(grpId, partId);
+
+                                DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                                pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                                try {
+                                    renameTempPartitionFile(workDir, partId);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    throw new IgniteException(e);
+                                }
+                            }
+                        };
+
+                        GridFutureAdapter<?> cpFut = defragmentationCheckpoint
+                            .forceCheckpoint("partition defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+
+                        cpFut.listen(cpLsnr);
+
+                        cmpFut.add((IgniteInternalFuture<Object>)cpFut);
+                    }
+
+                    // A bit too general for now, but I like it more then saving only the last checkpoint future.
+                    cmpFut.markInitialized().get();
+
+                    idxDfrgFut = new GridFinishedFuture<>();
+
+                    if (filePageStoreMgr.hasIndexStore(grpId)) {
+                        defragmentIndexPartition(oldGrpCtx, newGrpCtx, linkMapByPart);
+
+                        idxDfrgFut = defragmentationCheckpoint
+                            .forceCheckpoint("index defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+                    }
+
+                    idxDfrgFut.listen(fut -> {
+                        oldPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx partPageMem = (PageMemoryEx)partDataRegion.pageMemory();
+
+                        partPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().removePageStore(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx mappingPageMem = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+                        pageMgr = (DefragmentationPageReadWriteManager)mappingPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().clear(grpId);
+
+                        try {
+                            renameTempIndexFile(workDir);
+
+                            writeDefragmentationCompletionMarker(filePageStoreMgr.getPageStoreFileIoFactory(), workDir, log);
+
+                            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    });
+                }
+
+                // I guess we should wait for it?
+                if (idxDfrgFut != null)
+                    idxDfrgFut.get();
+            }
+
+            mntcReg.unregisterMaintenanceTask(DEFRAGMENTATION_MNTC_TASK_NAME);
+
+            log.info("Defragmentation completed. All partitions are defragmented.");
+        }
+        finally {
+            defragmentationCheckpoint.stop(true);
+        }
+    }
+
+    /** */
+    public void createIndexPageStore(
+        int grpId,
+        File workDir,
+        FilePageStoreFactory pageStoreFactory,
+        DataRegion partRegion,
+        LongConsumer allocatedTracker
+    ) throws IgniteCheckedException {
+        // Index partition file has to be deleted before we begin, otherwise there's a chance of reading corrupted file.
+        // There is a time period when index is already defragmented but marker file is not created yet. If node is
+        // failed in that time window then index will be deframented once again. That's fine, situation is rare but code
+        // to fix that would add unnecessary complications.
+        U.delete(defragmentedIndexTmpFile(workDir));
+
+        PageStore idxPageStore;
+
+        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+        try {
+            idxPageStore = pageStoreFactory.createPageStore(
+                FLAG_IDX,
+                () -> defragmentedIndexTmpFile(workDir).toPath(),
+                allocatedTracker
+            );
+        }
+        finally {
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+        }
+
+        idxPageStore.sync();
+
+        PageMemoryEx partPageMem = (PageMemoryEx)partRegion.pageMemory();
+
+        DefragmentationPageReadWriteManager partMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+        partMgr.pageStoreMap().addPageStore(grpId, PageIdAllocator.INDEX_PARTITION, idxPageStore);
+    }
+
+    /**
+     * Cancel the process of defragmentation.
+     */
+    public void cancel(){
+
+    }
+
+    /** */
+    public enum PartStages {
+        START,
+        CP_LOCK,
+        ITERATE,
+        READ_ROW,
+        INSERT_ROW,
+        STORE_MAP,
+        STORE_PK,
+        STORE_PENDING,
+        METADATA
+    }
+
+    /**
+     * Defragmentate partition.
+     *
+     * @param partCtx
+     * @param treeIter
+     * @throws IgniteCheckedException If failed.
+     */
+    private void copyPartitionData(
+        PartitionContext partCtx,
+        TreeIterator treeIter,
+        GridSpinBusyLock busyLock
+    ) throws IgniteCheckedException {
+        partCtx.createNewCacheDataStore(busyLock);
+
+        CacheDataTree tree = partCtx.oldCacheDataStore.tree();
+
+        CacheDataTree newTree = partCtx.newCacheDataStore.tree();
+        PendingEntriesTree newPendingTree = partCtx.newCacheDataStore.pendingTree();
+        AbstractFreeList<CacheDataRow> freeList = partCtx.newCacheDataStore.getCacheStoreFreeList();
+
+        long cpLockThreshold = 250L;
+
+        TimeTracker<PartStages> tracker = new TimeTracker<>(PartStages.class);
+
+        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+        tracker.complete(CP_LOCK);
+
+        try {
+            AtomicLong lastCpLockTs = new AtomicLong(System.currentTimeMillis());
+            AtomicInteger entriesProcessed = new AtomicInteger();
+
+            treeIter.iterate(tree, partCtx.cachePageMemory, (tree0, io, pageAddr, idx) -> {
+                tracker.complete(ITERATE);
+
+                if (System.currentTimeMillis() - lastCpLockTs.get() >= cpLockThreshold) {
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+                    tracker.complete(CP_LOCK);
+
+                    lastCpLockTs.set(System.currentTimeMillis());
+                }
+
+                AbstractDataLeafIO leafIo = (AbstractDataLeafIO)io;
+                CacheDataRow row = tree.getRow(io, pageAddr, idx);
+
+                tracker.complete(READ_ROW);
+
+                int cacheId = row.cacheId();
+
+                // Reuse row that we just read.
+                row.link(0);
+
+                // "insertDataRow" will corrupt page memory if we don't do this.
+                if (row instanceof DataRow && !partCtx.oldGrpCtx.storeCacheIdInDataPage())
+                    ((DataRow)row).cacheId(CU.UNDEFINED_CACHE_ID);
+
+                freeList.insertDataRow(row, IoStatisticsHolderNoOp.INSTANCE);
+
+                // Put it back.
+                if (row instanceof DataRow)
+                    ((DataRow)row).cacheId(cacheId);
+
+                tracker.complete(INSERT_ROW);
+
+                newTree.putx(row);
+
+                long newLink = row.link();
+
+                tracker.complete(STORE_MAP);
+
+                partCtx.linkMap.put(leafIo.getLink(pageAddr, idx), newLink);
+
+                tracker.complete(STORE_PK);
+
+                if (row.expireTime() != 0)
+                    newPendingTree.putx(new PendingRow(cacheId, row.expireTime(), newLink));
+
+                tracker.complete(STORE_PENDING);
+
+                entriesProcessed.incrementAndGet();
+
+                return true;
+            });
+
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+            tracker.complete(CP_LOCK);
+
+            freeList.saveMetadata(IoStatisticsHolderNoOp.INSTANCE);
+
+            copyCacheMetadata(partCtx);
+
+            tracker.complete(METADATA);
+        }
+        finally {
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+        }
+
+        System.out.println(tracker.toString());
+    }
+
+    /** */
+    private void copyCacheMetadata(
+        PartitionContext partCtx
+    ) throws IgniteCheckedException {
+        // Same for all page memories. Why does it need to be in PageMemory?
+        long partMetaPageId = partCtx.cachePageMemory.partitionMetaPageId(partCtx.grpId, partCtx.partId);
+
+        access(ACCESS_READ, partCtx.cachePageMemory, partCtx.grpId, partMetaPageId, oldPartMetaPageAddr -> {
+            PagePartitionMetaIO oldPartMetaIo = PageIO.getPageIO(oldPartMetaPageAddr);
+
+            // Newer meta versions may contain new data that we don't copy during defragmentation.
+            assert Arrays.asList(1, 2, 3).contains(oldPartMetaIo.getVersion()) : oldPartMetaIo.getVersion();

Review comment:
       We should add a better message here.




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r526714026



##########
File path: modules/core/src/main/java/org/apache/ignite/configuration/DefragmentationConfiguration.java
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.configuration;
+
+import java.io.Serializable;
+
+/**
+ *
+ */
+public final class DefragmentationConfiguration implements Serializable {
+    /** Serial version uid. */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     *
+     */
+    public static final long DFLT_MAPPIN_REGION_SIZE = 256L * 1024 * 1024;
+
+    /**
+     *
+     */
+    private long regionSize = DataStorageConfiguration.DFLT_DATA_REGION_MAX_SIZE;
+
+    /**
+     *
+     */
+    private long mappingRegionSize = DFLT_MAPPIN_REGION_SIZE;

Review comment:
       @akalash will merge similar fix 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.

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r531457238



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
##########
@@ -447,6 +472,32 @@ private DataRegionConfiguration createMetastoreDataRegionConfig(DataStorageConfi
         return cfg;
     }
 
+    /** */
+    private DataRegionConfiguration createDefragmentationDataRegionConfig(long regionSize) {

Review comment:
       No, node must be restarted when defragmentation is completed. It's not part of the topology during this process, so everything's fine here.




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r526711488



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/DefragmentationFileUtils.java
##########
@@ -0,0 +1,400 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static java.nio.file.StandardCopyOption.ATOMIC_MOVE;
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static java.nio.file.StandardOpenOption.CREATE_NEW;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.FILE_SUFFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_TEMPLATE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.TMP_SUFFIX;
+
+/**
+ * Everything related to file management during defragmentation process.
+ */
+public class DefragmentationFileUtils {
+    /** Prefix for link mapping files. */
+    private static final String DFRG_LINK_MAPPING_FILE_PREFIX = PART_FILE_PREFIX + "map-";
+
+    /** Link mapping file template. */
+    private static final String DFRG_LINK_MAPPING_FILE_TEMPLATE = DFRG_LINK_MAPPING_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmentation complation marker file name. */
+    private static final String DFRG_COMPLETION_MARKER_FILE_NAME = "dfrg-completion-marker";
+
+    /** Name of defragmentated index partition file. */
+    private static final String DFRG_INDEX_FILE_NAME = INDEX_FILE_PREFIX + "-dfrg" + FILE_SUFFIX;
+
+    /** Name of defragmentated index partition temporary file. */
+    private static final String DFRG_INDEX_TMP_FILE_NAME = DFRG_INDEX_FILE_NAME + TMP_SUFFIX;
+
+    /** Prefix for defragmented partition files. */
+    private static final String DFRG_PARTITION_FILE_PREFIX = PART_FILE_PREFIX + "dfrg-";
+
+    /** Defragmented partition file template. */
+    private static final String DFRG_PARTITION_FILE_TEMPLATE = DFRG_PARTITION_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmented partition temp file template. */
+    private static final String DFRG_PARTITION_TMP_FILE_TEMPLATE = DFRG_PARTITION_FILE_TEMPLATE + TMP_SUFFIX;
+
+    /**
+     * Performs cleanup of work dir before initializing file page stores.
+     * Will finish batch renaming if defragmentation was completed or delete garbage if it wasn't.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     */
+    public static void beforeInitPageStores(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+        U.delete(defragmentationCompletionMarkerFile(workDir));
+
+        for (File file : workDir.listFiles()) {
+            String fileName = file.getName();
+
+            if (
+                fileName.startsWith(DFRG_PARTITION_FILE_PREFIX)
+                || fileName.startsWith(DFRG_INDEX_FILE_NAME)
+                || fileName.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)
+            )
+                U.delete(file);
+        }
+    }
+
+    /**
+     * Checks whether cache group defragmentation completed or not. Completes it if all that's left is renaming.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param log Logger to write messages.
+     * @return {@code true} if given cache group is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentationCompletionMarkerFile(File)
+     */
+    public static boolean skipAlreadyDefragmentedCacheGroup(File workDir, int grpId, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (completionMarkerFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented page group",
+                    "grpId", grpId, false,
+                    "markerFileName", completionMarkerFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+            return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * Checks whether partition has already been defragmented or not. Cleans corrupted data if previous failed
+     * defragmentation attempt was found.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param partId Partionion index to check.
+     * @param log Logger to write messages.
+     * @return {@code true} if given partition is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartMappingFile(File, int)
+     */
+    public static boolean skipAlreadyDefragmentedPartition(File workDir, int grpId, int partId, IgniteLogger log) throws IgniteCheckedException {
+        File defragmentedPartFile = defragmentedPartFile(workDir, partId);
+        File defragmentedPartMappingFile = defragmentedPartMappingFile(workDir, partId);
+
+        if (defragmentedPartFile.exists() && defragmentedPartMappingFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented partition",
+                    "grpId", grpId, false,
+                    "partId", partId, false,
+                    "partFileName", defragmentedPartFile.getName(), false,
+                    "mappingFileName", defragmentedPartMappingFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            return true;
+        }
+
+        File defragmentedPartTmpFile = defragmentedPartTmpFile(workDir, partId);
+
+        try {
+            Files.deleteIfExists(defragmentedPartTmpFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartMappingFile.toPath());
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+
+        return false;
+    }
+
+    /**
+     * Failure-tolerant batch rename of defragmented partition files.
+     *
+     * Deletes all link mapping files old partition and index files, renaming defragmentated files in the process. Can
+     * be run on the same folder multiple times if failed for some reason.
+     *
+     * Does something only if completion marker is present in the folder. This marker won't be deleted in the end.
+     * Deletion of the marker must be done outside of defragmentation mode to prevent cache groups to be defragmentated
+     * several times in case of failures.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#writeDefragmentationCompletionMarker(FileIOFactory, File, IgniteLogger)
+     */
+    public static void batchRenameDefragmentedCacheGroupPartitions(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (!completionMarkerFile.exists())
+            return;
+
+        try {
+            for (File mappingFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)))
+                Files.delete(mappingFile.toPath());
+
+            for (File partFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_PARTITION_FILE_PREFIX))) {
+                int partId = extractPartId(partFile.getName());
+
+                File oldPartFile = new File(workDir, String.format(PART_FILE_TEMPLATE, partId));
+
+                Files.move(partFile.toPath(), oldPartFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+
+            File idxFile = new File(workDir, DFRG_INDEX_FILE_NAME);
+
+            if (idxFile.exists()) {
+                File oldIdxFile = new File(workDir, INDEX_FILE_NAME);
+
+                Files.move(idxFile.toPath(), oldIdxFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Extracts partition number from file names like {@code part-dfrg-%d.bin}.
+     *
+     * @param dfrgPartFileName Defragmented partition file name.
+     * @return Partition index.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    private static int extractPartId(String dfrgPartFileName) {
+        assert dfrgPartFileName.startsWith(DFRG_PARTITION_FILE_PREFIX) : dfrgPartFileName;
+        assert dfrgPartFileName.endsWith(FILE_SUFFIX) : dfrgPartFileName;
+
+        String partIdStr = dfrgPartFileName.substring(
+            DFRG_PARTITION_FILE_PREFIX.length(),
+            dfrgPartFileName.length() - FILE_SUFFIX.length()
+        );
+
+        return Integer.parseInt(partIdStr);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin.tmp} in given folder. It will be used for storing defragmented index
+     * partition during the process.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexFile(File)
+     */
+    public static File defragmentedIndexTmpFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_TMP_FILE_NAME);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin} in given folder. It will be used for storing defragmented index
+     * partition when the process is over.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexTmpFile(File)
+     */
+    public static File defragmentedIndexFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_FILE_NAME);
+    }
+
+    /**
+     * Rename temporary index defragmenation file to a finalized one.

Review comment:
       Done, thank you for catching these typos!

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/DefragmentationFileUtils.java
##########
@@ -0,0 +1,400 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static java.nio.file.StandardCopyOption.ATOMIC_MOVE;
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static java.nio.file.StandardOpenOption.CREATE_NEW;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.FILE_SUFFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_TEMPLATE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.TMP_SUFFIX;
+
+/**
+ * Everything related to file management during defragmentation process.
+ */
+public class DefragmentationFileUtils {
+    /** Prefix for link mapping files. */
+    private static final String DFRG_LINK_MAPPING_FILE_PREFIX = PART_FILE_PREFIX + "map-";
+
+    /** Link mapping file template. */
+    private static final String DFRG_LINK_MAPPING_FILE_TEMPLATE = DFRG_LINK_MAPPING_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmentation complation marker file name. */
+    private static final String DFRG_COMPLETION_MARKER_FILE_NAME = "dfrg-completion-marker";
+
+    /** Name of defragmentated index partition file. */
+    private static final String DFRG_INDEX_FILE_NAME = INDEX_FILE_PREFIX + "-dfrg" + FILE_SUFFIX;
+
+    /** Name of defragmentated index partition temporary file. */
+    private static final String DFRG_INDEX_TMP_FILE_NAME = DFRG_INDEX_FILE_NAME + TMP_SUFFIX;
+
+    /** Prefix for defragmented partition files. */
+    private static final String DFRG_PARTITION_FILE_PREFIX = PART_FILE_PREFIX + "dfrg-";
+
+    /** Defragmented partition file template. */
+    private static final String DFRG_PARTITION_FILE_TEMPLATE = DFRG_PARTITION_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmented partition temp file template. */
+    private static final String DFRG_PARTITION_TMP_FILE_TEMPLATE = DFRG_PARTITION_FILE_TEMPLATE + TMP_SUFFIX;
+
+    /**
+     * Performs cleanup of work dir before initializing file page stores.
+     * Will finish batch renaming if defragmentation was completed or delete garbage if it wasn't.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     */
+    public static void beforeInitPageStores(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+        U.delete(defragmentationCompletionMarkerFile(workDir));
+
+        for (File file : workDir.listFiles()) {
+            String fileName = file.getName();
+
+            if (
+                fileName.startsWith(DFRG_PARTITION_FILE_PREFIX)
+                || fileName.startsWith(DFRG_INDEX_FILE_NAME)
+                || fileName.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)
+            )
+                U.delete(file);
+        }
+    }
+
+    /**
+     * Checks whether cache group defragmentation completed or not. Completes it if all that's left is renaming.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param log Logger to write messages.
+     * @return {@code true} if given cache group is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentationCompletionMarkerFile(File)
+     */
+    public static boolean skipAlreadyDefragmentedCacheGroup(File workDir, int grpId, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (completionMarkerFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented page group",
+                    "grpId", grpId, false,
+                    "markerFileName", completionMarkerFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+            return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * Checks whether partition has already been defragmented or not. Cleans corrupted data if previous failed
+     * defragmentation attempt was found.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param partId Partionion index to check.
+     * @param log Logger to write messages.
+     * @return {@code true} if given partition is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartMappingFile(File, int)
+     */
+    public static boolean skipAlreadyDefragmentedPartition(File workDir, int grpId, int partId, IgniteLogger log) throws IgniteCheckedException {
+        File defragmentedPartFile = defragmentedPartFile(workDir, partId);
+        File defragmentedPartMappingFile = defragmentedPartMappingFile(workDir, partId);
+
+        if (defragmentedPartFile.exists() && defragmentedPartMappingFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented partition",
+                    "grpId", grpId, false,
+                    "partId", partId, false,
+                    "partFileName", defragmentedPartFile.getName(), false,
+                    "mappingFileName", defragmentedPartMappingFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            return true;
+        }
+
+        File defragmentedPartTmpFile = defragmentedPartTmpFile(workDir, partId);
+
+        try {
+            Files.deleteIfExists(defragmentedPartTmpFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartMappingFile.toPath());
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+
+        return false;
+    }
+
+    /**
+     * Failure-tolerant batch rename of defragmented partition files.
+     *
+     * Deletes all link mapping files old partition and index files, renaming defragmentated files in the process. Can
+     * be run on the same folder multiple times if failed for some reason.
+     *
+     * Does something only if completion marker is present in the folder. This marker won't be deleted in the end.
+     * Deletion of the marker must be done outside of defragmentation mode to prevent cache groups to be defragmentated
+     * several times in case of failures.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#writeDefragmentationCompletionMarker(FileIOFactory, File, IgniteLogger)
+     */
+    public static void batchRenameDefragmentedCacheGroupPartitions(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (!completionMarkerFile.exists())
+            return;
+
+        try {
+            for (File mappingFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)))
+                Files.delete(mappingFile.toPath());
+
+            for (File partFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_PARTITION_FILE_PREFIX))) {
+                int partId = extractPartId(partFile.getName());
+
+                File oldPartFile = new File(workDir, String.format(PART_FILE_TEMPLATE, partId));
+
+                Files.move(partFile.toPath(), oldPartFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+
+            File idxFile = new File(workDir, DFRG_INDEX_FILE_NAME);
+
+            if (idxFile.exists()) {
+                File oldIdxFile = new File(workDir, INDEX_FILE_NAME);
+
+                Files.move(idxFile.toPath(), oldIdxFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Extracts partition number from file names like {@code part-dfrg-%d.bin}.
+     *
+     * @param dfrgPartFileName Defragmented partition file name.
+     * @return Partition index.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    private static int extractPartId(String dfrgPartFileName) {
+        assert dfrgPartFileName.startsWith(DFRG_PARTITION_FILE_PREFIX) : dfrgPartFileName;
+        assert dfrgPartFileName.endsWith(FILE_SUFFIX) : dfrgPartFileName;
+
+        String partIdStr = dfrgPartFileName.substring(
+            DFRG_PARTITION_FILE_PREFIX.length(),
+            dfrgPartFileName.length() - FILE_SUFFIX.length()
+        );
+
+        return Integer.parseInt(partIdStr);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin.tmp} in given folder. It will be used for storing defragmented index
+     * partition during the process.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexFile(File)
+     */
+    public static File defragmentedIndexTmpFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_TMP_FILE_NAME);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin} in given folder. It will be used for storing defragmented index
+     * partition when the process is over.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexTmpFile(File)
+     */
+    public static File defragmentedIndexFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_FILE_NAME);
+    }
+
+    /**
+     * Rename temporary index defragmenation file to a finalized one.
+     *
+     * @param workDir Cache group working directory.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexTmpFile(File)
+     * @see DefragmentationFileUtils#defragmentedIndexFile(File)
+     */
+    public static void renameTempIndexFile(File workDir) throws IgniteCheckedException {
+        File defragmentedIdxTmpFile = defragmentedIndexTmpFile(workDir);
+        File defragmentedIdxFile = defragmentedIndexFile(workDir);
+
+        try {
+            Files.move(defragmentedIdxTmpFile.toPath(), defragmentedIdxFile.toPath(), ATOMIC_MOVE);
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Return file named {@code part-dfrg-%d.bin.tmp} in given folder. It will be used for storing defragmented data
+     * partition during the process.
+     *
+     * @param workDir Cache group working directory.
+     * @param partId Parition index, will be substituted into file name.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    public static File defragmentedPartTmpFile(File workDir, int partId) {
+        return new File(workDir, String.format(DFRG_PARTITION_TMP_FILE_TEMPLATE, partId));
+    }
+
+    /**
+     * Return file named {@code part-dfrg-%d.bin} in given folder. It will be used for storing defragmented data
+     * partition when the process is over.
+     *
+     * @param workDir Cache group working directory.
+     * @param partId Parition index, will be substituted into file name.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     */
+    public static File defragmentedPartFile(File workDir, int partId) {
+        return new File(workDir, String.format(DFRG_PARTITION_FILE_TEMPLATE, partId));
+    }
+
+    /**
+     * Rename temporary partition defragmenation file to a finalized one.
+     *
+     * @param workDir Cache group working directory.
+     * @param partId Parition index.

Review comment:
       Done

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/DefragmentationFileUtils.java
##########
@@ -0,0 +1,400 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static java.nio.file.StandardCopyOption.ATOMIC_MOVE;
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static java.nio.file.StandardOpenOption.CREATE_NEW;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.FILE_SUFFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_TEMPLATE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.TMP_SUFFIX;
+
+/**
+ * Everything related to file management during defragmentation process.
+ */
+public class DefragmentationFileUtils {
+    /** Prefix for link mapping files. */
+    private static final String DFRG_LINK_MAPPING_FILE_PREFIX = PART_FILE_PREFIX + "map-";
+
+    /** Link mapping file template. */
+    private static final String DFRG_LINK_MAPPING_FILE_TEMPLATE = DFRG_LINK_MAPPING_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmentation complation marker file name. */
+    private static final String DFRG_COMPLETION_MARKER_FILE_NAME = "dfrg-completion-marker";
+
+    /** Name of defragmentated index partition file. */
+    private static final String DFRG_INDEX_FILE_NAME = INDEX_FILE_PREFIX + "-dfrg" + FILE_SUFFIX;
+
+    /** Name of defragmentated index partition temporary file. */
+    private static final String DFRG_INDEX_TMP_FILE_NAME = DFRG_INDEX_FILE_NAME + TMP_SUFFIX;
+
+    /** Prefix for defragmented partition files. */
+    private static final String DFRG_PARTITION_FILE_PREFIX = PART_FILE_PREFIX + "dfrg-";
+
+    /** Defragmented partition file template. */
+    private static final String DFRG_PARTITION_FILE_TEMPLATE = DFRG_PARTITION_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmented partition temp file template. */
+    private static final String DFRG_PARTITION_TMP_FILE_TEMPLATE = DFRG_PARTITION_FILE_TEMPLATE + TMP_SUFFIX;
+
+    /**
+     * Performs cleanup of work dir before initializing file page stores.
+     * Will finish batch renaming if defragmentation was completed or delete garbage if it wasn't.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     */
+    public static void beforeInitPageStores(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+        U.delete(defragmentationCompletionMarkerFile(workDir));
+
+        for (File file : workDir.listFiles()) {
+            String fileName = file.getName();
+
+            if (
+                fileName.startsWith(DFRG_PARTITION_FILE_PREFIX)
+                || fileName.startsWith(DFRG_INDEX_FILE_NAME)
+                || fileName.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)
+            )
+                U.delete(file);
+        }
+    }
+
+    /**
+     * Checks whether cache group defragmentation completed or not. Completes it if all that's left is renaming.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param log Logger to write messages.
+     * @return {@code true} if given cache group is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentationCompletionMarkerFile(File)
+     */
+    public static boolean skipAlreadyDefragmentedCacheGroup(File workDir, int grpId, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (completionMarkerFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented page group",
+                    "grpId", grpId, false,
+                    "markerFileName", completionMarkerFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+            return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * Checks whether partition has already been defragmented or not. Cleans corrupted data if previous failed
+     * defragmentation attempt was found.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param partId Partionion index to check.

Review comment:
       Done




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r530205083



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/DefragmentationFileUtils.java
##########
@@ -0,0 +1,395 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static java.nio.file.StandardCopyOption.ATOMIC_MOVE;
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static java.nio.file.StandardOpenOption.CREATE_NEW;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.FILE_SUFFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_TEMPLATE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.TMP_SUFFIX;
+
+/**
+ * Everything related to file management during defragmentation process.
+ */
+public class DefragmentationFileUtils {
+    /** Prefix for link mapping files. */
+    private static final String DFRG_LINK_MAPPING_FILE_PREFIX = PART_FILE_PREFIX + "map-";
+
+    /** Link mapping file template. */
+    private static final String DFRG_LINK_MAPPING_FILE_TEMPLATE = DFRG_LINK_MAPPING_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmentation complation marker file name. */
+    private static final String DFRG_COMPLETION_MARKER_FILE_NAME = "dfrg-completion-marker";
+
+    /** Name of defragmentated index partition file. */
+    private static final String DFRG_INDEX_FILE_NAME = INDEX_FILE_PREFIX + "-dfrg" + FILE_SUFFIX;
+
+    /** Name of defragmentated index partition temporary file. */
+    private static final String DFRG_INDEX_TMP_FILE_NAME = DFRG_INDEX_FILE_NAME + TMP_SUFFIX;
+
+    /** Prefix for defragmented partition files. */
+    private static final String DFRG_PARTITION_FILE_PREFIX = PART_FILE_PREFIX + "dfrg-";
+
+    /** Defragmented partition file template. */
+    private static final String DFRG_PARTITION_FILE_TEMPLATE = DFRG_PARTITION_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmented partition temp file template. */
+    private static final String DFRG_PARTITION_TMP_FILE_TEMPLATE = DFRG_PARTITION_FILE_TEMPLATE + TMP_SUFFIX;
+
+    /**
+     * Performs cleanup of work dir before initializing file page stores.
+     * Will finish batch renaming if defragmentation was completed or delete garbage if it wasn't.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     */
+    public static void beforeInitPageStores(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+        U.delete(defragmentationCompletionMarkerFile(workDir));
+
+        for (File file : workDir.listFiles()) {
+            String fileName = file.getName();
+
+            if (
+                fileName.startsWith(DFRG_PARTITION_FILE_PREFIX)
+                || fileName.startsWith(DFRG_INDEX_FILE_NAME)
+                || fileName.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)
+            )
+                U.delete(file);
+        }
+    }
+
+    /**
+     * Checks whether cache group defragmentation completed or not. Completes it if all that's left is renaming.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param log Logger to write messages.
+     * @return {@code true} if given cache group is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentationCompletionMarkerFile(File)
+     */
+    public static boolean skipAlreadyDefragmentedCacheGroup(File workDir, int grpId, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (completionMarkerFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented page group",
+                    "grpId", grpId, false,
+                    "markerFileName", completionMarkerFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+            return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * Checks whether partition has already been defragmented or not. Cleans corrupted data if previous failed
+     * defragmentation attempt was found.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param partId Partition index to check.
+     * @param log Logger to write messages.
+     * @return {@code true} if given partition is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartMappingFile(File, int)
+     */
+    public static boolean skipAlreadyDefragmentedPartition(File workDir, int grpId, int partId, IgniteLogger log) throws IgniteCheckedException {
+        File defragmentedPartFile = defragmentedPartFile(workDir, partId);
+        File defragmentedPartMappingFile = defragmentedPartMappingFile(workDir, partId);
+
+        if (defragmentedPartFile.exists() && defragmentedPartMappingFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented partition",
+                    "grpId", grpId, false,
+                    "partId", partId, false,
+                    "partFileName", defragmentedPartFile.getName(), false,
+                    "mappingFileName", defragmentedPartMappingFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            return true;
+        }
+
+        File defragmentedPartTmpFile = defragmentedPartTmpFile(workDir, partId);
+
+        try {
+            Files.deleteIfExists(defragmentedPartTmpFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartMappingFile.toPath());
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException(e);
+        }
+
+        return false;
+    }
+
+    /**
+     * Failure-tolerant batch rename of defragmented partition files.
+     *
+     * Deletes all link mapping files old partition and index files, renaming defragmentated files in the process. Can
+     * be run on the same folder multiple times if failed for some reason.
+     *
+     * Does something only if completion marker is present in the folder. This marker won't be deleted in the end.
+     * Deletion of the marker must be done outside of defragmentation mode to prevent cache groups to be defragmentated
+     * several times in case of failures.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#writeDefragmentationCompletionMarker(FileIOFactory, File, IgniteLogger)
+     */
+    public static void batchRenameDefragmentedCacheGroupPartitions(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (!completionMarkerFile.exists())
+            return;
+
+        try {
+            for (File mappingFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)))
+                Files.delete(mappingFile.toPath());
+
+            for (File partFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_PARTITION_FILE_PREFIX))) {
+                int partId = extractPartId(partFile.getName());
+
+                File oldPartFile = new File(workDir, String.format(PART_FILE_TEMPLATE, partId));
+
+                Files.move(partFile.toPath(), oldPartFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+
+            File idxFile = new File(workDir, DFRG_INDEX_FILE_NAME);
+
+            if (idxFile.exists()) {
+                File oldIdxFile = new File(workDir, INDEX_FILE_NAME);
+
+                Files.move(idxFile.toPath(), oldIdxFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException(e);
+        }
+    }
+
+    /**
+     * Extracts partition number from file names like {@code part-dfrg-%d.bin}.
+     *
+     * @param dfrgPartFileName Defragmented partition file name.
+     * @return Partition index.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    private static int extractPartId(String dfrgPartFileName) {
+        assert dfrgPartFileName.startsWith(DFRG_PARTITION_FILE_PREFIX) : dfrgPartFileName;
+        assert dfrgPartFileName.endsWith(FILE_SUFFIX) : dfrgPartFileName;
+
+        String partIdStr = dfrgPartFileName.substring(
+            DFRG_PARTITION_FILE_PREFIX.length(),
+            dfrgPartFileName.length() - FILE_SUFFIX.length()
+        );
+
+        return Integer.parseInt(partIdStr);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin.tmp} in given folder. It will be used for storing defragmented index
+     * partition during the process.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexFile(File)
+     */
+    public static File defragmentedIndexTmpFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_TMP_FILE_NAME);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin} in given folder. It will be used for storing defragmented index
+     * partition when the process is over.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexTmpFile(File)
+     */
+    public static File defragmentedIndexFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_FILE_NAME);
+    }
+
+    /**
+     * Rename temporary index defragmentation file to a finalized one.
+     *
+     * @param workDir Cache group working directory.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexTmpFile(File)
+     * @see DefragmentationFileUtils#defragmentedIndexFile(File)
+     */
+    public static void renameTempIndexFile(File workDir) throws IgniteCheckedException {

Review comment:
       Because code looks better when chunks of code have names. Method that calls it is already about 250 lines, that's too much.
   EDIT: current class has everything related to files handling, it's convenient to have all these methods in a single place.




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r531457700



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;
+
+                                if (log.isDebugEnabled()) {
+                                    log.debug(S.toString(
+                                        "Partition defragmented",
+                                        "grpId", grpId, false,
+                                        "partId", partId, false,
+                                        "oldPages", oldPageStore.pages(), false,
+                                        "newPages", partCtx.partPagesAllocated.get() + 1, false,
+                                        "mappingPages", partCtx.mappingPagesAllocated.get() + 1, false,
+                                        "pageSize", pageSize, false,
+                                        "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                        "workDir", workDir, false
+                                    ));
+                                }
+
+                                oldPageMem.invalidate(grpId, partId);
+
+                                partCtx.partPageMemory.invalidate(grpId, partId);
+
+                                DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                                pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                                try {
+                                    renameTempPartitionFile(workDir, partId);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    throw new IgniteException(e);

Review comment:
       Ok, I'm cool with it, not a big deal. I'll make other places more complicated though




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

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r529648796



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/DefragmentationFileUtils.java
##########
@@ -0,0 +1,395 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static java.nio.file.StandardCopyOption.ATOMIC_MOVE;
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static java.nio.file.StandardOpenOption.CREATE_NEW;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.FILE_SUFFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_TEMPLATE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.TMP_SUFFIX;
+
+/**
+ * Everything related to file management during defragmentation process.
+ */
+public class DefragmentationFileUtils {
+    /** Prefix for link mapping files. */
+    private static final String DFRG_LINK_MAPPING_FILE_PREFIX = PART_FILE_PREFIX + "map-";
+
+    /** Link mapping file template. */
+    private static final String DFRG_LINK_MAPPING_FILE_TEMPLATE = DFRG_LINK_MAPPING_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmentation complation marker file name. */
+    private static final String DFRG_COMPLETION_MARKER_FILE_NAME = "dfrg-completion-marker";
+
+    /** Name of defragmentated index partition file. */
+    private static final String DFRG_INDEX_FILE_NAME = INDEX_FILE_PREFIX + "-dfrg" + FILE_SUFFIX;
+
+    /** Name of defragmentated index partition temporary file. */
+    private static final String DFRG_INDEX_TMP_FILE_NAME = DFRG_INDEX_FILE_NAME + TMP_SUFFIX;
+
+    /** Prefix for defragmented partition files. */
+    private static final String DFRG_PARTITION_FILE_PREFIX = PART_FILE_PREFIX + "dfrg-";
+
+    /** Defragmented partition file template. */
+    private static final String DFRG_PARTITION_FILE_TEMPLATE = DFRG_PARTITION_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmented partition temp file template. */
+    private static final String DFRG_PARTITION_TMP_FILE_TEMPLATE = DFRG_PARTITION_FILE_TEMPLATE + TMP_SUFFIX;
+
+    /**
+     * Performs cleanup of work dir before initializing file page stores.
+     * Will finish batch renaming if defragmentation was completed or delete garbage if it wasn't.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     */
+    public static void beforeInitPageStores(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+        U.delete(defragmentationCompletionMarkerFile(workDir));
+
+        for (File file : workDir.listFiles()) {
+            String fileName = file.getName();
+
+            if (
+                fileName.startsWith(DFRG_PARTITION_FILE_PREFIX)
+                || fileName.startsWith(DFRG_INDEX_FILE_NAME)
+                || fileName.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)
+            )
+                U.delete(file);
+        }
+    }
+
+    /**
+     * Checks whether cache group defragmentation completed or not. Completes it if all that's left is renaming.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param log Logger to write messages.
+     * @return {@code true} if given cache group is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentationCompletionMarkerFile(File)
+     */
+    public static boolean skipAlreadyDefragmentedCacheGroup(File workDir, int grpId, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (completionMarkerFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented page group",
+                    "grpId", grpId, false,
+                    "markerFileName", completionMarkerFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+            return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * Checks whether partition has already been defragmented or not. Cleans corrupted data if previous failed
+     * defragmentation attempt was found.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param partId Partition index to check.
+     * @param log Logger to write messages.
+     * @return {@code true} if given partition is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartMappingFile(File, int)
+     */
+    public static boolean skipAlreadyDefragmentedPartition(File workDir, int grpId, int partId, IgniteLogger log) throws IgniteCheckedException {
+        File defragmentedPartFile = defragmentedPartFile(workDir, partId);
+        File defragmentedPartMappingFile = defragmentedPartMappingFile(workDir, partId);
+
+        if (defragmentedPartFile.exists() && defragmentedPartMappingFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented partition",
+                    "grpId", grpId, false,
+                    "partId", partId, false,
+                    "partFileName", defragmentedPartFile.getName(), false,
+                    "mappingFileName", defragmentedPartMappingFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            return true;
+        }
+
+        File defragmentedPartTmpFile = defragmentedPartTmpFile(workDir, partId);
+
+        try {
+            Files.deleteIfExists(defragmentedPartTmpFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartMappingFile.toPath());
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException(e);
+        }
+
+        return false;
+    }
+
+    /**
+     * Failure-tolerant batch rename of defragmented partition files.
+     *
+     * Deletes all link mapping files old partition and index files, renaming defragmentated files in the process. Can
+     * be run on the same folder multiple times if failed for some reason.
+     *
+     * Does something only if completion marker is present in the folder. This marker won't be deleted in the end.
+     * Deletion of the marker must be done outside of defragmentation mode to prevent cache groups to be defragmentated
+     * several times in case of failures.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#writeDefragmentationCompletionMarker(FileIOFactory, File, IgniteLogger)
+     */
+    public static void batchRenameDefragmentedCacheGroupPartitions(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (!completionMarkerFile.exists())
+            return;
+
+        try {
+            for (File mappingFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)))
+                Files.delete(mappingFile.toPath());
+
+            for (File partFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_PARTITION_FILE_PREFIX))) {
+                int partId = extractPartId(partFile.getName());
+
+                File oldPartFile = new File(workDir, String.format(PART_FILE_TEMPLATE, partId));
+
+                Files.move(partFile.toPath(), oldPartFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+
+            File idxFile = new File(workDir, DFRG_INDEX_FILE_NAME);
+
+            if (idxFile.exists()) {
+                File oldIdxFile = new File(workDir, INDEX_FILE_NAME);
+
+                Files.move(idxFile.toPath(), oldIdxFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException(e);
+        }
+    }
+
+    /**
+     * Extracts partition number from file names like {@code part-dfrg-%d.bin}.
+     *
+     * @param dfrgPartFileName Defragmented partition file name.
+     * @return Partition index.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    private static int extractPartId(String dfrgPartFileName) {
+        assert dfrgPartFileName.startsWith(DFRG_PARTITION_FILE_PREFIX) : dfrgPartFileName;
+        assert dfrgPartFileName.endsWith(FILE_SUFFIX) : dfrgPartFileName;
+
+        String partIdStr = dfrgPartFileName.substring(
+            DFRG_PARTITION_FILE_PREFIX.length(),
+            dfrgPartFileName.length() - FILE_SUFFIX.length()
+        );
+
+        return Integer.parseInt(partIdStr);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin.tmp} in given folder. It will be used for storing defragmented index
+     * partition during the process.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexFile(File)
+     */
+    public static File defragmentedIndexTmpFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_TMP_FILE_NAME);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin} in given folder. It will be used for storing defragmented index
+     * partition when the process is over.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexTmpFile(File)
+     */
+    public static File defragmentedIndexFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_FILE_NAME);
+    }
+
+    /**
+     * Rename temporary index defragmentation file to a finalized one.
+     *
+     * @param workDir Cache group working directory.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexTmpFile(File)
+     * @see DefragmentationFileUtils#defragmentedIndexFile(File)
+     */
+    public static void renameTempIndexFile(File workDir) throws IgniteCheckedException {

Review comment:
       Used only once, do we need it at all? Why such simple actions can't be inlined?




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r530395551



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;
+
+                                if (log.isDebugEnabled()) {
+                                    log.debug(S.toString(
+                                        "Partition defragmented",
+                                        "grpId", grpId, false,
+                                        "partId", partId, false,
+                                        "oldPages", oldPageStore.pages(), false,
+                                        "newPages", partCtx.partPagesAllocated.get() + 1, false,
+                                        "mappingPages", partCtx.mappingPagesAllocated.get() + 1, false,
+                                        "pageSize", pageSize, false,
+                                        "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                        "workDir", workDir, false
+                                    ));
+                                }
+
+                                oldPageMem.invalidate(grpId, partId);
+
+                                partCtx.partPageMemory.invalidate(grpId, partId);
+
+                                DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                                pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                                try {
+                                    renameTempPartitionFile(workDir, partId);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    throw new IgniteException(e);
+                                }
+                            }
+                        };
+
+                        GridFutureAdapter<?> cpFut = defragmentationCheckpoint
+                            .forceCheckpoint("partition defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+
+                        cpFut.listen(cpLsnr);
+
+                        cmpFut.add((IgniteInternalFuture<Object>)cpFut);
+                    }
+
+                    // A bit too general for now, but I like it more then saving only the last checkpoint future.
+                    cmpFut.markInitialized().get();
+
+                    idxDfrgFut = new GridFinishedFuture<>();
+
+                    if (filePageStoreMgr.hasIndexStore(grpId)) {
+                        defragmentIndexPartition(oldGrpCtx, newGrpCtx, linkMapByPart);
+
+                        idxDfrgFut = defragmentationCheckpoint
+                            .forceCheckpoint("index defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+                    }
+
+                    idxDfrgFut.listen(fut -> {
+                        oldPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx partPageMem = (PageMemoryEx)partDataRegion.pageMemory();
+
+                        partPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().removePageStore(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx mappingPageMem = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+                        pageMgr = (DefragmentationPageReadWriteManager)mappingPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().clear(grpId);
+
+                        try {
+                            renameTempIndexFile(workDir);
+
+                            writeDefragmentationCompletionMarker(filePageStoreMgr.getPageStoreFileIoFactory(), workDir, log);

Review comment:
       > Will the process be finished?
   Yes, it'll be finished after restart.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;
+
+                                if (log.isDebugEnabled()) {
+                                    log.debug(S.toString(
+                                        "Partition defragmented",
+                                        "grpId", grpId, false,
+                                        "partId", partId, false,
+                                        "oldPages", oldPageStore.pages(), false,
+                                        "newPages", partCtx.partPagesAllocated.get() + 1, false,
+                                        "mappingPages", partCtx.mappingPagesAllocated.get() + 1, false,
+                                        "pageSize", pageSize, false,
+                                        "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                        "workDir", workDir, false
+                                    ));
+                                }
+
+                                oldPageMem.invalidate(grpId, partId);
+
+                                partCtx.partPageMemory.invalidate(grpId, partId);
+
+                                DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                                pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                                try {
+                                    renameTempPartitionFile(workDir, partId);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    throw new IgniteException(e);
+                                }
+                            }
+                        };
+
+                        GridFutureAdapter<?> cpFut = defragmentationCheckpoint
+                            .forceCheckpoint("partition defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+
+                        cpFut.listen(cpLsnr);
+
+                        cmpFut.add((IgniteInternalFuture<Object>)cpFut);
+                    }
+
+                    // A bit too general for now, but I like it more then saving only the last checkpoint future.
+                    cmpFut.markInitialized().get();
+
+                    idxDfrgFut = new GridFinishedFuture<>();
+
+                    if (filePageStoreMgr.hasIndexStore(grpId)) {
+                        defragmentIndexPartition(oldGrpCtx, newGrpCtx, linkMapByPart);
+
+                        idxDfrgFut = defragmentationCheckpoint
+                            .forceCheckpoint("index defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+                    }
+
+                    idxDfrgFut.listen(fut -> {
+                        oldPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx partPageMem = (PageMemoryEx)partDataRegion.pageMemory();
+
+                        partPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().removePageStore(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx mappingPageMem = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+                        pageMgr = (DefragmentationPageReadWriteManager)mappingPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().clear(grpId);
+
+                        try {
+                            renameTempIndexFile(workDir);
+
+                            writeDefragmentationCompletionMarker(filePageStoreMgr.getPageStoreFileIoFactory(), workDir, log);

Review comment:
       > Will the process be finished?
   
   Yes, it'll be finished after restart.




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r524319939



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,887 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database schared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();

Review comment:
       It actually affects the way cache data entries are stored - they either have cacheId inside or they don't. Tests show that persistent caches are no exception




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r530196859



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/TreeIterator.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.nio.ByteBuffer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.util.GridUnsafe;
+
+/** */
+public class TreeIterator {
+    /** Direct memory buffer with a size of one page. */
+    private final ByteBuffer pageBuf;
+
+    /** Offheap page size. */
+    private final int pageSize;
+
+    /** */
+    public TreeIterator(int size) {
+        pageSize = size;
+
+        pageBuf = ByteBuffer.allocateDirect(pageSize);
+    }
+
+    // Performance impact of constant closures allocation is not clear. So this method should be avoided in massive
+    // operations like tree leaves access.
+    /** */
+    public static <T> T access(
+        PageAccessType access,
+        PageMemoryEx pageMemory,
+        int grpId,
+        long pageId,
+        PageAccessor<T> accessor
+    ) throws IgniteCheckedException {
+        assert access != null;

Review comment:
       I doubt that anyone will add anything, these types correspond to "lockRead" and "lockWrite" for the pages.
   Technically there is a "restore write lock", or whatever it is called, so I'll change the assert, thank you!




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

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



[GitHub] [ignite] agoncharuk commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
agoncharuk commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r524281992



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,887 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database schared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();

Review comment:
       How is defragmentation affected by the page eviction mode? Tha page eviction should only make sense for in-memory caches, so this should not be relevant for persistence.




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

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



[GitHub] [ignite] agoncharuk commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
agoncharuk commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r524301405



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,887 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database schared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {

Review comment:
       Some performance considerations here:
   1) We iterate over tree data in PK order which will result in tons of random reads on cold start. Since we run defragmentation per-partition, it makes sense to try to preload partition beforehand
   2) Are all the partitions processed in one thread? Since we do actual inserts to the data tree, there will be a noticable CPU overhead for this, thus it makes sense to execute defragmentation for multiple partitions in parallel
   
   We can implement these improvements in separate 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.

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r530993086



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,896 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database schared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;
+
+                                if (log.isDebugEnabled()) {
+                                    log.debug(S.toString(
+                                        "Partition defragmented",
+                                        "grpId", grpId, false,
+                                        "partId", partId, false,
+                                        "oldPages", oldPageStore.pages(), false,
+                                        "newPages", partCtx.partPagesAllocated.get() + 1, false,
+                                        "mappingPages", partCtx.mappingPagesAllocated.get() + 1, false,
+                                        "pageSize", pageSize, false,
+                                        "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                        "workDir", workDir, false
+                                    ));
+                                }
+
+                                oldPageMem.invalidate(grpId, partId);
+
+                                partCtx.partPageMemory.invalidate(grpId, partId);
+
+                                DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                                pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                                try {
+                                    renameTempPartitionFile(workDir, partId);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    throw new IgniteException(e);
+                                }
+                            }
+                        };
+
+                        GridFutureAdapter<?> cpFut = defragmentationCheckpoint
+                            .forceCheckpoint("partition defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+
+                        cpFut.listen(cpLsnr);
+
+                        cmpFut.add((IgniteInternalFuture<Object>)cpFut);
+                    }
+
+                    // A bit too general for now, but I like it more then saving only the last checkpoint future.
+                    cmpFut.markInitialized().get();
+
+                    idxDfrgFut = new GridFinishedFuture<>();
+
+                    if (filePageStoreMgr.hasIndexStore(grpId)) {
+                        defragmentIndexPartition(oldGrpCtx, newGrpCtx, linkMapByPart);
+
+                        idxDfrgFut = defragmentationCheckpoint
+                            .forceCheckpoint("index defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+                    }
+
+                    idxDfrgFut.listen(fut -> {
+                        oldPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx partPageMem = (PageMemoryEx)partDataRegion.pageMemory();
+
+                        partPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().removePageStore(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx mappingPageMem = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+                        pageMgr = (DefragmentationPageReadWriteManager)mappingPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().clear(grpId);
+
+                        try {
+                            renameTempIndexFile(workDir);
+
+                            writeDefragmentationCompletionMarker(filePageStoreMgr.getPageStoreFileIoFactory(), workDir, log);
+
+                            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    });
+                }
+
+                // I guess we should wait for it?
+                if (idxDfrgFut != null)
+                    idxDfrgFut.get();
+            }
+
+            mntcReg.unregisterMaintenanceTask(DEFRAGMENTATION_MNTC_TASK_NAME);
+
+            log.info("Defragmentation completed. All partitions are defragmented.");
+        }
+        finally {
+            defragmentationCheckpoint.stop(true);
+        }
+    }
+
+    /** */
+    public void createIndexPageStore(
+        int grpId,
+        File workDir,
+        FilePageStoreFactory pageStoreFactory,
+        DataRegion partRegion,
+        LongConsumer allocatedTracker
+    ) throws IgniteCheckedException {
+        // Index partition file has to be deleted before we begin, otherwise there's a chance of reading corrupted file.
+        // There is a time period when index is already defragmented but marker file is not created yet. If node is
+        // failed in that time window then index will be deframented once again. That's fine, situation is rare but code
+        // to fix that would add unnecessary complications.
+        U.delete(defragmentedIndexTmpFile(workDir));
+
+        PageStore idxPageStore;
+
+        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+        try {
+            idxPageStore = pageStoreFactory.createPageStore(
+                FLAG_IDX,
+                () -> defragmentedIndexTmpFile(workDir).toPath(),
+                allocatedTracker
+            );
+        }
+        finally {
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+        }
+
+        idxPageStore.sync();
+
+        PageMemoryEx partPageMem = (PageMemoryEx)partRegion.pageMemory();
+
+        DefragmentationPageReadWriteManager partMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+        partMgr.pageStoreMap().addPageStore(grpId, PageIdAllocator.INDEX_PARTITION, idxPageStore);
+    }
+
+    /**
+     * Cancel the process of defragmentation.
+     */
+    public void cancel(){

Review comment:
       Done




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

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r529609303



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/TreeIterator.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.nio.ByteBuffer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.util.GridUnsafe;
+
+/** */
+public class TreeIterator {
+    /** Direct memory buffer with a size of one page. */
+    private final ByteBuffer pageBuf;
+
+    /** Offheap page size. */
+    private final int pageSize;
+
+    /** */
+    public TreeIterator(int size) {
+        pageSize = size;
+
+        pageBuf = ByteBuffer.allocateDirect(pageSize);
+    }
+
+    // Performance impact of constant closures allocation is not clear. So this method should be avoided in massive
+    // operations like tree leaves access.
+    /** */
+    public static <T> T access(
+        PageAccessType access,
+        PageMemoryEx pageMemory,
+        int grpId,
+        long pageId,
+        PageAccessor<T> accessor
+    ) throws IgniteCheckedException {
+        assert access != null;
+        long page = pageMemory.acquirePage(grpId, pageId);
+
+        try {
+            long pageAddr = access == PageAccessType.ACCESS_READ
+                    ? pageMemory.readLock(grpId, pageId, page)
+                    : pageMemory.writeLock(grpId, pageId, page);
+
+            try {
+                return accessor.access(pageAddr);
+            }
+            finally {
+                if (access == PageAccessType.ACCESS_READ)
+                    pageMemory.readUnlock(grpId, pageId, page);
+                else
+                    pageMemory.writeUnlock(grpId, pageId, page, null, true);
+            }
+        }
+        finally {
+            pageMemory.releasePage(grpId, pageId, page);
+        }
+    }
+
+    /** */
+    @SuppressWarnings("PublicInnerClass")
+    public enum PageAccessType {

Review comment:
       Let's remove such customization and create simple read/write methods.




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r524320192



##########
File path: modules/core/src/main/java/org/apache/ignite/configuration/DefragmentationConfiguration.java
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.configuration;
+
+import java.io.Serializable;
+
+/**
+ *
+ */
+public final class DefragmentationConfiguration implements Serializable {

Review comment:
       Yes, we can, it's a subject for discussion 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.

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r530212951



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;
+
+                                if (log.isDebugEnabled()) {
+                                    log.debug(S.toString(
+                                        "Partition defragmented",
+                                        "grpId", grpId, false,
+                                        "partId", partId, false,
+                                        "oldPages", oldPageStore.pages(), false,
+                                        "newPages", partCtx.partPagesAllocated.get() + 1, false,
+                                        "mappingPages", partCtx.mappingPagesAllocated.get() + 1, false,
+                                        "pageSize", pageSize, false,
+                                        "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                        "workDir", workDir, false
+                                    ));
+                                }
+
+                                oldPageMem.invalidate(grpId, partId);
+
+                                partCtx.partPageMemory.invalidate(grpId, partId);
+
+                                DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                                pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                                try {
+                                    renameTempPartitionFile(workDir, partId);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    throw new IgniteException(e);
+                                }
+                            }
+                        };
+
+                        GridFutureAdapter<?> cpFut = defragmentationCheckpoint
+                            .forceCheckpoint("partition defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+
+                        cpFut.listen(cpLsnr);
+
+                        cmpFut.add((IgniteInternalFuture<Object>)cpFut);
+                    }
+
+                    // A bit too general for now, but I like it more then saving only the last checkpoint future.
+                    cmpFut.markInitialized().get();
+
+                    idxDfrgFut = new GridFinishedFuture<>();
+
+                    if (filePageStoreMgr.hasIndexStore(grpId)) {
+                        defragmentIndexPartition(oldGrpCtx, newGrpCtx, linkMapByPart);
+
+                        idxDfrgFut = defragmentationCheckpoint
+                            .forceCheckpoint("index defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+                    }
+
+                    idxDfrgFut.listen(fut -> {
+                        oldPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx partPageMem = (PageMemoryEx)partDataRegion.pageMemory();
+
+                        partPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().removePageStore(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx mappingPageMem = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+                        pageMgr = (DefragmentationPageReadWriteManager)mappingPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().clear(grpId);
+
+                        try {
+                            renameTempIndexFile(workDir);
+
+                            writeDefragmentationCompletionMarker(filePageStoreMgr.getPageStoreFileIoFactory(), workDir, log);

Review comment:
       You just described the exact test that we already have :)




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r527617593



##########
File path: modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsIndexingDefragmentationTest.java
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.processors.cache.persistence;
+
+import java.io.File;
+import java.util.Collections;
+import java.util.Objects;
+import java.util.function.Function;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.visor.verify.ValidateIndexesClosure;
+import org.apache.ignite.internal.visor.verify.VisorValidateIndexesJobResult;
+import org.apache.ignite.testframework.junits.WithSystemProperty;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
+/**
+ * Defragmentation tests with enabled ignite-indexing.
+ */
+public class IgnitePdsIndexingDefragmentationTest extends IgnitePdsDefragmentationTest {
+    /** Use MVCC in tests. */
+    private static final String USE_MVCC = "USE_MVCC";
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setConsistentId(igniteInstanceName);
+
+        DataStorageConfiguration dsCfg = new DataStorageConfiguration();
+        dsCfg.setWalSegmentSize(4 * 1024 * 1024);
+
+        dsCfg.setDefaultDataRegionConfiguration(
+            new DataRegionConfiguration()
+                .setInitialSize(100L * 1024 * 1024)
+                .setMaxSize(1024L * 1024 * 1024)
+                .setPersistenceEnabled(true)
+        );
+
+        cfg.setDataStorageConfiguration(dsCfg);
+
+        CacheConfiguration<?, ?> cache1Cfg = new CacheConfiguration<>(DEFAULT_CACHE_NAME)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setGroupName(GRP_NAME)
+            .setIndexedTypes(
+                ObjKey.class, byte[].class,
+                Integer.class, byte[].class
+            )
+            .setAffinity(new RendezvousAffinityFunction(false, PARTS));
+
+        CacheConfiguration<?, ?> cache2Cfg = new CacheConfiguration<>(CACHE_2_NAME)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setGroupName(GRP_NAME)
+            .setIndexedTypes(
+                ObjKey.class, byte[].class,
+                Integer.class, byte[].class

Review comment:
       More tests added.

##########
File path: modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsIndexingDefragmentationTest.java
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.processors.cache.persistence;
+
+import java.io.File;
+import java.util.Collections;
+import java.util.Objects;
+import java.util.function.Function;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.visor.verify.ValidateIndexesClosure;
+import org.apache.ignite.internal.visor.verify.VisorValidateIndexesJobResult;
+import org.apache.ignite.testframework.junits.WithSystemProperty;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
+/**
+ * Defragmentation tests with enabled ignite-indexing.
+ */
+public class IgnitePdsIndexingDefragmentationTest extends IgnitePdsDefragmentationTest {
+    /** Use MVCC in tests. */
+    private static final String USE_MVCC = "USE_MVCC";
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setConsistentId(igniteInstanceName);
+
+        DataStorageConfiguration dsCfg = new DataStorageConfiguration();
+        dsCfg.setWalSegmentSize(4 * 1024 * 1024);
+
+        dsCfg.setDefaultDataRegionConfiguration(
+            new DataRegionConfiguration()
+                .setInitialSize(100L * 1024 * 1024)
+                .setMaxSize(1024L * 1024 * 1024)
+                .setPersistenceEnabled(true)
+        );
+
+        cfg.setDataStorageConfiguration(dsCfg);
+
+        CacheConfiguration<?, ?> cache1Cfg = new CacheConfiguration<>(DEFAULT_CACHE_NAME)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setGroupName(GRP_NAME)
+            .setIndexedTypes(
+                ObjKey.class, byte[].class,
+                Integer.class, byte[].class
+            )
+            .setAffinity(new RendezvousAffinityFunction(false, PARTS));
+
+        CacheConfiguration<?, ?> cache2Cfg = new CacheConfiguration<>(CACHE_2_NAME)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setGroupName(GRP_NAME)
+            .setIndexedTypes(
+                ObjKey.class, byte[].class,
+                Integer.class, byte[].class
+            )
+            .setAffinity(new RendezvousAffinityFunction(false, PARTS));
+
+        if (Boolean.TRUE.toString().equals(System.getProperty(USE_MVCC))) {
+            cache1Cfg.setAtomicityMode(TRANSACTIONAL_SNAPSHOT);
+            cache2Cfg.setAtomicityMode(TRANSACTIONAL_SNAPSHOT);
+        } else
+            cache2Cfg.setExpiryPolicyFactory(new PolicyFactory());
+
+        cfg.setCacheConfiguration(cache1Cfg, cache2Cfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        GridQueryProcessor.idxCls = null;
+    }
+
+    /**
+     * Fill cache, remove half of the entries, defragmentate PDS and check index.
+     *
+     * @param keyMapper Function that provides key based on the index of entry.
+     * @param <T> Type of cache key.
+     *
+     * @throws Exception If failed.
+     */
+    private <T> void test(Function<Integer, T> keyMapper) throws Exception {
+        IgniteEx ig = startGrid(0);
+
+        ig.cluster().state(ClusterState.ACTIVE);
+
+        fillCache(keyMapper, ig.cache(DEFAULT_CACHE_NAME));
+
+        forceCheckpoint(ig);
+
+        createMaintenanceRecord();
+
+        stopGrid(0);
+
+        File dbWorkDir = U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false);
+        File nodeWorkDir = new File(dbWorkDir, U.maskForFileName(ig.name()));
+        File workDir = new File(nodeWorkDir, FilePageStoreManager.CACHE_GRP_DIR_PREFIX + GRP_NAME);
+
+        long oldIdxFileLen = new File(workDir, FilePageStoreManager.INDEX_FILE_NAME).length();
+
+        startGrid(0);
+
+        long newIdxFileLen = new File(workDir, FilePageStoreManager.INDEX_FILE_NAME).length();
+
+        assertTrue(newIdxFileLen <= oldIdxFileLen);
+
+        File completionMarkerFile = DefragmentationFileUtils.defragmentationCompletionMarkerFile(workDir);
+        assertTrue(completionMarkerFile.exists());
+
+        stopGrid(0);
+
+        GridQueryProcessor.idxCls = CaptureRebuildGridQueryIndexing.class;
+
+        IgniteEx node = startGrid(0);
+
+        awaitPartitionMapExchange();
+
+        CaptureRebuildGridQueryIndexing indexing = (CaptureRebuildGridQueryIndexing) node.context().query().getIndexing();
+
+        assertFalse(indexing.didRebuildIndexes());
+
+        IgniteCache<Object, Object> cache = node.cache(DEFAULT_CACHE_NAME);
+
+        assertFalse(completionMarkerFile.exists());
+
+        validateIndexes(node);

Review comment:
       Done in another test.




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r524325758



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,887 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database schared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {

Review comment:
       Number 2 is going to be implemented in separate ticket, that's for sure.
   Number 1 - will it really be faster to preload before copying? If we make this in parallel then replacement is possible, it's really tricky. I actually experimented with "reactive" approach - one thread reads disk to the buffer, other thread reads from the buffer. With one reader the difference is pretty much negligible and it makes no sense to make more readers since we're going to defragment several partitions in parallel anyway. Maybe I miss something, so I'll be glad to here more ideas.




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r531463578



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();

Review comment:
       I removed this and created factory method in offheap manager, thank you! Sorry for misunderstanding from my side.




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r525112647



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/DefragmentationFileUtils.java
##########
@@ -0,0 +1,400 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static java.nio.file.StandardCopyOption.ATOMIC_MOVE;
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static java.nio.file.StandardOpenOption.CREATE_NEW;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.FILE_SUFFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_TEMPLATE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.TMP_SUFFIX;
+
+/**
+ * Everything related to file management during defragmentation process.
+ */
+public class DefragmentationFileUtils {
+    /** Prefix for link mapping files. */
+    private static final String DFRG_LINK_MAPPING_FILE_PREFIX = PART_FILE_PREFIX + "map-";
+
+    /** Link mapping file template. */
+    private static final String DFRG_LINK_MAPPING_FILE_TEMPLATE = DFRG_LINK_MAPPING_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmentation complation marker file name. */
+    private static final String DFRG_COMPLETION_MARKER_FILE_NAME = "dfrg-completion-marker";
+
+    /** Name of defragmentated index partition file. */
+    private static final String DFRG_INDEX_FILE_NAME = INDEX_FILE_PREFIX + "-dfrg" + FILE_SUFFIX;
+
+    /** Name of defragmentated index partition temporary file. */
+    private static final String DFRG_INDEX_TMP_FILE_NAME = DFRG_INDEX_FILE_NAME + TMP_SUFFIX;
+
+    /** Prefix for defragmented partition files. */
+    private static final String DFRG_PARTITION_FILE_PREFIX = PART_FILE_PREFIX + "dfrg-";
+
+    /** Defragmented partition file template. */
+    private static final String DFRG_PARTITION_FILE_TEMPLATE = DFRG_PARTITION_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmented partition temp file template. */
+    private static final String DFRG_PARTITION_TMP_FILE_TEMPLATE = DFRG_PARTITION_FILE_TEMPLATE + TMP_SUFFIX;
+
+    /**
+     * Performs cleanup of work dir before initializing file page stores.
+     * Will finish batch renaming if defragmentation was completed or delete garbage if it wasn't.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     */
+    public static void beforeInitPageStores(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+        U.delete(defragmentationCompletionMarkerFile(workDir));
+
+        for (File file : workDir.listFiles()) {
+            String fileName = file.getName();
+
+            if (
+                fileName.startsWith(DFRG_PARTITION_FILE_PREFIX)
+                || fileName.startsWith(DFRG_INDEX_FILE_NAME)
+                || fileName.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)
+            )
+                U.delete(file);
+        }
+    }
+
+    /**
+     * Checks whether cache group defragmentation completed or not. Completes it if all that's left is renaming.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param log Logger to write messages.
+     * @return {@code true} if given cache group is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentationCompletionMarkerFile(File)
+     */
+    public static boolean skipAlreadyDefragmentedCacheGroup(File workDir, int grpId, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (completionMarkerFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented page group",
+                    "grpId", grpId, false,
+                    "markerFileName", completionMarkerFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+            return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * Checks whether partition has already been defragmented or not. Cleans corrupted data if previous failed
+     * defragmentation attempt was found.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param partId Partionion index to check.
+     * @param log Logger to write messages.
+     * @return {@code true} if given partition is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartMappingFile(File, int)
+     */
+    public static boolean skipAlreadyDefragmentedPartition(File workDir, int grpId, int partId, IgniteLogger log) throws IgniteCheckedException {
+        File defragmentedPartFile = defragmentedPartFile(workDir, partId);
+        File defragmentedPartMappingFile = defragmentedPartMappingFile(workDir, partId);
+
+        if (defragmentedPartFile.exists() && defragmentedPartMappingFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented partition",
+                    "grpId", grpId, false,
+                    "partId", partId, false,
+                    "partFileName", defragmentedPartFile.getName(), false,
+                    "mappingFileName", defragmentedPartMappingFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            return true;
+        }
+
+        File defragmentedPartTmpFile = defragmentedPartTmpFile(workDir, partId);
+
+        try {
+            Files.deleteIfExists(defragmentedPartTmpFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartMappingFile.toPath());
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+
+        return false;
+    }
+
+    /**
+     * Failure-tolerant batch rename of defragmented partition files.
+     *
+     * Deletes all link mapping files old partition and index files, renaming defragmentated files in the process. Can
+     * be run on the same folder multiple times if failed for some reason.
+     *
+     * Does something only if completion marker is present in the folder. This marker won't be deleted in the end.
+     * Deletion of the marker must be done outside of defragmentation mode to prevent cache groups to be defragmentated
+     * several times in case of failures.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#writeDefragmentationCompletionMarker(FileIOFactory, File, IgniteLogger)
+     */
+    public static void batchRenameDefragmentedCacheGroupPartitions(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (!completionMarkerFile.exists())
+            return;
+
+        try {
+            for (File mappingFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)))
+                Files.delete(mappingFile.toPath());
+
+            for (File partFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_PARTITION_FILE_PREFIX))) {
+                int partId = extractPartId(partFile.getName());
+
+                File oldPartFile = new File(workDir, String.format(PART_FILE_TEMPLATE, partId));
+
+                Files.move(partFile.toPath(), oldPartFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+
+            File idxFile = new File(workDir, DFRG_INDEX_FILE_NAME);
+
+            if (idxFile.exists()) {
+                File oldIdxFile = new File(workDir, INDEX_FILE_NAME);
+
+                Files.move(idxFile.toPath(), oldIdxFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Extracts partition number from file names like {@code part-dfrg-%d.bin}.
+     *
+     * @param dfrgPartFileName Defragmented partition file name.
+     * @return Partition index.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    private static int extractPartId(String dfrgPartFileName) {

Review comment:
       Why? Code looks better when methods are small and have simple names. It's already messy, are there other reasons besides "it's only used once"?




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r530220929



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {

Review comment:
       I know it might look like I don't want to fix anything. I assure you, it's not intentional.
   
   The only thing that I **strongly** disagree with is inlining of methods that have only one 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.

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r530992273



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/TimeTracker.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.util.Arrays;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/** */
+public class TimeTracker<Stage extends Enum<Stage>> {

Review comment:
       1. How can I use custom aggregating function for TimeBag? I don't see anything like that in the code. It all is very convoluted, TimeBag is designed for processes that have no repeating stages.
   2. I don't want to have 100 thousand entries in log to aggregate them later, it's too much. And current class aggregates time as effective as it's possible, pretty much. It doesn't use lists and maps and all these heave things.




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r525114108



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,896 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database schared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;
+
+                                if (log.isDebugEnabled()) {
+                                    log.debug(S.toString(
+                                        "Partition defragmented",
+                                        "grpId", grpId, false,
+                                        "partId", partId, false,
+                                        "oldPages", oldPageStore.pages(), false,
+                                        "newPages", partCtx.partPagesAllocated.get() + 1, false,
+                                        "mappingPages", partCtx.mappingPagesAllocated.get() + 1, false,
+                                        "pageSize", pageSize, false,
+                                        "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                        "workDir", workDir, false
+                                    ));
+                                }
+
+                                oldPageMem.invalidate(grpId, partId);
+
+                                partCtx.partPageMemory.invalidate(grpId, partId);
+
+                                DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                                pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                                try {
+                                    renameTempPartitionFile(workDir, partId);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    throw new IgniteException(e);
+                                }
+                            }
+                        };
+
+                        GridFutureAdapter<?> cpFut = defragmentationCheckpoint
+                            .forceCheckpoint("partition defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+
+                        cpFut.listen(cpLsnr);
+
+                        cmpFut.add((IgniteInternalFuture<Object>)cpFut);
+                    }
+
+                    // A bit too general for now, but I like it more then saving only the last checkpoint future.
+                    cmpFut.markInitialized().get();
+
+                    idxDfrgFut = new GridFinishedFuture<>();
+
+                    if (filePageStoreMgr.hasIndexStore(grpId)) {
+                        defragmentIndexPartition(oldGrpCtx, newGrpCtx, linkMapByPart);
+
+                        idxDfrgFut = defragmentationCheckpoint
+                            .forceCheckpoint("index defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+                    }
+
+                    idxDfrgFut.listen(fut -> {
+                        oldPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx partPageMem = (PageMemoryEx)partDataRegion.pageMemory();
+
+                        partPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().removePageStore(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx mappingPageMem = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+                        pageMgr = (DefragmentationPageReadWriteManager)mappingPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().clear(grpId);
+
+                        try {
+                            renameTempIndexFile(workDir);
+
+                            writeDefragmentationCompletionMarker(filePageStoreMgr.getPageStoreFileIoFactory(), workDir, log);
+
+                            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    });
+                }
+
+                // I guess we should wait for it?
+                if (idxDfrgFut != null)
+                    idxDfrgFut.get();
+            }
+
+            mntcReg.unregisterMaintenanceTask(DEFRAGMENTATION_MNTC_TASK_NAME);
+
+            log.info("Defragmentation completed. All partitions are defragmented.");
+        }
+        finally {
+            defragmentationCheckpoint.stop(true);
+        }
+    }
+
+    /** */
+    public void createIndexPageStore(
+        int grpId,
+        File workDir,
+        FilePageStoreFactory pageStoreFactory,
+        DataRegion partRegion,
+        LongConsumer allocatedTracker
+    ) throws IgniteCheckedException {
+        // Index partition file has to be deleted before we begin, otherwise there's a chance of reading corrupted file.
+        // There is a time period when index is already defragmented but marker file is not created yet. If node is
+        // failed in that time window then index will be deframented once again. That's fine, situation is rare but code
+        // to fix that would add unnecessary complications.
+        U.delete(defragmentedIndexTmpFile(workDir));
+
+        PageStore idxPageStore;
+
+        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+        try {
+            idxPageStore = pageStoreFactory.createPageStore(
+                FLAG_IDX,
+                () -> defragmentedIndexTmpFile(workDir).toPath(),
+                allocatedTracker
+            );
+        }
+        finally {
+            defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+        }
+
+        idxPageStore.sync();
+
+        PageMemoryEx partPageMem = (PageMemoryEx)partRegion.pageMemory();
+
+        DefragmentationPageReadWriteManager partMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+        partMgr.pageStoreMap().addPageStore(grpId, PageIdAllocator.INDEX_PARTITION, idxPageStore);
+    }
+
+    /**
+     * Cancel the process of defragmentation.
+     */
+    public void cancel(){

Review comment:
       Yes, we will, but for now I'll get rid of it. There's a separate issue for 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.

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r527618339



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/DefragmentationFileUtils.java
##########
@@ -0,0 +1,400 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static java.nio.file.StandardCopyOption.ATOMIC_MOVE;
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static java.nio.file.StandardOpenOption.CREATE_NEW;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.FILE_SUFFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_TEMPLATE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.TMP_SUFFIX;
+
+/**
+ * Everything related to file management during defragmentation process.
+ */
+public class DefragmentationFileUtils {
+    /** Prefix for link mapping files. */
+    private static final String DFRG_LINK_MAPPING_FILE_PREFIX = PART_FILE_PREFIX + "map-";
+
+    /** Link mapping file template. */
+    private static final String DFRG_LINK_MAPPING_FILE_TEMPLATE = DFRG_LINK_MAPPING_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmentation complation marker file name. */
+    private static final String DFRG_COMPLETION_MARKER_FILE_NAME = "dfrg-completion-marker";
+
+    /** Name of defragmentated index partition file. */
+    private static final String DFRG_INDEX_FILE_NAME = INDEX_FILE_PREFIX + "-dfrg" + FILE_SUFFIX;
+
+    /** Name of defragmentated index partition temporary file. */
+    private static final String DFRG_INDEX_TMP_FILE_NAME = DFRG_INDEX_FILE_NAME + TMP_SUFFIX;
+
+    /** Prefix for defragmented partition files. */
+    private static final String DFRG_PARTITION_FILE_PREFIX = PART_FILE_PREFIX + "dfrg-";
+
+    /** Defragmented partition file template. */
+    private static final String DFRG_PARTITION_FILE_TEMPLATE = DFRG_PARTITION_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmented partition temp file template. */
+    private static final String DFRG_PARTITION_TMP_FILE_TEMPLATE = DFRG_PARTITION_FILE_TEMPLATE + TMP_SUFFIX;
+
+    /**
+     * Performs cleanup of work dir before initializing file page stores.
+     * Will finish batch renaming if defragmentation was completed or delete garbage if it wasn't.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     */
+    public static void beforeInitPageStores(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+        U.delete(defragmentationCompletionMarkerFile(workDir));
+
+        for (File file : workDir.listFiles()) {
+            String fileName = file.getName();
+
+            if (
+                fileName.startsWith(DFRG_PARTITION_FILE_PREFIX)
+                || fileName.startsWith(DFRG_INDEX_FILE_NAME)
+                || fileName.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)
+            )
+                U.delete(file);
+        }
+    }
+
+    /**
+     * Checks whether cache group defragmentation completed or not. Completes it if all that's left is renaming.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param log Logger to write messages.
+     * @return {@code true} if given cache group is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentationCompletionMarkerFile(File)
+     */
+    public static boolean skipAlreadyDefragmentedCacheGroup(File workDir, int grpId, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (completionMarkerFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented page group",
+                    "grpId", grpId, false,
+                    "markerFileName", completionMarkerFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+            return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * Checks whether partition has already been defragmented or not. Cleans corrupted data if previous failed
+     * defragmentation attempt was found.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param partId Partionion index to check.
+     * @param log Logger to write messages.
+     * @return {@code true} if given partition is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartMappingFile(File, int)
+     */
+    public static boolean skipAlreadyDefragmentedPartition(File workDir, int grpId, int partId, IgniteLogger log) throws IgniteCheckedException {
+        File defragmentedPartFile = defragmentedPartFile(workDir, partId);
+        File defragmentedPartMappingFile = defragmentedPartMappingFile(workDir, partId);
+
+        if (defragmentedPartFile.exists() && defragmentedPartMappingFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented partition",
+                    "grpId", grpId, false,
+                    "partId", partId, false,
+                    "partFileName", defragmentedPartFile.getName(), false,
+                    "mappingFileName", defragmentedPartMappingFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            return true;
+        }
+
+        File defragmentedPartTmpFile = defragmentedPartTmpFile(workDir, partId);
+
+        try {
+            Files.deleteIfExists(defragmentedPartTmpFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartMappingFile.toPath());
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+
+        return false;
+    }
+
+    /**
+     * Failure-tolerant batch rename of defragmented partition files.
+     *
+     * Deletes all link mapping files old partition and index files, renaming defragmentated files in the process. Can
+     * be run on the same folder multiple times if failed for some reason.
+     *
+     * Does something only if completion marker is present in the folder. This marker won't be deleted in the end.
+     * Deletion of the marker must be done outside of defragmentation mode to prevent cache groups to be defragmentated
+     * several times in case of failures.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#writeDefragmentationCompletionMarker(FileIOFactory, File, IgniteLogger)
+     */
+    public static void batchRenameDefragmentedCacheGroupPartitions(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (!completionMarkerFile.exists())
+            return;
+
+        try {
+            for (File mappingFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)))
+                Files.delete(mappingFile.toPath());
+
+            for (File partFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_PARTITION_FILE_PREFIX))) {
+                int partId = extractPartId(partFile.getName());
+
+                File oldPartFile = new File(workDir, String.format(PART_FILE_TEMPLATE, partId));
+
+                Files.move(partFile.toPath(), oldPartFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+
+            File idxFile = new File(workDir, DFRG_INDEX_FILE_NAME);
+
+            if (idxFile.exists()) {
+                File oldIdxFile = new File(workDir, INDEX_FILE_NAME);
+
+                Files.move(idxFile.toPath(), oldIdxFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Extracts partition number from file names like {@code part-dfrg-%d.bin}.
+     *
+     * @param dfrgPartFileName Defragmented partition file name.
+     * @return Partition index.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    private static int extractPartId(String dfrgPartFileName) {

Review comment:
       What you described is the exact reason why we have 1000-lines methods)




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r530195555



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {

Review comment:
       ExecuteDefragmentationAction is a class for Maintenance only, I don't think that we should move all this code there.
   
   I agree that Manager is a term that is usually used for extensions of GridCacheManagerAdapter.
   Can we do the rename in other ignite issue? That's a personal question, we already have a chain of, like, 4 branches, created one from the other and they all have changes in this class. Renaming would be a nightmare :(




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r525132991



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/TimeTracker.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.util.Arrays;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/** */
+public class TimeTracker<Stage extends Enum<Stage>> {

Review comment:
       I mean that I need sum of all times that we waited for lock to be acquired, not every single time value.
   EDIT: I think it should be clear from implementation, class is very small and simple.




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

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r525123270



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/DefragmentationFileUtils.java
##########
@@ -0,0 +1,400 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static java.nio.file.StandardCopyOption.ATOMIC_MOVE;
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static java.nio.file.StandardOpenOption.CREATE_NEW;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.FILE_SUFFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_TEMPLATE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.TMP_SUFFIX;
+
+/**
+ * Everything related to file management during defragmentation process.
+ */
+public class DefragmentationFileUtils {
+    /** Prefix for link mapping files. */
+    private static final String DFRG_LINK_MAPPING_FILE_PREFIX = PART_FILE_PREFIX + "map-";
+
+    /** Link mapping file template. */
+    private static final String DFRG_LINK_MAPPING_FILE_TEMPLATE = DFRG_LINK_MAPPING_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmentation complation marker file name. */
+    private static final String DFRG_COMPLETION_MARKER_FILE_NAME = "dfrg-completion-marker";
+
+    /** Name of defragmentated index partition file. */
+    private static final String DFRG_INDEX_FILE_NAME = INDEX_FILE_PREFIX + "-dfrg" + FILE_SUFFIX;
+
+    /** Name of defragmentated index partition temporary file. */
+    private static final String DFRG_INDEX_TMP_FILE_NAME = DFRG_INDEX_FILE_NAME + TMP_SUFFIX;
+
+    /** Prefix for defragmented partition files. */
+    private static final String DFRG_PARTITION_FILE_PREFIX = PART_FILE_PREFIX + "dfrg-";
+
+    /** Defragmented partition file template. */
+    private static final String DFRG_PARTITION_FILE_TEMPLATE = DFRG_PARTITION_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmented partition temp file template. */
+    private static final String DFRG_PARTITION_TMP_FILE_TEMPLATE = DFRG_PARTITION_FILE_TEMPLATE + TMP_SUFFIX;
+
+    /**
+     * Performs cleanup of work dir before initializing file page stores.
+     * Will finish batch renaming if defragmentation was completed or delete garbage if it wasn't.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     */
+    public static void beforeInitPageStores(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+        U.delete(defragmentationCompletionMarkerFile(workDir));
+
+        for (File file : workDir.listFiles()) {
+            String fileName = file.getName();
+
+            if (
+                fileName.startsWith(DFRG_PARTITION_FILE_PREFIX)
+                || fileName.startsWith(DFRG_INDEX_FILE_NAME)
+                || fileName.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)
+            )
+                U.delete(file);
+        }
+    }
+
+    /**
+     * Checks whether cache group defragmentation completed or not. Completes it if all that's left is renaming.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param log Logger to write messages.
+     * @return {@code true} if given cache group is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentationCompletionMarkerFile(File)
+     */
+    public static boolean skipAlreadyDefragmentedCacheGroup(File workDir, int grpId, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (completionMarkerFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented page group",
+                    "grpId", grpId, false,
+                    "markerFileName", completionMarkerFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+            return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * Checks whether partition has already been defragmented or not. Cleans corrupted data if previous failed
+     * defragmentation attempt was found.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param partId Partionion index to check.
+     * @param log Logger to write messages.
+     * @return {@code true} if given partition is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartMappingFile(File, int)
+     */
+    public static boolean skipAlreadyDefragmentedPartition(File workDir, int grpId, int partId, IgniteLogger log) throws IgniteCheckedException {
+        File defragmentedPartFile = defragmentedPartFile(workDir, partId);
+        File defragmentedPartMappingFile = defragmentedPartMappingFile(workDir, partId);
+
+        if (defragmentedPartFile.exists() && defragmentedPartMappingFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented partition",
+                    "grpId", grpId, false,
+                    "partId", partId, false,
+                    "partFileName", defragmentedPartFile.getName(), false,
+                    "mappingFileName", defragmentedPartMappingFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            return true;
+        }
+
+        File defragmentedPartTmpFile = defragmentedPartTmpFile(workDir, partId);
+
+        try {
+            Files.deleteIfExists(defragmentedPartTmpFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartMappingFile.toPath());
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+
+        return false;
+    }
+
+    /**
+     * Failure-tolerant batch rename of defragmented partition files.
+     *
+     * Deletes all link mapping files old partition and index files, renaming defragmentated files in the process. Can
+     * be run on the same folder multiple times if failed for some reason.
+     *
+     * Does something only if completion marker is present in the folder. This marker won't be deleted in the end.
+     * Deletion of the marker must be done outside of defragmentation mode to prevent cache groups to be defragmentated
+     * several times in case of failures.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#writeDefragmentationCompletionMarker(FileIOFactory, File, IgniteLogger)
+     */
+    public static void batchRenameDefragmentedCacheGroupPartitions(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (!completionMarkerFile.exists())
+            return;
+
+        try {
+            for (File mappingFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)))
+                Files.delete(mappingFile.toPath());
+
+            for (File partFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_PARTITION_FILE_PREFIX))) {
+                int partId = extractPartId(partFile.getName());
+
+                File oldPartFile = new File(workDir, String.format(PART_FILE_TEMPLATE, partId));
+
+                Files.move(partFile.toPath(), oldPartFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+
+            File idxFile = new File(workDir, DFRG_INDEX_FILE_NAME);
+
+            if (idxFile.exists()) {
+                File oldIdxFile = new File(workDir, INDEX_FILE_NAME);
+
+                Files.move(idxFile.toPath(), oldIdxFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Extracts partition number from file names like {@code part-dfrg-%d.bin}.
+     *
+     * @param dfrgPartFileName Defragmented partition file name.
+     * @return Partition index.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    private static int extractPartId(String dfrgPartFileName) {

Review comment:
       Maybe I'm missing something, but from my understanding the logic is:
   - if the fragment of code is used only once there is no reason to do something
   - if the fragment of code is used may times this is the reason to create a private method inside the class
   - if the fragment of code is used may times between classes this is the reason to move it to util class
   
   If you have a method with 1000 lines and try to reduce it by 'naming' some pieces of code (creating methods for some fragments of code) than you should think about of completely reworking the method body. 
   
   So, my opinion is - such methods floods the source code and must be inlined. As an advantage of this - you will simplify the whole PR with reducing the changes. 
   
   But it's up to you :-)




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

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r529592983



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/TimeTracker.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.util.Arrays;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/** */
+public class TimeTracker<Stage extends Enum<Stage>> {

Review comment:
       I think we shouldn't have classes with 'almost the same behaviour'. 
   
   1. AFAIK, the `TimeBag` collects all the results per each Stage and then aggregates it. If you write a custom aggregate function will you get the same result as the `TimeTracker` does?
   
   1. Why do we need any aggregated results? Can we dump raw timings to log and aggregate than by some external tool (log-parser)? I think raw results would provide much valuable debug information




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r530215566



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsDefragmentationTest.java
##########
@@ -0,0 +1,539 @@
+/*
+ * 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.processors.cache.persistence;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.FileVisitResult;
+import java.nio.file.FileVisitor;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardCopyOption;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.Collections;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+import java.util.function.UnaryOperator;
+import java.util.stream.IntStream;
+import javax.cache.configuration.Factory;
+import javax.cache.expiry.Duration;
+import javax.cache.expiry.ExpiryPolicy;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.FailureHandler;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.maintenance.MaintenanceFileStore;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.util.lang.IgniteThrowableConsumer;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentationCompletionMarkerFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.maintenance.DefragmentationParameters.toStore;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+
+/** */
+public class IgnitePdsDefragmentationTest extends GridCommonAbstractTest {
+    /** */
+    public static final String CACHE_2_NAME = "cache2";
+
+    /** */
+    public static final int PARTS = 5;
+
+    /** */
+    public static final int ADDED_KEYS_COUNT = 150;
+
+    /** */
+    protected static final String GRP_NAME = "group";
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        stopAllGrids(true);
+
+        cleanPersistenceDir();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids(true);
+
+        cleanPersistenceDir();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected FailureHandler getFailureHandler(String igniteInstanceName) {
+        return new StopNodeFailureHandler();
+    }
+
+    /** */
+    protected static class PolicyFactory implements Factory<ExpiryPolicy> {
+        /** Serial version uid. */
+        private static final long serialVersionUID = 0L;
+
+        /** {@inheritDoc} */
+        @Override public ExpiryPolicy create() {
+            return new ExpiryPolicy() {
+                @Override public Duration getExpiryForCreation() {
+                    return new Duration(TimeUnit.MILLISECONDS, 13000);
+                }
+
+                /** {@inheritDoc} */
+                @Override public Duration getExpiryForAccess() {
+                    return new Duration(TimeUnit.MILLISECONDS, 13000);
+                }
+
+                /** {@inheritDoc} */
+                @Override public Duration getExpiryForUpdate() {
+                    return new Duration(TimeUnit.MILLISECONDS, 13000);
+                }
+            };
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setConsistentId(igniteInstanceName);
+
+        DataStorageConfiguration dsCfg = new DataStorageConfiguration();
+        dsCfg.setWalSegmentSize(4 * 1024 * 1024);
+
+        dsCfg.setDefaultDataRegionConfiguration(
+            new DataRegionConfiguration()
+                .setInitialSize(100L * 1024 * 1024)
+                .setMaxSize(1024L * 1024 * 1024)
+                .setPersistenceEnabled(true)
+        );
+
+        cfg.setDataStorageConfiguration(dsCfg);
+
+        CacheConfiguration<?, ?> cache1Cfg = new CacheConfiguration<>(DEFAULT_CACHE_NAME)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setGroupName(GRP_NAME)
+            .setAffinity(new RendezvousAffinityFunction(false, PARTS));
+
+        CacheConfiguration<?, ?> cache2Cfg = new CacheConfiguration<>(CACHE_2_NAME)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setGroupName(GRP_NAME)
+            .setExpiryPolicyFactory(new PolicyFactory())
+            .setAffinity(new RendezvousAffinityFunction(false, PARTS));
+
+        cfg.setCacheConfiguration(cache1Cfg, cache2Cfg);
+
+        return cfg;
+    }
+
+    /**
+     * Basic test scenario. Does following steps:
+     *  - Start node;
+     *  - Fill cache;
+     *  - Remove part of data;
+     *  - Stop node;
+     *  - Start node in defragmentation mode;
+     *  - Stop node;
+     *  - Start node;
+     *  - Check that partitions became smaller;
+     *  - Check that cache is accessible and works just fine.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testEssentials() throws Exception {
+        IgniteEx ig = startGrid(0);
+
+        ig.cluster().state(ClusterState.ACTIVE);
+
+        fillCache(ig.cache(DEFAULT_CACHE_NAME));
+
+        forceCheckpoint(ig);
+
+        createMaintenanceRecord();
+
+        stopGrid(0);
+
+        File workDir = resolveCacheWorkDir(ig);
+
+        long[] oldPartLen = partitionSizes(workDir);
+
+        long oldIdxFileLen = new File(workDir, FilePageStoreManager.INDEX_FILE_NAME).length();
+
+        startGrid(0);
+
+        long[] newPartLen = partitionSizes(workDir);
+
+        for (int p = 0; p < PARTS; p++)
+            assertTrue(newPartLen[p] < oldPartLen[p]); //TODO Fails.
+
+        long newIdxFileLen = new File(workDir, FilePageStoreManager.INDEX_FILE_NAME).length();
+
+        assertTrue(newIdxFileLen <= oldIdxFileLen);
+
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+        assertTrue(completionMarkerFile.exists());
+
+        stopGrid(0);
+
+        IgniteEx ig0 = startGrid(0);
+
+        ig0.cluster().state(ClusterState.ACTIVE);
+
+        assertFalse(completionMarkerFile.exists());
+
+        validateCache(grid(0).cache(DEFAULT_CACHE_NAME));

Review comment:
       Done




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

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r529597261



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {

Review comment:
       I think we should discuss class naming and/or extract `executeDefragmentation` action to some external tasks. From my understanding, each internal manager should have a global lifecycle by extending the general approach like `GridCacheSharedManager` or `GridCacheManager`. 




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r525132991



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/TimeTracker.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.util.Arrays;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/** */
+public class TimeTracker<Stage extends Enum<Stage>> {

Review comment:
       I mean that I need sum of all times that we waited for lock to be acquired, not every single time value.




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r526711666



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/DefragmentationFileUtils.java
##########
@@ -0,0 +1,400 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static java.nio.file.StandardCopyOption.ATOMIC_MOVE;
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static java.nio.file.StandardOpenOption.CREATE_NEW;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.FILE_SUFFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_TEMPLATE;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.TMP_SUFFIX;
+
+/**
+ * Everything related to file management during defragmentation process.
+ */
+public class DefragmentationFileUtils {
+    /** Prefix for link mapping files. */
+    private static final String DFRG_LINK_MAPPING_FILE_PREFIX = PART_FILE_PREFIX + "map-";
+
+    /** Link mapping file template. */
+    private static final String DFRG_LINK_MAPPING_FILE_TEMPLATE = DFRG_LINK_MAPPING_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmentation complation marker file name. */
+    private static final String DFRG_COMPLETION_MARKER_FILE_NAME = "dfrg-completion-marker";
+
+    /** Name of defragmentated index partition file. */
+    private static final String DFRG_INDEX_FILE_NAME = INDEX_FILE_PREFIX + "-dfrg" + FILE_SUFFIX;
+
+    /** Name of defragmentated index partition temporary file. */
+    private static final String DFRG_INDEX_TMP_FILE_NAME = DFRG_INDEX_FILE_NAME + TMP_SUFFIX;
+
+    /** Prefix for defragmented partition files. */
+    private static final String DFRG_PARTITION_FILE_PREFIX = PART_FILE_PREFIX + "dfrg-";
+
+    /** Defragmented partition file template. */
+    private static final String DFRG_PARTITION_FILE_TEMPLATE = DFRG_PARTITION_FILE_PREFIX + "%d" + FILE_SUFFIX;
+
+    /** Defragmented partition temp file template. */
+    private static final String DFRG_PARTITION_TMP_FILE_TEMPLATE = DFRG_PARTITION_FILE_TEMPLATE + TMP_SUFFIX;
+
+    /**
+     * Performs cleanup of work dir before initializing file page stores.
+     * Will finish batch renaming if defragmentation was completed or delete garbage if it wasn't.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     */
+    public static void beforeInitPageStores(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+        U.delete(defragmentationCompletionMarkerFile(workDir));
+
+        for (File file : workDir.listFiles()) {
+            String fileName = file.getName();
+
+            if (
+                fileName.startsWith(DFRG_PARTITION_FILE_PREFIX)
+                || fileName.startsWith(DFRG_INDEX_FILE_NAME)
+                || fileName.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)
+            )
+                U.delete(file);
+        }
+    }
+
+    /**
+     * Checks whether cache group defragmentation completed or not. Completes it if all that's left is renaming.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param log Logger to write messages.
+     * @return {@code true} if given cache group is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentationCompletionMarkerFile(File)
+     */
+    public static boolean skipAlreadyDefragmentedCacheGroup(File workDir, int grpId, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (completionMarkerFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented page group",
+                    "grpId", grpId, false,
+                    "markerFileName", completionMarkerFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+
+            return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * Checks whether partition has already been defragmented or not. Cleans corrupted data if previous failed
+     * defragmentation attempt was found.
+     *
+     * @param workDir Cache group working directory.
+     * @param grpId Cache group Id of cache group belonging to the given working directory.
+     * @param partId Partionion index to check.
+     * @param log Logger to write messages.
+     * @return {@code true} if given partition is already defragmented.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartMappingFile(File, int)
+     */
+    public static boolean skipAlreadyDefragmentedPartition(File workDir, int grpId, int partId, IgniteLogger log) throws IgniteCheckedException {
+        File defragmentedPartFile = defragmentedPartFile(workDir, partId);
+        File defragmentedPartMappingFile = defragmentedPartMappingFile(workDir, partId);
+
+        if (defragmentedPartFile.exists() && defragmentedPartMappingFile.exists()) {
+            if (log.isInfoEnabled()) {
+                log.info(S.toString(
+                    "Skipping already defragmented partition",
+                    "grpId", grpId, false,
+                    "partId", partId, false,
+                    "partFileName", defragmentedPartFile.getName(), false,
+                    "mappingFileName", defragmentedPartMappingFile.getName(), false,
+                    "workDir", workDir.getAbsolutePath(), false
+                ));
+            }
+
+            return true;
+        }
+
+        File defragmentedPartTmpFile = defragmentedPartTmpFile(workDir, partId);
+
+        try {
+            Files.deleteIfExists(defragmentedPartTmpFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartFile.toPath());
+
+            Files.deleteIfExists(defragmentedPartMappingFile.toPath());
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+
+        return false;
+    }
+
+    /**
+     * Failure-tolerant batch rename of defragmented partition files.
+     *
+     * Deletes all link mapping files old partition and index files, renaming defragmentated files in the process. Can
+     * be run on the same folder multiple times if failed for some reason.
+     *
+     * Does something only if completion marker is present in the folder. This marker won't be deleted in the end.
+     * Deletion of the marker must be done outside of defragmentation mode to prevent cache groups to be defragmentated
+     * several times in case of failures.
+     *
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#writeDefragmentationCompletionMarker(FileIOFactory, File, IgniteLogger)
+     */
+    public static void batchRenameDefragmentedCacheGroupPartitions(File workDir, IgniteLogger log) throws IgniteCheckedException {
+        File completionMarkerFile = defragmentationCompletionMarkerFile(workDir);
+
+        if (!completionMarkerFile.exists())
+            return;
+
+        try {
+            for (File mappingFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_LINK_MAPPING_FILE_PREFIX)))
+                Files.delete(mappingFile.toPath());
+
+            for (File partFile : workDir.listFiles((dir, name) -> name.startsWith(DFRG_PARTITION_FILE_PREFIX))) {
+                int partId = extractPartId(partFile.getName());
+
+                File oldPartFile = new File(workDir, String.format(PART_FILE_TEMPLATE, partId));
+
+                Files.move(partFile.toPath(), oldPartFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+
+            File idxFile = new File(workDir, DFRG_INDEX_FILE_NAME);
+
+            if (idxFile.exists()) {
+                File oldIdxFile = new File(workDir, INDEX_FILE_NAME);
+
+                Files.move(idxFile.toPath(), oldIdxFile.toPath(), ATOMIC_MOVE, REPLACE_EXISTING);
+            }
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Extracts partition number from file names like {@code part-dfrg-%d.bin}.
+     *
+     * @param dfrgPartFileName Defragmented partition file name.
+     * @return Partition index.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    private static int extractPartId(String dfrgPartFileName) {
+        assert dfrgPartFileName.startsWith(DFRG_PARTITION_FILE_PREFIX) : dfrgPartFileName;
+        assert dfrgPartFileName.endsWith(FILE_SUFFIX) : dfrgPartFileName;
+
+        String partIdStr = dfrgPartFileName.substring(
+            DFRG_PARTITION_FILE_PREFIX.length(),
+            dfrgPartFileName.length() - FILE_SUFFIX.length()
+        );
+
+        return Integer.parseInt(partIdStr);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin.tmp} in given folder. It will be used for storing defragmented index
+     * partition during the process.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexFile(File)
+     */
+    public static File defragmentedIndexTmpFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_TMP_FILE_NAME);
+    }
+
+    /**
+     * Return file named {@code index-dfrg.bin} in given folder. It will be used for storing defragmented index
+     * partition when the process is over.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexTmpFile(File)
+     */
+    public static File defragmentedIndexFile(File workDir) {
+        return new File(workDir, DFRG_INDEX_FILE_NAME);
+    }
+
+    /**
+     * Rename temporary index defragmenation file to a finalized one.
+     *
+     * @param workDir Cache group working directory.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedIndexTmpFile(File)
+     * @see DefragmentationFileUtils#defragmentedIndexFile(File)
+     */
+    public static void renameTempIndexFile(File workDir) throws IgniteCheckedException {
+        File defragmentedIdxTmpFile = defragmentedIndexTmpFile(workDir);
+        File defragmentedIdxFile = defragmentedIndexFile(workDir);
+
+        try {
+            Files.move(defragmentedIdxTmpFile.toPath(), defragmentedIdxFile.toPath(), ATOMIC_MOVE);
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Return file named {@code part-dfrg-%d.bin.tmp} in given folder. It will be used for storing defragmented data
+     * partition during the process.
+     *
+     * @param workDir Cache group working directory.
+     * @param partId Parition index, will be substituted into file name.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    public static File defragmentedPartTmpFile(File workDir, int partId) {
+        return new File(workDir, String.format(DFRG_PARTITION_TMP_FILE_TEMPLATE, partId));
+    }
+
+    /**
+     * Return file named {@code part-dfrg-%d.bin} in given folder. It will be used for storing defragmented data
+     * partition when the process is over.
+     *
+     * @param workDir Cache group working directory.
+     * @param partId Parition index, will be substituted into file name.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     */
+    public static File defragmentedPartFile(File workDir, int partId) {
+        return new File(workDir, String.format(DFRG_PARTITION_FILE_TEMPLATE, partId));
+    }
+
+    /**
+     * Rename temporary partition defragmenation file to a finalized one.
+     *
+     * @param workDir Cache group working directory.
+     * @param partId Parition index.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentedPartTmpFile(File, int)
+     * @see DefragmentationFileUtils#defragmentedPartFile(File, int)
+     */
+    public static void renameTempPartitionFile(File workDir, int partId) throws IgniteCheckedException {
+        File defragmentedPartTmpFile = defragmentedPartTmpFile(workDir, partId);
+        File defragmentedPartFile = defragmentedPartFile(workDir, partId);
+
+        assert !defragmentedPartFile.exists() : defragmentedPartFile;
+
+        try {
+            Files.move(defragmentedPartTmpFile.toPath(), defragmentedPartFile.toPath(), ATOMIC_MOVE);
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /**
+     * Return file named {@code part-map-%d.bin} in given folder. It will be used for storing defragmention links
+     * mapping for given partition during and after defragmentation process. No temporary counterpart is required here.
+     *
+     * @param workDir Cache group working directory.
+     * @param partId Parition index, will be substituted into file name.
+     * @return File.
+     *
+     * @see LinkMap
+     */
+    public static File defragmentedPartMappingFile(File workDir, int partId) {
+        return new File(workDir, String.format(DFRG_LINK_MAPPING_FILE_TEMPLATE, partId));
+    }
+
+    /**
+     * Return defragmentation completion marker file. This file can only be created when all partitions and index are
+     * defragmented and renamed from their original {@code *.tmp} versions. Presence of this file signals that no data
+     * will be lost if original partitions are deleted and batch rename process can be safely initiated.
+     *
+     * @param workDir Cache group working directory.
+     * @return File.
+     *
+     * @see DefragmentationFileUtils#writeDefragmentationCompletionMarker(FileIOFactory, File, IgniteLogger)
+     * @see DefragmentationFileUtils#batchRenameDefragmentedCacheGroupPartitions(File, IgniteLogger)
+     */
+    public static File defragmentationCompletionMarkerFile(File workDir) {
+        return new File(workDir, DFRG_COMPLETION_MARKER_FILE_NAME);
+    }
+
+    /**
+     * Creates empty completion marker file in given directory.
+     *
+     * @param ioFactory File IO factory.
+     * @param workDir Cache group working directory.
+     * @param log Logger to write messages.
+     * @throws IgniteCheckedException If {@link IOException} occurred.
+     *
+     * @see DefragmentationFileUtils#defragmentationCompletionMarkerFile(File)
+     */
+    public static void writeDefragmentationCompletionMarker(
+        FileIOFactory ioFactory,
+        File workDir,
+        IgniteLogger log
+    ) throws IgniteCheckedException {
+        File completionMarker = defragmentationCompletionMarkerFile(workDir);
+
+        try (FileIO io = ioFactory.create(completionMarker, CREATE_NEW, WRITE)) {
+            io.force(true);
+        }
+        catch (IOException e) {
+            handleIoException(e);
+        }
+    }
+
+    /** */
+    private static void handleIoException(IOException e) throws IgniteCheckedException {

Review comment:
       Done

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,896 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database schared manager. */

Review comment:
       Done




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

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



[GitHub] [ignite] agoncharuk commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
agoncharuk commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r524310015



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java
##########
@@ -1678,6 +1678,51 @@ public static String toString(String str,
         }
     }
 
+    /**
+     * Produces uniformed output of string with context properties
+     *
+     * @param str Output prefix or {@code null} if empty.
+     * @param triplets Triplets {@code {name, value, sencitivity}}.
+     * @return String presentation.
+     */
+    public static String toString(String str, Object... triplets) {
+        assert triplets.length % 3 == 0;

Review comment:
       Looks like this should be an exception




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r531443625



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/GridQueryIndexingDefragmentation.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.processors.cache.persistence.defragmentation;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.util.collection.IntMap;
+
+/**
+ *
+ */
+public interface GridQueryIndexingDefragmentation {

Review comment:
       Ok




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

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r529735384



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
##########
@@ -447,6 +472,32 @@ private DataRegionConfiguration createMetastoreDataRegionConfig(DataStorageConfi
         return cfg;
     }
 
+    /** */
+    private DataRegionConfiguration createDefragmentationDataRegionConfig(long regionSize) {

Review comment:
       What are the drawbacks of such an approach? 
   Will the region be deallocated when the defragmentation ends?




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r530199249



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/TreeIterator.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.nio.ByteBuffer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.util.GridUnsafe;
+
+/** */
+public class TreeIterator {
+    /** Direct memory buffer with a size of one page. */
+    private final ByteBuffer pageBuf;
+
+    /** Offheap page size. */
+    private final int pageSize;
+
+    /** */
+    public TreeIterator(int size) {
+        pageSize = size;
+
+        pageBuf = ByteBuffer.allocateDirect(pageSize);
+    }
+
+    // Performance impact of constant closures allocation is not clear. So this method should be avoided in massive

Review comment:
       In current usages performance impact is practically zero, that's why I use simpler code. This is just a warning that you shouldn't use this method for all pages, that would be a disaster.




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r530215326



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database shared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();

Review comment:
       Sharing busy lock was the only available option. I'll test later, semantics of node start procedure (during defragmentation) is changed in "control.sh API" issue, it would be better to introduce such test 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.

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r532411996



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/TreeIterator.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.nio.ByteBuffer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.util.GridUnsafe;
+
+/** */
+public class TreeIterator {
+    /** Direct memory buffer with a size of one page. */
+    private final ByteBuffer pageBuf;
+
+    /** Offheap page size. */
+    private final int pageSize;
+
+    /** */
+    public TreeIterator(int size) {
+        pageSize = size;
+
+        pageBuf = ByteBuffer.allocateDirect(pageSize);
+    }
+
+    // Performance impact of constant closures allocation is not clear. So this method should be avoided in massive
+    // operations like tree leaves access.
+    /** */
+    public static <T> T access(
+        PageAccessType access,
+        PageMemoryEx pageMemory,
+        int grpId,
+        long pageId,
+        PageAccessor<T> accessor
+    ) throws IgniteCheckedException {
+        assert access == PageAccessType.ACCESS_READ || access == PageAccessType.ACCESS_WRITE;
+        long page = pageMemory.acquirePage(grpId, pageId);
+
+        try {
+            long pageAddr = access == PageAccessType.ACCESS_READ
+                    ? pageMemory.readLock(grpId, pageId, page)
+                    : pageMemory.writeLock(grpId, pageId, page);
+
+            try {
+                return accessor.access(pageAddr);
+            }
+            finally {
+                if (access == PageAccessType.ACCESS_READ)
+                    pageMemory.readUnlock(grpId, pageId, page);
+                else
+                    pageMemory.writeUnlock(grpId, pageId, page, null, true);
+            }
+        }
+        finally {
+            pageMemory.releasePage(grpId, pageId, page);
+        }
+    }
+
+    /** */
+    @SuppressWarnings("PublicInnerClass")
+    public enum PageAccessType {
+        /** Read access. */
+        ACCESS_READ,
+
+        /** Write access. */
+        ACCESS_WRITE;
+    }
+
+    /** */
+    @SuppressWarnings("PublicInnerClass")
+    @FunctionalInterface
+    public interface PageAccessor<T> {
+        /** */
+        public T access(long pageAddr) throws IgniteCheckedException;
+    }
+
+    /** */
+    // TODO Prefetch future pages?

Review comment:
       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.

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r524997891



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,887 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database schared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {
+                if (!oldGrpCtx.userCache())
+                    continue;
+
+                int grpId = oldGrpCtx.groupId();
+
+                if (!cacheGroupsForDefragmentation.isEmpty() && !cacheGroupsForDefragmentation.contains(grpId))
+                    continue;
+
+                File workDir = filePageStoreMgr.cacheWorkDir(oldGrpCtx.sharedGroup(), oldGrpCtx.cacheOrGroupName());
+
+                if (skipAlreadyDefragmentedCacheGroup(workDir, grpId, log))
+                    continue;
+
+                GridCacheOffheapManager offheap = (GridCacheOffheapManager)oldGrpCtx.offheap();
+
+                GridSpinBusyLock busyLock = offheap.busyLock();
+
+                List<CacheDataStore> oldCacheDataStores = stream(offheap.cacheDataStores().spliterator(), false)
+                    .filter(store -> {
+                        try {
+                            return filePageStoreMgr.exists(grpId, store.partId());
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    })
+                    .collect(Collectors.toList());
+
+                if (workDir != null && !oldCacheDataStores.isEmpty()) {
+                    // We can't start defragmentation of new group on the region that has wrong eviction mode.
+                    // So waiting of the previous cache group defragmentation is inevitable.
+                    DataPageEvictionMode curPageEvictionMode = oldGrpCtx.dataRegion().config().getPageEvictionMode();
+
+                    if (prevPageEvictionMode == null || prevPageEvictionMode != curPageEvictionMode) {
+                        prevPageEvictionMode = curPageEvictionMode;
+
+                        partDataRegion.config().setPageEvictionMode(curPageEvictionMode);
+
+                        if (idxDfrgFut != null)
+                            idxDfrgFut.get();
+                    }
+
+                    IntMap<CacheDataStore> cacheDataStores = new IntHashMap<>();
+
+                    for (CacheDataStore store : offheap.cacheDataStores()) {
+                        // Tree can be null for not yet initialized partitions.
+                        // This would mean that these partitions are empty.
+                        assert store.tree() == null || store.tree().groupId() == grpId;
+
+                        if (store.tree() != null)
+                            cacheDataStores.put(store.partId(), store);
+                    }
+
+                    //TODO ensure that there are no races.
+                    dbMgr.checkpointedDataRegions().remove(oldGrpCtx.dataRegion());
+
+                    // Another cheat. Ttl cleanup manager knows too much shit.
+                    oldGrpCtx.caches().stream()
+                        .filter(cacheCtx -> cacheCtx.groupId() == grpId)
+                        .forEach(cacheCtx -> cacheCtx.ttl().unregister());
+
+                    // Technically wal is already disabled, but "PageHandler.isWalDeltaRecordNeeded" doesn't care and
+                    // WAL records will be allocated anyway just to be ignored later if we don't disable WAL for
+                    // cache group explicitly.
+                    oldGrpCtx.localWalEnabled(false, false);
+
+                    boolean encrypted = oldGrpCtx.config().isEncryptionEnabled();
+
+                    FilePageStoreFactory pageStoreFactory = filePageStoreMgr.getPageStoreFactory(grpId, encrypted);
+
+                    createIndexPageStore(grpId, workDir, pageStoreFactory, partDataRegion, val -> {
+                    }); //TODO Allocated tracker.
+
+                    GridCompoundFuture<Object, Object> cmpFut = new GridCompoundFuture<>();
+
+                    PageMemoryEx oldPageMem = (PageMemoryEx)oldGrpCtx.dataRegion().pageMemory();
+
+                    CacheGroupContext newGrpCtx = new CacheGroupContext(
+                        sharedCtx,
+                        grpId,
+                        oldGrpCtx.receivedFrom(),
+                        CacheType.USER,
+                        oldGrpCtx.config(),
+                        oldGrpCtx.affinityNode(),
+                        partDataRegion,
+                        oldGrpCtx.cacheObjectContext(),
+                        null,
+                        null,
+                        oldGrpCtx.localStartVersion(),
+                        true,
+                        false,
+                        true
+                    );
+
+                    defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadLock();
+
+                    try {
+                        // This will initialize partition meta in index partition - meta tree and reuse list.
+                        newGrpCtx.start();
+                    }
+                    finally {
+                        defragmentationCheckpoint.checkpointTimeoutLock().checkpointReadUnlock();
+                    }
+
+                    IntMap<LinkMap> linkMapByPart = new IntHashMap<>();
+
+                    for (CacheDataStore oldCacheDataStore : oldCacheDataStores) {
+                        int partId = oldCacheDataStore.partId();
+
+                        PartitionContext partCtx = new PartitionContext(
+                            workDir,
+                            grpId,
+                            partId,
+                            partDataRegion,
+                            mappingDataRegion,
+                            oldGrpCtx,
+                            newGrpCtx,
+                            cacheDataStores.get(partId),
+                            pageStoreFactory
+                        );
+
+                        if (skipAlreadyDefragmentedPartition(workDir, grpId, partId, log)) {
+                            partCtx.createMappingPageStore();
+
+                            linkMapByPart.put(partId, partCtx.createLinkMapTree(false));
+
+                            continue;
+                        }
+
+                        partCtx.createMappingPageStore();
+
+                        linkMapByPart.put(partId, partCtx.createLinkMapTree(true));
+
+                        partCtx.createPartPageStore();
+
+                        copyPartitionData(partCtx, treeIter, busyLock);
+
+                        //TODO Move inside of defragmentSinglePartition.
+                        IgniteInClosure<IgniteInternalFuture<?>> cpLsnr = fut -> {
+                            if (fut.error() == null) {
+                                PageStore oldPageStore = null;
+
+                                try {
+                                    oldPageStore = filePageStoreMgr.getStore(grpId, partId);
+                                }
+                                catch (IgniteCheckedException ignore) {
+                                }
+
+                                assert oldPageStore != null;
+
+                                log.info(S.toString(
+                                    "Partition defragmented",
+                                    "grpId", grpId, false,
+                                    "partId", partId, false,
+                                    "oldPages", oldPageStore.pages(), false,
+                                    "newPages", partCtx.partPagesAllocated.get(), false,
+                                    "bytesSaved", (oldPageStore.pages() - partCtx.partPagesAllocated.get()) * pageSize, false,
+                                    "mappingPages", partCtx.mappingPagesAllocated.get(), false,
+                                    "partFile", defragmentedPartFile(workDir, partId).getName(), false,
+                                    "workDir", workDir, false
+                                ));
+
+                                oldPageMem.invalidate(grpId, partId);
+
+                                partCtx.partPageMemory.invalidate(grpId, partId);
+
+                                DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partCtx.partPageMemory.pageManager();
+
+                                pageMgr.pageStoreMap().removePageStore(grpId, partId); // Yes, it'll be invalid in a second.
+
+                                try {
+                                    renameTempPartitionFile(workDir, partId);
+                                }
+                                catch (IgniteCheckedException e) {
+                                    throw new IgniteException(e);
+                                }
+                            }
+                        };
+
+                        GridFutureAdapter<?> cpFut = defragmentationCheckpoint
+                            .forceCheckpoint("partition defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+
+                        cpFut.listen(cpLsnr);
+
+                        cmpFut.add((IgniteInternalFuture<Object>)cpFut);
+                    }
+
+                    // A bit too general for now, but I like it more then saving only the last checkpoint future.
+                    cmpFut.markInitialized().get();
+
+                    idxDfrgFut = new GridFinishedFuture<>();
+
+                    if (filePageStoreMgr.hasIndexStore(grpId)) {
+                        defragmentIndexPartition(oldGrpCtx, newGrpCtx, linkMapByPart);
+
+                        idxDfrgFut = defragmentationCheckpoint
+                            .forceCheckpoint("index defragmented", null)
+                            .futureFor(CheckpointState.FINISHED);
+                    }
+
+                    idxDfrgFut.listen(fut -> {
+                        oldPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx partPageMem = (PageMemoryEx)partDataRegion.pageMemory();
+
+                        partPageMem.invalidate(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        DefragmentationPageReadWriteManager pageMgr = (DefragmentationPageReadWriteManager)partPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().removePageStore(grpId, PageIdAllocator.INDEX_PARTITION);
+
+                        PageMemoryEx mappingPageMem = (PageMemoryEx)mappingDataRegion.pageMemory();
+
+                        pageMgr = (DefragmentationPageReadWriteManager)mappingPageMem.pageManager();
+
+                        pageMgr.pageStoreMap().clear(grpId);
+
+                        try {
+                            renameTempIndexFile(workDir);
+
+                            writeDefragmentationCompletionMarker(filePageStoreMgr.getPageStoreFileIoFactory(), workDir, log);
+
+                            batchRenameDefragmentedCacheGroupPartitions(workDir, log);
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IgniteException(e);
+                        }
+                    });
+                }
+
+                // I guess we should wait for it?
+                if (idxDfrgFut != null)
+                    idxDfrgFut.get();
+            }
+
+            mntcReg.unregisterMaintenanceTask(DEFRAGMENTATION_MNTC_TASK_NAME);
+
+            log.info("Defragmentation completed. All partitions are defragmented.");

Review comment:
       I'd make this change in issue where I implement control.sh "status" command, it has some aggregated statistics implemented. Current log will be moved to 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.

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



[GitHub] [ignite] agoncharuk commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
agoncharuk commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r524294661



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/defragmentation/IndexingDefragmentation.java
##########
@@ -0,0 +1,454 @@
+/*
+ * 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.processors.query.h2.defragmentation;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.GridQueryIndexingDefragmentation;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.LinkMap;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.TimeTracker;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIoResolver;
+import org.apache.ignite.internal.processors.cache.persistence.tree.util.InsertLast;
+import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccDataRow;
+import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+import org.apache.ignite.internal.processors.query.h2.database.H2Tree;
+import org.apache.ignite.internal.processors.query.h2.database.H2TreeIndex;
+import org.apache.ignite.internal.processors.query.h2.database.InlineIndexColumn;
+import org.apache.ignite.internal.processors.query.h2.database.inlinecolumn.AbstractInlineIndexColumn;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2ExtrasInnerIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2ExtrasLeafIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2InnerIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.AbstractH2LeafIO;
+import org.apache.ignite.internal.processors.query.h2.database.io.H2RowLinkIO;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.processors.query.h2.opt.H2CacheRow;
+import org.apache.ignite.internal.processors.query.h2.opt.H2Row;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.h2.index.Index;
+import org.h2.value.Value;
+
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.INIT_TREE;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.ITERATE;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.READ_MAP;
+import static org.apache.ignite.internal.processors.query.h2.defragmentation.IndexingDefragmentation.IndexStages.READ_ROW;
+
+/**
+ *
+ */
+public class IndexingDefragmentation implements GridQueryIndexingDefragmentation {
+    /** Indexing. */
+    private final IgniteH2Indexing indexing;
+
+    /** Constructor. */
+    public IndexingDefragmentation(IgniteH2Indexing indexing) {
+        this.indexing = indexing;
+    }
+
+    /** */
+    public enum IndexStages {
+        START,
+        CP_LOCK,
+        INIT_TREE,
+        ITERATE,
+        READ_ROW,
+        READ_MAP,
+        INSERT_ROW
+    }
+
+    /** {@inheritDoc} */
+    @Override public void defragmentate(
+        CacheGroupContext grpCtx,
+        CacheGroupContext newCtx,
+        PageMemoryEx partPageMem,
+        IntMap<LinkMap> mappingByPartition,
+        CheckpointTimeoutLock cpLock,
+        IgniteLogger log
+    ) throws IgniteCheckedException {
+        int pageSize = grpCtx.cacheObjectContext().kernalContext().grid().configuration().getDataStorageConfiguration().getPageSize();
+
+        TreeIterator treeIterator = new TreeIterator(pageSize);
+
+        PageMemoryEx oldCachePageMem = (PageMemoryEx)grpCtx.dataRegion().pageMemory();
+
+        PageMemory newCachePageMemory = partPageMem;
+
+        Collection<GridH2Table> tables = indexing.schemaManager().dataTables();
+
+        long cpLockThreshold = 250L;
+
+        TimeTracker<IndexStages> tracker = new TimeTracker<>(IndexStages.class);
+
+        cpLock.checkpointReadLock();
+        tracker.complete(CP_LOCK);
+
+        try {
+            AtomicLong lastCpLockTs = new AtomicLong(System.currentTimeMillis());
+
+            for (GridH2Table table : tables) {
+                GridCacheContext<?, ?> cctx = table.cacheContext();
+
+                if (cctx.groupId() != grpCtx.groupId())
+                    continue; // Not our index.
+
+                GridH2RowDescriptor rowDesc = table.rowDescriptor();
+
+                List<Index> indexes = table.getIndexes();
+                H2TreeIndex oldH2Idx = (H2TreeIndex)indexes.get(2);
+
+                int segments = oldH2Idx.segmentsCount();
+
+                H2Tree firstTree = oldH2Idx.treeForRead(0);
+
+                PageIoResolver pageIoRslvr = pageAddr -> {
+                    PageIO io = PageIoResolver.DEFAULT_PAGE_IO_RESOLVER.resolve(pageAddr);
+
+                    if (io instanceof BPlusMetaIO)
+                        return io;
+
+                    //noinspection unchecked,rawtypes,rawtypes
+                    return wrap((BPlusIO)io);
+                };
+
+                //TODO Create new proper GridCacheContext for it?
+                H2TreeIndex newIdx = H2TreeIndex.createIndex(
+                    cctx,
+                    null,
+                    table,
+                    oldH2Idx.getName(),
+                    firstTree.getPk(),
+                    firstTree.getAffinityKey(),
+                    Arrays.asList(firstTree.cols()),
+                    Arrays.asList(firstTree.cols()),
+                    oldH2Idx.inlineSize(),
+                    segments,
+                    newCachePageMemory,
+                    newCtx.offheap(),
+                    pageIoRslvr,
+                    log
+                );
+
+                tracker.complete(INIT_TREE);
+
+                for (int i = 0; i < segments; i++) {
+                    H2Tree tree = oldH2Idx.treeForRead(i);
+
+                    treeIterator.iterate(tree, oldCachePageMem, (theTree, io, pageAddr, idx) -> {
+                        tracker.complete(ITERATE);
+
+                        if (System.currentTimeMillis() - lastCpLockTs.get() >= cpLockThreshold) {
+                            cpLock.checkpointReadUnlock();
+
+                            cpLock.checkpointReadLock();
+                            tracker.complete(CP_LOCK);
+
+                            lastCpLockTs.set(System.currentTimeMillis());
+                        }
+
+                        assert 1 == io.getVersion();
+
+                        BPlusIO<H2Row> h2IO = wrap(io);
+
+                        H2Row row = theTree.getRow(h2IO, pageAddr, idx);
+
+                        tracker.complete(READ_ROW);
+
+                        if (row instanceof H2CacheRowWithIndex) {
+                            H2CacheRowWithIndex h2CacheRow = (H2CacheRowWithIndex)row;
+
+                            CacheDataRow cacheDataRow = h2CacheRow.getRow();
+
+                            int partition = cacheDataRow.partition();
+
+                            long link = h2CacheRow.link();
+
+                            LinkMap map = mappingByPartition.get(partition);
+
+                            long newLink = map.get(link);
+
+                            tracker.complete(READ_MAP);
+
+                            H2CacheRowWithIndex newRow = H2CacheRowWithIndex.create(
+                                rowDesc,
+                                newLink,
+                                h2CacheRow,
+                                ((H2RowLinkIO)io).storeMvccInfo()
+                            );
+
+                            newIdx.putx(newRow);
+
+                            tracker.complete(INSERT_ROW);
+                        }
+
+                        return true;
+                    });
+                }
+            }
+        }
+        finally {
+            cpLock.checkpointReadUnlock();
+        }
+
+        System.out.println(tracker.toString());

Review comment:
       A left out println?




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r525125608



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/CachePartitionDefragmentationManager.java
##########
@@ -0,0 +1,887 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.LongConsumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager.CacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CheckpointState;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager.GridCacheDataStore;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.LightweightCheckpointManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.SimpleDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV3;
+import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
+import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
+import org.apache.ignite.internal.processors.cache.tree.DataRow;
+import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
+import org.apache.ignite.internal.processors.cache.tree.PendingRow;
+import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
+import org.apache.ignite.internal.util.collection.IntHashMap;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+
+import static java.util.stream.StreamSupport.stream;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
+import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_MAPPING_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DEFRAGMENTATION_PART_REGION_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.CP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.INSERT_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.ITERATE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.METADATA;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.READ_ROW;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_MAP;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PENDING;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.CachePartitionDefragmentationManager.PartStages.STORE_PK;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.batchRenameDefragmentedCacheGroupPartitions;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedIndexTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartMappingFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.defragmentedPartTmpFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempIndexFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.renameTempPartitionFile;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedCacheGroup;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.skipAlreadyDefragmentedPartition;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.DefragmentationFileUtils.writeDefragmentationCompletionMarker;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_READ;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.PageAccessType.ACCESS_WRITE;
+import static org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator.access;
+
+/**
+ * Defragmentation manager is the core class that contains main defragmentation procedure.
+ */
+public class CachePartitionDefragmentationManager {
+    /** */
+    public static final String DEFRAGMENTATION_MNTC_TASK_NAME = "defragmentationMaintenanceTask";
+
+    /** */
+    private final Set<Integer> cacheGroupsForDefragmentation;
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> sharedCtx;
+
+    /** Maintenance registry. */
+    private final MaintenanceRegistry mntcReg;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Database schared manager. */
+    private final GridCacheDatabaseSharedManager dbMgr;
+
+    /** File page store manager. */
+    private final FilePageStoreManager filePageStoreMgr;
+
+    /**
+     * Checkpoint for specific defragmentation regions which would store the data to new partitions
+     * during the defragmentation.
+     */
+    private final LightweightCheckpointManager defragmentationCheckpoint;
+
+    /** Default checkpoint for current node. */
+    private final CheckpointManager nodeCheckpoint;
+
+    /** Page size. */
+    private final int pageSize;
+
+    /** */
+    private final DataRegion partDataRegion;
+
+    /** */
+    private final DataRegion mappingDataRegion;
+
+    /**
+     * @param cacheGrpIds
+     * @param sharedCtx Cache shared context.
+     * @param dbMgr Database manager.
+     * @param filePageStoreMgr File page store manager.
+     * @param nodeCheckpoint Default checkpoint for this node.
+     * @param defragmentationCheckpoint Specific checkpoint for defragmentation.
+     * @param pageSize Page size.
+     */
+    public CachePartitionDefragmentationManager(
+        List<Integer> cacheGrpIds,
+        GridCacheSharedContext<?, ?> sharedCtx,
+        GridCacheDatabaseSharedManager dbMgr,
+        FilePageStoreManager filePageStoreMgr,
+        CheckpointManager nodeCheckpoint,
+        LightweightCheckpointManager defragmentationCheckpoint,
+        int pageSize
+    ) throws IgniteCheckedException {
+        cacheGroupsForDefragmentation = new HashSet<>(cacheGrpIds);
+
+        this.dbMgr = dbMgr;
+        this.filePageStoreMgr = filePageStoreMgr;
+        this.pageSize = pageSize;
+        this.sharedCtx = sharedCtx;
+
+        this.mntcReg = sharedCtx.kernalContext().maintenanceRegistry();
+        this.log = sharedCtx.logger(getClass());
+        this.defragmentationCheckpoint = defragmentationCheckpoint;
+        this.nodeCheckpoint = nodeCheckpoint;
+
+        partDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_PART_REGION_NAME);
+        mappingDataRegion = dbMgr.dataRegion(DEFRAGMENTATION_MAPPING_REGION_NAME);
+    }
+
+    /** */
+    //TODO How will we handle constant fail and restart scenario?
+    public void executeDefragmentation() throws IgniteCheckedException {
+        log.info("Defragmentation started.");
+
+        try {
+            // Checkpointer must be enabled so all pages on disk are in their latest valid state.
+            dbMgr.resumeWalLogging();
+
+            dbMgr.onStateRestored(null);
+
+            nodeCheckpoint.forceCheckpoint("beforeDefragmentation", null).futureFor(FINISHED).get();
+
+            sharedCtx.wal().onDeActivate(sharedCtx.kernalContext());
+
+            // Now the actual process starts.
+            TreeIterator treeIter = new TreeIterator(pageSize);
+
+            IgniteInternalFuture<?> idxDfrgFut = null;
+            DataPageEvictionMode prevPageEvictionMode = null;
+
+            for (CacheGroupContext oldGrpCtx : sharedCtx.cache().cacheGroups()) {

Review comment:
       Yeah, I misunderstood your comment at first. You're right, we may have benefits on HDDs if everything fits in memory, that's for sure. But should we be bothered by 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.

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



[GitHub] [ignite] asfgit closed pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #7984:
URL: https://github.com/apache/ignite/pull/7984


   


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

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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r525082853



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/TimeTracker.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.util.Arrays;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/** */
+public class TimeTracker<Stage extends Enum<Stage>> {

Review comment:
       Is this the same as `TimeBag`?




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r530992273



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/TimeTracker.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.util.Arrays;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/** */
+public class TimeTracker<Stage extends Enum<Stage>> {

Review comment:
       1. How can I use custom aggregating function for TimeBag? I don't see anything like that in the code. It all is very convoluted, TimeBag is designed for processes that have no repeating stages.
   2. I don't want to have 100 thousand entries in log to aggregate them later, it's too much. And current class aggregates time as effective as it's possible, pretty much. It doesn't use lists and maps and all these heavy structures.




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

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r532425010



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
##########
@@ -487,4 +493,25 @@ default long indexSize(String schemaName, String tblName, String idxName) throws
     default Map<String, Integer> secondaryIndexesInlineSize() {
         return Collections.emptyMap();
     }
+
+    /**
+     * Defragment index partition.
+     *
+     * @param grpCtx Old group context.
+     * @param newCtx New group context.
+     * @param partPageMem Partition page memory.
+     * @param mappingByPart Mapping page memory.
+     * @param cpLock Defragmentation checkpoint read lock.
+     * @param log Log.
+     *
+     * @throws IgniteCheckedException If failed.
+     */
+    void defragment(
+        CacheGroupContext grpCtx,
+        CacheGroupContext newCtx,
+        PageMemoryEx partPageMem,
+        IntMap<LinkMap> mappingByPart,
+        CheckpointTimeoutLock cpLock,

Review comment:
       I agree about log, it'll be removed. On the other hand - this is not the same checkpoint lock that you think, it's lock from specific defragmentation checkpointer. I'm not removing 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.

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



[GitHub] [ignite] ibessonov commented on a change in pull request #7984: IGNITE-13190

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #7984:
URL: https://github.com/apache/ignite/pull/7984#discussion_r533388229



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/defragmentation/TimeTracker.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.processors.cache.persistence.defragmentation;
+
+import java.util.Arrays;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/** */
+public class TimeTracker<Stage extends Enum<Stage>> {

Review comment:
       Ok, I still don't get why you want this code being removed so badly. I'm removing 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.

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