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/06/17 17:51:11 UTC

[GitHub] [ignite] xtern opened a new pull request #7941: IGNITE-12843

xtern opened a new pull request #7941:
URL: https://github.com/apache/ignite/pull/7941


   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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/encryption/CacheGroupKeyChangeTest.java
##########
@@ -0,0 +1,988 @@
+/*
+ * 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.encryption;
+
+import java.io.File;
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.IgniteException;
+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.configuration.WALMode;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
+import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
+import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
+import org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType;
+import org.apache.ignite.internal.util.distributed.InitMessage;
+import org.apache.ignite.internal.util.distributed.SingleNodeMessage;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.spi.discovery.tcp.TestTcpDiscoverySpi;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.GridTestUtils.DiscoveryHook;
+import org.junit.Test;
+
+import static org.apache.ignite.configuration.WALMode.LOG_ONLY;
+import static org.apache.ignite.configuration.WALMode.NONE;
+import static org.apache.ignite.internal.managers.encryption.GridEncryptionManager.INITIAL_KEY_ID;
+import static org.apache.ignite.spi.encryption.keystore.KeystoreEncryptionSpi.DEFAULT_MASTER_KEY_NAME;
+import static org.apache.ignite.testframework.GridTestUtils.assertThrowsAnyCause;
+import static org.apache.ignite.testframework.GridTestUtils.assertThrowsWithCause;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+import static org.apache.ignite.testframework.GridTestUtils.waitForCondition;
+
+/**
+ * Cache group key change distributed process tests.
+ */
+public class CacheGroupKeyChangeTest extends AbstractEncryptionTest {
+    /** Timeout. */
+    private static final long MAX_AWAIT_MILLIS = 15_000;
+
+    /** */
+    private static final String GRID_2 = "grid-2";
+
+    /** Discovery hook for distributed process. */
+    private InitMessageDiscoveryHook discoveryHook;
+
+    /** Count of cache backups. */
+    private int backups;
+
+    /** WAL mode. */
+    private WALMode walMode = LOG_ONLY;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        if (discoveryHook != null)
+            ((TestTcpDiscoverySpi)cfg.getDiscoverySpi()).discoveryHook(discoveryHook);
+
+        DataStorageConfiguration memCfg = new DataStorageConfiguration()
+            .setDefaultDataRegionConfiguration(
+                new DataRegionConfiguration()
+                    .setMaxSize(100L * 1024 * 1024)
+                    .setPersistenceEnabled(true))
+            .setPageSize(4 * 1024)
+            .setWalSegmentSize(1024 * 1024)
+            .setWalSegments(10)
+            .setMaxWalArchiveSize(20 * 1024 * 1024)
+            .setCheckpointFrequency(30 * 1000L)
+            .setWalMode(walMode);
+
+        cfg.setDataStorageConfiguration(memCfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected <K, V> CacheConfiguration<K, V> cacheConfiguration(String name, String grp) {
+        CacheConfiguration<K, V> cfg = super.cacheConfiguration(name, grp);
+
+        return cfg.setAffinity(new RendezvousAffinityFunction(false, 8)).setBackups(backups);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    @SuppressWarnings("ThrowableNotThrown")
+    public void testRejectNodeJoinDuringRotation() throws Exception {
+        T2<IgniteEx, IgniteEx> grids = startTestGrids(true);
+
+        createEncryptedCache(grids.get1(), grids.get2(), cacheName(), null);
+
+        int grpId = CU.cacheId(cacheName());
+
+        assertEquals(0, grids.get1().context().encryption().groupKey(grpId).id());
+
+        TestRecordingCommunicationSpi commSpi = TestRecordingCommunicationSpi.spi(grids.get2());
+
+        commSpi.blockMessages((node, msg) -> msg instanceof SingleNodeMessage);
+
+        IgniteFuture<Void> fut = grids.get1().encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+
+        commSpi.waitForBlocked();
+
+        assertThrowsWithCause(() -> startGrid(3), IgniteCheckedException.class);
+
+        commSpi.stopBlock();
+
+        fut.get();
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        checkEncryptedCaches(grids.get1(), grids.get2());
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNotAllBltNodesPresent() throws Exception {
+        startTestGrids(true);
+
+        createEncryptedCache(grid(GRID_0), grid(GRID_1), cacheName(), null);
+
+        stopGrid(GRID_1);
+
+        grid(GRID_0).encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get();
+
+        startGrid(GRID_1);
+
+        checkGroupKey(CU.cacheId(cacheName()), INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNodeFailsBeforePrepare() throws Exception {
+        checkNodeFailsDuringRotation(false, true, true);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNodeFailsBeforePerform() throws Exception {
+        checkNodeFailsDuringRotation(false, false, true);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNodeFailsAfterPrepare() throws Exception {
+        checkNodeFailsDuringRotation(false, true, false);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testCrdFailsAfterPrepare() throws Exception {
+        checkNodeFailsDuringRotation(true, true, false);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNodeFailsAfterPerform() throws Exception {
+        checkNodeFailsDuringRotation(false, false, false);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testCrdFailsAfterPerform() throws Exception {
+        checkNodeFailsDuringRotation(true, false, false);
+    }
+
+    /**
+     * @param stopCrd {@code True} to stop coordinator.
+     * @param prepare {@code True} to stop on the prepare phase. {@code False} to stop on the perform phase.
+     * @param discoBlock  {@code True} to block discovery, {@code False} to block communication SPI.
+     */
+    private void checkNodeFailsDuringRotation(boolean stopCrd, boolean prepare, boolean discoBlock) throws Exception {
+        cleanPersistenceDir();
+
+        DistributedProcessType type = prepare ?
+            DistributedProcessType.CACHE_GROUP_KEY_CHANGE_PREPARE : DistributedProcessType.CACHE_GROUP_KEY_CHANGE_FINISH;
+
+        InitMessageDiscoveryHook locHook = new InitMessageDiscoveryHook(type);
+
+        if (discoBlock && stopCrd)
+            discoveryHook = locHook;
+
+        IgniteEx grid0 = startGrid(GRID_0);
+
+        if (discoBlock && !stopCrd)
+            discoveryHook = locHook;
+
+        IgniteEx grid1 = startGrid(GRID_1);
+
+        grid0.cluster().state(ClusterState.ACTIVE);
+
+        createEncryptedCache(grid0, grid1, cacheName(), null);
+
+        int grpId = CU.cacheId(cacheName());
+
+        checkGroupKey(grpId, INITIAL_KEY_ID, MAX_AWAIT_MILLIS);
+
+        TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid1);
+
+        if (!discoBlock) {
+            AtomicBoolean preparePhase = new AtomicBoolean(true);
+
+            spi.blockMessages((node, msg) -> {
+                if (msg instanceof SingleNodeMessage) {
+                    boolean isPrepare = preparePhase.compareAndSet(true, false);
+
+                    return prepare || !isPrepare;
+                }
+
+                return false;
+            });
+        }
+
+        String alive = stopCrd ? GRID_1 : GRID_0;
+        String stopped = stopCrd ? GRID_0 : GRID_1;
+
+        IgniteFuture<Void> changeFut = grid(alive).encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+
+        IgniteInternalFuture<?> stopFut = new GridFinishedFuture<>();
+
+        if (discoBlock) {
+            locHook.waitForBlocked(MAX_AWAIT_MILLIS);
+
+            stopGrid(stopped, true);
+
+            locHook.stopBlock();
+        }
+        else {
+            spi.waitForBlocked();
+
+            stopFut = runAsync(() -> stopGrid(stopped, true));
+        }
+
+        changeFut.get(MAX_AWAIT_MILLIS);
+        stopFut.get(MAX_AWAIT_MILLIS);
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        IgniteEx stoppedNode = startGrid(stopped);
+
+        stoppedNode.resetLostPartitions(Collections.singleton(ENCRYPTED_CACHE));
+
+        awaitPartitionMapExchange();
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        stoppedNode.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get(MAX_AWAIT_MILLIS);
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 2, MAX_AWAIT_MILLIS);
+    }
+
+    /**
+     * Ensures that we can rotate the key more than 255 times.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testKeyIdentifierOverflow() throws Exception {
+        IgniteEx node = startTestGrids(true).get1();
+
+        createEncryptedCache(node, null, cacheName(), null, false);
+
+        int grpId = CU.cacheId(cacheName());
+
+        byte keyId = INITIAL_KEY_ID;
+
+        do {
+            node.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get();
+
+            // Validates reencryption of index partition.
+            checkGroupKey(grpId, ++keyId & 0xff, MAX_AWAIT_MILLIS);
+        } while (keyId != INITIAL_KEY_ID);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testMasterAndCacheGroupKeySimultaneousChange() throws Exception {
+        startTestGrids(true);
+
+        IgniteEx node0 = grid(GRID_0);
+        IgniteEx node1 = grid(GRID_1);
+
+        createEncryptedCache(node0, node1, cacheName(), null);
+
+        int grpId = CU.cacheId(cacheName());
+
+        assertTrue(checkMasterKeyName(DEFAULT_MASTER_KEY_NAME));
+
+        Random rnd = ThreadLocalRandom.current();
+
+        for (byte keyId = 1; keyId < 50; keyId++) {
+            String currMkName = node0.context().config().getEncryptionSpi().getMasterKeyName();
+            String newMkName = currMkName.equals(MASTER_KEY_NAME_2) ? MASTER_KEY_NAME_3 : MASTER_KEY_NAME_2;
+
+            boolean changeGrpFirst = rnd.nextBoolean();
+
+            IgniteFuture<Void> grpKeyFut;
+            IgniteFuture<Void> masterKeyFut;
+
+            if (changeGrpFirst) {
+                grpKeyFut = node0.encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+                masterKeyFut = node0.encryption().changeMasterKey(newMkName);
+            }
+            else {
+                masterKeyFut = node0.encryption().changeMasterKey(newMkName);
+                grpKeyFut = node0.encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+            }
+
+            masterKeyFut.get(MAX_AWAIT_MILLIS);
+            assertTrue(checkMasterKeyName(newMkName));
+
+            try {
+                grpKeyFut.get(MAX_AWAIT_MILLIS);
+                checkGroupKey(grpId, keyId, MAX_AWAIT_MILLIS);
+            } catch (IgniteException e) {
+                assertTrue(e.getMessage().contains("Cache group key change was rejected. Master key has been changed."));
+
+                keyId -= 1;
+            }
+        }
+    }
+
+    /**
+     * Ensures that after rotation, the node has correct key identifier.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testNodeJoinAfterChange() throws Exception {
+        startTestGrids(true);
+
+        IgniteEx node0 = grid(GRID_0);
+        IgniteEx node1 = grid(GRID_1);
+
+        createEncryptedCache(node0, node1, cacheName(), null);
+
+        node0.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get();
+
+        startGrid(GRID_2);
+
+        resetBaselineTopology();
+
+        awaitPartitionMapExchange();
+
+        checkGroupKey(CU.cacheId(cacheName()), INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testCacheStartDuringRotation() throws Exception {
+        T2<IgniteEx, IgniteEx> grids = startTestGrids(true);
+
+        createEncryptedCache(grids.get1(), grids.get2(), cacheName(), null);
+
+        TestRecordingCommunicationSpi commSpi = TestRecordingCommunicationSpi.spi(grids.get2());
+
+        commSpi.blockMessages((node, msg) -> msg instanceof SingleNodeMessage);
+
+        IgniteFuture<Void> fut = grids.get1().encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+
+        commSpi.waitForBlocked();
+
+        IgniteCache<Integer, Integer> cache = grids.get1().createCache(cacheConfiguration("cache1", null));
+
+        for (int i = 0; i < 100; i++)
+            cache.put(i, i);
+
+        commSpi.stopBlock();
+
+        fut.get();
+
+        checkGroupKey(CU.cacheId(cacheName()), INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        checkGroupKey(CU.cacheId("cache1"), INITIAL_KEY_ID, MAX_AWAIT_MILLIS);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testCacheStartSameGroupDuringRotation() throws Exception {
+        T2<IgniteEx, IgniteEx> grids = startTestGrids(true);
+
+        String grpName = "shared";
+
+        createEncryptedCache(grids.get1(), grids.get2(), cacheName(), grpName);
+
+        TestRecordingCommunicationSpi commSpi = TestRecordingCommunicationSpi.spi(grids.get2());
+
+        commSpi.blockMessages((node, msg) -> msg instanceof SingleNodeMessage);
+
+        IgniteFuture<Void> fut = grids.get1().encryption().changeCacheGroupKey(Collections.singleton(grpName));
+
+        commSpi.waitForBlocked();
+
+        IgniteCache<Integer, Integer> cache =
+            grids.get1().createCache(cacheConfiguration("cache1", grpName));
+
+        commSpi.stopBlock();
+
+        for (int i = 0; i < 100; i++)
+            cache.put(i, i);
+
+        fut.get();
+
+        checkGroupKey(CU.cacheId(grpName), INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testChangeKeyDuringRebalancing() throws Exception {
+        T2<IgniteEx, IgniteEx> grids = startTestGrids(true);
+
+        IgniteEx node0 = grids.get1();
+        IgniteEx node1 = grids.get2();
+
+        createEncryptedCache(node0, node1, cacheName(), null);
+
+        loadData(500_000);
+
+        IgniteEx node2 = startGrid(GRID_2);
+
+        resetBaselineTopology();
+
+        int grpId = CU.cacheId(cacheName());
+
+        IgniteFuture<Void> fut = node2.encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+
+        fut.get(MAX_AWAIT_MILLIS);
+
+        stopAllGrids();
+
+        startGrid(GRID_0);
+        startGrid(GRID_1);
+        startGrid(GRID_2);
+
+        grid(GRID_0).cluster().state(ClusterState.ACTIVE);
+
+        awaitPartitionMapExchange();
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testBasicChange() throws Exception {
+        startTestGrids(true);
+
+        IgniteEx node1 = grid(GRID_0);
+        IgniteEx node2 = grid(GRID_1);
+
+        createEncryptedCache(node1, node2, cacheName(), null);
+
+        forceCheckpoint();
+
+        IgniteInternalCache<Object, Object> cache = node1.cachex(cacheName());
+
+        int grpId = cache.context().groupId();
+
+        node1.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get();
+
+        Set<Integer> keys1 = new TreeSet<>(node1.context().encryption().groupKeyIds(grpId));
+        Set<Integer> keys2 = new TreeSet<>(node2.context().encryption().groupKeyIds(grpId));
+
+        assertEquals(2, keys1.size());
+
+        assertEquals(keys1, keys2);
+
+        info("New key was set on all nodes [grpId=" + grpId + ", keys=" + keys1 + "]");
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        stopAllGrids();
+
+        node1 = startGrid(GRID_0);
+        node2 = startGrid(GRID_1);
+
+        node1.cluster().state(ClusterState.ACTIVE);
+
+        // Previous leys must be deleted when the corresponding WAL segment is deleted.
+        try (IgniteDataStreamer<Integer, String> streamer = node1.dataStreamer(cacheName())) {
+            for (int i = node1.cache(cacheName()).size(); i < 500_000; i++) {
+                streamer.addData(i, String.valueOf(i));
+
+                if (i % 1_000 == 0 &&
+                    node1.context().encryption().groupKeyIds(grpId).size() == 1 &&
+                    node2.context().encryption().groupKeyIds(grpId).size() == 1)
+                    break;
+            }
+        }
+
+        assertEquals(1, node1.context().encryption().groupKeyIds(grpId).size());
+        assertEquals(1, node2.context().encryption().groupKeyIds(grpId).size());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testBasicChangeWithConstantLoad() throws Exception {
+        startTestGrids(true);
+
+        IgniteEx node0 = grid(GRID_0);
+        IgniteEx node1 = grid(GRID_1);
+
+        createEncryptedCache(node0, node1, cacheName(), null);
+
+        forceCheckpoint();
+
+        IgniteInternalCache<Object, Object> cache = node0.cachex(cacheName());
+
+        AtomicInteger cntr = new AtomicInteger(cache.size());
+
+        CountDownLatch startLatch = new CountDownLatch(1);
+
+        final Ignite somenode = node0;
+
+        IgniteInternalFuture<?> loadFut = GridTestUtils.runAsync(() -> {
+            try (IgniteDataStreamer<Integer, String> streamer = somenode.dataStreamer(cacheName())) {
+                while (!Thread.currentThread().isInterrupted()) {
+                    int n = cntr.getAndIncrement();
+
+                    streamer.addData(n, String.valueOf(n));
+
+                    if (n == 5000)
+                        startLatch.countDown();
+                }
+            }
+        });
+
+        startLatch.await(MAX_AWAIT_MILLIS, TimeUnit.MILLISECONDS);
+
+        int grpId = cache.context().groupId();
+
+        node0.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get(MAX_AWAIT_MILLIS);
+
+        Set<Integer> keys1 = new TreeSet<>(node0.context().encryption().groupKeyIds(grpId));
+        Set<Integer> keys2 = new TreeSet<>(node1.context().encryption().groupKeyIds(grpId));
+
+        assertEquals(keys1, keys2);
+
+        awaitEncryption(G.allGrids(), grpId, MAX_AWAIT_MILLIS);
+
+        forceCheckpoint();
+
+        loadFut.cancel();
+
+        // Ensure that data is encrypted with the new key.
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        stopAllGrids();
+
+        node0 = startGrid(GRID_0);
+        node1 = startGrid(GRID_1);
+
+        node0.cluster().state(ClusterState.ACTIVE);
+
+        // Wait for WAL segment remove.
+        try (IgniteDataStreamer<Integer, String> streamer = node0.dataStreamer(cacheName())) {
+            int start = cntr.get();
+
+            for (; ; ) {
+                int n = cntr.getAndIncrement();
+
+                streamer.addData(n, String.valueOf(n));
+
+                if (n % 1000 == 0 &&
+                    node0.context().encryption().groupKeyIds(grpId).size() == 1 &&
+                    node1.context().encryption().groupKeyIds(grpId).size() == 1)
+                    break;
+
+                if (n - start == 1_000_000)
+                    break;
+            }
+        }
+
+        assertEquals(1, node0.context().encryption().groupKeyIds(grpId).size());
+        assertEquals(1, node1.context().encryption().groupKeyIds(grpId).size());
+    }
+
+    /**
+     * Ensures that unused key will be removed even if user cleaned wal archive folder manually.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testWalArchiveCleanup() throws Exception {
+        startTestGrids(true);
+
+        IgniteEx node1 = grid(GRID_0);
+        IgniteEx node2 = grid(GRID_1);
+
+        createEncryptedCache(node1, node2, cacheName(), null);
+
+        node1.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get();
+
+        long walIdx = node1.context().cache().context().wal().currentSegment();
+
+        AtomicBoolean stopLoad = new AtomicBoolean();
+
+        IgniteInternalFuture<?> fut = runAsync(() -> {
+            Ignite grid = grid(GRID_0);
+
+            long cntr = grid.cache(cacheName()).size();
+
+            try (IgniteDataStreamer<Long, String> streamer = grid.dataStreamer(cacheName())) {
+                while (!stopLoad.get() && !Thread.currentThread().isInterrupted()) {
+                    streamer.addData(cntr, String.valueOf(cntr));
+
+                    streamer.flush();
+
+                    ++cntr;
+                }
+            }
+        });
+
+        try {
+            IgniteWriteAheadLogManager walMgr = grid(GRID_0).context().cache().context().wal();
+
+            boolean success = waitForCondition(() -> walMgr.lastArchivedSegment() >= walIdx, MAX_AWAIT_MILLIS);
+
+            assertTrue(success);
+        } finally {
+            stopLoad.set(true);
+
+            fut.get(MAX_AWAIT_MILLIS);
+        }
+
+        forceCheckpoint();
+
+        int grpId = CU.cacheId(cacheName());
+
+        assertEquals(2, node1.context().encryption().groupKeyIds(grpId).size());

Review comment:
       Added WAL reservation to prevent deletion.




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GroupKeyChangeProcess.java
##########
@@ -0,0 +1,350 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteFeatures;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.managers.encryption.GridEncryptionManager.EmptyResult;
+import org.apache.ignite.internal.managers.encryption.GridEncryptionManager.KeyChangeFuture;
+import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor;
+import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
+import org.apache.ignite.internal.util.distributed.DistributedProcess;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.future.IgniteFinishedFutureImpl;
+import org.apache.ignite.internal.util.future.IgniteFutureImpl;
+import org.apache.ignite.internal.util.typedef.F;
+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.IgniteFuture;
+import org.apache.ignite.lang.IgniteFutureCancelledException;
+
+import static org.apache.ignite.internal.IgniteFeatures.CACHE_GROUP_KEY_CHANGE;
+import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.CACHE_GROUP_KEY_CHANGE_FINISH;
+import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.CACHE_GROUP_KEY_CHANGE_PREPARE;
+
+/**
+ * A two-phase distributed process that rotates the encryption keys of specified cache groups and initiates
+ * re-encryption of those cache groups.
+ */
+class GroupKeyChangeProcess {
+    /** Grid kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Cache group encyption key change prepare phase. */
+    private final DistributedProcess<ChangeCacheEncryptionRequest, EmptyResult> prepareGKChangeProc;
+
+    /** Cache group encyption key change perform phase. */
+    private final DistributedProcess<ChangeCacheEncryptionRequest, EmptyResult> performGKChangeProc;
+
+    /** Group encryption keys. */
+    private final CacheGroupEncryptionKeys keys;
+
+    /** Cache group key change future. */
+    private volatile GroupKeyChangeFuture fut;
+
+    /** Cache group key change request. */
+    private volatile ChangeCacheEncryptionRequest req;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    GroupKeyChangeProcess(GridKernalContext ctx, CacheGroupEncryptionKeys keys) {
+        this.ctx = ctx;
+        this.keys = keys;
+
+        prepareGKChangeProc =
+            new DistributedProcess<>(ctx, CACHE_GROUP_KEY_CHANGE_PREPARE, this::prepare, this::finishPrepare);
+        performGKChangeProc =
+            new DistributedProcess<>(ctx, CACHE_GROUP_KEY_CHANGE_FINISH, this::perform, this::finishPerform);
+    }
+
+    /**
+     * @return {@code True} if operation is still in progress.
+     */
+    public boolean inProgress() {
+        return req != null;
+    }
+
+    /**
+     * @param msg Error message.
+     */
+    public void cancel(String msg) {
+        GridFutureAdapter<Void> keyChangeFut = fut;
+
+        if (keyChangeFut != null && !keyChangeFut.isDone())
+            keyChangeFut.onDone(new IgniteFutureCancelledException(msg));
+    }
+
+    /**
+     * Starts cache group encryption key change process.
+     *
+     * @param cacheOrGrpNames Cache or group names.
+     */
+    public IgniteFuture<Void> start(Collection<String> cacheOrGrpNames) {
+        if (ctx.clientNode())
+            throw new UnsupportedOperationException("Client and daemon nodes can not perform this operation.");
+
+        if (!IgniteFeatures.allNodesSupports(ctx.grid().cluster().nodes(), CACHE_GROUP_KEY_CHANGE))
+            throw new IllegalStateException("Not all nodes in the cluster support this operation.");
+
+        if (!ctx.state().clusterState().state().active())
+            throw new IgniteException("Operation was rejected. The cluster is inactive.");
+
+        IgniteInternalFuture<Void> fut0 = fut;
+
+        if (fut0 != null && !fut0.isDone()) {
+            return new IgniteFinishedFutureImpl<>(new IgniteException("Cache group key change was rejected. " +
+                "The previous change was not completed."));
+        }
+
+        int[] grpIds = new int[cacheOrGrpNames.size()];
+        byte[] keyIds = new byte[grpIds.length];
+
+        int n = 0;
+
+        for (String cacheOrGroupName : cacheOrGrpNames) {
+            CacheGroupDescriptor grpDesc = ctx.cache().cacheGroupDescriptor(CU.cacheId(cacheOrGroupName));
+
+            if (grpDesc == null) {
+                DynamicCacheDescriptor cacheDesc = ctx.cache().cacheDescriptor(cacheOrGroupName);
+
+                if (cacheDesc == null) {
+                    throw new IgniteException("Cache group key change was rejected. " +
+                        "Cache or group \"" + cacheOrGroupName + "\" doesn't exists");
+                }
+
+                int grpId = cacheDesc.groupId();
+
+                grpDesc = ctx.cache().cacheGroupDescriptor(grpId);
+
+                if (grpDesc.sharedGroup()) {
+                    throw new IgniteException("Cache group key change was rejected. " +
+                        "Cache or group \"" + cacheOrGroupName + "\" is a part of group \"" +
+                        grpDesc.groupName() + "\". Provide group name instead of cache name for shared groups.");
+                }
+            }
+
+            if (!grpDesc.config().isEncryptionEnabled()) {
+                throw new IgniteException("Cache group key change was rejected. " +
+                    "Cache or group \"" + cacheOrGroupName + "\" is not encrypted.");
+            }
+
+            if (ctx.encryption().reencryptionInProgress(grpDesc.groupId())) {
+                throw new IgniteException("Cache group key change was rejected. " +
+                    "Cache group reencryption is in progress [grp=" + cacheOrGroupName + "]");
+            }
+
+            grpIds[n] = grpDesc.groupId();
+            keyIds[n] = (byte)(ctx.encryption().groupKey(grpDesc.groupId()).unsignedId() + 1);
+
+            n += 1;
+        }
+
+        byte[][] keys = ctx.encryption().createKeys(grpIds.length).get1().toArray(new byte[grpIds.length][]);
+
+        ChangeCacheEncryptionRequest req =
+            new ChangeCacheEncryptionRequest(grpIds, keys, keyIds, ctx.config().getEncryptionSpi().getMasterKeyName());
+
+        fut = new GroupKeyChangeFuture(req);
+
+        prepareGKChangeProc.start(req.requestId(), req);
+
+        return new IgniteFutureImpl<>(fut);
+    }
+
+    /**
+     * Validates existing keys.
+     *
+     * @param req Request.
+     * @return Result future.
+     */
+    private IgniteInternalFuture<EmptyResult> prepare(ChangeCacheEncryptionRequest req) {
+        if (ctx.clientNode())
+            return new GridFinishedFuture<>();
+
+        if (inProgress()) {
+            return new GridFinishedFuture<>(new IgniteException("Cache group key change was rejected. " +
+                "The previous change was not completed."));
+        }
+
+        this.req = req;
+
+        try {
+            for (int i = 0; i < req.groupIds().length; i++) {
+                int grpId = req.groupIds()[i];
+                int keyId = req.keyIds()[i] & 0xff;
+
+                if (ctx.encryption().reencryptionInProgress(grpId)) {
+                    return new GridFinishedFuture<>(new IgniteException("Cache group key change was rejected. " +
+                            "Cache group reencryption is in progress [grpId=" + grpId + "]"));
+                }
+
+                List<Integer> keyIds = ctx.encryption().groupKeyIds(grpId);
+
+                if (keyIds == null) {
+                    return new GridFinishedFuture<>(new IgniteException("Cache group key change was rejected." +
+                            "Encrypted cache group not found [grpId=" + grpId + "]"));
+                }
+
+                GroupKey currKey = ctx.encryption().groupKey(grpId);
+
+                for (int locKeyId : keyIds) {
+                    if (locKeyId != keyId)
+                        continue;
+
+                    Long walSegment = keys.reservedSegment(grpId, keyId);
+
+                    // Can overwrite inactive key if it was added during prepare phase.
+                    if (walSegment == null && currKey.id() != (byte)keyId)
+                        continue;
+
+                    return new GridFinishedFuture<>(
+                        new IgniteException("Cache group key change was rejected. Cannot add new key identifier, " +
+                        "it's already present. There existing WAL segments that encrypted with this key [" +
+                        "grpId=" + grpId + ", newId=" + keyId + ", currId=" + currKey.unsignedId() +
+                        ", walSegment=" + walSegment + "]."));
+                }
+            }
+
+            return ctx.encryption().withMasterKeyChangeReadLock(() -> {
+                String curMasterKeyName = ctx.config().getEncryptionSpi().getMasterKeyName();
+
+                if (!curMasterKeyName.equals(req.masterKeyName())) {
+                    return new GridFinishedFuture<>(new IgniteException("Cache group key change was rejected. " +
+                        "Master key has been changed."));
+                }
+
+                for (int i = 0; i < req.groupIds().length; i++) {
+                    // Store new key as inactive.
+                    GroupKeyEncrypted grpKey = new GroupKeyEncrypted(req.keyIds()[i] & 0xff, req.keys()[i]);
+
+                    ctx.encryption().addGroupKey(req.groupIds()[i], grpKey);

Review comment:
       Because the master key may changes between prepare/perform stages and we'll not be able to decrypt received keys.




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupPageScanner.java
##########
@@ -0,0 +1,453 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.EncryptionConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId;
+import org.apache.ignite.internal.util.BasicRateLimiter;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+
+import static org.apache.ignite.internal.util.IgniteUtils.MB;
+
+/**
+ * Cache group page stores scanner.
+ * Scans a range of pages and marks them as dirty to re-encrypt them with the last encryption key on disk.
+ */
+public class CacheGroupPageScanner implements DbCheckpointListener {
+    /** Encryption configuration. */
+    private final EncryptionConfiguration encrCfg;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to group scanning task. */
+    private final Map<Integer, GroupScanTask> grps = new ConcurrentHashMap<>();
+
+    /** Collection of groups waiting for a checkpoint. */
+    private final Collection<GroupScanTask> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Page scanning speed limiter. */
+    private final BasicRateLimiter limiter;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupPageScanner(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        encrCfg = ctx.config().getDataStorageConfiguration().getEncryptionConfiguration();
+
+        DataStorageConfiguration dsCfg = ctx.config().getDataStorageConfiguration();
+
+        limiter = CU.isPersistenceEnabled(dsCfg) && encrCfg.getReencryptionRateLimit() > 0 ?
+            new BasicRateLimiter(encrCfg.getReencryptionRateLimit() * MB / dsCfg.getPageSize()) : null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCheckpointBegin(Context cpCtx) {
+        Set<GroupScanTask> completeCandidates = new HashSet<>();
+
+        cpWaitGrps.removeIf(completeCandidates::add);
+
+        cpCtx.finishedStateFut().listen(
+            f -> {
+                // Retry if error occurs.
+                if (f.error() != null || f.isCancelled()) {
+                    cpWaitGrps.addAll(completeCandidates);
+
+                    return;
+                }
+
+                lock.lock();
+
+                try {
+                    for (GroupScanTask grpScanTask : completeCandidates) {
+                        grps.remove(grpScanTask.groupId());
+
+                        grpScanTask.onDone();
+
+                        if (log.isInfoEnabled())
+                            log.info("Cache group reencryption is finished [grpId=" + grpScanTask.groupId() + "]");
+                    }
+
+                    if (!grps.isEmpty())
+                        return;
+
+                    ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).
+                        removeCheckpointListener(this);
+                }
+                finally {
+                    lock.unlock();
+                }
+            }
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public void beforeCheckpointBegin(Context cpCtx) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onMarkCheckpointBegin(Context ctx) {
+        // No-op.
+    }
+
+    /**
+     * @return {@code True} If reencryption is disabled.
+     */
+    public boolean disabled() {
+        return encrCfg.isReencryptionDisabled();
+    }
+
+    /**
+     * Schedule scanning partitions.
+     *
+     * @param grpId Cache group ID.
+     */
+    public IgniteInternalFuture<Void> schedule(int grpId) throws IgniteCheckedException {
+        if (disabled())
+            throw new IgniteCheckedException("Reencryption is disabled.");
+
+        CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+        if (grp == null) {
+            if (log.isDebugEnabled())
+                log.debug("Skip reencryption, cache group was destroyed [grp=" + grpId + "]");
+
+            return new GridFinishedFuture<>();
+        }
+
+        lock.lock();
+
+        try {
+            if (stopped)
+                throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
+
+            if (grps.isEmpty())
+                ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).addCheckpointListener(this);
+
+            GroupScanTask prevState = grps.get(grpId);
+
+            if (prevState != null) {
+                if (log.isDebugEnabled())
+                    log.debug("Reencryption already scheduled [grpId=" + grpId + "]");
+
+                return prevState;
+            }
+
+            Set<Integer> parts = new HashSet<>();
+
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) {
+                    if (ctx.encryption().getEncryptionState(grpId, partId) == 0) {
+                        if (log.isDebugEnabled())
+                            log.debug("Skipping partition reencryption [grp=" + grpId + ", p=" + partId + "]");
+
+                        return;
+                    }
+
+                    parts.add(partId);
+                }
+            });
+
+            GroupScanTask grpScan = new GroupScanTask(grp, parts);
+
+            ctx.getSystemExecutorService().submit(grpScan);
+
+            if (log.isInfoEnabled())
+                log.info("Scheduled reencryption [grpId=" + grpId + "]");
+
+            grps.put(grpId, grpScan);
+
+            return grpScan;
+        }
+        finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @return Future that will be completed when all partitions have been scanned and pages have been written to disk.
+     */
+    public IgniteInternalFuture<Void> statusFuture(int grpId) {
+        GroupScanTask ctx0 = grps.get(grpId);
+
+        return ctx0 == null ? new GridFinishedFuture<>() : ctx0;
+    }
+
+    /**
+     * Shutdown scanning and disable new tasks scheduling.
+     */
+    public void stop() throws IgniteCheckedException {
+        lock.lock();
+
+        try {
+            stopped = true;
+
+            for (GroupScanTask grpScanTask : grps.values())
+                grpScanTask.cancel();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Stop scannig the specified partition.
+     *
+     * @param grpId Cache group ID.
+     * @param partId Partition ID.
+     * @return {@code True} if reencryption was cancelled.
+     */
+    public boolean cancel(int grpId, int partId) {
+        GroupScanTask grpScanTask = grps.get(grpId);
+
+        if (grpScanTask == null)
+            return false;
+
+        return grpScanTask.cancel(partId);
+    }
+
+    /**
+     * Collect current number of pages in the specified cache group.
+     *
+     * @param grp Cache group.
+     * @return Partitions with current page count.
+     * @throws IgniteCheckedException If failed.
+     */
+    public long[] pagesCount(CacheGroupContext grp) throws IgniteCheckedException {
+        long[] partStates = new long[grp.affinity().partitions() + 1];

Review comment:
       added `// The last element of the array is used to store the status of the index partition.`




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PagePartitionMetaIOV3.java
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.tree.io;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageUtils;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState;
+import org.apache.ignite.internal.util.GridStringBuilder;
+
+/**
+ * IO for partition metadata pages.
+ * Persistent partition contains it's own PendingTree.
+ */
+public class PagePartitionMetaIOV3 extends PagePartitionMetaIOV2 {
+    /** Last reencrypted page index offset. */
+    private static final int ENCRYPT_PAGE_IDX_OFF = END_OF_PARTITION_PAGE_META_V2;
+
+    /** Total pages to be reencrypted offset. */
+    private static final int ENCRYPT_PAGE_MAX_OFF = ENCRYPT_PAGE_IDX_OFF + 4;
+
+    /**
+     * @param ver Version.
+     */
+    public PagePartitionMetaIOV3(int ver) {
+        super(ver);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void initNewPage(long pageAddr, long pageId, int pageSize) {
+        super.initNewPage(pageAddr, pageId, pageSize);
+
+        setEncryptedPageIndex(pageAddr, 0);
+        setEncryptedPageCount(pageAddr, 0);
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @return Index of the last reencrypted page.
+     */
+    public int getEncryptedPageIndex(long pageAddr) {
+        return PageUtils.getInt(pageAddr, ENCRYPT_PAGE_IDX_OFF);
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @param pageIdx Index of the last reencrypted page.
+     *
+     * @return {@code true} if value has changed as a result of this method's invocation.
+     */
+    public boolean setEncryptedPageIndex(long pageAddr, int pageIdx) {
+        if (getEncryptedPageIndex(pageAddr) == pageIdx)
+            return false;
+
+        PageUtils.putLong(pageAddr, ENCRYPT_PAGE_IDX_OFF, pageIdx);
+
+        return true;
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @return Total pages to be reencrypted.
+     */
+    public int getEncryptedPageCount(long pageAddr) {
+        return PageUtils.getInt(pageAddr, ENCRYPT_PAGE_MAX_OFF);
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @param pagesCnt Total pages to be reencrypted.
+     *
+     * @return {@code true} if value has changed as a result of this method's invocation.
+     */
+    public boolean setEncryptedPageCount(long pageAddr, int pagesCnt) {
+        if (getEncryptedPageCount(pageAddr) == pagesCnt)
+            return false;
+
+        PageUtils.putInt(pageAddr, ENCRYPT_PAGE_MAX_OFF, pagesCnt);
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void printPage(long pageAddr, int pageSize, GridStringBuilder sb) throws IgniteCheckedException {
+        byte state = getPartitionState(pageAddr);
+
+        sb.a("PagePartitionMeta[\n\ttreeRoot=").a(getReuseListRoot(pageAddr));
+        sb.a(",\n\tpendingTreeRoot=").a(getLastSuccessfulFullSnapshotId(pageAddr));
+        sb.a(",\n\tlastSuccessfulFullSnapshotId=").a(getLastSuccessfulFullSnapshotId(pageAddr));
+        sb.a(",\n\tlastSuccessfulSnapshotId=").a(getLastSuccessfulSnapshotId(pageAddr));
+        sb.a(",\n\tnextSnapshotTag=").a(getNextSnapshotTag(pageAddr));
+        sb.a(",\n\tlastSuccessfulSnapshotTag=").a(getLastSuccessfulSnapshotTag(pageAddr));
+        sb.a(",\n\tlastAllocatedPageCount=").a(getLastAllocatedPageCount(pageAddr));
+        sb.a(",\n\tcandidatePageCount=").a(getCandidatePageCount(pageAddr));
+        sb.a(",\n\tencryptedPageIndex=").a(getEncryptedPageIndex(pageAddr));
+        sb.a(",\n\tencryptedPageCount=").a(getEncryptedPageCount(pageAddr));

Review comment:
       Fixed




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -483,16 +534,18 @@ public void onLocalJoin() {
         if (dataBag.isJoiningNodeClient() || dataBag.commonDataCollectedFor(ENCRYPTION_MGR.ordinal()))
             return;
 
-        HashMap<Integer, byte[]> knownEncKeys = knownEncryptionKeys();
+        HashMap<Integer, GroupKeyEncrypted> knownEncKeys = grpKeys.getAll();

Review comment:
       These methods cannot be executed concurrently with the master key change, as the node join is rejected during the master key change
      onLocalJoin
      onJoiningNodeDataReceived
      collectGridNodeData




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/WALRecord.java
##########
@@ -233,7 +233,25 @@
         TRACKING_PAGE_REPAIR_DELTA(61, PHYSICAL),
 
         /** Atomic out-of-order update. */
-        OUT_OF_ORDER_UPDATE(62, LOGICAL);
+        OUT_OF_ORDER_UPDATE(62, LOGICAL),
+
+        /** Encrypted WAL-record. */
+        ENCRYPTED_RECORD_V2(63, PHYSICAL),
+
+        /** Ecnrypted data record. */
+        ENCRYPTED_DATA_RECORD_V2(64, LOGICAL),
+
+        /** Master key change record containing multiple keys for single cache group. */
+        MASTER_KEY_CHANGE_RECORD_V2(65, LOGICAL),
+
+        /** Logical record to restart reencryption with the latest encryption key. */
+        REENCRYPTION_START_RECORD(66, LOGICAL),
+
+        /** Partition meta page delta record includes encryption status data. */
+        PARTITION_META_PAGE_UPDATE_COUNTERS_V3(67, PHYSICAL),

Review comment:
       Renamed PARTITION_META_PAGE_UPDATE_COUNTERS_V3 => PARTITION_META_PAGE_DELTA_RECORD_V3




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -821,11 +1181,57 @@ private void writeKeysToMetaStore(boolean writeAll) throws IgniteCheckedExceptio
         if (writeAll)
             metaStorage.write(MASTER_KEY_NAME_PREFIX, getSpi().getMasterKeyName());
 
-        for (Map.Entry<Integer, Serializable> entry : grpEncKeys.entrySet()) {
-            if (!writeAll && metaStorage.read(ENCRYPTION_KEY_PREFIX + entry.getKey()) != null)
+        if (!reencryptGroupsForced.isEmpty())
+            writeTrackedWalIdxsToMetaStore();
+
+        for (Integer grpId : grpKeys.groupIds()) {
+            if (!writeAll && !reencryptGroupsForced.containsKey(grpId) &&
+                metaStorage.read(ENCRYPTION_KEYS_PREFIX + grpId) != null)
                 continue;
 
-            writeToMetaStore(entry.getKey(), getSpi().encryptKey(entry.getValue()));
+            writeGroupKeysToMetaStore(grpId);
+        }
+    }
+
+    /**
+     * Writes cache group encryption keys to metastore.
+     *
+     * @param grpId Cache group ID.
+     */
+    private void writeGroupKeysToMetaStore(int grpId) throws IgniteCheckedException {
+        assert Thread.holdsLock(metaStorageMux);
+
+        if (metaStorage == null || !writeToMetaStoreEnabled || stopped)
+            return;
+
+        List<GroupKeyEncrypted> keysEncrypted = withMasterKeyChangeReadLock(() -> grpKeys.getAll(grpId));

Review comment:
       Locks reordered.




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -811,6 +1124,53 @@ private void sendGenerateEncryptionKeyRequest(GenerateEncryptionKeyFuture fut) t
         ctx.io().sendToGridTopic(rndNode.id(), TOPIC_GEN_ENC_KEY, req, SYSTEM_POOL);
     }
 
+    /**
+     * @param grpIds Cache group IDs.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void startReencryption(Collection<Integer> grpIds) throws IgniteCheckedException {
+        if (pageScanner.disabled())
+            return;
+
+        for (int grpId : grpIds) {
+            IgniteInternalFuture<?> fut = pageScanner.schedule(grpId);
+
+            fut.listen(f -> {
+                try {
+                    f.get();

Review comment:
       What is the reason to change local `try/catch` to multiple  `if/else`?  Fir me it looks like changing `switch/case` to `if/else`

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -811,6 +1124,53 @@ private void sendGenerateEncryptionKeyRequest(GenerateEncryptionKeyFuture fut) t
         ctx.io().sendToGridTopic(rndNode.id(), TOPIC_GEN_ENC_KEY, req, SYSTEM_POOL);
     }
 
+    /**
+     * @param grpIds Cache group IDs.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void startReencryption(Collection<Integer> grpIds) throws IgniteCheckedException {
+        if (pageScanner.disabled())
+            return;
+
+        for (int grpId : grpIds) {
+            IgniteInternalFuture<?> fut = pageScanner.schedule(grpId);
+
+            fut.listen(f -> {
+                try {
+                    f.get();

Review comment:
       What is the reason to change local `try/catch` to multiple  `if/else`?  For me it looks like changing `switch/case` to `if/else`




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupPageScanner.java
##########
@@ -0,0 +1,479 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId;
+import org.apache.ignite.internal.util.BasicRateLimiter;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+import org.apache.ignite.thread.OomExceptionHandler;
+
+import static org.apache.ignite.internal.util.IgniteUtils.MB;
+
+/**
+ * Cache group page stores scanner.
+ * Scans a range of pages and marks them as dirty to re-encrypt them with the last encryption key on disk.
+ */
+public class CacheGroupPageScanner implements CheckpointListener {
+    /** Thread prefix for scanning tasks. */
+    private static final String REENCRYPT_THREAD_PREFIX = "reencrypt";
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to group scanning task. */
+    private final Map<Integer, GroupScanTask> grps = new ConcurrentHashMap<>();
+
+    /** Collection of groups waiting for a checkpoint. */
+    private final Collection<GroupScanTask> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Page scanning speed limiter. */
+    private final BasicRateLimiter limiter;
+
+    /** Single-threaded executor to run cache group scan task. */
+    private final ThreadPoolExecutor singleExecSvc;
+
+    /** Number of pages that is scanned during reencryption under checkpoint lock. */
+    private final int batchSize;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupPageScanner(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        DataStorageConfiguration dsCfg = ctx.config().getDataStorageConfiguration();
+
+        if (!CU.isPersistenceEnabled(dsCfg)) {
+            batchSize = -1;
+            limiter = null;
+            singleExecSvc = null;
+
+            return;
+        }
+
+        double rateLimit = dsCfg.getEncryptionConfiguration().getReencryptionRateLimit();
+
+        limiter = rateLimit > 0 ? new BasicRateLimiter(rateLimit * MB /
+            (dsCfg.getPageSize() == 0 ? DataStorageConfiguration.DFLT_PAGE_SIZE : dsCfg.getPageSize())) : null;
+
+        batchSize = dsCfg.getEncryptionConfiguration().getReencryptionBatchSize();
+
+        singleExecSvc = new IgniteThreadPoolExecutor(REENCRYPT_THREAD_PREFIX,
+            ctx.igniteInstanceName(),
+            1,
+            1,
+            IgniteConfiguration.DFLT_THREAD_KEEP_ALIVE_TIME,
+            new LinkedBlockingQueue<>(),
+            GridIoPolicy.SYSTEM_POOL,
+            new OomExceptionHandler(ctx));
+
+        singleExecSvc.allowCoreThreadTimeOut(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCheckpointBegin(Context cpCtx) {
+        Set<GroupScanTask> completeCandidates = new HashSet<>();
+
+        cpWaitGrps.removeIf(completeCandidates::add);
+
+        cpCtx.finishedStateFut().listen(
+            f -> {
+                // Retry if error occurs.
+                if (f.error() != null || f.isCancelled()) {
+                    cpWaitGrps.addAll(completeCandidates);
+
+                    return;
+                }
+
+                lock.lock();
+
+                try {
+                    for (GroupScanTask grpScanTask : completeCandidates) {
+                        grps.remove(grpScanTask.groupId());
+
+                        grpScanTask.onDone();
+
+                        if (log.isInfoEnabled())
+                            log.info("Cache group reencryption is finished [grpId=" + grpScanTask.groupId() + "]");
+                    }
+
+                    if (!grps.isEmpty())
+                        return;
+
+                    ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).
+                        removeCheckpointListener(this);
+                }
+                finally {
+                    lock.unlock();
+                }
+            }
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public void beforeCheckpointBegin(Context cpCtx) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onMarkCheckpointBegin(Context ctx) {
+        // No-op.
+    }
+
+    /**
+     * Schedule scanning partitions.
+     *
+     * @param grpId Cache group ID.
+     */
+    public IgniteInternalFuture<Void> schedule(int grpId) throws IgniteCheckedException {
+        CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+        if (grp == null || !grp.affinityNode()) {
+            if (log.isInfoEnabled())
+                log.info("Skip reencryption, cache group doesn't exist on the local node [grp=" + grpId + "]");
+
+            return new GridFinishedFuture<>();
+        }
+
+        lock.lock();
+
+        try {
+            if (stopped)
+                throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
+
+            if (grps.isEmpty())
+                ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).addCheckpointListener(this);
+
+            GroupScanTask prevState = grps.get(grpId);
+
+            if (prevState != null && !prevState.isDone()) {
+                if (log.isDebugEnabled())
+                    log.debug("Reencryption already scheduled [grpId=" + grpId + "]");
+
+                return prevState;
+            }
+
+            Set<Integer> parts = new HashSet<>();
+
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) {
+                    if (ctx.encryption().getEncryptionState(grpId, partId) == 0) {
+                        if (log.isDebugEnabled())
+                            log.debug("Skipping partition reencryption [grp=" + grpId + ", p=" + partId + "]");
+
+                        return;
+                    }
+
+                    parts.add(partId);
+                }
+            });
+
+            GroupScanTask grpScan = new GroupScanTask(grp, parts);
+
+            singleExecSvc.submit(grpScan);
+
+            if (log.isInfoEnabled())
+                log.info("Scheduled reencryption [grpId=" + grpId + "]");
+
+            grps.put(grpId, grpScan);
+
+            return grpScan;
+        }
+        finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @return Future that will be completed when all partitions have been scanned and pages have been written to disk.
+     */
+    public IgniteInternalFuture<Void> statusFuture(int grpId) {
+        GroupScanTask grpScanTask = grps.get(grpId);
+
+        return grpScanTask == null ? new GridFinishedFuture<>() : grpScanTask;
+    }
+
+    /**
+     * Shutdown scanning and disable new tasks scheduling.
+     */
+    public void stop() throws IgniteCheckedException {
+        lock.lock();
+
+        try {
+            stopped = true;
+
+            for (GroupScanTask grpScanTask : grps.values())
+                grpScanTask.cancel();
+
+            if (singleExecSvc != null)
+                singleExecSvc.shutdownNow();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Stop scannig the specified partition.
+     *
+     * @param grpId Cache group ID.
+     * @param partId Partition ID.
+     * @return {@code True} if reencryption was cancelled.
+     */
+    public boolean excludePartition(int grpId, int partId) {
+        GroupScanTask grpScanTask = grps.get(grpId);
+
+        if (grpScanTask == null)
+            return false;
+
+        return grpScanTask.excludePartition(partId);
+    }
+
+    /**
+     * Collect current number of pages in the specified cache group.
+     *
+     * @param grp Cache group.
+     * @return Partitions with current page count.
+     * @throws IgniteCheckedException If failed.
+     */
+    public long[] pagesCount(CacheGroupContext grp) throws IgniteCheckedException {
+        // The last element of the array is used to store the status of the index partition.
+        long[] partStates = new long[grp.affinity().partitions() + 1];
+
+        ctx.cache().context().database().checkpointReadLock();
+
+        try {
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) throws IgniteCheckedException {
+                    int pagesCnt = ctx.cache().context().pageStore().pages(grp.groupId(), partId);
+
+                    partStates[Math.min(partId, partStates.length - 1)] = pagesCnt;
+                }
+            });
+        } finally {
+            ctx.cache().context().database().checkpointReadUnlock();
+        }
+
+        return partStates;
+    }
+
+    /**
+     * @param grp Cache group.
+     * @param hnd Partition handler.
+     */
+    private void forEachPageStore(CacheGroupContext grp, IgniteInClosureX<Integer> hnd) throws IgniteCheckedException {
+        int parts = grp.affinity().partitions();
+
+        IgnitePageStoreManager pageStoreMgr = ctx.cache().context().pageStore();
+
+        for (int p = 0; p < parts; p++) {
+            if (!pageStoreMgr.exists(grp.groupId(), p))
+                continue;
+
+            hnd.applyx(p);
+        }
+
+        hnd.applyx(PageIdAllocator.INDEX_PARTITION);
+    }
+
+    /**
+     * Cache group partition scanning task.
+     */
+    private class GroupScanTask extends GridFutureAdapter<Void> implements Runnable {
+        /** Cache group ID. */
+        private final CacheGroupContext grp;
+
+        /** Partition IDs. */
+        private final Set<Integer> parts;
+
+        /** Page memory. */
+        private final PageMemoryEx pageMem;
+
+        /**
+         * @param grp Cache group.
+         */
+        public GroupScanTask(CacheGroupContext grp, Set<Integer> parts) {
+            this.grp = grp;
+            this.parts = new GridConcurrentHashSet<>(parts);
+
+            pageMem = (PageMemoryEx)grp.dataRegion().pageMemory();
+        }
+
+        /** {@inheritDoc} */
+        @Override public synchronized boolean cancel() throws IgniteCheckedException {
+            return onCancelled();
+        }
+
+        /**
+         * Stop reencryption of the specified partition.
+         *
+         * @param partId Partition ID.
+         * @return {@code True} if reencryption was cancelled.
+         */
+        public synchronized boolean excludePartition(int partId) {
+            return parts.remove(partId);
+        }
+
+        /**
+         * @return Cache group ID.
+         */
+        public int groupId() {
+            return grp.groupId();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void run() {
+            try {
+                for (int partId : parts) {
+                    long state = ctx.encryption().getEncryptionState(grp.groupId(), partId);
+
+                    if (state == 0)
+                        continue;
+
+                    scanPartition(partId, ReencryptStateUtils.pageIndex(state), ReencryptStateUtils.pageCount(state));
+
+                    if (isDone())
+                        return;
+                }
+
+                boolean added = cpWaitGrps.add(this);

Review comment:
       cpWaitGrps is a collection of groups waiting for a checkpoint to start.
   
   PageScanner performs the following steps:
   1. mark all pages in the group as dirty
   2. wait for the start of the checkpoint (onCheckpointBegin)
   3. wait for finishedStateFut() of the current checkpoint to complete
   
   I though that after finishedStateFut finishes, all pages (dirty at the time of starting the checkpoint) are written to disk, am I missing something?




----------------------------------------------------------------
This is an automated message from the 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] alex-plekhanov commented on a change in pull request #7941: IGNITE-12843

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #7941:
URL: https://github.com/apache/ignite/pull/7941#discussion_r477165652



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/ReencryptionStatusRecord.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.pagemem.wal.record;
+
+import java.util.Map;
+
+/**
+ * Logical record to restart reencryption with the latest encryption key.
+ */
+public class ReencryptionStatusRecord extends WALRecord {

Review comment:
       It doesn't contain any status, this record is just to say that reencryption was started, perhaps class should be renamed.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PagePartitionMetaIOV3.java
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.tree.io;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageUtils;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState;
+import org.apache.ignite.internal.util.GridStringBuilder;
+
+/**
+ * IO for partition metadata pages.
+ * Persistent partition contains it's own PendingTree.
+ */
+public class PagePartitionMetaIOV3 extends PagePartitionMetaIOV2 {
+    /** Last reencrypted page index offset. */
+    private static final int ENCRYPT_PAGE_IDX_OFF = END_OF_PARTITION_PAGE_META_V2;
+
+    /** Total pages to be reencrypted offset. */
+    private static final int ENCRYPT_PAGE_MAX_OFF = ENCRYPT_PAGE_IDX_OFF + 4;
+
+    /**
+     * @param ver Version.
+     */
+    public PagePartitionMetaIOV3(int ver) {
+        super(ver);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void initNewPage(long pageAddr, long pageId, int pageSize) {
+        super.initNewPage(pageAddr, pageId, pageSize);
+
+        setEncryptedPageIndex(pageAddr, 0);
+        setEncryptedPageCount(pageAddr, 0);
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @return Index of the last reencrypted page.
+     */
+    public int getEncryptedPageIndex(long pageAddr) {
+        return PageUtils.getInt(pageAddr, ENCRYPT_PAGE_IDX_OFF);
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @param pageIdx Index of the last reencrypted page.
+     *
+     * @return {@code true} if value has changed as a result of this method's invocation.
+     */
+    public boolean setEncryptedPageIndex(long pageAddr, int pageIdx) {
+        if (getEncryptedPageIndex(pageAddr) == pageIdx)
+            return false;
+
+        PageUtils.putLong(pageAddr, ENCRYPT_PAGE_IDX_OFF, pageIdx);
+
+        return true;
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @return Total pages to be reencrypted.
+     */
+    public int getEncryptedPageCount(long pageAddr) {
+        return PageUtils.getInt(pageAddr, ENCRYPT_PAGE_MAX_OFF);
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @param pagesCnt Total pages to be reencrypted.
+     *
+     * @return {@code true} if value has changed as a result of this method's invocation.
+     */
+    public boolean setEncryptedPageCount(long pageAddr, int pagesCnt) {
+        if (getEncryptedPageCount(pageAddr) == pagesCnt)
+            return false;
+
+        PageUtils.putInt(pageAddr, ENCRYPT_PAGE_MAX_OFF, pagesCnt);
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void printPage(long pageAddr, int pageSize, GridStringBuilder sb) throws IgniteCheckedException {
+        byte state = getPartitionState(pageAddr);
+
+        sb.a("PagePartitionMeta[\n\ttreeRoot=").a(getReuseListRoot(pageAddr));
+        sb.a(",\n\tpendingTreeRoot=").a(getLastSuccessfulFullSnapshotId(pageAddr));
+        sb.a(",\n\tlastSuccessfulFullSnapshotId=").a(getLastSuccessfulFullSnapshotId(pageAddr));
+        sb.a(",\n\tlastSuccessfulSnapshotId=").a(getLastSuccessfulSnapshotId(pageAddr));
+        sb.a(",\n\tnextSnapshotTag=").a(getNextSnapshotTag(pageAddr));
+        sb.a(",\n\tlastSuccessfulSnapshotTag=").a(getLastSuccessfulSnapshotTag(pageAddr));
+        sb.a(",\n\tlastAllocatedPageCount=").a(getLastAllocatedPageCount(pageAddr));
+        sb.a(",\n\tcandidatePageCount=").a(getCandidatePageCount(pageAddr));
+        sb.a(",\n\tencryptedPageIndex=").a(getEncryptedPageIndex(pageAddr));
+        sb.a(",\n\tencryptedPageCount=").a(getEncryptedPageCount(pageAddr));

Review comment:
       `encryptedPageIndex` and `encryptedPageCount` printed twice.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PagePartitionMetaIOV3.java
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.tree.io;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageUtils;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState;
+import org.apache.ignite.internal.util.GridStringBuilder;
+
+/**
+ * IO for partition metadata pages.
+ * Persistent partition contains it's own PendingTree.
+ */
+public class PagePartitionMetaIOV3 extends PagePartitionMetaIOV2 {
+    /** Last reencrypted page index offset. */
+    private static final int ENCRYPT_PAGE_IDX_OFF = END_OF_PARTITION_PAGE_META_V2;
+
+    /** Total pages to be reencrypted offset. */
+    private static final int ENCRYPT_PAGE_MAX_OFF = ENCRYPT_PAGE_IDX_OFF + 4;
+
+    /**
+     * @param ver Version.
+     */
+    public PagePartitionMetaIOV3(int ver) {
+        super(ver);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void initNewPage(long pageAddr, long pageId, int pageSize) {
+        super.initNewPage(pageAddr, pageId, pageSize);
+
+        setEncryptedPageIndex(pageAddr, 0);
+        setEncryptedPageCount(pageAddr, 0);
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @return Index of the last reencrypted page.
+     */
+    public int getEncryptedPageIndex(long pageAddr) {
+        return PageUtils.getInt(pageAddr, ENCRYPT_PAGE_IDX_OFF);
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @param pageIdx Index of the last reencrypted page.
+     *
+     * @return {@code true} if value has changed as a result of this method's invocation.
+     */
+    public boolean setEncryptedPageIndex(long pageAddr, int pageIdx) {
+        if (getEncryptedPageIndex(pageAddr) == pageIdx)
+            return false;
+
+        PageUtils.putLong(pageAddr, ENCRYPT_PAGE_IDX_OFF, pageIdx);
+
+        return true;
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @return Total pages to be reencrypted.
+     */
+    public int getEncryptedPageCount(long pageAddr) {
+        return PageUtils.getInt(pageAddr, ENCRYPT_PAGE_MAX_OFF);
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @param pagesCnt Total pages to be reencrypted.
+     *
+     * @return {@code true} if value has changed as a result of this method's invocation.
+     */
+    public boolean setEncryptedPageCount(long pageAddr, int pagesCnt) {
+        if (getEncryptedPageCount(pageAddr) == pagesCnt)
+            return false;
+
+        PageUtils.putInt(pageAddr, ENCRYPT_PAGE_MAX_OFF, pagesCnt);
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void printPage(long pageAddr, int pageSize, GridStringBuilder sb) throws IgniteCheckedException {

Review comment:
       Let's simplify this method: for example, introduce new method to print fields which can reuse the same method from superclass and just add new fields of current class.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PagePartitionMetaIOV3.java
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.tree.io;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageUtils;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState;
+import org.apache.ignite.internal.util.GridStringBuilder;
+
+/**
+ * IO for partition metadata pages.
+ * Persistent partition contains it's own PendingTree.
+ */
+public class PagePartitionMetaIOV3 extends PagePartitionMetaIOV2 {
+    /** Last reencrypted page index offset. */
+    private static final int ENCRYPT_PAGE_IDX_OFF = END_OF_PARTITION_PAGE_META_V2;
+
+    /** Total pages to be reencrypted offset. */
+    private static final int ENCRYPT_PAGE_MAX_OFF = ENCRYPT_PAGE_IDX_OFF + 4;
+
+    /**
+     * @param ver Version.
+     */
+    public PagePartitionMetaIOV3(int ver) {
+        super(ver);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void initNewPage(long pageAddr, long pageId, int pageSize) {
+        super.initNewPage(pageAddr, pageId, pageSize);
+
+        setEncryptedPageIndex(pageAddr, 0);
+        setEncryptedPageCount(pageAddr, 0);
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @return Index of the last reencrypted page.
+     */
+    public int getEncryptedPageIndex(long pageAddr) {
+        return PageUtils.getInt(pageAddr, ENCRYPT_PAGE_IDX_OFF);
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @param pageIdx Index of the last reencrypted page.
+     *
+     * @return {@code true} if value has changed as a result of this method's invocation.
+     */
+    public boolean setEncryptedPageIndex(long pageAddr, int pageIdx) {
+        if (getEncryptedPageIndex(pageAddr) == pageIdx)
+            return false;
+
+        PageUtils.putLong(pageAddr, ENCRYPT_PAGE_IDX_OFF, pageIdx);
+
+        return true;
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @return Total pages to be reencrypted.
+     */
+    public int getEncryptedPageCount(long pageAddr) {
+        return PageUtils.getInt(pageAddr, ENCRYPT_PAGE_MAX_OFF);
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @param pagesCnt Total pages to be reencrypted.
+     *
+     * @return {@code true} if value has changed as a result of this method's invocation.
+     */
+    public boolean setEncryptedPageCount(long pageAddr, int pagesCnt) {
+        if (getEncryptedPageCount(pageAddr) == pagesCnt)
+            return false;
+
+        PageUtils.putInt(pageAddr, ENCRYPT_PAGE_MAX_OFF, pagesCnt);
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void printPage(long pageAddr, int pageSize, GridStringBuilder sb) throws IgniteCheckedException {
+        byte state = getPartitionState(pageAddr);
+
+        sb.a("PagePartitionMeta[\n\ttreeRoot=").a(getReuseListRoot(pageAddr));
+        sb.a(",\n\tpendingTreeRoot=").a(getLastSuccessfulFullSnapshotId(pageAddr));
+        sb.a(",\n\tlastSuccessfulFullSnapshotId=").a(getLastSuccessfulFullSnapshotId(pageAddr));
+        sb.a(",\n\tlastSuccessfulSnapshotId=").a(getLastSuccessfulSnapshotId(pageAddr));
+        sb.a(",\n\tnextSnapshotTag=").a(getNextSnapshotTag(pageAddr));
+        sb.a(",\n\tlastSuccessfulSnapshotTag=").a(getLastSuccessfulSnapshotTag(pageAddr));
+        sb.a(",\n\tlastAllocatedPageCount=").a(getLastAllocatedPageCount(pageAddr));
+        sb.a(",\n\tcandidatePageCount=").a(getCandidatePageCount(pageAddr));
+        sb.a(",\n\tencryptedPageIndex=").a(getEncryptedPageIndex(pageAddr));
+        sb.a(",\n\tencryptedPageCount=").a(getEncryptedPageCount(pageAddr));
+        sb.a(",\n\tsize=").a(getSize(pageAddr));
+        sb.a(",\n\tupdateCounter=").a(getUpdateCounter(pageAddr));
+        sb.a(",\n\tglobalRemoveId=").a(getGlobalRemoveId(pageAddr));
+        sb.a(",\n\tpartitionState=").a(state).a("(").a(GridDhtPartitionState.fromOrdinal(state)).a(")");
+        sb.a(",\n\tcountersPageId=").a(getCountersPageId(pageAddr));
+        sb.a(",\n\tcntrUpdDataPageId=").a(getGapsLink(pageAddr));
+        sb.a(",\n\tencryptedPageIndex=").a(getEncryptedPageIndex(pageAddr));
+        sb.a(",\n\tencryptedPageCount=").a(getEncryptedPageCount(pageAddr));
+        sb.a("\n]");
+    }
+
+    /**
+     * Upgrade page to PagePartitionMetaIOV3.
+     *
+     * @param pageAddr Page address.
+     */
+    @Override public void upgradePage(long pageAddr) {
+        assert PageIO.getType(pageAddr) == getType();
+        assert PageIO.getVersion(pageAddr) < 3;
+

Review comment:
       Let's invoke super method if version < 2




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -676,6 +962,100 @@ public void onCacheGroupDestroyed(int grpId) {
         removeGroupKey(grpId);
     }
 
+    /**
+     * @param grpId Cache group ID.
+     * @param partId Partition ID.
+     */
+    public void onDestroyPartitionStore(int grpId, int partId) {
+        try {
+            pageScan.cancel(grpId, partId);
+
+            setEncryptionState(grpId, partId, 0, 0);
+        }
+        catch (IgniteCheckedException e) {
+            log.warning("Unable to cancel reencryption [grpId=" + grpId + ", partId=" + partId + "]", e);
+        }
+    }
+
+    /**
+     * Callabck when WAL segment is removed.
+     *
+     * @param segmentIdx WAL segment index.
+     */
+    public void onWalSegmentRemoved(long segmentIdx) {
+        Map<Integer, Set<Integer>> rmvKeys = removeWalSegments(segmentIdx);
+
+        if (rmvKeys == null)
+            return;
+
+        synchronized (metaStorageMux) {
+            try {
+                writeToMetaStore(0, false, true);
+
+                for (Map.Entry<Integer, Set<Integer>> entry : rmvKeys.entrySet()) {
+                    int grpId = entry.getKey();
+
+                    if (reencryptGroups.containsKey(grpId))
+                        continue;
+
+                    Set<Integer> keyIds = entry.getValue();
+
+                    boolean rmv = removeKeysById(grpEncKeys.get(grpId), keyIds);
+
+                    assert rmv : keyIds;
+
+                    writeToMetaStore(grpId, true, false);

Review comment:
       Seems, it should be under master key change read lock




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

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



[GitHub] [ignite] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupEncryptionKeys.java
##########
@@ -0,0 +1,376 @@
+/*
+ * 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.managers.encryption;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.CopyOnWriteArrayList;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.encryption.EncryptionSpi;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Serves for managing encryption keys and related datastructure located in the heap.
+ */
+class CacheGroupEncryptionKeys {
+    /** Group encryption keys. */
+    private final Map<Integer, List<GroupKey>> grpKeys = new ConcurrentHashMap<>();
+
+    /**
+     * WAL segments encrypted with previous encryption keys prevent keys from being deleted
+     * until the associated segment is deleted.
+     */
+    private final Collection<TrackedWalSegment> trackedWalSegments = new ConcurrentLinkedQueue<>();

Review comment:
       It is possible that the old key will not be deleted until the next key change.
   
   For example, we start with encryption key with ID = 0.
   
   creates new WAL segment 0 (encrypted with key ID = 0)
   ..
   creates new WAL segment 10 (encrypted with key ID = 0)
   
   change encryption key ID = 1
   
   creates new WAL segment 11 (encrypted with key ID = 1)
   ..
   creates new WAL segment 15 (encrypted with key ID = 1)
   
   change encryption key ID = 2
   
   creates new WAL segment 16 (encrypted with key ID = 2)
   ...
   
   and so on.
   
   So, in order to read segment 3 of WAL (for example, for delta rebalancing), we need to keep the key with ID = 0 until the segment 10 is removed from the disk.




----------------------------------------------------------------
This is an automated message from the 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] vldpyatkov commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupEncryptionKeys.java
##########
@@ -0,0 +1,376 @@
+/*
+ * 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.managers.encryption;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.CopyOnWriteArrayList;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.encryption.EncryptionSpi;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Serves for managing encryption keys and related datastructure located in the heap.
+ */
+class CacheGroupEncryptionKeys {
+    /** Group encryption keys. */
+    private final Map<Integer, List<GroupKey>> grpKeys = new ConcurrentHashMap<>();
+
+    /**
+     * WAL segments encrypted with previous encryption keys prevent keys from being deleted
+     * until the associated segment is deleted.
+     */
+    private final Collection<TrackedWalSegment> trackedWalSegments = new ConcurrentLinkedQueue<>();

Review comment:
       These map looks like CheckpointHistory. Can we thought about move it 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] NSAmelchev commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -627,22 +716,104 @@ public void groupKey(int grpId, byte[] encGrpKey) {
         return withMasterKeyChangeReadLock(() -> getSpi().getMasterKeyName());
     }
 
+    /** {@inheritDoc} */
+    @Override public IgniteFuture<Void> changeCacheGroupKey(Collection<String> cacheOrGrpNames) {
+        A.notEmpty(cacheOrGrpNames, "cacheOrGrpNames");
+
+        synchronized (opsMux) {
+            if (stopped) {

Review comment:
       We should check `disconnected` too




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupPageScanner.java
##########
@@ -0,0 +1,567 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.EncryptionConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.util.BasicRateLimiter;
+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.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+import org.apache.ignite.thread.OomExceptionHandler;
+
+import static org.apache.ignite.internal.util.IgniteUtils.MB;
+
+/**
+ * Cache group page stores scanner.
+ * Scans a range of pages and marks them as dirty to re-encrypt them with the last encryption key on disk.
+ */
+public class CacheGroupPageScanner implements DbCheckpointListener {
+    /** Thread prefix for scanning tasks. */
+    private static final String REENCRYPT_THREAD_PREFIX = "reencrypt";
+
+    /** Encryption configuration. */
+    private final EncryptionConfiguration encrCfg;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to group scanning context. */
+    private final Map<Integer, GroupScanContext> grps = new ConcurrentHashMap<>();
+
+    /** Queue of groups waiting for a checkpoint. */
+    private final Queue<Integer> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Executor to start partition scan tasks. */
+    private final IgniteThreadPoolExecutor execSvc;
+
+    /** Page scanning speed limiter. */
+    private final BasicRateLimiter limiter;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupPageScanner(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        encrCfg = ctx.config().getDataStorageConfiguration().getEncryptionConfiguration();
+
+        execSvc = new IgniteThreadPoolExecutor(REENCRYPT_THREAD_PREFIX,
+            ctx.igniteInstanceName(),
+            encrCfg.getReencryptionThreadCnt(),
+            encrCfg.getReencryptionThreadCnt(),
+            IgniteConfiguration.DFLT_THREAD_KEEP_ALIVE_TIME,
+            new LinkedBlockingQueue<>(),
+            GridIoPolicy.SYSTEM_POOL,
+            new OomExceptionHandler(ctx));
+
+        execSvc.allowCoreThreadTimeOut(true);
+
+        DataStorageConfiguration dsCfg = ctx.config().getDataStorageConfiguration();
+
+        limiter = CU.isPersistenceEnabled(dsCfg) && encrCfg.getReencryptionRateLimit() > 0 ?
+            new BasicRateLimiter(encrCfg.getReencryptionRateLimit() * MB / dsCfg.getPageSize()) : null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCheckpointBegin(Context cpCtx) {
+        Set<Integer> completeCandidates = new HashSet<>();
+
+        Integer grpId;
+
+        while ((grpId = cpWaitGrps.poll()) != null)
+            completeCandidates.add(grpId);
+
+        cpCtx.finishedStateFut().listen(
+            f -> {
+                if (f.error() != null || f.isCancelled()) {
+                    cpWaitGrps.addAll(completeCandidates);
+
+                    return;
+                }
+
+                lock.lock();
+
+                try {
+                    for (int groupId : completeCandidates) {
+                        GroupScanContext scanCtx = grps.remove(groupId);
+
+                        boolean finished = scanCtx.finish();
+
+                        assert finished : groupId;
+
+                        if (log.isInfoEnabled())
+                            log.info("Cache group reencryption is finished [grpId=" + groupId + "]");
+                    }
+
+                    if (!grps.isEmpty())
+                        return;
+
+                    ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).
+                        removeCheckpointListener(this);
+                }
+                finally {
+                    lock.unlock();
+                }
+            }
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public void beforeCheckpointBegin(Context cpCtx) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onMarkCheckpointBegin(Context ctx) {
+        // No-op.
+    }
+
+    /**
+     * @return {@code True} If reencryption is disabled.
+     */
+    public boolean disabled() {
+        return encrCfg.isReencryptionDisabled();
+    }
+
+    /**
+     * Schedule scanning partitions.
+     *
+     * @param grpId Cache group ID.
+     */
+    public IgniteInternalFuture<Void> schedule(int grpId) throws IgniteCheckedException {
+        if (disabled())
+            throw new IgniteCheckedException("Reencryption is disabled.");
+
+        CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+        if (grp == null) {
+            if (log.isDebugEnabled())
+                log.debug("Skip reencryption, cache group was destroyed [grp=" + grpId + "]");
+
+            return new GridFinishedFuture<>();
+        }
+
+        lock.lock();
+
+        try {
+            if (stopped)
+                throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
+
+            if (grps.isEmpty())
+                ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).addCheckpointListener(this);
+
+            GroupScanContext prevState = grps.get(grpId);
+
+            if (prevState != null) {
+                if (log.isDebugEnabled())
+                    log.debug("Reencryption already scheduled [grpId=" + grpId + "]");
+
+                return prevState.finishFuture();
+            }
+
+            GroupScanContext ctx0 = new GroupScanContext(grpId);
+
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) {
+                    if (ctx.encryption().getEncryptionState(grpId, partId) == 0) {
+                        if (log.isDebugEnabled())
+                            log.debug("Skipping partition reencryption [grp=" + grpId + ", p=" + partId + "]");
+
+                        return;
+                    }
+
+                    PageStoreScanTask scanTask = new PageStoreScanTask(ctx0, partId);
+
+                    ctx0.add(partId, scanTask);
+
+                    execSvc.submit(scanTask);
+                }
+            });
+
+            ctx0.initialize().listen(f -> {
+                Throwable t = f.error();
+
+                if (t != null) {
+                    log.error("Reencryption is failed [grpId=" + grpId + "]", t);
+
+                    ctx0.fail(t);
+
+                    return;
+                }
+
+                boolean added = cpWaitGrps.offer(grpId);
+
+                assert added;
+            });
+
+            if (log.isInfoEnabled())
+                log.info("Scheduled reencryption [grpId=" + grpId + "]");
+
+            grps.put(grpId, ctx0);
+
+            return ctx0.finishFuture();
+        }
+        finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @return Future that will be completed when all partitions have been scanned and pages have been written to disk.
+     */
+    public IgniteInternalFuture<Void> statusFuture(int grpId) {
+        GroupScanContext ctx0 = grps.get(grpId);
+
+        return ctx0 == null ? new GridFinishedFuture<>() : ctx0.finishFuture();
+    }
+
+    /**
+     * Shutdown scanning and disable new tasks scheduling.
+     */
+    public void stop() throws IgniteCheckedException {
+        lock.lock();
+
+        try {
+            stopped = true;
+
+            for (GroupScanContext ctx0 : grps.values())
+                ctx0.finishFuture().cancel();
+
+            execSvc.shutdown();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Stop scannig the specified partition.
+     *
+     * @param grpId Cache group ID.
+     * @param partId Partition ID.
+     * @return {@code True} if reencryption was cancelled.
+     * @throws IgniteCheckedException If failed.
+     */
+    public boolean cancel(int grpId, int partId) throws IgniteCheckedException {
+        GroupScanContext ctx = grps.get(grpId);
+
+        if (ctx == null)
+            return false;
+
+        return ctx.cancel(partId);
+    }
+
+    /**
+     * Collect current number of pages in the specified cache group.
+     *
+     * @param grp Cache group.
+     * @return Map of partitions with current page count.
+     * @throws IgniteCheckedException If failed.
+     */
+    public Map<Integer, Long> pagesCount(CacheGroupContext grp) throws IgniteCheckedException {
+        Map<Integer, Long> partStates = new HashMap<>();
+
+        ctx.cache().context().database().checkpointReadLock();
+
+        try {
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) throws IgniteCheckedException {
+                    int pagesCnt = ctx.cache().context().pageStore().pages(grp.groupId(), partId);
+
+                    partStates.put(partId, (long)pagesCnt);
+                }
+            });
+        } finally {
+            ctx.cache().context().database().checkpointReadUnlock();
+        }
+
+        return partStates;
+    }
+
+    /**
+     * @param grp Cache group.
+     * @param hnd Partition handler.
+     */
+    private void forEachPageStore(CacheGroupContext grp, IgniteInClosureX<Integer> hnd) throws IgniteCheckedException {
+        int parts = grp.affinity().partitions();
+
+        IgnitePageStoreManager pageStoreMgr = ctx.cache().context().pageStore();
+
+        for (int p = 0; p < parts; p++) {
+            if (!pageStoreMgr.exists(grp.groupId(), p))
+                continue;
+
+            hnd.applyx(p);
+        }
+
+        hnd.applyx(PageIdAllocator.INDEX_PARTITION);
+    }
+
+    /**
+     * Cache group scanning context.
+     */
+    private static class GroupScanContext {
+        /** Partition scanning futures. */
+        private final Map<Integer, IgniteInternalFuture<Void>> futMap = new ConcurrentHashMap<>();
+
+        /** Compound future, that will be completed when all partitions scanned. */
+        private final GridCompoundFuture<Void, Void> compFut = new GridCompoundFuture<>();
+
+        /** Cache group ID. */
+        private final int grpId;
+
+        /** Future that ends after all partitions are done and a checkpoint is finished. */
+        private final GridFutureAdapter<Void> cpFut = new GridFutureAdapter<Void>() {
+            @Override public boolean cancel() throws IgniteCheckedException {
+                compFut.cancel();
+
+                return onDone(null, null, true);
+            }
+        };
+
+        /**
+         * @param grpId Cache group ID.
+         */
+        public GroupScanContext(int grpId) {
+            this.grpId = grpId;
+        }
+
+        /**
+         * @return Cache group ID.
+         */
+        public int groupId() {
+            return grpId;
+        }
+
+        /**
+         * @param partId Partition ID.
+         * @param fut Partition scanning future.
+         */
+        public void add(int partId, IgniteInternalFuture<Void> fut) {
+            compFut.add(fut);
+
+            futMap.put(partId, fut);
+        }
+
+        /**
+         * @return Compound future, that will be completed when all partitions scanned.
+         */
+        public IgniteInternalFuture<Void> initialize() {
+            return compFut.markInitialized();
+        }
+
+        /**
+         * @return Future that ends after all partitions are done and a checkpoint is finished.
+         */
+        public IgniteInternalFuture<Void> finishFuture() {
+            return cpFut;
+        }
+
+        /**
+         * Finish reencryption future.
+         *
+         * @return {@code True} if the future was finished by this call.
+         */
+        public boolean finish() {
+            return cpFut.onDone(compFut.result());
+        }
+
+        /**
+         * Stop reencryption of the specified partition.
+         *
+         * @param partId Partition ID.
+         * @return {@code True} if reencryption was cancelled.
+         * @throws IgniteCheckedException If failed.
+         */
+        public boolean cancel(int partId) throws IgniteCheckedException {
+            IgniteInternalFuture<Void> fut = futMap.get(partId);
+
+            if (fut == null)
+                return false;
+
+            return fut.cancel();
+        }
+
+        /**
+         * @param t Throwable.
+         */
+        public void fail(Throwable t) {
+            cpFut.onDone(t);
+        }
+    }
+
+    /**
+     * Page store scanning task.
+     */
+    private class PageStoreScanTask extends GridFutureAdapter<Void> implements Runnable {
+        /** Partiion ID. */
+        private final int partId;
+
+        /** Cache group scan context. */
+        private final GroupScanContext scanCtx;
+
+        /**
+         * @param scanCtx Cache group scanning context.
+         * @param partId Partition ID.
+         */
+        public PageStoreScanTask(GroupScanContext scanCtx, int partId) {
+            this.scanCtx = scanCtx;
+            this.partId = partId;
+        }
+
+        /** {@inheritDoc} */
+        @Override public synchronized boolean cancel() throws IgniteCheckedException {
+            return onDone();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void run() {
+            try {
+                int grpId = scanCtx.groupId();
+
+                CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+                if (grp == null) {
+                    onDone();
+
+                    return;
+                }
+
+                PageMemoryEx pageMem = (PageMemoryEx)grp.dataRegion().pageMemory();
+                long metaPageId = pageMem.partitionMetaPageId(grpId, partId);
+                long state = ctx.encryption().getEncryptionState(grpId, partId);
+                int batchSize = encrCfg.getReencryptionBatchSize();
+
+                int off = (int)(state >> Integer.SIZE);
+                int cnt = (int)state;

Review comment:
       Added utility class ReencryptStateUtils




----------------------------------------------------------------
This is an automated message from the 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] ingvard commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java
##########
@@ -258,6 +258,9 @@
     /** */
     public static final short T_MARKER_PAGE = 33;
 
+    /** */
+    public static final short T_INDEX_META = 34;

Review comment:
       There are pages which will be broken after that. Like MetaPageUpdateLastAllocatedIndex , because assert 
   ```
     assert type == PageIO.T_META || type == PageIO.T_PART_META;
   ```
   You should use 
   ```
    assert type == PageIO.T_META || type == PageIO.T_PART_META || type == PageIO.T_INDEX_META
   ```
   Please check other places.




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupPageScanner.java
##########
@@ -0,0 +1,453 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.EncryptionConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId;
+import org.apache.ignite.internal.util.BasicRateLimiter;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+
+import static org.apache.ignite.internal.util.IgniteUtils.MB;
+
+/**
+ * Cache group page stores scanner.
+ * Scans a range of pages and marks them as dirty to re-encrypt them with the last encryption key on disk.
+ */
+public class CacheGroupPageScanner implements DbCheckpointListener {
+    /** Encryption configuration. */
+    private final EncryptionConfiguration encrCfg;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to group scanning task. */
+    private final Map<Integer, GroupScanTask> grps = new ConcurrentHashMap<>();
+
+    /** Collection of groups waiting for a checkpoint. */
+    private final Collection<GroupScanTask> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Page scanning speed limiter. */
+    private final BasicRateLimiter limiter;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupPageScanner(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        encrCfg = ctx.config().getDataStorageConfiguration().getEncryptionConfiguration();
+
+        DataStorageConfiguration dsCfg = ctx.config().getDataStorageConfiguration();
+
+        limiter = CU.isPersistenceEnabled(dsCfg) && encrCfg.getReencryptionRateLimit() > 0 ?
+            new BasicRateLimiter(encrCfg.getReencryptionRateLimit() * MB / dsCfg.getPageSize()) : null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCheckpointBegin(Context cpCtx) {
+        Set<GroupScanTask> completeCandidates = new HashSet<>();
+
+        cpWaitGrps.removeIf(completeCandidates::add);
+
+        cpCtx.finishedStateFut().listen(
+            f -> {
+                // Retry if error occurs.
+                if (f.error() != null || f.isCancelled()) {
+                    cpWaitGrps.addAll(completeCandidates);
+
+                    return;
+                }
+
+                lock.lock();
+
+                try {
+                    for (GroupScanTask grpScanTask : completeCandidates) {
+                        grps.remove(grpScanTask.groupId());
+
+                        grpScanTask.onDone();
+
+                        if (log.isInfoEnabled())
+                            log.info("Cache group reencryption is finished [grpId=" + grpScanTask.groupId() + "]");
+                    }
+
+                    if (!grps.isEmpty())
+                        return;
+
+                    ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).
+                        removeCheckpointListener(this);
+                }
+                finally {
+                    lock.unlock();
+                }
+            }
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public void beforeCheckpointBegin(Context cpCtx) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onMarkCheckpointBegin(Context ctx) {
+        // No-op.
+    }
+
+    /**
+     * @return {@code True} If reencryption is disabled.
+     */
+    public boolean disabled() {
+        return encrCfg.isReencryptionDisabled();
+    }
+
+    /**
+     * Schedule scanning partitions.
+     *
+     * @param grpId Cache group ID.
+     */
+    public IgniteInternalFuture<Void> schedule(int grpId) throws IgniteCheckedException {
+        if (disabled())
+            throw new IgniteCheckedException("Reencryption is disabled.");
+
+        CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+        if (grp == null) {
+            if (log.isDebugEnabled())
+                log.debug("Skip reencryption, cache group was destroyed [grp=" + grpId + "]");
+
+            return new GridFinishedFuture<>();
+        }
+
+        lock.lock();
+
+        try {
+            if (stopped)
+                throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
+
+            if (grps.isEmpty())
+                ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).addCheckpointListener(this);
+
+            GroupScanTask prevState = grps.get(grpId);
+
+            if (prevState != null) {
+                if (log.isDebugEnabled())
+                    log.debug("Reencryption already scheduled [grpId=" + grpId + "]");
+
+                return prevState;
+            }
+
+            Set<Integer> parts = new HashSet<>();
+
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) {
+                    if (ctx.encryption().getEncryptionState(grpId, partId) == 0) {
+                        if (log.isDebugEnabled())
+                            log.debug("Skipping partition reencryption [grp=" + grpId + ", p=" + partId + "]");
+
+                        return;
+                    }
+
+                    parts.add(partId);
+                }
+            });
+
+            GroupScanTask grpScan = new GroupScanTask(grp, parts);
+
+            ctx.getSystemExecutorService().submit(grpScan);
+
+            if (log.isInfoEnabled())
+                log.info("Scheduled reencryption [grpId=" + grpId + "]");
+
+            grps.put(grpId, grpScan);
+
+            return grpScan;
+        }
+        finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @return Future that will be completed when all partitions have been scanned and pages have been written to disk.
+     */
+    public IgniteInternalFuture<Void> statusFuture(int grpId) {
+        GroupScanTask ctx0 = grps.get(grpId);
+
+        return ctx0 == null ? new GridFinishedFuture<>() : ctx0;
+    }
+
+    /**
+     * Shutdown scanning and disable new tasks scheduling.
+     */
+    public void stop() throws IgniteCheckedException {
+        lock.lock();
+
+        try {
+            stopped = true;
+
+            for (GroupScanTask grpScanTask : grps.values())
+                grpScanTask.cancel();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Stop scannig the specified partition.
+     *
+     * @param grpId Cache group ID.
+     * @param partId Partition ID.
+     * @return {@code True} if reencryption was cancelled.
+     */
+    public boolean cancel(int grpId, int partId) {
+        GroupScanTask grpScanTask = grps.get(grpId);
+
+        if (grpScanTask == null)
+            return false;
+
+        return grpScanTask.cancel(partId);
+    }
+
+    /**
+     * Collect current number of pages in the specified cache group.
+     *
+     * @param grp Cache group.
+     * @return Partitions with current page count.
+     * @throws IgniteCheckedException If failed.
+     */
+    public long[] pagesCount(CacheGroupContext grp) throws IgniteCheckedException {
+        long[] partStates = new long[grp.affinity().partitions() + 1];
+
+        ctx.cache().context().database().checkpointReadLock();
+
+        try {
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) throws IgniteCheckedException {
+                    int pagesCnt = ctx.cache().context().pageStore().pages(grp.groupId(), partId);
+
+                    partStates[Math.min(partId, partStates.length - 1)] = pagesCnt;
+                }
+            });
+        } finally {
+            ctx.cache().context().database().checkpointReadUnlock();
+        }
+
+        return partStates;
+    }
+
+    /**
+     * @param grp Cache group.
+     * @param hnd Partition handler.
+     */
+    private void forEachPageStore(CacheGroupContext grp, IgniteInClosureX<Integer> hnd) throws IgniteCheckedException {
+        int parts = grp.affinity().partitions();
+
+        IgnitePageStoreManager pageStoreMgr = ctx.cache().context().pageStore();
+
+        for (int p = 0; p < parts; p++) {
+            if (!pageStoreMgr.exists(grp.groupId(), p))
+                continue;
+
+            hnd.applyx(p);
+        }
+
+        hnd.applyx(PageIdAllocator.INDEX_PARTITION);
+    }
+
+    /**
+     * Cache group partition scanning task.
+     */
+    private class GroupScanTask extends GridFutureAdapter<Void> implements Runnable {
+        /** Cache group ID. */
+        private final CacheGroupContext grp;
+
+        /** Partition IDs. */
+        private final Set<Integer> parts;
+
+        /** Page memory. */
+        private final PageMemoryEx pageMem;
+
+        /**
+         * @param grp Cache group.
+         */
+        public GroupScanTask(CacheGroupContext grp, Set<Integer> parts) {
+            this.grp = grp;
+            this.parts = new GridConcurrentHashSet<>(parts);
+
+            pageMem = (PageMemoryEx)grp.dataRegion().pageMemory();
+        }
+
+        /** {@inheritDoc} */
+        @Override public synchronized boolean cancel() throws IgniteCheckedException {
+            return onDone(null, null, true);
+        }
+
+        /**
+         * Stop reencryption of the specified partition.
+         *
+         * @param partId Partition ID.
+         * @return {@code True} if reencryption was cancelled.
+         */
+        public synchronized boolean cancel(int partId) {

Review comment:
       renamed to excludePartition




----------------------------------------------------------------
This is an automated message from the 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] NSAmelchev commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -182,7 +195,7 @@
     private volatile boolean recoveryMasterKeyName;
 
     /** Master key change future. Not {@code null} on request initiator. */

Review comment:
       Javadoc shpuld be fixed too




----------------------------------------------------------------
This is an automated message from the 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] sk0x50 commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/EncryptedRecord.java
##########
@@ -43,7 +43,7 @@ public EncryptedRecord(int grpId, RecordType plainRecType) {
 
     /** {@inheritDoc} */
     @Override public RecordType type() {
-        return RecordType.ENCRYPTED_RECORD;
+        return RecordType.ENCRYPTED_RECORD_V2;

Review comment:
       Could you please describe in short why you need to change the type of this WAL record?
   WAL record was not changed itself, so it looks weird to me.




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java
##########
@@ -258,6 +258,9 @@
     /** */
     public static final short T_MARKER_PAGE = 33;
 
+    /** */
+    public static final short T_INDEX_META = 34;

Review comment:
       I want to agree on naming for this change:
   
   Type: T_META
   Io: MetaPageIOV2 _(extends MetaPageIO)_
   
   Wal record: MetaPageUpdate**Index**DataRecord 
   Wal record type: **INDEX**_META_PAGE_DELTA_RECORD
   
   is it 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] NSAmelchev commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -503,58 +549,96 @@ else if (newKeys != null) {
 
     /** {@inheritDoc} */
     @Override public void onGridDataReceived(GridDiscoveryData data) {
+        assert !writeToMetaStoreEnabled;
+
         if (ctx.clientNode())
             return;
 
-        Map<Integer, byte[]> encKeysFromCluster = (Map<Integer, byte[]>)data.commonData();
+        Map<Integer, Object> encKeysFromCluster = (Map<Integer, Object>)data.commonData();
 
         if (F.isEmpty(encKeysFromCluster))
             return;
 
-        for (Map.Entry<Integer, byte[]> entry : encKeysFromCluster.entrySet()) {
-            if (groupKey(entry.getKey()) == null) {
-                U.quietAndInfo(log, "Store group key received from coordinator [grp=" + entry.getKey() + "]");
+        for (Map.Entry<Integer, Object> entry : encKeysFromCluster.entrySet()) {
+            int grpId = entry.getKey();
 
-                groupKey(entry.getKey(), entry.getValue());
-            }
-            else {
+            GroupKeyEncrypted rmtKey;
+
+            if (entry.getValue() instanceof GroupKeyEncrypted)
+                rmtKey = (GroupKeyEncrypted)entry.getValue();
+            else
+                rmtKey = new GroupKeyEncrypted(INITIAL_KEY_ID, (byte[])entry.getValue());
+
+            GroupKey locGrpKey = groupKey(grpId);
+
+            if (locGrpKey != null && locGrpKey.unsignedId() == rmtKey.id()) {
                 U.quietAndInfo(log, "Skip group key received from coordinator. Already exists. [grp=" +
-                    entry.getKey() + "]");
+                    grpId + ", keyId=" + rmtKey.id() + "]");
+
+                continue;
             }
+
+            U.quietAndInfo(log, "Store group key received from coordinator [grp=" + grpId +
+                ", keyId=" + rmtKey.id() + "]");
+
+            //changeActiveKey
+            GroupKey prevKey = grpKeys.changeActiveKey(grpId, rmtKey);
+
+            if (prevKey == null)
+                continue;
+
+            grpKeys.reserveWalKey(grpId, prevKey.unsignedId(), ctx.cache().context().wal().currentSegment());
+
+            reencryptGroupsForced.put(grpId, rmtKey.id());
         }
     }
 
     /**
      * Returns group encryption key.
      *
-     * @param grpId Group id.
-     * @return Group encryption key.
+     * @param grpId Cache group ID.
+     * @return Group encryption key with identifier, that was set for writing.
      */
-    @Nullable public Serializable groupKey(int grpId) {
-        if (grpEncKeys.isEmpty())
-            return null;
-
-        return grpEncKeys.get(grpId);
+    @Nullable public GroupKey groupKey(int grpId) {
+        return grpKeys.getActiveKey(grpId);
     }
 
     /**
-     * Store group encryption key.
+     * Returns group encryption key with specified identifier.
      *
-     * @param grpId Group id.
-     * @param encGrpKey Encrypted group key.
+     * @param grpId Cache group ID.
+     * @param keyId Encryption key ID.
+     * @return Group encryption key.
      */
-    public void groupKey(int grpId, byte[] encGrpKey) {
-        assert !grpEncKeys.containsKey(grpId);
-
-        Serializable encKey = withMasterKeyChangeReadLock(() -> getSpi().decryptKey(encGrpKey));
+    @Nullable public GroupKey groupKey(int grpId, int keyId) {
+        return grpKeys.getKey(grpId, keyId);
+    }
 
-        synchronized (metaStorageMux) {
-            if (log.isDebugEnabled())
-                log.debug("Key added. [grp=" + grpId + "]");
+    /**
+     * Gets the existing encryption key IDs for the specified cache group.
+     *
+     * @param grpId Cache group ID.
+     * @return List of the key identifiers.
+     */
+    @Nullable public List<Integer> groupKeyIds(int grpId) {
+        return grpKeys.keyIds(grpId);
+    }
 
-            grpEncKeys.put(grpId, encKey);
+    /**
+     * Adds new cache group encryption key.
+     *
+     * @param grpId Cache group ID.
+     * @param key Encryption key.
+     */
+    private void addGroupKey(int grpId, GroupKeyEncrypted key) {
+        try {
+            synchronized (metaStorageMux) {
+                withMasterKeyChangeReadLock(() -> grpKeys.changeActiveKey(grpId, key));

Review comment:
       There is possible deadlock with the `doChangeMasterKey` method in order of locks acquire 




----------------------------------------------------------------
This is an automated message from the 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] NSAmelchev commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
##########
@@ -993,6 +1035,11 @@ private Metas getOrAllocateCacheMetas() throws IgniteCheckedException {
                     null,
                     null);
             }
+            catch (Throwable e) {
+                e.printStackTrace();

Review comment:
       Use `log`, please 




----------------------------------------------------------------
This is an automated message from the 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] vldpyatkov commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupEncryptionKeys.java
##########
@@ -0,0 +1,376 @@
+/*
+ * 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.managers.encryption;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.CopyOnWriteArrayList;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.encryption.EncryptionSpi;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Serves for managing encryption keys and related datastructure located in the heap.
+ */
+class CacheGroupEncryptionKeys {
+    /** Group encryption keys. */
+    private final Map<Integer, List<GroupKey>> grpKeys = new ConcurrentHashMap<>();
+
+    /**
+     * WAL segments encrypted with previous encryption keys prevent keys from being deleted
+     * until the associated segment is deleted.
+     */
+    private final Collection<TrackedWalSegment> trackedWalSegments = new ConcurrentLinkedQueue<>();

Review comment:
       Also, it is a misusing queue. I looked at the code and don't found any queue specific operation.
   But I found this method releaseWalKeys
   I think `NavigableMap<Long, TrackedWalSegment>` fits better here.
   `trackedWalSegments.tailMap(walIdx, true)`




----------------------------------------------------------------
This is an automated message from the 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] sk0x50 commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/EncryptedRecord.java
##########
@@ -43,7 +43,7 @@ public EncryptedRecord(int grpId, RecordType plainRecType) {
 
     /** {@inheritDoc} */
     @Override public RecordType type() {
-        return RecordType.ENCRYPTED_RECORD;
+        return RecordType.ENCRYPTED_RECORD_V2;

Review comment:
       Hello @xtern,
   
   I think changing already existing WAL records is an error-prone approach.
   Today we changed a type; tomorrow we will change/add a field and skip the corresponding changes in the serializer code.
   It is just good practice not to change existing records, IMHO. Anyway, it is 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
##########
@@ -993,6 +1035,11 @@ private Metas getOrAllocateCacheMetas() throws IgniteCheckedException {
                     null,
                     null);
             }
+            catch (Throwable e) {
+                e.printStackTrace();

Review comment:
       thanks, removed it - was used for debugging




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupPageScanner.java
##########
@@ -0,0 +1,453 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.EncryptionConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId;
+import org.apache.ignite.internal.util.BasicRateLimiter;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+
+import static org.apache.ignite.internal.util.IgniteUtils.MB;
+
+/**
+ * Cache group page stores scanner.
+ * Scans a range of pages and marks them as dirty to re-encrypt them with the last encryption key on disk.
+ */
+public class CacheGroupPageScanner implements DbCheckpointListener {
+    /** Encryption configuration. */
+    private final EncryptionConfiguration encrCfg;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to group scanning task. */
+    private final Map<Integer, GroupScanTask> grps = new ConcurrentHashMap<>();
+
+    /** Collection of groups waiting for a checkpoint. */
+    private final Collection<GroupScanTask> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Page scanning speed limiter. */
+    private final BasicRateLimiter limiter;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupPageScanner(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        encrCfg = ctx.config().getDataStorageConfiguration().getEncryptionConfiguration();
+
+        DataStorageConfiguration dsCfg = ctx.config().getDataStorageConfiguration();
+
+        limiter = CU.isPersistenceEnabled(dsCfg) && encrCfg.getReencryptionRateLimit() > 0 ?
+            new BasicRateLimiter(encrCfg.getReencryptionRateLimit() * MB / dsCfg.getPageSize()) : null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCheckpointBegin(Context cpCtx) {
+        Set<GroupScanTask> completeCandidates = new HashSet<>();
+
+        cpWaitGrps.removeIf(completeCandidates::add);
+
+        cpCtx.finishedStateFut().listen(
+            f -> {
+                // Retry if error occurs.
+                if (f.error() != null || f.isCancelled()) {
+                    cpWaitGrps.addAll(completeCandidates);
+
+                    return;
+                }
+
+                lock.lock();
+
+                try {
+                    for (GroupScanTask grpScanTask : completeCandidates) {
+                        grps.remove(grpScanTask.groupId());
+
+                        grpScanTask.onDone();
+
+                        if (log.isInfoEnabled())
+                            log.info("Cache group reencryption is finished [grpId=" + grpScanTask.groupId() + "]");
+                    }
+
+                    if (!grps.isEmpty())
+                        return;
+
+                    ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).
+                        removeCheckpointListener(this);
+                }
+                finally {
+                    lock.unlock();
+                }
+            }
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public void beforeCheckpointBegin(Context cpCtx) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onMarkCheckpointBegin(Context ctx) {
+        // No-op.
+    }
+
+    /**
+     * @return {@code True} If reencryption is disabled.
+     */
+    public boolean disabled() {
+        return encrCfg.isReencryptionDisabled();
+    }
+
+    /**
+     * Schedule scanning partitions.
+     *
+     * @param grpId Cache group ID.
+     */
+    public IgniteInternalFuture<Void> schedule(int grpId) throws IgniteCheckedException {
+        if (disabled())
+            throw new IgniteCheckedException("Reencryption is disabled.");
+
+        CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+        if (grp == null) {
+            if (log.isDebugEnabled())
+                log.debug("Skip reencryption, cache group was destroyed [grp=" + grpId + "]");
+
+            return new GridFinishedFuture<>();
+        }
+
+        lock.lock();
+
+        try {
+            if (stopped)
+                throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
+
+            if (grps.isEmpty())
+                ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).addCheckpointListener(this);
+
+            GroupScanTask prevState = grps.get(grpId);
+
+            if (prevState != null) {
+                if (log.isDebugEnabled())
+                    log.debug("Reencryption already scheduled [grpId=" + grpId + "]");
+
+                return prevState;
+            }
+
+            Set<Integer> parts = new HashSet<>();
+
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) {
+                    if (ctx.encryption().getEncryptionState(grpId, partId) == 0) {
+                        if (log.isDebugEnabled())
+                            log.debug("Skipping partition reencryption [grp=" + grpId + ", p=" + partId + "]");
+
+                        return;
+                    }
+
+                    parts.add(partId);
+                }
+            });
+
+            GroupScanTask grpScan = new GroupScanTask(grp, parts);
+
+            ctx.getSystemExecutorService().submit(grpScan);
+
+            if (log.isInfoEnabled())
+                log.info("Scheduled reencryption [grpId=" + grpId + "]");
+
+            grps.put(grpId, grpScan);
+
+            return grpScan;
+        }
+        finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @return Future that will be completed when all partitions have been scanned and pages have been written to disk.
+     */
+    public IgniteInternalFuture<Void> statusFuture(int grpId) {
+        GroupScanTask ctx0 = grps.get(grpId);
+
+        return ctx0 == null ? new GridFinishedFuture<>() : ctx0;
+    }
+
+    /**
+     * Shutdown scanning and disable new tasks scheduling.
+     */
+    public void stop() throws IgniteCheckedException {
+        lock.lock();
+
+        try {
+            stopped = true;
+
+            for (GroupScanTask grpScanTask : grps.values())
+                grpScanTask.cancel();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Stop scannig the specified partition.
+     *
+     * @param grpId Cache group ID.
+     * @param partId Partition ID.
+     * @return {@code True} if reencryption was cancelled.
+     */
+    public boolean cancel(int grpId, int partId) {
+        GroupScanTask grpScanTask = grps.get(grpId);
+
+        if (grpScanTask == null)
+            return false;
+
+        return grpScanTask.cancel(partId);
+    }
+
+    /**
+     * Collect current number of pages in the specified cache group.
+     *
+     * @param grp Cache group.
+     * @return Partitions with current page count.
+     * @throws IgniteCheckedException If failed.
+     */
+    public long[] pagesCount(CacheGroupContext grp) throws IgniteCheckedException {
+        long[] partStates = new long[grp.affinity().partitions() + 1];

Review comment:
       added `// The last element of the array is used to store the status of the index section.`




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupPageScanner.java
##########
@@ -0,0 +1,453 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.EncryptionConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId;
+import org.apache.ignite.internal.util.BasicRateLimiter;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+
+import static org.apache.ignite.internal.util.IgniteUtils.MB;
+
+/**
+ * Cache group page stores scanner.
+ * Scans a range of pages and marks them as dirty to re-encrypt them with the last encryption key on disk.
+ */
+public class CacheGroupPageScanner implements DbCheckpointListener {
+    /** Encryption configuration. */
+    private final EncryptionConfiguration encrCfg;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to group scanning task. */
+    private final Map<Integer, GroupScanTask> grps = new ConcurrentHashMap<>();
+
+    /** Collection of groups waiting for a checkpoint. */
+    private final Collection<GroupScanTask> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Page scanning speed limiter. */
+    private final BasicRateLimiter limiter;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupPageScanner(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        encrCfg = ctx.config().getDataStorageConfiguration().getEncryptionConfiguration();
+
+        DataStorageConfiguration dsCfg = ctx.config().getDataStorageConfiguration();
+
+        limiter = CU.isPersistenceEnabled(dsCfg) && encrCfg.getReencryptionRateLimit() > 0 ?
+            new BasicRateLimiter(encrCfg.getReencryptionRateLimit() * MB / dsCfg.getPageSize()) : null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCheckpointBegin(Context cpCtx) {
+        Set<GroupScanTask> completeCandidates = new HashSet<>();
+
+        cpWaitGrps.removeIf(completeCandidates::add);
+
+        cpCtx.finishedStateFut().listen(
+            f -> {
+                // Retry if error occurs.
+                if (f.error() != null || f.isCancelled()) {
+                    cpWaitGrps.addAll(completeCandidates);
+
+                    return;
+                }
+
+                lock.lock();
+
+                try {
+                    for (GroupScanTask grpScanTask : completeCandidates) {
+                        grps.remove(grpScanTask.groupId());
+
+                        grpScanTask.onDone();
+
+                        if (log.isInfoEnabled())
+                            log.info("Cache group reencryption is finished [grpId=" + grpScanTask.groupId() + "]");
+                    }
+
+                    if (!grps.isEmpty())
+                        return;
+
+                    ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).
+                        removeCheckpointListener(this);
+                }
+                finally {
+                    lock.unlock();
+                }
+            }
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public void beforeCheckpointBegin(Context cpCtx) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onMarkCheckpointBegin(Context ctx) {
+        // No-op.
+    }
+
+    /**
+     * @return {@code True} If reencryption is disabled.
+     */
+    public boolean disabled() {
+        return encrCfg.isReencryptionDisabled();
+    }
+
+    /**
+     * Schedule scanning partitions.
+     *
+     * @param grpId Cache group ID.
+     */
+    public IgniteInternalFuture<Void> schedule(int grpId) throws IgniteCheckedException {
+        if (disabled())
+            throw new IgniteCheckedException("Reencryption is disabled.");
+
+        CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+        if (grp == null) {
+            if (log.isDebugEnabled())
+                log.debug("Skip reencryption, cache group was destroyed [grp=" + grpId + "]");
+
+            return new GridFinishedFuture<>();
+        }
+
+        lock.lock();
+
+        try {
+            if (stopped)
+                throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
+
+            if (grps.isEmpty())
+                ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).addCheckpointListener(this);
+
+            GroupScanTask prevState = grps.get(grpId);
+
+            if (prevState != null) {
+                if (log.isDebugEnabled())
+                    log.debug("Reencryption already scheduled [grpId=" + grpId + "]");
+
+                return prevState;
+            }
+
+            Set<Integer> parts = new HashSet<>();
+
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) {
+                    if (ctx.encryption().getEncryptionState(grpId, partId) == 0) {
+                        if (log.isDebugEnabled())
+                            log.debug("Skipping partition reencryption [grp=" + grpId + ", p=" + partId + "]");
+
+                        return;
+                    }
+
+                    parts.add(partId);
+                }
+            });
+
+            GroupScanTask grpScan = new GroupScanTask(grp, parts);
+
+            ctx.getSystemExecutorService().submit(grpScan);
+
+            if (log.isInfoEnabled())
+                log.info("Scheduled reencryption [grpId=" + grpId + "]");
+
+            grps.put(grpId, grpScan);
+
+            return grpScan;
+        }
+        finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @return Future that will be completed when all partitions have been scanned and pages have been written to disk.
+     */
+    public IgniteInternalFuture<Void> statusFuture(int grpId) {
+        GroupScanTask ctx0 = grps.get(grpId);
+
+        return ctx0 == null ? new GridFinishedFuture<>() : ctx0;
+    }
+
+    /**
+     * Shutdown scanning and disable new tasks scheduling.
+     */
+    public void stop() throws IgniteCheckedException {
+        lock.lock();
+
+        try {
+            stopped = true;
+
+            for (GroupScanTask grpScanTask : grps.values())
+                grpScanTask.cancel();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Stop scannig the specified partition.
+     *
+     * @param grpId Cache group ID.
+     * @param partId Partition ID.
+     * @return {@code True} if reencryption was cancelled.
+     */
+    public boolean cancel(int grpId, int partId) {
+        GroupScanTask grpScanTask = grps.get(grpId);
+
+        if (grpScanTask == null)
+            return false;
+
+        return grpScanTask.cancel(partId);
+    }
+
+    /**
+     * Collect current number of pages in the specified cache group.
+     *
+     * @param grp Cache group.
+     * @return Partitions with current page count.
+     * @throws IgniteCheckedException If failed.
+     */
+    public long[] pagesCount(CacheGroupContext grp) throws IgniteCheckedException {
+        long[] partStates = new long[grp.affinity().partitions() + 1];
+
+        ctx.cache().context().database().checkpointReadLock();
+
+        try {
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) throws IgniteCheckedException {
+                    int pagesCnt = ctx.cache().context().pageStore().pages(grp.groupId(), partId);
+
+                    partStates[Math.min(partId, partStates.length - 1)] = pagesCnt;
+                }
+            });
+        } finally {
+            ctx.cache().context().database().checkpointReadUnlock();
+        }
+
+        return partStates;
+    }
+
+    /**
+     * @param grp Cache group.
+     * @param hnd Partition handler.
+     */
+    private void forEachPageStore(CacheGroupContext grp, IgniteInClosureX<Integer> hnd) throws IgniteCheckedException {
+        int parts = grp.affinity().partitions();
+
+        IgnitePageStoreManager pageStoreMgr = ctx.cache().context().pageStore();
+
+        for (int p = 0; p < parts; p++) {
+            if (!pageStoreMgr.exists(grp.groupId(), p))
+                continue;
+
+            hnd.applyx(p);
+        }
+
+        hnd.applyx(PageIdAllocator.INDEX_PARTITION);
+    }
+
+    /**
+     * Cache group partition scanning task.
+     */
+    private class GroupScanTask extends GridFutureAdapter<Void> implements Runnable {
+        /** Cache group ID. */
+        private final CacheGroupContext grp;
+
+        /** Partition IDs. */
+        private final Set<Integer> parts;
+
+        /** Page memory. */
+        private final PageMemoryEx pageMem;
+
+        /**
+         * @param grp Cache group.
+         */
+        public GroupScanTask(CacheGroupContext grp, Set<Integer> parts) {
+            this.grp = grp;
+            this.parts = new GridConcurrentHashSet<>(parts);
+
+            pageMem = (PageMemoryEx)grp.dataRegion().pageMemory();
+        }
+
+        /** {@inheritDoc} */
+        @Override public synchronized boolean cancel() throws IgniteCheckedException {
+            return onDone(null, null, true);
+        }
+
+        /**
+         * Stop reencryption of the specified partition.
+         *
+         * @param partId Partition ID.
+         * @return {@code True} if reencryption was cancelled.
+         */
+        public synchronized boolean cancel(int partId) {

Review comment:
       In this case, `synchronized` guarantees that after the call to cancel, the scan will not continue (we will get nasty exceptions in the log if we continue to scan the destroyed partition)




----------------------------------------------------------------
This is an automated message from the 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] alex-plekhanov commented on a change in pull request #7941: IGNITE-12843

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #7941:
URL: https://github.com/apache/ignite/pull/7941#discussion_r494384542



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -821,11 +1181,57 @@ private void writeKeysToMetaStore(boolean writeAll) throws IgniteCheckedExceptio
         if (writeAll)
             metaStorage.write(MASTER_KEY_NAME_PREFIX, getSpi().getMasterKeyName());
 
-        for (Map.Entry<Integer, Serializable> entry : grpEncKeys.entrySet()) {
-            if (!writeAll && metaStorage.read(ENCRYPTION_KEY_PREFIX + entry.getKey()) != null)
+        if (!reencryptGroupsForced.isEmpty())
+            writeTrackedWalIdxsToMetaStore();
+
+        for (Integer grpId : grpKeys.groupIds()) {
+            if (!writeAll && !reencryptGroupsForced.containsKey(grpId) &&
+                metaStorage.read(ENCRYPTION_KEYS_PREFIX + grpId) != null)
                 continue;
 
-            writeToMetaStore(entry.getKey(), getSpi().encryptKey(entry.getValue()));
+            writeGroupKeysToMetaStore(grpId);
+        }
+    }
+
+    /**
+     * Writes cache group encryption keys to metastore.
+     *
+     * @param grpId Cache group ID.
+     */
+    private void writeGroupKeysToMetaStore(int grpId) throws IgniteCheckedException {
+        assert Thread.holdsLock(metaStorageMux);
+
+        if (metaStorage == null || !writeToMetaStoreEnabled || stopped)
+            return;
+
+        List<GroupKeyEncrypted> keysEncrypted = withMasterKeyChangeReadLock(() -> grpKeys.getAll(grpId));

Review comment:
       Different locks order, deadlock is possible. This method is invoked under metaStorageMux and then master key change log acquired, but in doChangeMasterKey() master key change lock acquired first and then metaStorageMux.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/WALRecord.java
##########
@@ -233,7 +233,25 @@
         TRACKING_PAGE_REPAIR_DELTA(61, PHYSICAL),
 
         /** Atomic out-of-order update. */
-        OUT_OF_ORDER_UPDATE(62, LOGICAL);
+        OUT_OF_ORDER_UPDATE(62, LOGICAL),
+
+        /** Encrypted WAL-record. */
+        ENCRYPTED_RECORD_V2(63, PHYSICAL),
+
+        /** Ecnrypted data record. */
+        ENCRYPTED_DATA_RECORD_V2(64, LOGICAL),
+
+        /** Master key change record containing multiple keys for single cache group. */
+        MASTER_KEY_CHANGE_RECORD_V2(65, LOGICAL),
+
+        /** Logical record to restart reencryption with the latest encryption key. */
+        REENCRYPTION_START_RECORD(66, LOGICAL),
+
+        /** Partition meta page delta record includes encryption status data. */
+        PARTITION_META_PAGE_UPDATE_COUNTERS_V3(67, PHYSICAL),

Review comment:
       Maybe `PARTITION_META_PAGE_DELTA_RECORD_V3`? WDYT?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -483,16 +534,18 @@ public void onLocalJoin() {
         if (dataBag.isJoiningNodeClient() || dataBag.commonDataCollectedFor(ENCRYPTION_MGR.ordinal()))
             return;
 
-        HashMap<Integer, byte[]> knownEncKeys = knownEncryptionKeys();
+        HashMap<Integer, GroupKeyEncrypted> knownEncKeys = grpKeys.getAll();

Review comment:
       As far as I understand master key can be changed concurrently with this method and getAll() encrypt keys with master key, so we should use  grpKeys.getAll() under master key change read lock (please pay attention to locks order to avoid deadlocks). At least in this method, perhaps in other methods too (please review also usages of getAll(grpId) and setGroupKeys methods, perhaps these methods should be invoked under lock too)   

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
##########
@@ -2863,11 +2869,16 @@ private RestoreLogicalState applyLogicalUpdates(
 
                         break;
 
-                    case MASTER_KEY_CHANGE_RECORD:
+                    case MASTER_KEY_CHANGE_RECORD_V2:

Review comment:
       Let's also keep `case MASTER_KEY_CHANGE_RECORD:`

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -811,6 +1124,53 @@ private void sendGenerateEncryptionKeyRequest(GenerateEncryptionKeyFuture fut) t
         ctx.io().sendToGridTopic(rndNode.id(), TOPIC_GEN_ENC_KEY, req, SYSTEM_POOL);
     }
 
+    /**
+     * @param grpIds Cache group IDs.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void startReencryption(Collection<Integer> grpIds) throws IgniteCheckedException {
+        if (pageScanner.disabled())
+            return;
+
+        for (int grpId : grpIds) {
+            IgniteInternalFuture<?> fut = pageScanner.schedule(grpId);
+
+            fut.listen(f -> {
+                try {
+                    f.get();

Review comment:
       Because exception driven flow control is an anti-pattern




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/util/BasicRateLimiterTest.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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;
+
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Test;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Rate limiter tests.
+ */
+public class BasicRateLimiterTest {
+    /**
+     * Check rate limit with multiple threads.
+     */
+    @Test
+    public void checkLimitMultithreaded() throws Exception {
+        int opsPerSec = 1_000;
+        int totalOps = 10_000;
+
+        BasicRateLimiter limiter = new BasicRateLimiter(opsPerSec);
+
+        int threads = Runtime.getRuntime().availableProcessors();
+
+        CyclicBarrier ready = new CyclicBarrier(threads + 1);
+
+        AtomicInteger cntr = new AtomicInteger();
+
+        IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(() -> {
+            ready.await();
+
+            do {
+                limiter.acquire(1);
+            }
+            while (!Thread.currentThread().isInterrupted() && cntr.incrementAndGet() < totalOps);
+
+            return null;
+        }, threads, "worker");
+
+        ready.await();
+
+        long startTime = System.currentTimeMillis();
+
+        fut.get();
+
+        long timeSpent = System.currentTimeMillis() - startTime;
+
+        assertEquals(totalOps / opsPerSec, SECONDS.convert(timeSpent, MILLISECONDS));
+    }
+
+    /**
+     * Check that the average speed is limited correctly even if we are acquiring more permits than allowed per second.
+     */
+    @Test
+    public void checkAcquireWithOverflow() throws IgniteInterruptedCheckedException {
+        double permitsPerSec = 0.5;
+        int permitsPerOp = 1;
+        int totalOps = 5;
+
+        BasicRateLimiter limiter = new BasicRateLimiter(permitsPerSec);
+
+        long startTime = System.currentTimeMillis();
+
+        for (int i = 0; i <= totalOps; i++)

Review comment:
       Ratelimiter targets for an average rate of permits per second, these tests have been reworked. To prevent instability of tests, I added a 50% error probability (in general, the error does not exceed 20%).




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/EncryptedRecord.java
##########
@@ -43,7 +43,7 @@ public EncryptedRecord(int grpId, RecordType plainRecType) {
 
     /** {@inheritDoc} */
     @Override public RecordType type() {
-        return RecordType.ENCRYPTED_RECORD;
+        return RecordType.ENCRYPTED_RECORD_V2;

Review comment:
       Thanks for the suggestion, as far as I understand you are suggesting not to modify any existing WAL record classes.
   Could you please give an example of a compatibility issue that we may face in the future (in theory)?




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -627,22 +739,104 @@ public void groupKey(int grpId, byte[] encGrpKey) {
         return withMasterKeyChangeReadLock(() -> getSpi().getMasterKeyName());
     }
 
+    /** {@inheritDoc} */
+    @Override public IgniteFuture<Void> changeCacheGroupKey(Collection<String> cacheOrGrpNames) {
+        A.notEmpty(cacheOrGrpNames, "cacheOrGrpNames");
+
+        synchronized (opsMux) {
+            if (stopped) {
+                return new IgniteFinishedFutureImpl<>(new IgniteException("Cache group key change was rejected. " +
+                    "Node is stopping."));
+            }
+
+            return grpKeyChangeProc.start(cacheOrGrpNames);
+        }
+    }
+
+    /**
+     * @param grpIds Cache group IDs.
+     * @param keyIds Encryption key IDs.
+     * @param keys Encryption keys.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected void changeCacheGroupKeyLocal(int[] grpIds, byte[] keyIds, byte[][] keys) throws IgniteCheckedException {
+        Map<Integer, Byte> encryptionStatus = U.newHashMap(grpIds.length);
+
+        for (int i = 0; i < grpIds.length; i++)
+            encryptionStatus.put(grpIds[i], keyIds[i]);
+
+        WALPointer ptr = ctx.cache().context().wal().log(new ReencryptionStartRecord(encryptionStatus));
+
+        if (ptr != null)
+            ctx.cache().context().wal().flush(ptr, false);
+
+        for (int i = 0; i < grpIds.length; i++) {
+            int grpId = grpIds[i];
+            int newKeyId = keyIds[i] & 0xff;
+
+            synchronized (metaStorageMux) {
+                // Set new key as key for writing. Note that we cannot pass the encrypted key here because the master
+                // key may have changed in which case we will not be able to decrypt the cache encryption key.
+                GroupKey prevGrpKey = grpKeys.changeActiveKey(grpId, newKeyId);
+
+                List<GroupKeyEncrypted> keysEncrypted = withMasterKeyChangeReadLock(() -> grpKeys.getAll(grpId));

Review comment:
       Thanks, fixed it.




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

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



[GitHub] [ignite] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -821,11 +1181,57 @@ private void writeKeysToMetaStore(boolean writeAll) throws IgniteCheckedExceptio
         if (writeAll)
             metaStorage.write(MASTER_KEY_NAME_PREFIX, getSpi().getMasterKeyName());
 
-        for (Map.Entry<Integer, Serializable> entry : grpEncKeys.entrySet()) {
-            if (!writeAll && metaStorage.read(ENCRYPTION_KEY_PREFIX + entry.getKey()) != null)
+        if (!reencryptGroupsForced.isEmpty())
+            writeTrackedWalIdxsToMetaStore();
+
+        for (Integer grpId : grpKeys.groupIds()) {
+            if (!writeAll && !reencryptGroupsForced.containsKey(grpId) &&
+                metaStorage.read(ENCRYPTION_KEYS_PREFIX + grpId) != null)
                 continue;
 
-            writeToMetaStore(entry.getKey(), getSpi().encryptKey(entry.getValue()));
+            writeGroupKeysToMetaStore(grpId);
+        }
+    }
+
+    /**
+     * Writes cache group encryption keys to metastore.
+     *
+     * @param grpId Cache group ID.
+     */
+    private void writeGroupKeysToMetaStore(int grpId) throws IgniteCheckedException {
+        assert Thread.holdsLock(metaStorageMux);
+
+        if (metaStorage == null || !writeToMetaStoreEnabled || stopped)
+            return;
+
+        List<GroupKeyEncrypted> keysEncrypted = withMasterKeyChangeReadLock(() -> grpKeys.getAll(grpId));

Review comment:
       Thanks, reworked locks ordering.




----------------------------------------------------------------
This is an automated message from the 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] ingvard commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java
##########
@@ -258,6 +258,9 @@
     /** */
     public static final short T_MARKER_PAGE = 33;
 
+    /** */
+    public static final short T_INDEX_META = 34;

Review comment:
       There are pages which will be broken after that. Like MetaPageUpdateLastAllocatedIndex , because assert 
   ```
     assert type == PageIO.T_META || type == PageIO.T_PART_META;
   ```
   You should use 
   ```
    assert type == PageIO.T_META || type == PageIO.T_PART_META || type == PageIO.T_INDEX_META
   ```
   Please check other places.




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/ReencryptionStatusRecord.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.pagemem.wal.record;
+
+import java.util.Map;
+
+/**
+ * Logical record to restart reencryption with the latest encryption key.
+ */
+public class ReencryptionStatusRecord extends WALRecord {

Review comment:
       rernamed to "ReencryptionStartRecord"




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupPageScanner.java
##########
@@ -0,0 +1,567 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.EncryptionConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.util.BasicRateLimiter;
+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.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+import org.apache.ignite.thread.OomExceptionHandler;
+
+import static org.apache.ignite.internal.util.IgniteUtils.MB;
+
+/**
+ * Cache group page stores scanner.
+ * Scans a range of pages and marks them as dirty to re-encrypt them with the last encryption key on disk.
+ */
+public class CacheGroupPageScanner implements DbCheckpointListener {
+    /** Thread prefix for scanning tasks. */
+    private static final String REENCRYPT_THREAD_PREFIX = "reencrypt";
+
+    /** Encryption configuration. */
+    private final EncryptionConfiguration encrCfg;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to group scanning context. */
+    private final Map<Integer, GroupScanContext> grps = new ConcurrentHashMap<>();
+
+    /** Queue of groups waiting for a checkpoint. */
+    private final Queue<Integer> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Executor to start partition scan tasks. */
+    private final IgniteThreadPoolExecutor execSvc;
+
+    /** Page scanning speed limiter. */
+    private final BasicRateLimiter limiter;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupPageScanner(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        encrCfg = ctx.config().getDataStorageConfiguration().getEncryptionConfiguration();
+
+        execSvc = new IgniteThreadPoolExecutor(REENCRYPT_THREAD_PREFIX,
+            ctx.igniteInstanceName(),
+            encrCfg.getReencryptionThreadCnt(),
+            encrCfg.getReencryptionThreadCnt(),
+            IgniteConfiguration.DFLT_THREAD_KEEP_ALIVE_TIME,
+            new LinkedBlockingQueue<>(),
+            GridIoPolicy.SYSTEM_POOL,
+            new OomExceptionHandler(ctx));
+
+        execSvc.allowCoreThreadTimeOut(true);
+
+        DataStorageConfiguration dsCfg = ctx.config().getDataStorageConfiguration();
+
+        limiter = CU.isPersistenceEnabled(dsCfg) && encrCfg.getReencryptionRateLimit() > 0 ?
+            new BasicRateLimiter(encrCfg.getReencryptionRateLimit() * MB / dsCfg.getPageSize()) : null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCheckpointBegin(Context cpCtx) {
+        Set<Integer> completeCandidates = new HashSet<>();
+
+        Integer grpId;
+
+        while ((grpId = cpWaitGrps.poll()) != null)
+            completeCandidates.add(grpId);
+
+        cpCtx.finishedStateFut().listen(
+            f -> {
+                if (f.error() != null || f.isCancelled()) {
+                    cpWaitGrps.addAll(completeCandidates);
+
+                    return;
+                }
+
+                lock.lock();
+
+                try {
+                    for (int groupId : completeCandidates) {
+                        GroupScanContext scanCtx = grps.remove(groupId);
+
+                        boolean finished = scanCtx.finish();
+
+                        assert finished : groupId;
+
+                        if (log.isInfoEnabled())
+                            log.info("Cache group reencryption is finished [grpId=" + groupId + "]");
+                    }
+
+                    if (!grps.isEmpty())
+                        return;
+
+                    ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).
+                        removeCheckpointListener(this);
+                }
+                finally {
+                    lock.unlock();
+                }
+            }
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public void beforeCheckpointBegin(Context cpCtx) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onMarkCheckpointBegin(Context ctx) {
+        // No-op.
+    }
+
+    /**
+     * @return {@code True} If reencryption is disabled.
+     */
+    public boolean disabled() {
+        return encrCfg.isReencryptionDisabled();
+    }
+
+    /**
+     * Schedule scanning partitions.
+     *
+     * @param grpId Cache group ID.
+     */
+    public IgniteInternalFuture<Void> schedule(int grpId) throws IgniteCheckedException {
+        if (disabled())
+            throw new IgniteCheckedException("Reencryption is disabled.");
+
+        CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+        if (grp == null) {
+            if (log.isDebugEnabled())
+                log.debug("Skip reencryption, cache group was destroyed [grp=" + grpId + "]");
+
+            return new GridFinishedFuture<>();
+        }
+
+        lock.lock();
+
+        try {
+            if (stopped)
+                throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
+
+            if (grps.isEmpty())
+                ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).addCheckpointListener(this);
+
+            GroupScanContext prevState = grps.get(grpId);
+
+            if (prevState != null) {
+                if (log.isDebugEnabled())
+                    log.debug("Reencryption already scheduled [grpId=" + grpId + "]");
+
+                return prevState.finishFuture();
+            }
+
+            GroupScanContext ctx0 = new GroupScanContext(grpId);
+
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) {
+                    if (ctx.encryption().getEncryptionState(grpId, partId) == 0) {
+                        if (log.isDebugEnabled())
+                            log.debug("Skipping partition reencryption [grp=" + grpId + ", p=" + partId + "]");
+
+                        return;
+                    }
+
+                    PageStoreScanTask scanTask = new PageStoreScanTask(ctx0, partId);
+
+                    ctx0.add(partId, scanTask);
+
+                    execSvc.submit(scanTask);
+                }
+            });
+
+            ctx0.initialize().listen(f -> {
+                Throwable t = f.error();
+
+                if (t != null) {
+                    log.error("Reencryption is failed [grpId=" + grpId + "]", t);
+
+                    ctx0.fail(t);
+
+                    return;
+                }
+
+                boolean added = cpWaitGrps.offer(grpId);
+
+                assert added;
+            });
+
+            if (log.isInfoEnabled())
+                log.info("Scheduled reencryption [grpId=" + grpId + "]");
+
+            grps.put(grpId, ctx0);
+
+            return ctx0.finishFuture();
+        }
+        finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @return Future that will be completed when all partitions have been scanned and pages have been written to disk.
+     */
+    public IgniteInternalFuture<Void> statusFuture(int grpId) {
+        GroupScanContext ctx0 = grps.get(grpId);
+
+        return ctx0 == null ? new GridFinishedFuture<>() : ctx0.finishFuture();
+    }
+
+    /**
+     * Shutdown scanning and disable new tasks scheduling.
+     */
+    public void stop() throws IgniteCheckedException {
+        lock.lock();
+
+        try {
+            stopped = true;
+
+            for (GroupScanContext ctx0 : grps.values())
+                ctx0.finishFuture().cancel();
+
+            execSvc.shutdown();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Stop scannig the specified partition.
+     *
+     * @param grpId Cache group ID.
+     * @param partId Partition ID.
+     * @return {@code True} if reencryption was cancelled.
+     * @throws IgniteCheckedException If failed.
+     */
+    public boolean cancel(int grpId, int partId) throws IgniteCheckedException {
+        GroupScanContext ctx = grps.get(grpId);
+
+        if (ctx == null)
+            return false;
+
+        return ctx.cancel(partId);
+    }
+
+    /**
+     * Collect current number of pages in the specified cache group.
+     *
+     * @param grp Cache group.
+     * @return Map of partitions with current page count.
+     * @throws IgniteCheckedException If failed.
+     */
+    public Map<Integer, Long> pagesCount(CacheGroupContext grp) throws IgniteCheckedException {
+        Map<Integer, Long> partStates = new HashMap<>();

Review comment:
       Map<Integer, ReencryptState> replaced with long[grp.affinity().partitions() + 1] (+1 for index partition)




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java
##########
@@ -258,6 +258,9 @@
     /** */
     public static final short T_MARKER_PAGE = 33;
 
+    /** */
+    public static final short T_INDEX_META = 34;

Review comment:
       Yes, we adding two int fields into two pages.
   
   We have 2 types of memory pages:
   T_META (index partition)
   T_PART_META (other partition)
   
   We want to add 2 ints into each of them, but don't forget that IO for T_PART_META extends IO for T_META.
   
   In my first approach I added 2 ints into T_META, and added PagePartitionMetaIOV3 for T_PART_META.
   But since PagePartitionMetaIO "extends" PageMetaIO all offsets in PagePartitionMetaIO was shifted and there was a problem with inheritance (for example in PagePartitionMetaIO/V2 new methods were visible, all offsets was redefined in PagePartitionMetaIOV3), you can imagine what it looked like in commit with the second approach https://github.com/apache/ignite/pull/7941/commits/036b931bd40e2ffbd519779fde86fae7343da23e
   
   With current approach we simply create the new IO that is only used for the "index" partition.
   
   If you can suggest a better approach how for storing 2 extra int's itn the index meta page without losing binary compatibility, please describe.
   
   p.s. I explained this change in wiki https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=95652384#TDE.Phase3.Cachekeyrotation.-PageMetaIOandPagePartitionMetaIO




----------------------------------------------------------------
This is an automated message from the 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] ingvard commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupReencryption.java
##########
@@ -0,0 +1,583 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+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.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+import org.apache.ignite.thread.OomExceptionHandler;
+
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_REENCRYPTION_BATCH_SIZE;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_REENCRYPTION_DISABLED;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_REENCRYPTION_THREAD_POOL_SIZE;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_REENCRYPTION_THROTTLE;
+
+/**
+ * Cache group reencryption manager.
+ */
+public class CacheGroupReencryption implements DbCheckpointListener {

Review comment:
       CacheGroupReencryption may be add name like Listener or Processor?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupReencryption.java
##########
@@ -0,0 +1,583 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+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.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+import org.apache.ignite.thread.OomExceptionHandler;
+
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_REENCRYPTION_BATCH_SIZE;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_REENCRYPTION_DISABLED;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_REENCRYPTION_THREAD_POOL_SIZE;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_REENCRYPTION_THROTTLE;
+
+/**
+ * Cache group reencryption manager.
+ */
+public class CacheGroupReencryption implements DbCheckpointListener {
+    /** Thread prefix for reencryption tasks. */
+    private static final String REENCRYPT_THREAD_PREFIX = "reencrypt";
+
+    /** Max amount of pages that will be read into memory under checkpoint lock. */
+    private final int batchSize = IgniteSystemProperties.getInteger(IGNITE_REENCRYPTION_BATCH_SIZE, 1_000);
+
+    /** Timeout between batches. */
+    private final long timeoutBetweenBatches = IgniteSystemProperties.getLong(IGNITE_REENCRYPTION_THROTTLE, 0);
+
+    /** Disable background reencryption. */
+    private final boolean disabled = IgniteSystemProperties.getBoolean(IGNITE_REENCRYPTION_DISABLED, false);
+
+    /** Number of threads for partition scanning. */
+    private final int threadsCnt = IgniteSystemProperties.getInteger(IGNITE_REENCRYPTION_THREAD_POOL_SIZE,
+        Runtime.getRuntime().availableProcessors());
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to reencryption context. */
+    private final Map<Integer, GroupReencryptionContext> grps = new ConcurrentHashMap<>();
+
+    /** Queue of groups waiting for a checkpoint. */
+    private final Queue<Integer> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Executor to start partition scan tasks. */
+    private final IgniteThreadPoolExecutor execSvc;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupReencryption(GridKernalContext ctx) {

Review comment:
       Maybe we want to use current classes without context?  public CacheGroupReencryption(Logger log, etc?)

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -283,11 +334,17 @@ public GridEncryptionManager(GridKernalContext ctx) {
 
         performMKChangeProc = new DistributedProcess<>(ctx, MASTER_KEY_CHANGE_FINISH, this::performMasterKeyChange,
             this::finishPerformMasterKeyChange);
+
+        reencryption = new CacheGroupReencryption(ctx);

Review comment:
       could you put this dependency outside?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -194,10 +227,28 @@
      * Master key change prepare process. Checks that all server nodes have the same new master key and then starts
      * finish process.
      */
-    private DistributedProcess<MasterKeyChangeRequest, MasterKeyChangeResult> prepareMKChangeProc;
+    private DistributedProcess<MasterKeyChangeRequest, EmptyResult> prepareMKChangeProc;
 
     /** Process to perform the master key change. Changes master key and reencrypt group keys. */
-    private DistributedProcess<MasterKeyChangeRequest, MasterKeyChangeResult> performMKChangeProc;
+    private DistributedProcess<MasterKeyChangeRequest, EmptyResult> performMKChangeProc;
+
+    /** Two phase distributed process, that performs cache group encryption key rotation. */
+    private GroupKeyChangeProcess grpKeyChangeProc;
+
+    /** Cache groups for which encryption key was changed, and they must be re-encrypted. */
+    private final Map<Integer, Map<Integer, Long>> reencryptGroups = new ConcurrentHashMap<>();

Review comment:
        Map<Integer /** PartId **/ etc

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupReencryption.java
##########
@@ -0,0 +1,583 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+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.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+import org.apache.ignite.thread.OomExceptionHandler;
+
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_REENCRYPTION_BATCH_SIZE;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_REENCRYPTION_DISABLED;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_REENCRYPTION_THREAD_POOL_SIZE;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_REENCRYPTION_THROTTLE;
+
+/**
+ * Cache group reencryption manager.
+ */
+public class CacheGroupReencryption implements DbCheckpointListener {
+    /** Thread prefix for reencryption tasks. */
+    private static final String REENCRYPT_THREAD_PREFIX = "reencrypt";
+
+    /** Max amount of pages that will be read into memory under checkpoint lock. */
+    private final int batchSize = IgniteSystemProperties.getInteger(IGNITE_REENCRYPTION_BATCH_SIZE, 1_000);
+
+    /** Timeout between batches. */
+    private final long timeoutBetweenBatches = IgniteSystemProperties.getLong(IGNITE_REENCRYPTION_THROTTLE, 0);
+
+    /** Disable background reencryption. */
+    private final boolean disabled = IgniteSystemProperties.getBoolean(IGNITE_REENCRYPTION_DISABLED, false);
+
+    /** Number of threads for partition scanning. */
+    private final int threadsCnt = IgniteSystemProperties.getInteger(IGNITE_REENCRYPTION_THREAD_POOL_SIZE,
+        Runtime.getRuntime().availableProcessors());
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to reencryption context. */
+    private final Map<Integer, GroupReencryptionContext> grps = new ConcurrentHashMap<>();
+
+    /** Queue of groups waiting for a checkpoint. */
+    private final Queue<Integer> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Executor to start partition scan tasks. */
+    private final IgniteThreadPoolExecutor execSvc;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupReencryption(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        execSvc = new IgniteThreadPoolExecutor(REENCRYPT_THREAD_PREFIX,
+            ctx.igniteInstanceName(),
+            threadsCnt,
+            threadsCnt,
+            IgniteConfiguration.DFLT_THREAD_KEEP_ALIVE_TIME,
+            new LinkedBlockingQueue<>(),
+            GridIoPolicy.SYSTEM_POOL,
+            new OomExceptionHandler(ctx));
+
+        execSvc.allowCoreThreadTimeOut(true);
+    }
+
+    /**
+     * Shutdown reencryption and disable new tasks scheduling.
+     */
+    public void stop() throws IgniteCheckedException {
+        lock.lock();
+
+        try {
+            stopped = true;
+
+            for (GroupReencryptionContext ctx0 : grps.values())
+                ctx0.finishFuture().cancel();
+
+            execSvc.shutdown();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCheckpointBegin(Context cpCtx) {
+        Set<Integer> completeCandidates = new HashSet<>();
+
+        Integer grpId;
+
+        while ((grpId = cpWaitGrps.poll()) != null)
+            completeCandidates.add(grpId);
+
+        cpCtx.finishedStateFut().listen(
+            f -> {
+                if (f.error() != null || f.isCancelled()) {
+                    cpWaitGrps.addAll(completeCandidates);
+
+                    return;
+                }
+
+                lock.lock();
+
+                try {
+                    for (int groupId : completeCandidates) {
+                        GroupReencryptionContext encrCtx = grps.remove(groupId);
+
+                        boolean finished = encrCtx.finish();
+
+                        assert finished : groupId;
+
+                        if (log.isInfoEnabled())
+                            log.info("Cache group reencryption is finished [grpId=" + groupId + "]");
+                    }
+
+                    if (!grps.isEmpty())
+                        return;
+
+                    ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).
+                        removeCheckpointListener(this);
+                }
+                finally {
+                    lock.unlock();
+                }
+            }
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public void beforeCheckpointBegin(Context cpCtx) {
+        for (GroupReencryptionContext encrCtx : grps.values()) {
+            if (encrCtx.skipDirty())
+                return;
+
+            cpCtx.finishedStateFut().listen(f -> {
+                if (f.error() == null && !f.isCancelled())
+                    encrCtx.skipDirty(true);
+            });
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onMarkCheckpointBegin(Context ctx) {
+        // No-op.
+    }
+
+    /**
+     * @return {@code True} If reencryption is disabled.
+     */
+    public boolean disabled() {
+        return disabled;
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @param skipDirty Dirty page skip flag.
+     */
+    public IgniteInternalFuture schedule(int grpId, boolean skipDirty) throws IgniteCheckedException {
+        if (disabled)
+            throw new IgniteCheckedException("Reencryption is disabled.");
+
+        CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+        if (grp == null) {
+            if (log.isDebugEnabled())
+                log.debug("Skip reencryption, group was destroyed [grp=" + grpId + "]");
+
+            return new GridFinishedFuture();
+        }
+
+        lock.lock();
+
+        try {
+            if (stopped)
+                throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
+
+            if (grps.isEmpty())
+                ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).addCheckpointListener(this);
+
+            GroupReencryptionContext prevState = grps.get(grpId);
+
+            if (prevState != null) {
+                if (log.isDebugEnabled())
+                    log.debug("Reencryption already scheduled [grpId=" + grpId + "]");
+
+                return prevState.finishFuture();
+            }
+
+            GroupReencryptionContext ctx0 = new GroupReencryptionContext(grpId, skipDirty);
+
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) {
+                    if (ctx.encryption().getEncryptionState(grpId, partId) == 0) {
+                        if (log.isDebugEnabled())
+                            log.debug("Skipping partition reencryption [grp=" + grpId + ", p=" + partId + "]");
+
+                        return;
+                    }
+
+                    PageStoreScanTask scan = new PageStoreScanTask(ctx0, partId);
+
+                    ctx0.add(partId, scan);
+
+                    execSvc.submit(scan);
+                }
+            });
+
+            ctx0.initialize().listen(f -> {
+                Throwable t = f.error();
+
+                if (t != null) {
+                    log.error("Reencryption is failed [grpId=" + grpId + "]", t);
+
+                    ctx0.fail(t);
+
+                    return;
+                }
+
+                boolean added = cpWaitGrps.offer(grpId);
+
+                assert added;
+            });
+
+            if (log.isInfoEnabled())
+                log.info("Scheduled reencryption [grpId=" + grpId + "]");
+
+            grps.put(grpId, ctx0);
+
+            return ctx0.finishFuture();
+        }
+        finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @return Future that will be completed when the reencryption of the specified group ends.
+     */
+    public IgniteInternalFuture<Void> statusFuture(int grpId) {
+        GroupReencryptionContext ctx0 = grps.get(grpId);
+
+        return ctx0 == null ? new GridFinishedFuture<>() : ctx0.finishFuture();
+    }
+
+    /**
+     * Stop reencryption of the specified partition.
+     *
+     * @param grpId Cache group ID.
+     * @param partId Partition ID.
+     * @return {@code True} if reencryption was cancelled.
+     * @throws IgniteCheckedException If failed.
+     */
+    public boolean cancel(int grpId, int partId) throws IgniteCheckedException {
+        GroupReencryptionContext ctx = grps.get(grpId);
+
+        if (ctx == null)
+            return false;
+
+        return ctx.cancel(partId);
+    }
+
+    /**
+     * Save current pages count for reencryption.
+     *
+     * @param grp Cache group.
+     * @return Map of partitions with current page count.
+     * @throws IgniteCheckedException If failed.
+     */
+    public Map<Integer, Long> storePagesCount(CacheGroupContext grp) throws IgniteCheckedException {
+        Map<Integer, Long> offsets = new HashMap<>();
+
+        ctx.cache().context().database().checkpointReadLock();
+
+        try {
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) throws IgniteCheckedException {
+                    int pagesCnt = ctx.cache().context().pageStore().pages(grp.groupId(), partId);
+
+                    offsets.put(partId, (long)pagesCnt);
+                }
+            });
+        } finally {
+            ctx.cache().context().database().checkpointReadUnlock();
+        }
+
+        return offsets;
+    }
+
+    /**
+     * @param grp Cache group context.
+     * @param hnd Page store handler.
+     */
+    private void forEachPageStore(CacheGroupContext grp, IgniteInClosureX<Integer> hnd) throws IgniteCheckedException {
+        int parts = grp.affinity().partitions();
+
+        IgnitePageStoreManager pageStoreMgr = ctx.cache().context().pageStore();
+
+        for (int p = 0; p < parts; p++) {
+            if (!pageStoreMgr.exists(grp.groupId(), p))
+                continue;
+
+            hnd.applyx(p);
+        }
+
+        hnd.applyx(PageIdAllocator.INDEX_PARTITION);
+    }
+
+    /**
+     * Cache group reencryption context.
+     */
+    private static class GroupReencryptionContext {
+        /** Partition scanning futures. */
+        private final Map<Integer, IgniteInternalFuture<Void>> futMap = new ConcurrentHashMap<>();
+
+        /** Compound future, that will be completed when all partitions scanned. */
+        private final GridCompoundFuture<Void, Void> compFut = new GridCompoundFuture<>();
+
+        /** Cache group ID. */
+        private final int grpId;
+
+        /** Dirty page skip flag. */
+        private volatile boolean skipDirty;
+
+        /** Future that ends after all partitions are done and a checkpoint is finished. */
+        private final GridFutureAdapter<Void> cpFut = new GridFutureAdapter<Void>() {
+            @Override public boolean cancel() throws IgniteCheckedException {
+                compFut.cancel();
+
+                return onDone(null, null, true);
+            }
+        };
+
+        /**
+         * @param grpId Cache group ID.
+         * @param skipDirty Dirty page skip flag.
+         */
+        public GroupReencryptionContext(int grpId, boolean skipDirty) {
+            this.grpId = grpId;
+            this.skipDirty = skipDirty;
+        }
+
+        /**
+         * @return Cache group ID.
+         */
+        public int groupId() {
+            return grpId;
+        }
+
+        /**
+         * @return Dirty page skip flag.
+         */
+        public boolean skipDirty() {
+            return skipDirty;
+        }
+
+        /**
+         * @param skipDirty Dirty page skip flag.
+         */
+        public void skipDirty(boolean skipDirty) {
+            this.skipDirty = skipDirty;
+        }
+
+        /**
+         * @param partId Partition ID.
+         * @param fut Partition scanning future.
+         */
+        public void add(int partId, IgniteInternalFuture<Void> fut) {
+            compFut.add(fut);
+
+            futMap.put(partId, fut);
+        }
+
+        /**
+         * @return Compound future, that will be completed when all partitions scanned.
+         */
+        public IgniteInternalFuture<Void> initialize() {
+            return compFut.markInitialized();
+        }
+
+        /**
+         * @return Future that ends after all partitions are done and a checkpoint is finished.
+         */
+        public IgniteInternalFuture<Void> finishFuture() {
+            return cpFut;
+        }
+
+        /**
+         * Finish reencryption future.
+         *
+         * @return {@code True} if the future was finished by this call.
+         */
+        public boolean finish() {
+            return cpFut.onDone(compFut.result());
+        }
+
+        /**
+         * Stop reencryption of the specified partition.
+         *
+         * @param partId Partition ID.
+         * @return {@code True} if reencryption was cancelled.
+         * @throws IgniteCheckedException If failed.
+         */
+        public boolean cancel(int partId) throws IgniteCheckedException {
+            IgniteInternalFuture<Void> fut = futMap.get(partId);
+
+            if (fut == null)
+                return false;
+
+            return fut.cancel();
+        }
+
+        /**
+         * @param t Throwable.
+         */
+        public void fail(Throwable t) {
+            cpFut.onDone(t);
+        }
+    }
+
+    /**
+     * Page store scanning task.
+     */
+    private class PageStoreScanTask extends GridFutureAdapter<Void> implements Runnable {
+        /** Partiion ID. */
+        private final int partId;
+
+        /** Cache group reencryption context. */
+        private final GroupReencryptionContext encrCtx;
+
+        /**
+         * @param encrCtx Cache group reencryption context.
+         * @param partId Partition ID.
+         */
+        public PageStoreScanTask(GroupReencryptionContext encrCtx, int partId) {
+            this.encrCtx = encrCtx;
+            this.partId = partId;
+        }
+
+        /** {@inheritDoc} */
+        @Override public synchronized boolean cancel() throws IgniteCheckedException {
+            return onDone();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void run() {
+            try {
+                int grpId = encrCtx.groupId();
+
+                CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+                if (grp == null) {
+                    onDone();
+
+                    return;
+                }
+
+                PageMemoryEx pageMem = (PageMemoryEx)grp.dataRegion().pageMemory();
+                long metaPageId = pageMem.partitionMetaPageId(grpId, partId);
+                long state = ctx.encryption().getEncryptionState(grpId, partId);
+
+                int pageIdx = (int)(state >> 32);
+                int cnt = (int)state;
+
+                if (log.isDebugEnabled()) {
+                    log.debug("Partition reencryption is started [grpId=" + grpId +
+                        ", p=" + partId + ", remain=" + (cnt - pageIdx) + ", total=" + cnt + "]");
+                }
+
+                while (pageIdx < cnt) {
+                    synchronized (this) {
+                        ctx.cache().context().database().checkpointReadLock();
+
+                        try {
+                            if (isDone() || ctx.encryption().getEncryptionState(grpId, partId) == 0)
+                                break;
+
+                            int end = Math.min(pageIdx + batchSize, cnt);
+
+                            do {
+                                long pageId = metaPageId + pageIdx;
+
+                                pageIdx += 1;
+
+                                long page = pageMem.acquirePage(grpId, pageId);
+
+                                try {
+                                    // Can skip rewriting a dirty page if the checkpoint has been completed.
+                                    if (encrCtx.skipDirty() && pageMem.isDirty(grpId, pageId, page))
+                                        continue;
+
+                                    pageMem.writeLock(grpId, pageId, page, true);

Review comment:
       why it here?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -811,6 +1280,67 @@ private void sendGenerateEncryptionKeyRequest(GenerateEncryptionKeyFuture fut) t
         ctx.io().sendToGridTopic(rndNode.id(), TOPIC_GEN_ENC_KEY, req, SYSTEM_POOL);
     }
 
+    /**
+     * @param grpIds Cache group IDs.
+     * @param skipDirty Dirty page skip flag.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void startReencryption(Collection<Integer> grpIds, boolean skipDirty) throws IgniteCheckedException {

Review comment:
       Could you divide the class into subclasses and use a compositional approach?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -1267,6 +1910,322 @@ private String decryptKeyName(byte[] data) {
         });
     }
 
+    /**
+     * Two phase distributed process, that performs cache group encryption key rotation.
+     */
+    private class GroupKeyChangeProcess {

Review comment:
       Move to external class, e.g. with package visibility. GridEncryptionManager more 1000 lines are to match.




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GroupKeyChangeProcess.java
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteFeatures;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.managers.encryption.GridEncryptionManager.EmptyResult;
+import org.apache.ignite.internal.managers.encryption.GridEncryptionManager.KeyChangeFuture;
+import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor;
+import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
+import org.apache.ignite.internal.util.distributed.DistributedProcess;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.future.IgniteFinishedFutureImpl;
+import org.apache.ignite.internal.util.future.IgniteFutureImpl;
+import org.apache.ignite.internal.util.typedef.F;
+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.IgniteFuture;
+import org.apache.ignite.lang.IgniteFutureCancelledException;
+
+import static org.apache.ignite.internal.IgniteFeatures.CACHE_GROUP_KEY_CHANGE;
+import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.CACHE_GROUP_KEY_CHANGE_FINISH;
+import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.CACHE_GROUP_KEY_CHANGE_PREPARE;
+
+/**
+ * A two-phase distributed process that rotates the encryption keys of specified cache groups and initiates
+ * re-encryption of those cache groups.
+ */
+class GroupKeyChangeProcess {
+    /** Grid kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Cache group encyption key change prepare phase. */
+    private final DistributedProcess<ChangeCacheEncryptionRequest, EmptyResult> prepareGKChangeProc;
+
+    /** Cache group encyption key change perform phase. */
+    private final DistributedProcess<ChangeCacheEncryptionRequest, EmptyResult> performGKChangeProc;
+
+    /** Group encryption keys. */
+    private final CacheGroupEncryptionKeys keys;
+
+    /** Cache group key change future. */
+    private volatile GroupKeyChangeFuture fut;
+
+    /** Cache group key change request. */
+    private volatile ChangeCacheEncryptionRequest req;
+
+    /**
+     * @param ctx Grid kernal context.
+     * @param keys Cache group encryption keys.
+     */
+    GroupKeyChangeProcess(GridKernalContext ctx, CacheGroupEncryptionKeys keys) {
+        this.ctx = ctx;
+        this.keys = keys;
+
+        prepareGKChangeProc =
+            new DistributedProcess<>(ctx, CACHE_GROUP_KEY_CHANGE_PREPARE, this::prepare, this::finishPrepare);
+        performGKChangeProc =
+            new DistributedProcess<>(ctx, CACHE_GROUP_KEY_CHANGE_FINISH, this::perform, this::finishPerform);
+    }
+
+    /**
+     * @return {@code True} if operation is still in progress.
+     */
+    public boolean inProgress() {
+        return req != null;
+    }
+
+    /**
+     * @param msg Error message.
+     */
+    public void cancel(String msg) {
+        GridFutureAdapter<Void> keyChangeFut = fut;
+
+        if (keyChangeFut != null && !keyChangeFut.isDone())
+            keyChangeFut.onDone(new IgniteFutureCancelledException(msg));
+    }
+
+    /**
+     * Starts cache group encryption key change process.
+     *
+     * @param cacheOrGrpNames Cache or group names.
+     */
+    public IgniteFuture<Void> start(Collection<String> cacheOrGrpNames) {
+        if (ctx.clientNode())
+            throw new UnsupportedOperationException("Client and daemon nodes can not perform this operation.");
+
+        if (!IgniteFeatures.allNodesSupports(ctx.grid().cluster().nodes(), CACHE_GROUP_KEY_CHANGE))
+            throw new IllegalStateException("Not all nodes in the cluster support this operation.");
+
+        if (!ctx.state().clusterState().state().active())
+            throw new IgniteException("Operation was rejected. The cluster is inactive.");
+
+        IgniteInternalFuture<Void> fut0 = fut;
+
+        if (fut0 != null && !fut0.isDone()) {
+            return new IgniteFinishedFutureImpl<>(new IgniteException("Cache group key change was rejected. " +
+                "The previous change was not completed."));
+        }
+
+        int[] grpIds = new int[cacheOrGrpNames.size()];
+        byte[] keyIds = new byte[grpIds.length];
+
+        int n = 0;
+
+        for (String cacheOrGroupName : cacheOrGrpNames) {
+            CacheGroupDescriptor grpDesc = ctx.cache().cacheGroupDescriptor(CU.cacheId(cacheOrGroupName));
+
+            if (grpDesc == null) {
+                DynamicCacheDescriptor cacheDesc = ctx.cache().cacheDescriptor(cacheOrGroupName);
+
+                if (cacheDesc == null) {
+                    throw new IgniteException("Cache group key change was rejected. " +
+                        "Cache or group \"" + cacheOrGroupName + "\" doesn't exists");
+                }
+
+                int grpId = cacheDesc.groupId();
+
+                grpDesc = ctx.cache().cacheGroupDescriptor(grpId);
+
+                if (grpDesc.sharedGroup()) {
+                    throw new IgniteException("Cache group key change was rejected. " +
+                        "Cache or group \"" + cacheOrGroupName + "\" is a part of group \"" +
+                        grpDesc.groupName() + "\". Provide group name instead of cache name for shared groups.");
+                }
+            }
+
+            if (!grpDesc.config().isEncryptionEnabled()) {
+                throw new IgniteException("Cache group key change was rejected. " +
+                    "Cache or group \"" + cacheOrGroupName + "\" is not encrypted.");
+            }
+
+            if (ctx.encryption().reencryptionInProgress(grpDesc.groupId())) {
+                throw new IgniteException("Cache group key change was rejected. " +
+                    "Cache group reencryption is in progress [grp=" + cacheOrGroupName + "]");
+            }
+
+            grpIds[n] = grpDesc.groupId();
+            keyIds[n] = (byte)(ctx.encryption().groupKey(grpDesc.groupId()).unsignedId() + 1);
+
+            n += 1;
+        }
+
+        byte[][] keys = ctx.encryption().createKeys(grpIds.length).get1().toArray(new byte[grpIds.length][]);
+
+        ChangeCacheEncryptionRequest req =
+            new ChangeCacheEncryptionRequest(grpIds, keys, keyIds, ctx.config().getEncryptionSpi().getMasterKeyName());

Review comment:
       Done, thanks.




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupPageScanner.java
##########
@@ -0,0 +1,453 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.EncryptionConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId;
+import org.apache.ignite.internal.util.BasicRateLimiter;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+
+import static org.apache.ignite.internal.util.IgniteUtils.MB;
+
+/**
+ * Cache group page stores scanner.
+ * Scans a range of pages and marks them as dirty to re-encrypt them with the last encryption key on disk.
+ */
+public class CacheGroupPageScanner implements DbCheckpointListener {
+    /** Encryption configuration. */
+    private final EncryptionConfiguration encrCfg;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to group scanning task. */
+    private final Map<Integer, GroupScanTask> grps = new ConcurrentHashMap<>();
+
+    /** Collection of groups waiting for a checkpoint. */
+    private final Collection<GroupScanTask> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Page scanning speed limiter. */
+    private final BasicRateLimiter limiter;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupPageScanner(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        encrCfg = ctx.config().getDataStorageConfiguration().getEncryptionConfiguration();
+
+        DataStorageConfiguration dsCfg = ctx.config().getDataStorageConfiguration();
+
+        limiter = CU.isPersistenceEnabled(dsCfg) && encrCfg.getReencryptionRateLimit() > 0 ?
+            new BasicRateLimiter(encrCfg.getReencryptionRateLimit() * MB / dsCfg.getPageSize()) : null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCheckpointBegin(Context cpCtx) {
+        Set<GroupScanTask> completeCandidates = new HashSet<>();
+
+        cpWaitGrps.removeIf(completeCandidates::add);
+
+        cpCtx.finishedStateFut().listen(
+            f -> {
+                // Retry if error occurs.
+                if (f.error() != null || f.isCancelled()) {
+                    cpWaitGrps.addAll(completeCandidates);
+
+                    return;
+                }
+
+                lock.lock();
+
+                try {
+                    for (GroupScanTask grpScanTask : completeCandidates) {
+                        grps.remove(grpScanTask.groupId());
+
+                        grpScanTask.onDone();
+
+                        if (log.isInfoEnabled())
+                            log.info("Cache group reencryption is finished [grpId=" + grpScanTask.groupId() + "]");
+                    }
+
+                    if (!grps.isEmpty())
+                        return;
+
+                    ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).
+                        removeCheckpointListener(this);
+                }
+                finally {
+                    lock.unlock();
+                }
+            }
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public void beforeCheckpointBegin(Context cpCtx) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onMarkCheckpointBegin(Context ctx) {
+        // No-op.
+    }
+
+    /**
+     * @return {@code True} If reencryption is disabled.
+     */
+    public boolean disabled() {
+        return encrCfg.isReencryptionDisabled();
+    }
+
+    /**
+     * Schedule scanning partitions.
+     *
+     * @param grpId Cache group ID.
+     */
+    public IgniteInternalFuture<Void> schedule(int grpId) throws IgniteCheckedException {
+        if (disabled())
+            throw new IgniteCheckedException("Reencryption is disabled.");
+
+        CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+        if (grp == null) {
+            if (log.isDebugEnabled())
+                log.debug("Skip reencryption, cache group was destroyed [grp=" + grpId + "]");
+
+            return new GridFinishedFuture<>();
+        }
+
+        lock.lock();
+
+        try {
+            if (stopped)
+                throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
+
+            if (grps.isEmpty())
+                ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).addCheckpointListener(this);
+
+            GroupScanTask prevState = grps.get(grpId);
+
+            if (prevState != null) {
+                if (log.isDebugEnabled())
+                    log.debug("Reencryption already scheduled [grpId=" + grpId + "]");
+
+                return prevState;
+            }
+
+            Set<Integer> parts = new HashSet<>();
+
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) {
+                    if (ctx.encryption().getEncryptionState(grpId, partId) == 0) {
+                        if (log.isDebugEnabled())
+                            log.debug("Skipping partition reencryption [grp=" + grpId + ", p=" + partId + "]");
+
+                        return;
+                    }
+
+                    parts.add(partId);
+                }
+            });
+
+            GroupScanTask grpScan = new GroupScanTask(grp, parts);
+
+            ctx.getSystemExecutorService().submit(grpScan);
+
+            if (log.isInfoEnabled())
+                log.info("Scheduled reencryption [grpId=" + grpId + "]");
+
+            grps.put(grpId, grpScan);
+
+            return grpScan;
+        }
+        finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @return Future that will be completed when all partitions have been scanned and pages have been written to disk.
+     */
+    public IgniteInternalFuture<Void> statusFuture(int grpId) {
+        GroupScanTask ctx0 = grps.get(grpId);
+
+        return ctx0 == null ? new GridFinishedFuture<>() : ctx0;
+    }
+
+    /**
+     * Shutdown scanning and disable new tasks scheduling.
+     */
+    public void stop() throws IgniteCheckedException {
+        lock.lock();
+
+        try {
+            stopped = true;
+
+            for (GroupScanTask grpScanTask : grps.values())
+                grpScanTask.cancel();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Stop scannig the specified partition.
+     *
+     * @param grpId Cache group ID.
+     * @param partId Partition ID.
+     * @return {@code True} if reencryption was cancelled.
+     */
+    public boolean cancel(int grpId, int partId) {
+        GroupScanTask grpScanTask = grps.get(grpId);
+
+        if (grpScanTask == null)
+            return false;
+
+        return grpScanTask.cancel(partId);
+    }
+
+    /**
+     * Collect current number of pages in the specified cache group.
+     *
+     * @param grp Cache group.
+     * @return Partitions with current page count.
+     * @throws IgniteCheckedException If failed.
+     */
+    public long[] pagesCount(CacheGroupContext grp) throws IgniteCheckedException {
+        long[] partStates = new long[grp.affinity().partitions() + 1];
+
+        ctx.cache().context().database().checkpointReadLock();
+
+        try {
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) throws IgniteCheckedException {
+                    int pagesCnt = ctx.cache().context().pageStore().pages(grp.groupId(), partId);
+
+                    partStates[Math.min(partId, partStates.length - 1)] = pagesCnt;
+                }
+            });
+        } finally {
+            ctx.cache().context().database().checkpointReadUnlock();
+        }
+
+        return partStates;
+    }
+
+    /**
+     * @param grp Cache group.
+     * @param hnd Partition handler.
+     */
+    private void forEachPageStore(CacheGroupContext grp, IgniteInClosureX<Integer> hnd) throws IgniteCheckedException {
+        int parts = grp.affinity().partitions();
+
+        IgnitePageStoreManager pageStoreMgr = ctx.cache().context().pageStore();
+
+        for (int p = 0; p < parts; p++) {
+            if (!pageStoreMgr.exists(grp.groupId(), p))
+                continue;
+
+            hnd.applyx(p);
+        }
+
+        hnd.applyx(PageIdAllocator.INDEX_PARTITION);
+    }
+
+    /**
+     * Cache group partition scanning task.
+     */
+    private class GroupScanTask extends GridFutureAdapter<Void> implements Runnable {
+        /** Cache group ID. */
+        private final CacheGroupContext grp;
+
+        /** Partition IDs. */
+        private final Set<Integer> parts;
+
+        /** Page memory. */
+        private final PageMemoryEx pageMem;
+
+        /**
+         * @param grp Cache group.
+         */
+        public GroupScanTask(CacheGroupContext grp, Set<Integer> parts) {
+            this.grp = grp;
+            this.parts = new GridConcurrentHashSet<>(parts);
+
+            pageMem = (PageMemoryEx)grp.dataRegion().pageMemory();
+        }
+
+        /** {@inheritDoc} */
+        @Override public synchronized boolean cancel() throws IgniteCheckedException {

Review comment:
       In this case, `synchronized` guarantees that after the call to cancel, the scan will not continue




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateIndexDataRecord.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.pagemem.wal.record.delta;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIndexMetaIO;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * Meta page delta record, includes encryption status data.
+ */
+public class MetaPageUpdateIndexDataRecord extends PageDeltaRecord {
+    /** Index of the last reencrypted page. */
+    private int encryptPageIdx;
+
+    /** Total pages to be reencrypted. */
+    private int encryptPageCnt;
+
+    /**
+     * @param grpId  Cache group ID.
+     * @param pageId Page ID.
+     * @param encryptPageIdx Index of the last reencrypted page.
+     * @param encryptPageCnt Total pages to be reencrypted.
+     */
+    public MetaPageUpdateIndexDataRecord(int grpId, long pageId, int encryptPageIdx, int encryptPageCnt) {
+        super(grpId, pageId);
+
+        this.encryptPageIdx = encryptPageIdx;
+        this.encryptPageCnt = encryptPageCnt;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        PageIndexMetaIO io = PageIndexMetaIO.VERSIONS.forPage(pageAddr);

Review comment:
       Added type change
   ```
           // Upgrade meta page.
           if (PageIO.getType(pageAddr) == PageIO.T_META)
               PageIO.setType(pageAddr, PageIO.T_INDEX_META);
   ```




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupEncryptionKeys.java
##########
@@ -0,0 +1,376 @@
+/*
+ * 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.managers.encryption;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.CopyOnWriteArrayList;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.encryption.EncryptionSpi;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Serves for managing encryption keys and related datastructure located in the heap.
+ */
+class CacheGroupEncryptionKeys {
+    /** Group encryption keys. */
+    private final Map<Integer, List<GroupKey>> grpKeys = new ConcurrentHashMap<>();
+
+    /**
+     * WAL segments encrypted with previous encryption keys prevent keys from being deleted
+     * until the associated segment is deleted.
+     */
+    private final Collection<TrackedWalSegment> trackedWalSegments = new ConcurrentLinkedQueue<>();

Review comment:
       We only keep track of the last WAL segment encrypted with a specific encryption key, so we only store the current WAL segment index every time the encryption key is rotated.




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -811,6 +1280,67 @@ private void sendGenerateEncryptionKeyRequest(GenerateEncryptionKeyFuture fut) t
         ctx.io().sendToGridTopic(rndNode.id(), TOPIC_GEN_ENC_KEY, req, SYSTEM_POOL);
     }
 
+    /**
+     * @param grpIds Cache group IDs.
+     * @param skipDirty Dirty page skip flag.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void startReencryption(Collection<Integer> grpIds, boolean skipDirty) throws IgniteCheckedException {

Review comment:
       I do not quite understand the essence of the problem with this particular method and what kind of decomposition needs to be done with 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 #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java
##########
@@ -258,6 +258,9 @@
     /** */
     public static final short T_MARKER_PAGE = 33;
 
+    /** */
+    public static final short T_INDEX_META = 34;

Review comment:
       I suggest class PageIndexIO being just T_META with version 2, see no problems with this approach. Just as now, two different classes will add two independent sets of new integers, that's fine. The only thing that's changed is that PageIndexIO will have the same type as its base class.
   
   Types should be immutable while pages are alive. This saves us from a lot of troubles, I wouldn't abandon this tradition. That's exactly what versions are for.
   
   BTW your current code has bugs, you don't mark meta page as dirty in "getOrAllocateCacheMetas" while upgrading, please fix it.




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

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



[GitHub] [ignite] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/encryption/AbstractEncryptionTest.java
##########
@@ -271,4 +313,185 @@ protected boolean checkMasterKeyName(String name) {
 
         return true;
     }
+
+    /**
+     * Load data into cache "{@link #cacheName()}" using node "{@link #GRID_0}".
+     *
+     * @param cnt Count of entries.
+     */
+    protected void loadData(int cnt) {
+        loadData(cacheName(), cnt);
+    }
+
+    /**
+     * Load data into cache using node "{@link #GRID_0}".
+     *
+     * @param cnt Count of entries.
+     * @param cacheName Cache name.
+     */
+    protected void loadData(String cacheName, int cnt) {
+        info("Loading " + cnt + " entries into " + cacheName);
+
+        int start = grid(GRID_0).cache(cacheName).size();
+
+        try (IgniteDataStreamer<Long, Object> streamer = grid(GRID_0).dataStreamer(cacheName)) {
+            for (long i = start; i < (cnt + start); i++)
+                streamer.addData(i, generateValue(i));
+        }
+
+        info("Load data finished");
+    }
+
+    /**
+     * Ensures that all pages of page store have expected encryption key identifier.
+     *
+     * @param grpId Cache group ID.
+     * @param keyId Encryption key ID.
+     * @param timeout Timeout to wait for encryption to complete.
+     * @throws Exception If failed.
+     */
+    protected void checkGroupKey(int grpId, int keyId, long timeout) throws Exception {
+        awaitEncryption(G.allGrids(), grpId, timeout);
+
+        for (Ignite g : G.allGrids()) {
+            info("Validating encryption key [node=" + g.cluster().localNode().id() + ", grp=" + grpId + "]");
+
+            IgniteEx grid = (IgniteEx)g;
+
+            if (grid.context().clientNode())
+                continue;
+
+            GridEncryptionManager encryption = grid.context().encryption();
+
+            assertEquals(grid.localNode().id().toString(), (byte)keyId, encryption.groupKey(grpId).id());
+
+            IgniteInternalFuture<Void> fut = encryption.reencryptionFuture(grpId);
+
+            // The future will be completed after the checkpoint, forcecheckpoint does nothing
+            // if the checkpoint has already been scheduled.
+            GridTestUtils.waitForCondition(() -> {
+                if (fut.isDone())
+                    return true;
+
+                try {
+                    forceCheckpoint(g);
+                }
+                catch (IgniteCheckedException e) {
+                    throw new RuntimeException(e);
+                }
+
+                return fut.isDone();
+            }, timeout);
+
+            assertTrue(fut.isDone());
+
+            CacheGroupContext grp = grid.context().cache().cacheGroup(grpId);
+
+            List<Integer> parts = IntStream.range(0, grp.shared().affinity().affinity(grpId).partitions())
+                .boxed().collect(Collectors.toList());
+
+            parts.add(INDEX_PARTITION);
+
+            int realPageSize = grp.dataRegion().pageMemory().realPageSize(grpId);
+            int encryptionBlockSize = grp.shared().kernalContext().config().getEncryptionSpi().blockSize();
+
+            for (int p : parts) {
+                FilePageStore pageStore =
+                    (FilePageStore)((FilePageStoreManager)grp.shared().pageStore()).getStore(grpId, p);
+
+                if (!pageStore.exists())
+                    continue;
+
+                long state = grid.context().encryption().getEncryptionState(grpId, p);
+
+                String msg = String.format("p=%d, off=%d, total=%d",
+                    p, ReencryptStateUtils.pageIndex(state), ReencryptStateUtils.pageCount(state));
+
+                assertEquals(msg, 0, ReencryptStateUtils.pageCount(state));
+                assertEquals(msg, 0, ReencryptStateUtils.pageIndex(state));
+
+                long startPageId = PageIdUtils.pageId(p, PageIdAllocator.FLAG_DATA, 0);
+
+                int pagesCnt = pageStore.pages();
+                int pageSize = pageStore.getPageSize();
+
+                ByteBuffer pageBuf = ByteBuffer.allocate(pageSize);
+
+                Path path = new File(pageStore.getFileAbsolutePath()).toPath();
+
+                try (FileChannel ch = FileChannel.open(path, StandardOpenOption.READ)) {
+                    for (int n = 0; n < pagesCnt; n++) {
+                        long pageId = startPageId + n;
+                        long pageOff = pageStore.pageOffset(pageId);
+
+                        pageBuf.position(0);
+
+                        ch.position(pageOff);
+                        ch.read(pageBuf);
+
+                        pageBuf.position(realPageSize + encryptionBlockSize);
+
+                        // If crc present

Review comment:
       CRC can be calculated as "0" - it was my mistake.
   But in rare cases, when scanning a partition on a disk, we find a blank page. I added a special check for this case.




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java
##########
@@ -258,6 +258,9 @@
     /** */
     public static final short T_MARKER_PAGE = 33;
 
+    /** */
+    public static final short T_INDEX_META = 34;

Review comment:
       Thanks, I believe I've fixed it.




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -182,7 +215,7 @@
     private volatile boolean recoveryMasterKeyName;
 
     /** Master key change future. Not {@code null} on request initiator. */
-    private MasterKeyChangeFuture masterKeyChangeFut;
+    private volatile KeyChangeFuture masterKeyChangeFut;

Review comment:
       Why it should be volitile?




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java
##########
@@ -258,6 +258,9 @@
     /** */
     public static final short T_MARKER_PAGE = 33;
 
+    /** */
+    public static final short T_INDEX_META = 34;

Review comment:
       If I understood you correctly, you suggest to add PageMetaIOV2 (instead of PageIndexMetaIO) and keep page type T_META?
   PagePartitionMeta will continue to extend PageMetaIO(not v2), is this correct?




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/EncryptedRecord.java
##########
@@ -43,7 +43,7 @@ public EncryptedRecord(int grpId, RecordType plainRecType) {
 
     /** {@inheritDoc} */
     @Override public RecordType type() {
-        return RecordType.ENCRYPTED_RECORD;
+        return RecordType.ENCRYPTED_RECORD_V2;

Review comment:
       Thanks for the suggestion, as far as I understand you are suggesting not to modify any existing WAL record classes.
   Could you please give an example of a compatibility issue that we may face in the future (in theory) that can be avoided by keeping the old "unused" classes?




----------------------------------------------------------------
This is an automated message from the 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] alex-plekhanov commented on a change in pull request #7941: IGNITE-12843

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #7941:
URL: https://github.com/apache/ignite/pull/7941#discussion_r494546092



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateIndexDataRecord.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.pagemem.wal.record.delta;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIndexMetaIO;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * Meta page delta record, includes encryption status data.
+ */
+public class MetaPageUpdateIndexDataRecord extends PageDeltaRecord {
+    /** Index of the last reencrypted page. */
+    private int encryptPageIdx;
+
+    /** Total pages to be reencrypted. */
+    private int encryptPageCnt;
+
+    /**
+     * @param grpId  Cache group ID.
+     * @param pageId Page ID.
+     * @param encryptPageIdx Index of the last reencrypted page.
+     * @param encryptPageCnt Total pages to be reencrypted.
+     */
+    public MetaPageUpdateIndexDataRecord(int grpId, long pageId, int encryptPageIdx, int encryptPageCnt) {
+        super(grpId, pageId);
+
+        this.encryptPageIdx = encryptPageIdx;
+        this.encryptPageCnt = encryptPageCnt;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        PageIndexMetaIO io = PageIndexMetaIO.VERSIONS.forPage(pageAddr);

Review comment:
       Perhaps here we can face with a not upgraded page (in case node failed after page upgrade and before first 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] alex-plekhanov commented on a change in pull request #7941: IGNITE-12843

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #7941:
URL: https://github.com/apache/ignite/pull/7941#discussion_r494843840



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/util/BasicRateLimiterTest.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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;
+
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Test;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Rate limiter tests.
+ */
+public class BasicRateLimiterTest {

Review comment:
       Not included into any test suite

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -627,22 +720,104 @@ public void groupKey(int grpId, byte[] encGrpKey) {
         return withMasterKeyChangeReadLock(() -> getSpi().getMasterKeyName());
     }
 
+    /** {@inheritDoc} */
+    @Override public IgniteFuture<Void> changeCacheGroupKey(Collection<String> cacheOrGrpNames) {
+        A.notEmpty(cacheOrGrpNames, "cacheOrGrpNames");
+
+        synchronized (opsMux) {
+            if (stopped) {
+                return new IgniteFinishedFutureImpl<>(new IgniteException("Cache group key change was rejected. " +
+                    "Node is stopping."));
+            }
+
+            return grpKeyChangeProc.start(cacheOrGrpNames);
+        }
+    }
+
+    /**
+     * @param grpIds Cache group IDs.
+     * @param keyIds Encryption key IDs.
+     * @param keys Encryption keys.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected void changeCacheGroupKeyLocal(int[] grpIds, byte[] keyIds, byte[][] keys) throws IgniteCheckedException {
+        Map<Integer, Byte> encryptionStatus = U.newHashMap(grpIds.length);
+
+        for (int i = 0; i < grpIds.length; i++)
+            encryptionStatus.put(grpIds[i], keyIds[i]);
+
+        WALPointer ptr = ctx.cache().context().wal().log(new ReencryptionStartRecord(encryptionStatus));
+
+        if (ptr != null)
+            ctx.cache().context().wal().flush(ptr, false);
+
+        for (int i = 0; i < grpIds.length; i++) {
+            int grpId = grpIds[i];
+
+            CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+            if (grp == null)
+                continue;
+
+            int newKeyId = keyIds[i] & 0xff;
+
+            synchronized (metaStorageMux) {
+                // Set new key as key for writing.
+                GroupKey prevGrpKey = grpKeys.changeActiveKey(grpId, newKeyId);
+
+                writeGroupKeysToMetaStore(grpId);
+
+                if (ptr != null) {
+                    grpKeys.reserveWalKey(grpId, prevGrpKey.unsignedId(), ctx.cache().context().wal().currentSegment());
+
+                    writeTrackedWalIdxsToMetaStore();
+                }
+            }
+
+            reencryptGroups.put(grpId, pageScanner.pagesCount(grp));
+
+            if (log.isInfoEnabled())
+                log.info("New encryption key for group was added [grpId=" + grpId + ", keyId=" + newKeyId + "]");
+        }
+
+        startReencryption(encryptionStatus.keySet());
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @return Future that will be completed when reencryption of the specified group is finished.
+     */
+    public IgniteInternalFuture<Void> reencryptionFuture(int grpId) {
+        return pageScanner.statusFuture(grpId);
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @return {@code True} If the specified cache group should be re-encrypted.
+     */
+    public boolean reencryptionRequired(int grpId) {

Review comment:
       All usages of this method want to check that reencryption in progress. Let's rename it to something like `reencryptionInProgress`

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/util/BasicRateLimiterTest.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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;
+
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Test;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Rate limiter tests.
+ */
+public class BasicRateLimiterTest {
+    /**
+     * Check rate limit with multiple threads.
+     */
+    @Test
+    public void checkLimitMultithreaded() throws Exception {
+        int opsPerSec = 1_000;
+        int totalOps = 10_000;
+
+        BasicRateLimiter limiter = new BasicRateLimiter(opsPerSec);
+
+        int threads = Runtime.getRuntime().availableProcessors();
+
+        CyclicBarrier ready = new CyclicBarrier(threads + 1);
+
+        AtomicInteger cntr = new AtomicInteger();
+
+        IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(() -> {
+            ready.await();
+
+            do {
+                limiter.acquire(1);
+            }
+            while (!Thread.currentThread().isInterrupted() && cntr.incrementAndGet() < totalOps);
+
+            return null;
+        }, threads, "worker");
+
+        ready.await();
+
+        long startTime = System.currentTimeMillis();
+
+        fut.get();
+
+        long timeSpent = System.currentTimeMillis() - startTime;
+
+        assertEquals(totalOps / opsPerSec, SECONDS.convert(timeSpent, MILLISECONDS));
+    }
+
+    /**
+     * Check that the average speed is limited correctly even if we are acquiring more permits than allowed per second.
+     */
+    @Test
+    public void checkAcquireWithOverflow() throws IgniteInterruptedCheckedException {
+        double permitsPerSec = 0.5;
+        int permitsPerOp = 1;
+        int totalOps = 5;
+
+        BasicRateLimiter limiter = new BasicRateLimiter(permitsPerSec);
+
+        long startTime = System.currentTimeMillis();
+
+        for (int i = 0; i <= totalOps; i++)

Review comment:
       Actually, you check `totalOps + 1` operations here

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/util/BasicRateLimiterTest.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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;
+
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Test;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Rate limiter tests.
+ */
+public class BasicRateLimiterTest {
+    /**
+     * Check rate limit with multiple threads.
+     */
+    @Test
+    public void checkLimitMultithreaded() throws Exception {
+        int opsPerSec = 1_000;
+        int totalOps = 10_000;
+
+        BasicRateLimiter limiter = new BasicRateLimiter(opsPerSec);
+
+        int threads = Runtime.getRuntime().availableProcessors();
+
+        CyclicBarrier ready = new CyclicBarrier(threads + 1);
+
+        AtomicInteger cntr = new AtomicInteger();
+
+        IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(() -> {
+            ready.await();
+
+            do {
+                limiter.acquire(1);
+            }
+            while (!Thread.currentThread().isInterrupted() && cntr.incrementAndGet() < totalOps);
+
+            return null;
+        }, threads, "worker");
+
+        ready.await();
+
+        long startTime = System.currentTimeMillis();
+
+        fut.get();
+
+        long timeSpent = System.currentTimeMillis() - startTime;
+
+        assertEquals(totalOps / opsPerSec, SECONDS.convert(timeSpent, MILLISECONDS));

Review comment:
       In some circumstances, I think `timeSpent` can be a little bit less than 10 seconds




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupReencryption.java
##########
@@ -0,0 +1,583 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+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.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+import org.apache.ignite.thread.OomExceptionHandler;
+
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_REENCRYPTION_BATCH_SIZE;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_REENCRYPTION_DISABLED;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_REENCRYPTION_THREAD_POOL_SIZE;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_REENCRYPTION_THROTTLE;
+
+/**
+ * Cache group reencryption manager.
+ */
+public class CacheGroupReencryption implements DbCheckpointListener {
+    /** Thread prefix for reencryption tasks. */
+    private static final String REENCRYPT_THREAD_PREFIX = "reencrypt";
+
+    /** Max amount of pages that will be read into memory under checkpoint lock. */
+    private final int batchSize = IgniteSystemProperties.getInteger(IGNITE_REENCRYPTION_BATCH_SIZE, 1_000);
+
+    /** Timeout between batches. */
+    private final long timeoutBetweenBatches = IgniteSystemProperties.getLong(IGNITE_REENCRYPTION_THROTTLE, 0);
+
+    /** Disable background reencryption. */
+    private final boolean disabled = IgniteSystemProperties.getBoolean(IGNITE_REENCRYPTION_DISABLED, false);
+
+    /** Number of threads for partition scanning. */
+    private final int threadsCnt = IgniteSystemProperties.getInteger(IGNITE_REENCRYPTION_THREAD_POOL_SIZE,
+        Runtime.getRuntime().availableProcessors());
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to reencryption context. */
+    private final Map<Integer, GroupReencryptionContext> grps = new ConcurrentHashMap<>();
+
+    /** Queue of groups waiting for a checkpoint. */
+    private final Queue<Integer> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Executor to start partition scan tasks. */
+    private final IgniteThreadPoolExecutor execSvc;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupReencryption(GridKernalContext ctx) {

Review comment:
       Context is not fully initialized here, moreover I'm not sure that greatly increasing the number of arguments will improve the code. I understand that we want to loose the coupling, but I am not sure that it's easy to do here (class requires EncryptionManager, PageStoreManager, GridCacheDatabaseSharedManager and GridCacheProcessor at least).




----------------------------------------------------------------
This is an automated message from the 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] alex-plekhanov commented on a change in pull request #7941: IGNITE-12843

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #7941:
URL: https://github.com/apache/ignite/pull/7941#discussion_r498207521



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -627,22 +739,104 @@ public void groupKey(int grpId, byte[] encGrpKey) {
         return withMasterKeyChangeReadLock(() -> getSpi().getMasterKeyName());
     }
 
+    /** {@inheritDoc} */
+    @Override public IgniteFuture<Void> changeCacheGroupKey(Collection<String> cacheOrGrpNames) {
+        A.notEmpty(cacheOrGrpNames, "cacheOrGrpNames");
+
+        synchronized (opsMux) {
+            if (stopped) {
+                return new IgniteFinishedFutureImpl<>(new IgniteException("Cache group key change was rejected. " +
+                    "Node is stopping."));
+            }
+
+            return grpKeyChangeProc.start(cacheOrGrpNames);
+        }
+    }
+
+    /**
+     * @param grpIds Cache group IDs.
+     * @param keyIds Encryption key IDs.
+     * @param keys Encryption keys.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected void changeCacheGroupKeyLocal(int[] grpIds, byte[] keyIds, byte[][] keys) throws IgniteCheckedException {
+        Map<Integer, Byte> encryptionStatus = U.newHashMap(grpIds.length);
+
+        for (int i = 0; i < grpIds.length; i++)
+            encryptionStatus.put(grpIds[i], keyIds[i]);
+
+        WALPointer ptr = ctx.cache().context().wal().log(new ReencryptionStartRecord(encryptionStatus));
+
+        if (ptr != null)
+            ctx.cache().context().wal().flush(ptr, false);
+
+        for (int i = 0; i < grpIds.length; i++) {
+            int grpId = grpIds[i];
+            int newKeyId = keyIds[i] & 0xff;
+
+            synchronized (metaStorageMux) {
+                // Set new key as key for writing. Note that we cannot pass the encrypted key here because the master
+                // key may have changed in which case we will not be able to decrypt the cache encryption key.
+                GroupKey prevGrpKey = grpKeys.changeActiveKey(grpId, newKeyId);
+
+                List<GroupKeyEncrypted> keysEncrypted = withMasterKeyChangeReadLock(() -> grpKeys.getAll(grpId));

Review comment:
       Wrong lock order again




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupPageScanner.java
##########
@@ -0,0 +1,479 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId;
+import org.apache.ignite.internal.util.BasicRateLimiter;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+import org.apache.ignite.thread.OomExceptionHandler;
+
+import static org.apache.ignite.internal.util.IgniteUtils.MB;
+
+/**
+ * Cache group page stores scanner.
+ * Scans a range of pages and marks them as dirty to re-encrypt them with the last encryption key on disk.
+ */
+public class CacheGroupPageScanner implements CheckpointListener {
+    /** Thread prefix for scanning tasks. */
+    private static final String REENCRYPT_THREAD_PREFIX = "reencrypt";
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to group scanning task. */
+    private final Map<Integer, GroupScanTask> grps = new ConcurrentHashMap<>();
+
+    /** Collection of groups waiting for a checkpoint. */
+    private final Collection<GroupScanTask> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Page scanning speed limiter. */
+    private final BasicRateLimiter limiter;
+
+    /** Single-threaded executor to run cache group scan task. */
+    private final ThreadPoolExecutor singleExecSvc;
+
+    /** Number of pages that is scanned during reencryption under checkpoint lock. */
+    private final int batchSize;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupPageScanner(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        DataStorageConfiguration dsCfg = ctx.config().getDataStorageConfiguration();
+
+        if (!CU.isPersistenceEnabled(dsCfg)) {
+            batchSize = -1;
+            limiter = null;
+            singleExecSvc = null;
+
+            return;
+        }
+
+        double rateLimit = dsCfg.getEncryptionConfiguration().getReencryptionRateLimit();
+
+        limiter = rateLimit > 0 ? new BasicRateLimiter(rateLimit * MB /
+            (dsCfg.getPageSize() == 0 ? DataStorageConfiguration.DFLT_PAGE_SIZE : dsCfg.getPageSize())) : null;
+
+        batchSize = dsCfg.getEncryptionConfiguration().getReencryptionBatchSize();
+
+        singleExecSvc = new IgniteThreadPoolExecutor(REENCRYPT_THREAD_PREFIX,
+            ctx.igniteInstanceName(),
+            1,
+            1,
+            IgniteConfiguration.DFLT_THREAD_KEEP_ALIVE_TIME,
+            new LinkedBlockingQueue<>(),
+            GridIoPolicy.SYSTEM_POOL,
+            new OomExceptionHandler(ctx));
+
+        singleExecSvc.allowCoreThreadTimeOut(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCheckpointBegin(Context cpCtx) {
+        Set<GroupScanTask> completeCandidates = new HashSet<>();
+
+        cpWaitGrps.removeIf(completeCandidates::add);
+
+        cpCtx.finishedStateFut().listen(
+            f -> {
+                // Retry if error occurs.
+                if (f.error() != null || f.isCancelled()) {
+                    cpWaitGrps.addAll(completeCandidates);
+
+                    return;
+                }
+
+                lock.lock();
+
+                try {
+                    for (GroupScanTask grpScanTask : completeCandidates) {
+                        grps.remove(grpScanTask.groupId());
+
+                        grpScanTask.onDone();
+
+                        if (log.isInfoEnabled())
+                            log.info("Cache group reencryption is finished [grpId=" + grpScanTask.groupId() + "]");
+                    }
+
+                    if (!grps.isEmpty())
+                        return;
+
+                    ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).
+                        removeCheckpointListener(this);
+                }
+                finally {
+                    lock.unlock();
+                }
+            }
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public void beforeCheckpointBegin(Context cpCtx) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onMarkCheckpointBegin(Context ctx) {
+        // No-op.
+    }
+
+    /**
+     * Schedule scanning partitions.
+     *
+     * @param grpId Cache group ID.
+     */
+    public IgniteInternalFuture<Void> schedule(int grpId) throws IgniteCheckedException {
+        CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+        if (grp == null || !grp.affinityNode()) {
+            if (log.isInfoEnabled())
+                log.info("Skip reencryption, cache group doesn't exist on the local node [grp=" + grpId + "]");
+
+            return new GridFinishedFuture<>();
+        }
+
+        lock.lock();
+
+        try {
+            if (stopped)
+                throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
+
+            if (grps.isEmpty())
+                ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).addCheckpointListener(this);
+
+            GroupScanTask prevState = grps.get(grpId);
+
+            if (prevState != null && !prevState.isDone()) {
+                if (log.isDebugEnabled())
+                    log.debug("Reencryption already scheduled [grpId=" + grpId + "]");
+
+                return prevState;
+            }
+
+            Set<Integer> parts = new HashSet<>();
+
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) {
+                    if (ctx.encryption().getEncryptionState(grpId, partId) == 0) {
+                        if (log.isDebugEnabled())
+                            log.debug("Skipping partition reencryption [grp=" + grpId + ", p=" + partId + "]");
+
+                        return;
+                    }
+
+                    parts.add(partId);
+                }
+            });
+
+            GroupScanTask grpScan = new GroupScanTask(grp, parts);
+
+            singleExecSvc.submit(grpScan);
+
+            if (log.isInfoEnabled())
+                log.info("Scheduled reencryption [grpId=" + grpId + "]");
+
+            grps.put(grpId, grpScan);
+
+            return grpScan;
+        }
+        finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @return Future that will be completed when all partitions have been scanned and pages have been written to disk.
+     */
+    public IgniteInternalFuture<Void> statusFuture(int grpId) {
+        GroupScanTask grpScanTask = grps.get(grpId);
+
+        return grpScanTask == null ? new GridFinishedFuture<>() : grpScanTask;
+    }
+
+    /**
+     * Shutdown scanning and disable new tasks scheduling.
+     */
+    public void stop() throws IgniteCheckedException {
+        lock.lock();
+
+        try {
+            stopped = true;
+
+            for (GroupScanTask grpScanTask : grps.values())
+                grpScanTask.cancel();
+
+            if (singleExecSvc != null)
+                singleExecSvc.shutdownNow();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Stop scannig the specified partition.
+     *
+     * @param grpId Cache group ID.
+     * @param partId Partition ID.
+     * @return {@code True} if reencryption was cancelled.
+     */
+    public boolean excludePartition(int grpId, int partId) {
+        GroupScanTask grpScanTask = grps.get(grpId);
+
+        if (grpScanTask == null)
+            return false;
+
+        return grpScanTask.excludePartition(partId);
+    }
+
+    /**
+     * Collect current number of pages in the specified cache group.
+     *
+     * @param grp Cache group.
+     * @return Partitions with current page count.
+     * @throws IgniteCheckedException If failed.
+     */
+    public long[] pagesCount(CacheGroupContext grp) throws IgniteCheckedException {
+        // The last element of the array is used to store the status of the index partition.
+        long[] partStates = new long[grp.affinity().partitions() + 1];
+
+        ctx.cache().context().database().checkpointReadLock();
+
+        try {
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) throws IgniteCheckedException {
+                    int pagesCnt = ctx.cache().context().pageStore().pages(grp.groupId(), partId);
+
+                    partStates[Math.min(partId, partStates.length - 1)] = pagesCnt;
+                }
+            });
+        } finally {
+            ctx.cache().context().database().checkpointReadUnlock();
+        }
+
+        return partStates;
+    }
+
+    /**
+     * @param grp Cache group.
+     * @param hnd Partition handler.
+     */
+    private void forEachPageStore(CacheGroupContext grp, IgniteInClosureX<Integer> hnd) throws IgniteCheckedException {
+        int parts = grp.affinity().partitions();
+
+        IgnitePageStoreManager pageStoreMgr = ctx.cache().context().pageStore();
+
+        for (int p = 0; p < parts; p++) {
+            if (!pageStoreMgr.exists(grp.groupId(), p))
+                continue;
+
+            hnd.applyx(p);
+        }
+
+        hnd.applyx(PageIdAllocator.INDEX_PARTITION);
+    }
+
+    /**
+     * Cache group partition scanning task.
+     */
+    private class GroupScanTask extends GridFutureAdapter<Void> implements Runnable {
+        /** Cache group ID. */
+        private final CacheGroupContext grp;
+
+        /** Partition IDs. */
+        private final Set<Integer> parts;
+
+        /** Page memory. */
+        private final PageMemoryEx pageMem;
+
+        /**
+         * @param grp Cache group.
+         */
+        public GroupScanTask(CacheGroupContext grp, Set<Integer> parts) {
+            this.grp = grp;
+            this.parts = new GridConcurrentHashSet<>(parts);
+
+            pageMem = (PageMemoryEx)grp.dataRegion().pageMemory();
+        }
+
+        /** {@inheritDoc} */
+        @Override public synchronized boolean cancel() throws IgniteCheckedException {
+            return onCancelled();
+        }
+
+        /**
+         * Stop reencryption of the specified partition.
+         *
+         * @param partId Partition ID.
+         * @return {@code True} if reencryption was cancelled.
+         */
+        public synchronized boolean excludePartition(int partId) {
+            return parts.remove(partId);
+        }
+
+        /**
+         * @return Cache group ID.
+         */
+        public int groupId() {
+            return grp.groupId();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void run() {
+            try {
+                for (int partId : parts) {
+                    long state = ctx.encryption().getEncryptionState(grp.groupId(), partId);
+
+                    if (state == 0)
+                        continue;
+
+                    scanPartition(partId, ReencryptStateUtils.pageIndex(state), ReencryptStateUtils.pageCount(state));
+
+                    if (isDone())
+                        return;
+                }
+
+                boolean added = cpWaitGrps.add(this);

Review comment:
       @vldpyatkov thanks, seems that we should use beforeCheckpointBegin instead of onCheckpointBegin, 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GroupKeyChangeProcess.java
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteFeatures;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.managers.encryption.GridEncryptionManager.EmptyResult;
+import org.apache.ignite.internal.managers.encryption.GridEncryptionManager.KeyChangeFuture;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
+import org.apache.ignite.internal.util.distributed.DistributedProcess;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.future.IgniteFinishedFutureImpl;
+import org.apache.ignite.internal.util.future.IgniteFutureImpl;
+import org.apache.ignite.internal.util.typedef.F;
+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.IgniteFuture;
+import org.apache.ignite.lang.IgniteFutureCancelledException;
+
+import static org.apache.ignite.internal.IgniteFeatures.CACHE_GROUP_KEY_CHANGE;
+import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.CACHE_GROUP_KEY_CHANGE_FINISH;
+import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.CACHE_GROUP_KEY_CHANGE_PREPARE;
+
+/**
+ * Two phase distributed process, that performs cache group encryption key rotation.
+ */
+class GroupKeyChangeProcess {

Review comment:
       new javadoc
   `
   Cache group page stores scanner.
   Scans a range of pages and marks them as dirty to re-encrypt them with the last encryption key on disk.
   `
   




----------------------------------------------------------------
This is an automated message from the 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] sk0x50 commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/EncryptedRecord.java
##########
@@ -43,7 +43,7 @@ public EncryptedRecord(int grpId, RecordType plainRecType) {
 
     /** {@inheritDoc} */
     @Override public RecordType type() {
-        return RecordType.ENCRYPTED_RECORD;
+        return RecordType.ENCRYPTED_RECORD_V2;

Review comment:
       > (see change in RecordDataV1Serializer.readEncryptedData)
   
   Yep, I saw this, but it does not explain why you modify the existing WAL record.
   
   In my understanding, WAL records must be immutable in order to avoid compatibility issues in the future.
   I would prefer adding a new WAL record (it can extend an existing one).
   Perhaps, RecordDataV1Serializer implementation will be cleaner in that case.




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/EncryptedRecord.java
##########
@@ -43,7 +43,7 @@ public EncryptedRecord(int grpId, RecordType plainRecType) {
 
     /** {@inheritDoc} */
     @Override public RecordType type() {
-        return RecordType.ENCRYPTED_RECORD;
+        return RecordType.ENCRYPTED_RECORD_V2;

Review comment:
       For binary compatibility.
   For ENCRYPTED_RECORD_V2 we reading 1 extra byte (encryption key identifier).
   (see change in RecordDataV1Serializer.readEncryptedData)




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupPageScanner.java
##########
@@ -0,0 +1,479 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId;
+import org.apache.ignite.internal.util.BasicRateLimiter;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+import org.apache.ignite.thread.OomExceptionHandler;
+
+import static org.apache.ignite.internal.util.IgniteUtils.MB;
+
+/**
+ * Cache group page stores scanner.
+ * Scans a range of pages and marks them as dirty to re-encrypt them with the last encryption key on disk.
+ */
+public class CacheGroupPageScanner implements CheckpointListener {
+    /** Thread prefix for scanning tasks. */
+    private static final String REENCRYPT_THREAD_PREFIX = "reencrypt";
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to group scanning task. */
+    private final Map<Integer, GroupScanTask> grps = new ConcurrentHashMap<>();
+
+    /** Collection of groups waiting for a checkpoint. */
+    private final Collection<GroupScanTask> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Page scanning speed limiter. */
+    private final BasicRateLimiter limiter;
+
+    /** Single-threaded executor to run cache group scan task. */
+    private final ThreadPoolExecutor singleExecSvc;
+
+    /** Number of pages that is scanned during reencryption under checkpoint lock. */
+    private final int batchSize;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupPageScanner(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        DataStorageConfiguration dsCfg = ctx.config().getDataStorageConfiguration();
+
+        if (!CU.isPersistenceEnabled(dsCfg)) {
+            batchSize = -1;
+            limiter = null;
+            singleExecSvc = null;
+
+            return;
+        }
+
+        double rateLimit = dsCfg.getEncryptionConfiguration().getReencryptionRateLimit();
+
+        limiter = rateLimit > 0 ? new BasicRateLimiter(rateLimit * MB /
+            (dsCfg.getPageSize() == 0 ? DataStorageConfiguration.DFLT_PAGE_SIZE : dsCfg.getPageSize())) : null;
+
+        batchSize = dsCfg.getEncryptionConfiguration().getReencryptionBatchSize();
+
+        singleExecSvc = new IgniteThreadPoolExecutor(REENCRYPT_THREAD_PREFIX,
+            ctx.igniteInstanceName(),
+            1,
+            1,
+            IgniteConfiguration.DFLT_THREAD_KEEP_ALIVE_TIME,
+            new LinkedBlockingQueue<>(),
+            GridIoPolicy.SYSTEM_POOL,
+            new OomExceptionHandler(ctx));
+
+        singleExecSvc.allowCoreThreadTimeOut(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCheckpointBegin(Context cpCtx) {
+        Set<GroupScanTask> completeCandidates = new HashSet<>();
+
+        cpWaitGrps.removeIf(completeCandidates::add);
+
+        cpCtx.finishedStateFut().listen(
+            f -> {
+                // Retry if error occurs.
+                if (f.error() != null || f.isCancelled()) {
+                    cpWaitGrps.addAll(completeCandidates);
+
+                    return;
+                }
+
+                lock.lock();
+
+                try {
+                    for (GroupScanTask grpScanTask : completeCandidates) {
+                        grps.remove(grpScanTask.groupId());
+
+                        grpScanTask.onDone();
+
+                        if (log.isInfoEnabled())
+                            log.info("Cache group reencryption is finished [grpId=" + grpScanTask.groupId() + "]");
+                    }
+
+                    if (!grps.isEmpty())
+                        return;
+
+                    ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).
+                        removeCheckpointListener(this);
+                }
+                finally {
+                    lock.unlock();
+                }
+            }
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public void beforeCheckpointBegin(Context cpCtx) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onMarkCheckpointBegin(Context ctx) {
+        // No-op.
+    }
+
+    /**
+     * Schedule scanning partitions.
+     *
+     * @param grpId Cache group ID.
+     */
+    public IgniteInternalFuture<Void> schedule(int grpId) throws IgniteCheckedException {
+        CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+        if (grp == null || !grp.affinityNode()) {
+            if (log.isInfoEnabled())
+                log.info("Skip reencryption, cache group doesn't exist on the local node [grp=" + grpId + "]");
+
+            return new GridFinishedFuture<>();
+        }
+
+        lock.lock();
+
+        try {
+            if (stopped)
+                throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
+
+            if (grps.isEmpty())
+                ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).addCheckpointListener(this);
+
+            GroupScanTask prevState = grps.get(grpId);
+
+            if (prevState != null && !prevState.isDone()) {
+                if (log.isDebugEnabled())
+                    log.debug("Reencryption already scheduled [grpId=" + grpId + "]");
+
+                return prevState;
+            }
+
+            Set<Integer> parts = new HashSet<>();
+
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) {
+                    if (ctx.encryption().getEncryptionState(grpId, partId) == 0) {
+                        if (log.isDebugEnabled())
+                            log.debug("Skipping partition reencryption [grp=" + grpId + ", p=" + partId + "]");
+
+                        return;
+                    }
+
+                    parts.add(partId);
+                }
+            });
+
+            GroupScanTask grpScan = new GroupScanTask(grp, parts);
+
+            singleExecSvc.submit(grpScan);
+
+            if (log.isInfoEnabled())
+                log.info("Scheduled reencryption [grpId=" + grpId + "]");
+
+            grps.put(grpId, grpScan);
+
+            return grpScan;
+        }
+        finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @return Future that will be completed when all partitions have been scanned and pages have been written to disk.
+     */
+    public IgniteInternalFuture<Void> statusFuture(int grpId) {
+        GroupScanTask grpScanTask = grps.get(grpId);
+
+        return grpScanTask == null ? new GridFinishedFuture<>() : grpScanTask;
+    }
+
+    /**
+     * Shutdown scanning and disable new tasks scheduling.
+     */
+    public void stop() throws IgniteCheckedException {
+        lock.lock();
+
+        try {
+            stopped = true;
+
+            for (GroupScanTask grpScanTask : grps.values())
+                grpScanTask.cancel();
+
+            if (singleExecSvc != null)
+                singleExecSvc.shutdownNow();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Stop scannig the specified partition.
+     *
+     * @param grpId Cache group ID.
+     * @param partId Partition ID.
+     * @return {@code True} if reencryption was cancelled.
+     */
+    public boolean excludePartition(int grpId, int partId) {
+        GroupScanTask grpScanTask = grps.get(grpId);
+
+        if (grpScanTask == null)
+            return false;
+
+        return grpScanTask.excludePartition(partId);
+    }
+
+    /**
+     * Collect current number of pages in the specified cache group.
+     *
+     * @param grp Cache group.
+     * @return Partitions with current page count.
+     * @throws IgniteCheckedException If failed.
+     */
+    public long[] pagesCount(CacheGroupContext grp) throws IgniteCheckedException {
+        // The last element of the array is used to store the status of the index partition.
+        long[] partStates = new long[grp.affinity().partitions() + 1];
+
+        ctx.cache().context().database().checkpointReadLock();
+
+        try {
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) throws IgniteCheckedException {
+                    int pagesCnt = ctx.cache().context().pageStore().pages(grp.groupId(), partId);
+
+                    partStates[Math.min(partId, partStates.length - 1)] = pagesCnt;
+                }
+            });
+        } finally {
+            ctx.cache().context().database().checkpointReadUnlock();
+        }
+
+        return partStates;
+    }
+
+    /**
+     * @param grp Cache group.
+     * @param hnd Partition handler.
+     */
+    private void forEachPageStore(CacheGroupContext grp, IgniteInClosureX<Integer> hnd) throws IgniteCheckedException {
+        int parts = grp.affinity().partitions();
+
+        IgnitePageStoreManager pageStoreMgr = ctx.cache().context().pageStore();
+
+        for (int p = 0; p < parts; p++) {
+            if (!pageStoreMgr.exists(grp.groupId(), p))
+                continue;
+
+            hnd.applyx(p);
+        }
+
+        hnd.applyx(PageIdAllocator.INDEX_PARTITION);
+    }
+
+    /**
+     * Cache group partition scanning task.
+     */
+    private class GroupScanTask extends GridFutureAdapter<Void> implements Runnable {
+        /** Cache group ID. */
+        private final CacheGroupContext grp;
+
+        /** Partition IDs. */
+        private final Set<Integer> parts;
+
+        /** Page memory. */
+        private final PageMemoryEx pageMem;
+
+        /**
+         * @param grp Cache group.
+         */
+        public GroupScanTask(CacheGroupContext grp, Set<Integer> parts) {
+            this.grp = grp;
+            this.parts = new GridConcurrentHashSet<>(parts);
+
+            pageMem = (PageMemoryEx)grp.dataRegion().pageMemory();
+        }
+
+        /** {@inheritDoc} */
+        @Override public synchronized boolean cancel() throws IgniteCheckedException {
+            return onCancelled();
+        }
+
+        /**
+         * Stop reencryption of the specified partition.
+         *
+         * @param partId Partition ID.
+         * @return {@code True} if reencryption was cancelled.
+         */
+        public synchronized boolean excludePartition(int partId) {
+            return parts.remove(partId);
+        }
+
+        /**
+         * @return Cache group ID.
+         */
+        public int groupId() {
+            return grp.groupId();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void run() {
+            try {
+                for (int partId : parts) {
+                    long state = ctx.encryption().getEncryptionState(grp.groupId(), partId);
+
+                    if (state == 0)
+                        continue;
+
+                    scanPartition(partId, ReencryptStateUtils.pageIndex(state), ReencryptStateUtils.pageCount(state));
+
+                    if (isDone())
+                        return;
+                }
+
+                boolean added = cpWaitGrps.add(this);

Review comment:
       cpWaitGrps is a collection of groups waiting for a checkpoint to start.
   
   PageScanner performs the following steps:
   1. mark all existing at this moment pages in the group as dirty
   2. wait for the start of the checkpoint (onCheckpointBegin)
   3. wait for finishedStateFut() of the current checkpoint to complete
   
   I though that after finishedStateFut finishes, all pages (dirty at the time of starting the checkpoint) are written to disk, am I missing something?




----------------------------------------------------------------
This is an automated message from the 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 #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java
##########
@@ -258,6 +258,9 @@
     /** */
     public static final short T_MARKER_PAGE = 33;
 
+    /** */
+    public static final short T_INDEX_META = 34;

Review comment:
       Yes, this is correct. Naming of the classes doesn't bother me too much so I was fine with "Indexing" instead of V2 (although that page is also referred as "cache group meta page" sometimes, like in PageMemoryEx#metaPageId, for example).
   
   I think I understand the core problem, inheritance between MetaIO and PartitionMetaIO should have been replaced with extension of the same abstract superclass back in the day. But with current faulty design people just get confused :(




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PagePartitionMetaIOV3.java
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.tree.io;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageUtils;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState;
+import org.apache.ignite.internal.util.GridStringBuilder;
+
+/**
+ * IO for partition metadata pages.
+ * Persistent partition contains it's own PendingTree.
+ */
+public class PagePartitionMetaIOV3 extends PagePartitionMetaIOV2 {
+    /** Last reencrypted page index offset. */
+    private static final int ENCRYPT_PAGE_IDX_OFF = END_OF_PARTITION_PAGE_META_V2;
+
+    /** Total pages to be reencrypted offset. */
+    private static final int ENCRYPT_PAGE_MAX_OFF = ENCRYPT_PAGE_IDX_OFF + 4;
+
+    /**
+     * @param ver Version.
+     */
+    public PagePartitionMetaIOV3(int ver) {
+        super(ver);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void initNewPage(long pageAddr, long pageId, int pageSize) {
+        super.initNewPage(pageAddr, pageId, pageSize);
+
+        setEncryptedPageIndex(pageAddr, 0);
+        setEncryptedPageCount(pageAddr, 0);
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @return Index of the last reencrypted page.
+     */
+    public int getEncryptedPageIndex(long pageAddr) {
+        return PageUtils.getInt(pageAddr, ENCRYPT_PAGE_IDX_OFF);
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @param pageIdx Index of the last reencrypted page.
+     *
+     * @return {@code true} if value has changed as a result of this method's invocation.
+     */
+    public boolean setEncryptedPageIndex(long pageAddr, int pageIdx) {
+        if (getEncryptedPageIndex(pageAddr) == pageIdx)
+            return false;
+
+        PageUtils.putLong(pageAddr, ENCRYPT_PAGE_IDX_OFF, pageIdx);
+
+        return true;
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @return Total pages to be reencrypted.
+     */
+    public int getEncryptedPageCount(long pageAddr) {
+        return PageUtils.getInt(pageAddr, ENCRYPT_PAGE_MAX_OFF);
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @param pagesCnt Total pages to be reencrypted.
+     *
+     * @return {@code true} if value has changed as a result of this method's invocation.
+     */
+    public boolean setEncryptedPageCount(long pageAddr, int pagesCnt) {
+        if (getEncryptedPageCount(pageAddr) == pagesCnt)
+            return false;
+
+        PageUtils.putInt(pageAddr, ENCRYPT_PAGE_MAX_OFF, pagesCnt);
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void printPage(long pageAddr, int pageSize, GridStringBuilder sb) throws IgniteCheckedException {

Review comment:
       added additional method to print fields.




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java
##########
@@ -258,6 +258,9 @@
     /** */
     public static final short T_MARKER_PAGE = 33;
 
+    /** */
+    public static final short T_INDEX_META = 34;

Review comment:
       If I understood you correctly, you suggest to add PageMetaIOV2 (instead of PageIndexMetaIO)?
   PagePartitionMeta will continue to extend PageMetaIO(not v2), is this correct?




----------------------------------------------------------------
This is an automated message from the 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] alex-plekhanov commented on a change in pull request #7941: IGNITE-12843

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #7941:
URL: https://github.com/apache/ignite/pull/7941#discussion_r466480687



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/BasicRateLimiter.java
##########
@@ -0,0 +1,154 @@
+/*
+ * 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;
+
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.util.typedef.internal.A;
+
+import static java.lang.Math.max;
+import static java.util.concurrent.TimeUnit.MICROSECONDS;
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+
+/**
+ * The simplified version of Google Guava smooth rate limiter.<br><br>
+ *
+ * The primary feature of a rate limiter is its "stable rate", the maximum rate that is should
+ * allow at normal conditions. This is enforced by "throttling" incoming requests as needed, i.e.
+ * compute, for an incoming request, the appropriate throttle time, and make the calling thread
+ * wait as much.<br><br>
+ *
+ * The simplest way to maintain a rate of QPS is to keep the timestamp of the last granted
+ * request, and ensure that (1/QPS) seconds have elapsed since then. For example, for a rate of
+ * QPS=5 (5 tokens per second), if we ensure that a request isn't granted earlier than 200ms after
+ * the last one, then we achieve the intended rate. If a request comes and the last request was
+ * granted only 100ms ago, then we wait for another 100ms. At this rate, serving 15 fresh permits
+ * (i.e. for an acquire(15) request) naturally takes 3 seconds.<br><br>
+ *
+ * It is important to realize that such a limiter has a very superficial memory of the past:
+ * it only remembers the last request. if the limiter was unused for a long period of
+ * time, then a request arrived and was immediately granted? This limiter would immediately
+ * forget about that past underutilization.
+ */
+public class BasicRateLimiter {
+    /** Start timestamp. */
+    private final long startTime = System.nanoTime();
+
+    /** Synchronization mutex. */
+    private final Object mux = new Object();
+
+    /**
+     * The interval between two unit requests, at our stable rate. E.g., a stable rate of 5 permits
+     * per second has a stable interval of 200ms.
+     */
+    private double stableIntervalMicros;
+
+    /**
+     * The time when the next request (no matter its size) will be granted. After granting a request,
+     * this is pushed further in the future. Large requests push this further than small requests.
+     */
+    private long nextFreeTicketMicros;
+
+    /**
+     * @param permitsPerSecond Estimated number of permits per second.
+     */
+    public BasicRateLimiter(double permitsPerSecond) {
+        setRate(permitsPerSecond);
+    }
+
+    /**
+     * Updates the stable rate.
+     *
+     * @param permitsPerSecond the new stable rate of this {@code RateLimiter}

Review comment:
       Please fix all comments in this class according to Ignite codestyle (the point at the end, sentence start from the upper case) 

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupPageScanner.java
##########
@@ -0,0 +1,567 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.EncryptionConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.util.BasicRateLimiter;
+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.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+import org.apache.ignite.thread.OomExceptionHandler;
+
+import static org.apache.ignite.internal.util.IgniteUtils.MB;
+
+/**
+ * Cache group page stores scanner.
+ * Scans a range of pages and marks them as dirty to re-encrypt them with the last encryption key on disk.
+ */
+public class CacheGroupPageScanner implements DbCheckpointListener {
+    /** Thread prefix for scanning tasks. */
+    private static final String REENCRYPT_THREAD_PREFIX = "reencrypt";
+
+    /** Encryption configuration. */
+    private final EncryptionConfiguration encrCfg;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to group scanning context. */
+    private final Map<Integer, GroupScanContext> grps = new ConcurrentHashMap<>();
+
+    /** Queue of groups waiting for a checkpoint. */
+    private final Queue<Integer> cpWaitGrps = new ConcurrentLinkedQueue<>();

Review comment:
       Let's use `Queue<GroupScanContext>` to avoid redundant GroupScanContext search

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupPageScanner.java
##########
@@ -0,0 +1,567 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.EncryptionConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.util.BasicRateLimiter;
+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.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+import org.apache.ignite.thread.OomExceptionHandler;
+
+import static org.apache.ignite.internal.util.IgniteUtils.MB;
+
+/**
+ * Cache group page stores scanner.
+ * Scans a range of pages and marks them as dirty to re-encrypt them with the last encryption key on disk.
+ */
+public class CacheGroupPageScanner implements DbCheckpointListener {
+    /** Thread prefix for scanning tasks. */
+    private static final String REENCRYPT_THREAD_PREFIX = "reencrypt";
+
+    /** Encryption configuration. */
+    private final EncryptionConfiguration encrCfg;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to group scanning context. */
+    private final Map<Integer, GroupScanContext> grps = new ConcurrentHashMap<>();
+
+    /** Queue of groups waiting for a checkpoint. */
+    private final Queue<Integer> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Executor to start partition scan tasks. */
+    private final IgniteThreadPoolExecutor execSvc;
+
+    /** Page scanning speed limiter. */
+    private final BasicRateLimiter limiter;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupPageScanner(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        encrCfg = ctx.config().getDataStorageConfiguration().getEncryptionConfiguration();
+
+        execSvc = new IgniteThreadPoolExecutor(REENCRYPT_THREAD_PREFIX,
+            ctx.igniteInstanceName(),
+            encrCfg.getReencryptionThreadCnt(),
+            encrCfg.getReencryptionThreadCnt(),
+            IgniteConfiguration.DFLT_THREAD_KEEP_ALIVE_TIME,
+            new LinkedBlockingQueue<>(),
+            GridIoPolicy.SYSTEM_POOL,
+            new OomExceptionHandler(ctx));
+
+        execSvc.allowCoreThreadTimeOut(true);
+
+        DataStorageConfiguration dsCfg = ctx.config().getDataStorageConfiguration();
+
+        limiter = CU.isPersistenceEnabled(dsCfg) && encrCfg.getReencryptionRateLimit() > 0 ?
+            new BasicRateLimiter(encrCfg.getReencryptionRateLimit() * MB / dsCfg.getPageSize()) : null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCheckpointBegin(Context cpCtx) {
+        Set<Integer> completeCandidates = new HashSet<>();
+
+        Integer grpId;
+
+        while ((grpId = cpWaitGrps.poll()) != null)
+            completeCandidates.add(grpId);
+
+        cpCtx.finishedStateFut().listen(
+            f -> {
+                if (f.error() != null || f.isCancelled()) {
+                    cpWaitGrps.addAll(completeCandidates);
+
+                    return;
+                }
+
+                lock.lock();
+
+                try {
+                    for (int groupId : completeCandidates) {
+                        GroupScanContext scanCtx = grps.remove(groupId);
+
+                        boolean finished = scanCtx.finish();
+
+                        assert finished : groupId;
+
+                        if (log.isInfoEnabled())
+                            log.info("Cache group reencryption is finished [grpId=" + groupId + "]");
+                    }
+
+                    if (!grps.isEmpty())
+                        return;
+
+                    ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).
+                        removeCheckpointListener(this);
+                }
+                finally {
+                    lock.unlock();
+                }
+            }
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public void beforeCheckpointBegin(Context cpCtx) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onMarkCheckpointBegin(Context ctx) {
+        // No-op.
+    }
+
+    /**
+     * @return {@code True} If reencryption is disabled.
+     */
+    public boolean disabled() {
+        return encrCfg.isReencryptionDisabled();
+    }
+
+    /**
+     * Schedule scanning partitions.
+     *
+     * @param grpId Cache group ID.
+     */
+    public IgniteInternalFuture<Void> schedule(int grpId) throws IgniteCheckedException {
+        if (disabled())
+            throw new IgniteCheckedException("Reencryption is disabled.");
+
+        CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+        if (grp == null) {
+            if (log.isDebugEnabled())
+                log.debug("Skip reencryption, cache group was destroyed [grp=" + grpId + "]");
+
+            return new GridFinishedFuture<>();
+        }
+
+        lock.lock();
+
+        try {
+            if (stopped)
+                throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
+
+            if (grps.isEmpty())
+                ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).addCheckpointListener(this);
+
+            GroupScanContext prevState = grps.get(grpId);
+
+            if (prevState != null) {
+                if (log.isDebugEnabled())
+                    log.debug("Reencryption already scheduled [grpId=" + grpId + "]");
+
+                return prevState.finishFuture();
+            }
+
+            GroupScanContext ctx0 = new GroupScanContext(grpId);
+
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) {
+                    if (ctx.encryption().getEncryptionState(grpId, partId) == 0) {
+                        if (log.isDebugEnabled())
+                            log.debug("Skipping partition reencryption [grp=" + grpId + ", p=" + partId + "]");
+
+                        return;
+                    }
+
+                    PageStoreScanTask scanTask = new PageStoreScanTask(ctx0, partId);
+
+                    ctx0.add(partId, scanTask);
+
+                    execSvc.submit(scanTask);
+                }
+            });
+
+            ctx0.initialize().listen(f -> {
+                Throwable t = f.error();
+
+                if (t != null) {
+                    log.error("Reencryption is failed [grpId=" + grpId + "]", t);
+
+                    ctx0.fail(t);
+
+                    return;
+                }
+
+                boolean added = cpWaitGrps.offer(grpId);
+
+                assert added;
+            });
+
+            if (log.isInfoEnabled())
+                log.info("Scheduled reencryption [grpId=" + grpId + "]");
+
+            grps.put(grpId, ctx0);
+
+            return ctx0.finishFuture();
+        }
+        finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @return Future that will be completed when all partitions have been scanned and pages have been written to disk.
+     */
+    public IgniteInternalFuture<Void> statusFuture(int grpId) {
+        GroupScanContext ctx0 = grps.get(grpId);
+
+        return ctx0 == null ? new GridFinishedFuture<>() : ctx0.finishFuture();
+    }
+
+    /**
+     * Shutdown scanning and disable new tasks scheduling.
+     */
+    public void stop() throws IgniteCheckedException {
+        lock.lock();
+
+        try {
+            stopped = true;
+
+            for (GroupScanContext ctx0 : grps.values())
+                ctx0.finishFuture().cancel();
+
+            execSvc.shutdown();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Stop scannig the specified partition.
+     *
+     * @param grpId Cache group ID.
+     * @param partId Partition ID.
+     * @return {@code True} if reencryption was cancelled.
+     * @throws IgniteCheckedException If failed.
+     */
+    public boolean cancel(int grpId, int partId) throws IgniteCheckedException {
+        GroupScanContext ctx = grps.get(grpId);
+
+        if (ctx == null)
+            return false;
+
+        return ctx.cancel(partId);
+    }
+
+    /**
+     * Collect current number of pages in the specified cache group.
+     *
+     * @param grp Cache group.
+     * @return Map of partitions with current page count.
+     * @throws IgniteCheckedException If failed.
+     */
+    public Map<Integer, Long> pagesCount(CacheGroupContext grp) throws IgniteCheckedException {
+        Map<Integer, Long> partStates = new HashMap<>();
+
+        ctx.cache().context().database().checkpointReadLock();
+
+        try {
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) throws IgniteCheckedException {
+                    int pagesCnt = ctx.cache().context().pageStore().pages(grp.groupId(), partId);
+
+                    partStates.put(partId, (long)pagesCnt);
+                }
+            });
+        } finally {
+            ctx.cache().context().database().checkpointReadUnlock();
+        }
+
+        return partStates;
+    }
+
+    /**
+     * @param grp Cache group.
+     * @param hnd Partition handler.
+     */
+    private void forEachPageStore(CacheGroupContext grp, IgniteInClosureX<Integer> hnd) throws IgniteCheckedException {
+        int parts = grp.affinity().partitions();
+
+        IgnitePageStoreManager pageStoreMgr = ctx.cache().context().pageStore();
+
+        for (int p = 0; p < parts; p++) {
+            if (!pageStoreMgr.exists(grp.groupId(), p))
+                continue;
+
+            hnd.applyx(p);
+        }
+
+        hnd.applyx(PageIdAllocator.INDEX_PARTITION);
+    }
+
+    /**
+     * Cache group scanning context.
+     */
+    private static class GroupScanContext {
+        /** Partition scanning futures. */
+        private final Map<Integer, IgniteInternalFuture<Void>> futMap = new ConcurrentHashMap<>();
+
+        /** Compound future, that will be completed when all partitions scanned. */
+        private final GridCompoundFuture<Void, Void> compFut = new GridCompoundFuture<>();
+
+        /** Cache group ID. */
+        private final int grpId;
+
+        /** Future that ends after all partitions are done and a checkpoint is finished. */
+        private final GridFutureAdapter<Void> cpFut = new GridFutureAdapter<Void>() {
+            @Override public boolean cancel() throws IgniteCheckedException {
+                compFut.cancel();
+
+                return onDone(null, null, true);
+            }
+        };
+
+        /**
+         * @param grpId Cache group ID.
+         */
+        public GroupScanContext(int grpId) {
+            this.grpId = grpId;
+        }
+
+        /**
+         * @return Cache group ID.
+         */
+        public int groupId() {
+            return grpId;
+        }
+
+        /**
+         * @param partId Partition ID.
+         * @param fut Partition scanning future.
+         */
+        public void add(int partId, IgniteInternalFuture<Void> fut) {
+            compFut.add(fut);
+
+            futMap.put(partId, fut);
+        }
+
+        /**
+         * @return Compound future, that will be completed when all partitions scanned.
+         */
+        public IgniteInternalFuture<Void> initialize() {
+            return compFut.markInitialized();
+        }
+
+        /**
+         * @return Future that ends after all partitions are done and a checkpoint is finished.
+         */
+        public IgniteInternalFuture<Void> finishFuture() {
+            return cpFut;
+        }
+
+        /**
+         * Finish reencryption future.
+         *
+         * @return {@code True} if the future was finished by this call.
+         */
+        public boolean finish() {
+            return cpFut.onDone(compFut.result());
+        }
+
+        /**
+         * Stop reencryption of the specified partition.
+         *
+         * @param partId Partition ID.
+         * @return {@code True} if reencryption was cancelled.
+         * @throws IgniteCheckedException If failed.
+         */
+        public boolean cancel(int partId) throws IgniteCheckedException {
+            IgniteInternalFuture<Void> fut = futMap.get(partId);
+
+            if (fut == null)
+                return false;
+
+            return fut.cancel();
+        }
+
+        /**
+         * @param t Throwable.
+         */
+        public void fail(Throwable t) {
+            cpFut.onDone(t);
+        }
+    }
+
+    /**
+     * Page store scanning task.
+     */
+    private class PageStoreScanTask extends GridFutureAdapter<Void> implements Runnable {
+        /** Partiion ID. */
+        private final int partId;
+
+        /** Cache group scan context. */
+        private final GroupScanContext scanCtx;
+
+        /**
+         * @param scanCtx Cache group scanning context.
+         * @param partId Partition ID.
+         */
+        public PageStoreScanTask(GroupScanContext scanCtx, int partId) {
+            this.scanCtx = scanCtx;
+            this.partId = partId;
+        }
+
+        /** {@inheritDoc} */
+        @Override public synchronized boolean cancel() throws IgniteCheckedException {
+            return onDone();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void run() {
+            try {
+                int grpId = scanCtx.groupId();
+
+                CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+                if (grp == null) {
+                    onDone();
+
+                    return;
+                }
+
+                PageMemoryEx pageMem = (PageMemoryEx)grp.dataRegion().pageMemory();
+                long metaPageId = pageMem.partitionMetaPageId(grpId, partId);
+                long state = ctx.encryption().getEncryptionState(grpId, partId);
+                int batchSize = encrCfg.getReencryptionBatchSize();
+
+                int off = (int)(state >> Integer.SIZE);
+                int cnt = (int)state;

Review comment:
       Logic for extraction state is located at different classes. Perhaps there should be some helper class or helper methods in one class to extract and generate state out of offset and pages count.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupPageScanner.java
##########
@@ -0,0 +1,567 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.EncryptionConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.util.BasicRateLimiter;
+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.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+import org.apache.ignite.thread.OomExceptionHandler;
+
+import static org.apache.ignite.internal.util.IgniteUtils.MB;
+
+/**
+ * Cache group page stores scanner.
+ * Scans a range of pages and marks them as dirty to re-encrypt them with the last encryption key on disk.
+ */
+public class CacheGroupPageScanner implements DbCheckpointListener {
+    /** Thread prefix for scanning tasks. */
+    private static final String REENCRYPT_THREAD_PREFIX = "reencrypt";
+
+    /** Encryption configuration. */
+    private final EncryptionConfiguration encrCfg;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to group scanning context. */
+    private final Map<Integer, GroupScanContext> grps = new ConcurrentHashMap<>();
+
+    /** Queue of groups waiting for a checkpoint. */
+    private final Queue<Integer> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Executor to start partition scan tasks. */
+    private final IgniteThreadPoolExecutor execSvc;
+
+    /** Page scanning speed limiter. */
+    private final BasicRateLimiter limiter;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupPageScanner(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        encrCfg = ctx.config().getDataStorageConfiguration().getEncryptionConfiguration();
+
+        execSvc = new IgniteThreadPoolExecutor(REENCRYPT_THREAD_PREFIX,

Review comment:
       Perhaps single thread executor can be used here 

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupPageScanner.java
##########
@@ -0,0 +1,567 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.EncryptionConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.util.BasicRateLimiter;
+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.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+import org.apache.ignite.thread.OomExceptionHandler;
+
+import static org.apache.ignite.internal.util.IgniteUtils.MB;
+
+/**
+ * Cache group page stores scanner.
+ * Scans a range of pages and marks them as dirty to re-encrypt them with the last encryption key on disk.
+ */
+public class CacheGroupPageScanner implements DbCheckpointListener {
+    /** Thread prefix for scanning tasks. */
+    private static final String REENCRYPT_THREAD_PREFIX = "reencrypt";
+
+    /** Encryption configuration. */
+    private final EncryptionConfiguration encrCfg;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to group scanning context. */
+    private final Map<Integer, GroupScanContext> grps = new ConcurrentHashMap<>();
+
+    /** Queue of groups waiting for a checkpoint. */
+    private final Queue<Integer> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Executor to start partition scan tasks. */
+    private final IgniteThreadPoolExecutor execSvc;
+
+    /** Page scanning speed limiter. */
+    private final BasicRateLimiter limiter;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupPageScanner(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        encrCfg = ctx.config().getDataStorageConfiguration().getEncryptionConfiguration();
+
+        execSvc = new IgniteThreadPoolExecutor(REENCRYPT_THREAD_PREFIX,
+            ctx.igniteInstanceName(),
+            encrCfg.getReencryptionThreadCnt(),
+            encrCfg.getReencryptionThreadCnt(),
+            IgniteConfiguration.DFLT_THREAD_KEEP_ALIVE_TIME,
+            new LinkedBlockingQueue<>(),
+            GridIoPolicy.SYSTEM_POOL,
+            new OomExceptionHandler(ctx));
+
+        execSvc.allowCoreThreadTimeOut(true);
+
+        DataStorageConfiguration dsCfg = ctx.config().getDataStorageConfiguration();
+
+        limiter = CU.isPersistenceEnabled(dsCfg) && encrCfg.getReencryptionRateLimit() > 0 ?
+            new BasicRateLimiter(encrCfg.getReencryptionRateLimit() * MB / dsCfg.getPageSize()) : null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCheckpointBegin(Context cpCtx) {
+        Set<Integer> completeCandidates = new HashSet<>();
+
+        Integer grpId;
+
+        while ((grpId = cpWaitGrps.poll()) != null)

Review comment:
       `cpWaitGrps.removeIf(completeCandidates::add)`
   ?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupPageScanner.java
##########
@@ -0,0 +1,567 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.EncryptionConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.util.BasicRateLimiter;
+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.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+import org.apache.ignite.thread.OomExceptionHandler;
+
+import static org.apache.ignite.internal.util.IgniteUtils.MB;
+
+/**
+ * Cache group page stores scanner.
+ * Scans a range of pages and marks them as dirty to re-encrypt them with the last encryption key on disk.
+ */
+public class CacheGroupPageScanner implements DbCheckpointListener {
+    /** Thread prefix for scanning tasks. */
+    private static final String REENCRYPT_THREAD_PREFIX = "reencrypt";
+
+    /** Encryption configuration. */
+    private final EncryptionConfiguration encrCfg;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to group scanning context. */
+    private final Map<Integer, GroupScanContext> grps = new ConcurrentHashMap<>();
+
+    /** Queue of groups waiting for a checkpoint. */
+    private final Queue<Integer> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Executor to start partition scan tasks. */
+    private final IgniteThreadPoolExecutor execSvc;
+
+    /** Page scanning speed limiter. */
+    private final BasicRateLimiter limiter;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupPageScanner(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        encrCfg = ctx.config().getDataStorageConfiguration().getEncryptionConfiguration();
+
+        execSvc = new IgniteThreadPoolExecutor(REENCRYPT_THREAD_PREFIX,
+            ctx.igniteInstanceName(),
+            encrCfg.getReencryptionThreadCnt(),
+            encrCfg.getReencryptionThreadCnt(),
+            IgniteConfiguration.DFLT_THREAD_KEEP_ALIVE_TIME,
+            new LinkedBlockingQueue<>(),
+            GridIoPolicy.SYSTEM_POOL,
+            new OomExceptionHandler(ctx));
+
+        execSvc.allowCoreThreadTimeOut(true);
+
+        DataStorageConfiguration dsCfg = ctx.config().getDataStorageConfiguration();
+
+        limiter = CU.isPersistenceEnabled(dsCfg) && encrCfg.getReencryptionRateLimit() > 0 ?
+            new BasicRateLimiter(encrCfg.getReencryptionRateLimit() * MB / dsCfg.getPageSize()) : null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCheckpointBegin(Context cpCtx) {
+        Set<Integer> completeCandidates = new HashSet<>();
+
+        Integer grpId;
+
+        while ((grpId = cpWaitGrps.poll()) != null)
+            completeCandidates.add(grpId);
+
+        cpCtx.finishedStateFut().listen(
+            f -> {
+                if (f.error() != null || f.isCancelled()) {
+                    cpWaitGrps.addAll(completeCandidates);
+
+                    return;
+                }
+
+                lock.lock();
+
+                try {
+                    for (int groupId : completeCandidates) {
+                        GroupScanContext scanCtx = grps.remove(groupId);
+
+                        boolean finished = scanCtx.finish();
+
+                        assert finished : groupId;
+
+                        if (log.isInfoEnabled())
+                            log.info("Cache group reencryption is finished [grpId=" + groupId + "]");
+                    }
+
+                    if (!grps.isEmpty())
+                        return;
+
+                    ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).
+                        removeCheckpointListener(this);
+                }
+                finally {
+                    lock.unlock();
+                }
+            }
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public void beforeCheckpointBegin(Context cpCtx) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onMarkCheckpointBegin(Context ctx) {
+        // No-op.
+    }
+
+    /**
+     * @return {@code True} If reencryption is disabled.
+     */
+    public boolean disabled() {
+        return encrCfg.isReencryptionDisabled();
+    }
+
+    /**
+     * Schedule scanning partitions.
+     *
+     * @param grpId Cache group ID.
+     */
+    public IgniteInternalFuture<Void> schedule(int grpId) throws IgniteCheckedException {
+        if (disabled())
+            throw new IgniteCheckedException("Reencryption is disabled.");
+
+        CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+        if (grp == null) {
+            if (log.isDebugEnabled())
+                log.debug("Skip reencryption, cache group was destroyed [grp=" + grpId + "]");
+
+            return new GridFinishedFuture<>();
+        }
+
+        lock.lock();
+
+        try {
+            if (stopped)
+                throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
+
+            if (grps.isEmpty())
+                ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).addCheckpointListener(this);
+
+            GroupScanContext prevState = grps.get(grpId);
+
+            if (prevState != null) {
+                if (log.isDebugEnabled())
+                    log.debug("Reencryption already scheduled [grpId=" + grpId + "]");
+
+                return prevState.finishFuture();
+            }
+
+            GroupScanContext ctx0 = new GroupScanContext(grpId);
+
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) {
+                    if (ctx.encryption().getEncryptionState(grpId, partId) == 0) {
+                        if (log.isDebugEnabled())
+                            log.debug("Skipping partition reencryption [grp=" + grpId + ", p=" + partId + "]");
+
+                        return;
+                    }
+
+                    PageStoreScanTask scanTask = new PageStoreScanTask(ctx0, partId);
+
+                    ctx0.add(partId, scanTask);
+
+                    execSvc.submit(scanTask);
+                }
+            });
+
+            ctx0.initialize().listen(f -> {
+                Throwable t = f.error();
+
+                if (t != null) {
+                    log.error("Reencryption is failed [grpId=" + grpId + "]", t);
+
+                    ctx0.fail(t);
+
+                    return;
+                }
+
+                boolean added = cpWaitGrps.offer(grpId);
+
+                assert added;
+            });
+
+            if (log.isInfoEnabled())
+                log.info("Scheduled reencryption [grpId=" + grpId + "]");
+
+            grps.put(grpId, ctx0);
+
+            return ctx0.finishFuture();
+        }
+        finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @return Future that will be completed when all partitions have been scanned and pages have been written to disk.
+     */
+    public IgniteInternalFuture<Void> statusFuture(int grpId) {
+        GroupScanContext ctx0 = grps.get(grpId);
+
+        return ctx0 == null ? new GridFinishedFuture<>() : ctx0.finishFuture();
+    }
+
+    /**
+     * Shutdown scanning and disable new tasks scheduling.
+     */
+    public void stop() throws IgniteCheckedException {
+        lock.lock();
+
+        try {
+            stopped = true;
+
+            for (GroupScanContext ctx0 : grps.values())
+                ctx0.finishFuture().cancel();
+
+            execSvc.shutdown();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Stop scannig the specified partition.
+     *
+     * @param grpId Cache group ID.
+     * @param partId Partition ID.
+     * @return {@code True} if reencryption was cancelled.
+     * @throws IgniteCheckedException If failed.
+     */
+    public boolean cancel(int grpId, int partId) throws IgniteCheckedException {
+        GroupScanContext ctx = grps.get(grpId);
+
+        if (ctx == null)
+            return false;
+
+        return ctx.cancel(partId);
+    }
+
+    /**
+     * Collect current number of pages in the specified cache group.
+     *
+     * @param grp Cache group.
+     * @return Map of partitions with current page count.
+     * @throws IgniteCheckedException If failed.
+     */
+    public Map<Integer, Long> pagesCount(CacheGroupContext grp) throws IgniteCheckedException {
+        Map<Integer, Long> partStates = new HashMap<>();

Review comment:
       Perhaps it's more resource-friendly to use long[] here (value for each partition)

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupPageScanner.java
##########
@@ -0,0 +1,567 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.EncryptionConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.util.BasicRateLimiter;
+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.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+import org.apache.ignite.thread.OomExceptionHandler;
+
+import static org.apache.ignite.internal.util.IgniteUtils.MB;
+
+/**
+ * Cache group page stores scanner.
+ * Scans a range of pages and marks them as dirty to re-encrypt them with the last encryption key on disk.
+ */
+public class CacheGroupPageScanner implements DbCheckpointListener {
+    /** Thread prefix for scanning tasks. */
+    private static final String REENCRYPT_THREAD_PREFIX = "reencrypt";
+
+    /** Encryption configuration. */
+    private final EncryptionConfiguration encrCfg;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to group scanning context. */
+    private final Map<Integer, GroupScanContext> grps = new ConcurrentHashMap<>();
+
+    /** Queue of groups waiting for a checkpoint. */
+    private final Queue<Integer> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Executor to start partition scan tasks. */
+    private final IgniteThreadPoolExecutor execSvc;
+
+    /** Page scanning speed limiter. */
+    private final BasicRateLimiter limiter;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupPageScanner(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        encrCfg = ctx.config().getDataStorageConfiguration().getEncryptionConfiguration();
+
+        execSvc = new IgniteThreadPoolExecutor(REENCRYPT_THREAD_PREFIX,
+            ctx.igniteInstanceName(),
+            encrCfg.getReencryptionThreadCnt(),
+            encrCfg.getReencryptionThreadCnt(),
+            IgniteConfiguration.DFLT_THREAD_KEEP_ALIVE_TIME,
+            new LinkedBlockingQueue<>(),
+            GridIoPolicy.SYSTEM_POOL,
+            new OomExceptionHandler(ctx));
+
+        execSvc.allowCoreThreadTimeOut(true);
+
+        DataStorageConfiguration dsCfg = ctx.config().getDataStorageConfiguration();
+
+        limiter = CU.isPersistenceEnabled(dsCfg) && encrCfg.getReencryptionRateLimit() > 0 ?
+            new BasicRateLimiter(encrCfg.getReencryptionRateLimit() * MB / dsCfg.getPageSize()) : null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCheckpointBegin(Context cpCtx) {
+        Set<Integer> completeCandidates = new HashSet<>();
+
+        Integer grpId;
+
+        while ((grpId = cpWaitGrps.poll()) != null)
+            completeCandidates.add(grpId);
+
+        cpCtx.finishedStateFut().listen(
+            f -> {
+                if (f.error() != null || f.isCancelled()) {
+                    cpWaitGrps.addAll(completeCandidates);
+
+                    return;
+                }
+
+                lock.lock();
+
+                try {
+                    for (int groupId : completeCandidates) {
+                        GroupScanContext scanCtx = grps.remove(groupId);
+
+                        boolean finished = scanCtx.finish();
+
+                        assert finished : groupId;
+
+                        if (log.isInfoEnabled())
+                            log.info("Cache group reencryption is finished [grpId=" + groupId + "]");
+                    }
+
+                    if (!grps.isEmpty())
+                        return;
+
+                    ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).
+                        removeCheckpointListener(this);
+                }
+                finally {
+                    lock.unlock();
+                }
+            }
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public void beforeCheckpointBegin(Context cpCtx) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onMarkCheckpointBegin(Context ctx) {
+        // No-op.
+    }
+
+    /**
+     * @return {@code True} If reencryption is disabled.
+     */
+    public boolean disabled() {
+        return encrCfg.isReencryptionDisabled();
+    }
+
+    /**
+     * Schedule scanning partitions.
+     *
+     * @param grpId Cache group ID.
+     */
+    public IgniteInternalFuture<Void> schedule(int grpId) throws IgniteCheckedException {
+        if (disabled())
+            throw new IgniteCheckedException("Reencryption is disabled.");
+
+        CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+        if (grp == null) {
+            if (log.isDebugEnabled())
+                log.debug("Skip reencryption, cache group was destroyed [grp=" + grpId + "]");
+
+            return new GridFinishedFuture<>();
+        }
+
+        lock.lock();
+
+        try {
+            if (stopped)
+                throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
+
+            if (grps.isEmpty())
+                ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).addCheckpointListener(this);
+
+            GroupScanContext prevState = grps.get(grpId);
+
+            if (prevState != null) {
+                if (log.isDebugEnabled())
+                    log.debug("Reencryption already scheduled [grpId=" + grpId + "]");
+
+                return prevState.finishFuture();
+            }
+
+            GroupScanContext ctx0 = new GroupScanContext(grpId);
+
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) {
+                    if (ctx.encryption().getEncryptionState(grpId, partId) == 0) {
+                        if (log.isDebugEnabled())
+                            log.debug("Skipping partition reencryption [grp=" + grpId + ", p=" + partId + "]");
+
+                        return;
+                    }
+
+                    PageStoreScanTask scanTask = new PageStoreScanTask(ctx0, partId);
+
+                    ctx0.add(partId, scanTask);
+
+                    execSvc.submit(scanTask);
+                }
+            });
+
+            ctx0.initialize().listen(f -> {
+                Throwable t = f.error();
+
+                if (t != null) {
+                    log.error("Reencryption is failed [grpId=" + grpId + "]", t);
+
+                    ctx0.fail(t);
+
+                    return;
+                }
+
+                boolean added = cpWaitGrps.offer(grpId);
+
+                assert added;
+            });
+
+            if (log.isInfoEnabled())
+                log.info("Scheduled reencryption [grpId=" + grpId + "]");
+
+            grps.put(grpId, ctx0);
+
+            return ctx0.finishFuture();
+        }
+        finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @return Future that will be completed when all partitions have been scanned and pages have been written to disk.
+     */
+    public IgniteInternalFuture<Void> statusFuture(int grpId) {
+        GroupScanContext ctx0 = grps.get(grpId);
+
+        return ctx0 == null ? new GridFinishedFuture<>() : ctx0.finishFuture();
+    }
+
+    /**
+     * Shutdown scanning and disable new tasks scheduling.
+     */
+    public void stop() throws IgniteCheckedException {
+        lock.lock();
+
+        try {
+            stopped = true;
+
+            for (GroupScanContext ctx0 : grps.values())
+                ctx0.finishFuture().cancel();
+
+            execSvc.shutdown();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Stop scannig the specified partition.
+     *
+     * @param grpId Cache group ID.
+     * @param partId Partition ID.
+     * @return {@code True} if reencryption was cancelled.
+     * @throws IgniteCheckedException If failed.
+     */
+    public boolean cancel(int grpId, int partId) throws IgniteCheckedException {
+        GroupScanContext ctx = grps.get(grpId);
+
+        if (ctx == null)
+            return false;
+
+        return ctx.cancel(partId);
+    }
+
+    /**
+     * Collect current number of pages in the specified cache group.
+     *
+     * @param grp Cache group.
+     * @return Map of partitions with current page count.
+     * @throws IgniteCheckedException If failed.
+     */
+    public Map<Integer, Long> pagesCount(CacheGroupContext grp) throws IgniteCheckedException {
+        Map<Integer, Long> partStates = new HashMap<>();
+
+        ctx.cache().context().database().checkpointReadLock();
+
+        try {
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) throws IgniteCheckedException {
+                    int pagesCnt = ctx.cache().context().pageStore().pages(grp.groupId(), partId);
+
+                    partStates.put(partId, (long)pagesCnt);
+                }
+            });
+        } finally {
+            ctx.cache().context().database().checkpointReadUnlock();
+        }
+
+        return partStates;
+    }
+
+    /**
+     * @param grp Cache group.
+     * @param hnd Partition handler.
+     */
+    private void forEachPageStore(CacheGroupContext grp, IgniteInClosureX<Integer> hnd) throws IgniteCheckedException {
+        int parts = grp.affinity().partitions();
+
+        IgnitePageStoreManager pageStoreMgr = ctx.cache().context().pageStore();
+
+        for (int p = 0; p < parts; p++) {
+            if (!pageStoreMgr.exists(grp.groupId(), p))
+                continue;
+
+            hnd.applyx(p);
+        }
+
+        hnd.applyx(PageIdAllocator.INDEX_PARTITION);
+    }
+
+    /**
+     * Cache group scanning context.
+     */
+    private static class GroupScanContext {

Review comment:
       There are a lot of futures here listening to each other and it's hard to understand chain of these futures. Perhaps we can simplify it by excluding context class (make it implement future and rename it, also remove some futures included to this class). WDYT?
   Also if you will use single thread pool you can get rid of compound future and PageStoreScanTasks at all (you can iterate by partitions in this class)

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/BasicRateLimiter.java
##########
@@ -0,0 +1,154 @@
+/*
+ * 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;
+
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.util.typedef.internal.A;
+
+import static java.lang.Math.max;
+import static java.util.concurrent.TimeUnit.MICROSECONDS;
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+
+/**
+ * The simplified version of Google Guava smooth rate limiter.<br><br>
+ *
+ * The primary feature of a rate limiter is its "stable rate", the maximum rate that is should
+ * allow at normal conditions. This is enforced by "throttling" incoming requests as needed, i.e.
+ * compute, for an incoming request, the appropriate throttle time, and make the calling thread
+ * wait as much.<br><br>
+ *
+ * The simplest way to maintain a rate of QPS is to keep the timestamp of the last granted
+ * request, and ensure that (1/QPS) seconds have elapsed since then. For example, for a rate of
+ * QPS=5 (5 tokens per second), if we ensure that a request isn't granted earlier than 200ms after
+ * the last one, then we achieve the intended rate. If a request comes and the last request was
+ * granted only 100ms ago, then we wait for another 100ms. At this rate, serving 15 fresh permits
+ * (i.e. for an acquire(15) request) naturally takes 3 seconds.<br><br>
+ *
+ * It is important to realize that such a limiter has a very superficial memory of the past:
+ * it only remembers the last request. if the limiter was unused for a long period of
+ * time, then a request arrived and was immediately granted? This limiter would immediately
+ * forget about that past underutilization.
+ */
+public class BasicRateLimiter {
+    /** Start timestamp. */
+    private final long startTime = System.nanoTime();
+
+    /** Synchronization mutex. */
+    private final Object mux = new Object();
+
+    /**
+     * The interval between two unit requests, at our stable rate. E.g., a stable rate of 5 permits
+     * per second has a stable interval of 200ms.
+     */
+    private double stableIntervalMicros;
+
+    /**
+     * The time when the next request (no matter its size) will be granted. After granting a request,
+     * this is pushed further in the future. Large requests push this further than small requests.
+     */
+    private long nextFreeTicketMicros;
+
+    /**
+     * @param permitsPerSecond Estimated number of permits per second.
+     */
+    public BasicRateLimiter(double permitsPerSecond) {
+        setRate(permitsPerSecond);
+    }
+
+    /**
+     * Updates the stable rate.
+     *
+     * @param permitsPerSecond the new stable rate of this {@code RateLimiter}
+     * @throws IllegalArgumentException if {@code permitsPerSecond} is negative or zero
+     */
+    public void setRate(double permitsPerSecond) {
+        A.ensure(permitsPerSecond > 0, "Requested permits (" + permitsPerSecond + ") must be positive");
+
+        synchronized (mux) {
+            resync();
+
+            stableIntervalMicros = SECONDS.toMicros(1L) / permitsPerSecond;
+        }
+    }
+
+    /**
+     * @return The stable rate (as {@code permits per seconds}).
+     */
+    public double getRate() {
+        synchronized (mux) {
+            return SECONDS.toMicros(1L) / stableIntervalMicros;
+        }
+    }
+
+    /**
+     * Acquires the given number of permits from this {@code RateLimiter}, blocking until the request
+     * can be granted. Tells the amount of time slept, if any.
+     *
+     * @param permits the number of permits to acquire
+     * @throws IllegalArgumentException if the requested number of permits is negative or zero
+     * @since 16.0 (present in 13.0 with {@code void} return type})

Review comment:
       It's not related anyhow to Ignite versions, please remove it.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupPageScanner.java
##########
@@ -0,0 +1,567 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.EncryptionConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.util.BasicRateLimiter;
+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.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+import org.apache.ignite.thread.OomExceptionHandler;
+
+import static org.apache.ignite.internal.util.IgniteUtils.MB;
+
+/**
+ * Cache group page stores scanner.
+ * Scans a range of pages and marks them as dirty to re-encrypt them with the last encryption key on disk.
+ */
+public class CacheGroupPageScanner implements DbCheckpointListener {
+    /** Thread prefix for scanning tasks. */
+    private static final String REENCRYPT_THREAD_PREFIX = "reencrypt";
+
+    /** Encryption configuration. */
+    private final EncryptionConfiguration encrCfg;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to group scanning context. */
+    private final Map<Integer, GroupScanContext> grps = new ConcurrentHashMap<>();
+
+    /** Queue of groups waiting for a checkpoint. */
+    private final Queue<Integer> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Executor to start partition scan tasks. */
+    private final IgniteThreadPoolExecutor execSvc;
+
+    /** Page scanning speed limiter. */
+    private final BasicRateLimiter limiter;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupPageScanner(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        encrCfg = ctx.config().getDataStorageConfiguration().getEncryptionConfiguration();
+
+        execSvc = new IgniteThreadPoolExecutor(REENCRYPT_THREAD_PREFIX,
+            ctx.igniteInstanceName(),
+            encrCfg.getReencryptionThreadCnt(),
+            encrCfg.getReencryptionThreadCnt(),
+            IgniteConfiguration.DFLT_THREAD_KEEP_ALIVE_TIME,
+            new LinkedBlockingQueue<>(),
+            GridIoPolicy.SYSTEM_POOL,
+            new OomExceptionHandler(ctx));
+
+        execSvc.allowCoreThreadTimeOut(true);
+
+        DataStorageConfiguration dsCfg = ctx.config().getDataStorageConfiguration();
+
+        limiter = CU.isPersistenceEnabled(dsCfg) && encrCfg.getReencryptionRateLimit() > 0 ?
+            new BasicRateLimiter(encrCfg.getReencryptionRateLimit() * MB / dsCfg.getPageSize()) : null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCheckpointBegin(Context cpCtx) {
+        Set<Integer> completeCandidates = new HashSet<>();
+
+        Integer grpId;
+
+        while ((grpId = cpWaitGrps.poll()) != null)
+            completeCandidates.add(grpId);
+
+        cpCtx.finishedStateFut().listen(
+            f -> {
+                if (f.error() != null || f.isCancelled()) {
+                    cpWaitGrps.addAll(completeCandidates);
+
+                    return;
+                }
+
+                lock.lock();
+
+                try {
+                    for (int groupId : completeCandidates) {
+                        GroupScanContext scanCtx = grps.remove(groupId);
+
+                        boolean finished = scanCtx.finish();
+
+                        assert finished : groupId;
+
+                        if (log.isInfoEnabled())
+                            log.info("Cache group reencryption is finished [grpId=" + groupId + "]");
+                    }
+
+                    if (!grps.isEmpty())
+                        return;
+
+                    ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).
+                        removeCheckpointListener(this);
+                }
+                finally {
+                    lock.unlock();
+                }
+            }
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public void beforeCheckpointBegin(Context cpCtx) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onMarkCheckpointBegin(Context ctx) {
+        // No-op.
+    }
+
+    /**
+     * @return {@code True} If reencryption is disabled.
+     */
+    public boolean disabled() {
+        return encrCfg.isReencryptionDisabled();
+    }
+
+    /**
+     * Schedule scanning partitions.
+     *
+     * @param grpId Cache group ID.
+     */
+    public IgniteInternalFuture<Void> schedule(int grpId) throws IgniteCheckedException {
+        if (disabled())
+            throw new IgniteCheckedException("Reencryption is disabled.");
+
+        CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+        if (grp == null) {
+            if (log.isDebugEnabled())
+                log.debug("Skip reencryption, cache group was destroyed [grp=" + grpId + "]");
+
+            return new GridFinishedFuture<>();
+        }
+
+        lock.lock();
+
+        try {
+            if (stopped)
+                throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
+
+            if (grps.isEmpty())
+                ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).addCheckpointListener(this);
+
+            GroupScanContext prevState = grps.get(grpId);
+
+            if (prevState != null) {
+                if (log.isDebugEnabled())
+                    log.debug("Reencryption already scheduled [grpId=" + grpId + "]");
+
+                return prevState.finishFuture();
+            }
+
+            GroupScanContext ctx0 = new GroupScanContext(grpId);
+
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) {
+                    if (ctx.encryption().getEncryptionState(grpId, partId) == 0) {
+                        if (log.isDebugEnabled())
+                            log.debug("Skipping partition reencryption [grp=" + grpId + ", p=" + partId + "]");
+
+                        return;
+                    }
+
+                    PageStoreScanTask scanTask = new PageStoreScanTask(ctx0, partId);
+
+                    ctx0.add(partId, scanTask);
+
+                    execSvc.submit(scanTask);
+                }
+            });
+
+            ctx0.initialize().listen(f -> {
+                Throwable t = f.error();
+
+                if (t != null) {
+                    log.error("Reencryption is failed [grpId=" + grpId + "]", t);
+
+                    ctx0.fail(t);
+
+                    return;
+                }
+
+                boolean added = cpWaitGrps.offer(grpId);
+
+                assert added;
+            });
+
+            if (log.isInfoEnabled())
+                log.info("Scheduled reencryption [grpId=" + grpId + "]");
+
+            grps.put(grpId, ctx0);
+
+            return ctx0.finishFuture();
+        }
+        finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @return Future that will be completed when all partitions have been scanned and pages have been written to disk.
+     */
+    public IgniteInternalFuture<Void> statusFuture(int grpId) {
+        GroupScanContext ctx0 = grps.get(grpId);
+
+        return ctx0 == null ? new GridFinishedFuture<>() : ctx0.finishFuture();
+    }
+
+    /**
+     * Shutdown scanning and disable new tasks scheduling.
+     */
+    public void stop() throws IgniteCheckedException {
+        lock.lock();
+
+        try {
+            stopped = true;
+
+            for (GroupScanContext ctx0 : grps.values())
+                ctx0.finishFuture().cancel();
+
+            execSvc.shutdown();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Stop scannig the specified partition.
+     *
+     * @param grpId Cache group ID.
+     * @param partId Partition ID.
+     * @return {@code True} if reencryption was cancelled.
+     * @throws IgniteCheckedException If failed.
+     */
+    public boolean cancel(int grpId, int partId) throws IgniteCheckedException {
+        GroupScanContext ctx = grps.get(grpId);
+
+        if (ctx == null)
+            return false;
+
+        return ctx.cancel(partId);
+    }
+
+    /**
+     * Collect current number of pages in the specified cache group.
+     *
+     * @param grp Cache group.
+     * @return Map of partitions with current page count.
+     * @throws IgniteCheckedException If failed.
+     */
+    public Map<Integer, Long> pagesCount(CacheGroupContext grp) throws IgniteCheckedException {
+        Map<Integer, Long> partStates = new HashMap<>();
+
+        ctx.cache().context().database().checkpointReadLock();
+
+        try {
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) throws IgniteCheckedException {
+                    int pagesCnt = ctx.cache().context().pageStore().pages(grp.groupId(), partId);
+
+                    partStates.put(partId, (long)pagesCnt);
+                }
+            });
+        } finally {
+            ctx.cache().context().database().checkpointReadUnlock();
+        }
+
+        return partStates;
+    }
+
+    /**
+     * @param grp Cache group.
+     * @param hnd Partition handler.
+     */
+    private void forEachPageStore(CacheGroupContext grp, IgniteInClosureX<Integer> hnd) throws IgniteCheckedException {
+        int parts = grp.affinity().partitions();
+
+        IgnitePageStoreManager pageStoreMgr = ctx.cache().context().pageStore();
+
+        for (int p = 0; p < parts; p++) {
+            if (!pageStoreMgr.exists(grp.groupId(), p))
+                continue;
+
+            hnd.applyx(p);
+        }
+
+        hnd.applyx(PageIdAllocator.INDEX_PARTITION);
+    }
+
+    /**
+     * Cache group scanning context.
+     */
+    private static class GroupScanContext {
+        /** Partition scanning futures. */
+        private final Map<Integer, IgniteInternalFuture<Void>> futMap = new ConcurrentHashMap<>();
+
+        /** Compound future, that will be completed when all partitions scanned. */
+        private final GridCompoundFuture<Void, Void> compFut = new GridCompoundFuture<>();
+
+        /** Cache group ID. */
+        private final int grpId;
+
+        /** Future that ends after all partitions are done and a checkpoint is finished. */
+        private final GridFutureAdapter<Void> cpFut = new GridFutureAdapter<Void>() {
+            @Override public boolean cancel() throws IgniteCheckedException {
+                compFut.cancel();
+
+                return onDone(null, null, true);
+            }
+        };
+
+        /**
+         * @param grpId Cache group ID.
+         */
+        public GroupScanContext(int grpId) {
+            this.grpId = grpId;
+        }
+
+        /**
+         * @return Cache group ID.
+         */
+        public int groupId() {
+            return grpId;
+        }
+
+        /**
+         * @param partId Partition ID.
+         * @param fut Partition scanning future.
+         */
+        public void add(int partId, IgniteInternalFuture<Void> fut) {
+            compFut.add(fut);
+
+            futMap.put(partId, fut);
+        }
+
+        /**
+         * @return Compound future, that will be completed when all partitions scanned.
+         */
+        public IgniteInternalFuture<Void> initialize() {
+            return compFut.markInitialized();
+        }
+
+        /**
+         * @return Future that ends after all partitions are done and a checkpoint is finished.
+         */
+        public IgniteInternalFuture<Void> finishFuture() {
+            return cpFut;
+        }
+
+        /**
+         * Finish reencryption future.
+         *
+         * @return {@code True} if the future was finished by this call.
+         */
+        public boolean finish() {
+            return cpFut.onDone(compFut.result());
+        }
+
+        /**
+         * Stop reencryption of the specified partition.
+         *
+         * @param partId Partition ID.
+         * @return {@code True} if reencryption was cancelled.
+         * @throws IgniteCheckedException If failed.
+         */
+        public boolean cancel(int partId) throws IgniteCheckedException {
+            IgniteInternalFuture<Void> fut = futMap.get(partId);
+
+            if (fut == null)
+                return false;
+
+            return fut.cancel();
+        }
+
+        /**
+         * @param t Throwable.
+         */
+        public void fail(Throwable t) {
+            cpFut.onDone(t);
+        }
+    }
+
+    /**
+     * Page store scanning task.
+     */
+    private class PageStoreScanTask extends GridFutureAdapter<Void> implements Runnable {
+        /** Partiion ID. */
+        private final int partId;
+
+        /** Cache group scan context. */
+        private final GroupScanContext scanCtx;
+
+        /**
+         * @param scanCtx Cache group scanning context.
+         * @param partId Partition ID.
+         */
+        public PageStoreScanTask(GroupScanContext scanCtx, int partId) {
+            this.scanCtx = scanCtx;
+            this.partId = partId;
+        }
+
+        /** {@inheritDoc} */
+        @Override public synchronized boolean cancel() throws IgniteCheckedException {
+            return onDone();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void run() {
+            try {
+                int grpId = scanCtx.groupId();
+
+                CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+                if (grp == null) {
+                    onDone();
+
+                    return;
+                }
+
+                PageMemoryEx pageMem = (PageMemoryEx)grp.dataRegion().pageMemory();
+                long metaPageId = pageMem.partitionMetaPageId(grpId, partId);
+                long state = ctx.encryption().getEncryptionState(grpId, partId);
+                int batchSize = encrCfg.getReencryptionBatchSize();
+
+                int off = (int)(state >> Integer.SIZE);
+                int cnt = (int)state;
+
+                if (log.isDebugEnabled()) {
+                    log.debug("Partition reencryption is started [grpId=" + grpId +
+                        ", p=" + partId + ", remain=" + (cnt - off) + ", total=" + cnt + "]");
+                }
+
+                while (off < cnt) {
+                    int pagesCnt = Math.min(batchSize, cnt - off);
+
+                    if (limiter != null)
+                        limiter.acquire(pagesCnt);
+
+                    synchronized (this) {
+                        if (isDone())
+                            break;
+
+                        ctx.cache().context().database().checkpointReadLock();
+
+                        try {
+                            off += scanPages(pageMem, metaPageId + off, pagesCnt);

Review comment:
       Let's not rely on knowledge of pageId structure and pass just offset to scanPages and generate pageId each time from offset by page index.

##########
File path: modules/core/src/test/config/log4j-test.xml
##########
@@ -84,6 +84,10 @@
     </category>
     -->
 
+    <category name="org.apache.ignite.internal.managers.encryption">

Review comment:
       Don't forget to remove it after testing.




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

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



[GitHub] [ignite] vldpyatkov commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupPageScanner.java
##########
@@ -0,0 +1,479 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId;
+import org.apache.ignite.internal.util.BasicRateLimiter;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+import org.apache.ignite.thread.OomExceptionHandler;
+
+import static org.apache.ignite.internal.util.IgniteUtils.MB;
+
+/**
+ * Cache group page stores scanner.
+ * Scans a range of pages and marks them as dirty to re-encrypt them with the last encryption key on disk.
+ */
+public class CacheGroupPageScanner implements CheckpointListener {
+    /** Thread prefix for scanning tasks. */
+    private static final String REENCRYPT_THREAD_PREFIX = "reencrypt";
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to group scanning task. */
+    private final Map<Integer, GroupScanTask> grps = new ConcurrentHashMap<>();
+
+    /** Collection of groups waiting for a checkpoint. */
+    private final Collection<GroupScanTask> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Page scanning speed limiter. */
+    private final BasicRateLimiter limiter;
+
+    /** Single-threaded executor to run cache group scan task. */
+    private final ThreadPoolExecutor singleExecSvc;
+
+    /** Number of pages that is scanned during reencryption under checkpoint lock. */
+    private final int batchSize;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupPageScanner(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        DataStorageConfiguration dsCfg = ctx.config().getDataStorageConfiguration();
+
+        if (!CU.isPersistenceEnabled(dsCfg)) {
+            batchSize = -1;
+            limiter = null;
+            singleExecSvc = null;
+
+            return;
+        }
+
+        double rateLimit = dsCfg.getEncryptionConfiguration().getReencryptionRateLimit();
+
+        limiter = rateLimit > 0 ? new BasicRateLimiter(rateLimit * MB /
+            (dsCfg.getPageSize() == 0 ? DataStorageConfiguration.DFLT_PAGE_SIZE : dsCfg.getPageSize())) : null;
+
+        batchSize = dsCfg.getEncryptionConfiguration().getReencryptionBatchSize();
+
+        singleExecSvc = new IgniteThreadPoolExecutor(REENCRYPT_THREAD_PREFIX,
+            ctx.igniteInstanceName(),
+            1,
+            1,
+            IgniteConfiguration.DFLT_THREAD_KEEP_ALIVE_TIME,
+            new LinkedBlockingQueue<>(),
+            GridIoPolicy.SYSTEM_POOL,
+            new OomExceptionHandler(ctx));
+
+        singleExecSvc.allowCoreThreadTimeOut(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCheckpointBegin(Context cpCtx) {
+        Set<GroupScanTask> completeCandidates = new HashSet<>();
+
+        cpWaitGrps.removeIf(completeCandidates::add);
+
+        cpCtx.finishedStateFut().listen(
+            f -> {
+                // Retry if error occurs.
+                if (f.error() != null || f.isCancelled()) {
+                    cpWaitGrps.addAll(completeCandidates);
+
+                    return;
+                }
+
+                lock.lock();
+
+                try {
+                    for (GroupScanTask grpScanTask : completeCandidates) {
+                        grps.remove(grpScanTask.groupId());
+
+                        grpScanTask.onDone();
+
+                        if (log.isInfoEnabled())
+                            log.info("Cache group reencryption is finished [grpId=" + grpScanTask.groupId() + "]");
+                    }
+
+                    if (!grps.isEmpty())
+                        return;
+
+                    ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).
+                        removeCheckpointListener(this);
+                }
+                finally {
+                    lock.unlock();
+                }
+            }
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public void beforeCheckpointBegin(Context cpCtx) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onMarkCheckpointBegin(Context ctx) {
+        // No-op.
+    }
+
+    /**
+     * Schedule scanning partitions.
+     *
+     * @param grpId Cache group ID.
+     */
+    public IgniteInternalFuture<Void> schedule(int grpId) throws IgniteCheckedException {
+        CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+        if (grp == null || !grp.affinityNode()) {
+            if (log.isInfoEnabled())
+                log.info("Skip reencryption, cache group doesn't exist on the local node [grp=" + grpId + "]");
+
+            return new GridFinishedFuture<>();
+        }
+
+        lock.lock();
+
+        try {
+            if (stopped)
+                throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
+
+            if (grps.isEmpty())
+                ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).addCheckpointListener(this);
+
+            GroupScanTask prevState = grps.get(grpId);
+
+            if (prevState != null && !prevState.isDone()) {
+                if (log.isDebugEnabled())
+                    log.debug("Reencryption already scheduled [grpId=" + grpId + "]");
+
+                return prevState;
+            }
+
+            Set<Integer> parts = new HashSet<>();
+
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) {
+                    if (ctx.encryption().getEncryptionState(grpId, partId) == 0) {
+                        if (log.isDebugEnabled())
+                            log.debug("Skipping partition reencryption [grp=" + grpId + ", p=" + partId + "]");
+
+                        return;
+                    }
+
+                    parts.add(partId);
+                }
+            });
+
+            GroupScanTask grpScan = new GroupScanTask(grp, parts);
+
+            singleExecSvc.submit(grpScan);
+
+            if (log.isInfoEnabled())
+                log.info("Scheduled reencryption [grpId=" + grpId + "]");
+
+            grps.put(grpId, grpScan);
+
+            return grpScan;
+        }
+        finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @return Future that will be completed when all partitions have been scanned and pages have been written to disk.
+     */
+    public IgniteInternalFuture<Void> statusFuture(int grpId) {
+        GroupScanTask grpScanTask = grps.get(grpId);
+
+        return grpScanTask == null ? new GridFinishedFuture<>() : grpScanTask;
+    }
+
+    /**
+     * Shutdown scanning and disable new tasks scheduling.
+     */
+    public void stop() throws IgniteCheckedException {
+        lock.lock();
+
+        try {
+            stopped = true;
+
+            for (GroupScanTask grpScanTask : grps.values())
+                grpScanTask.cancel();
+
+            if (singleExecSvc != null)
+                singleExecSvc.shutdownNow();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Stop scannig the specified partition.
+     *
+     * @param grpId Cache group ID.
+     * @param partId Partition ID.
+     * @return {@code True} if reencryption was cancelled.
+     */
+    public boolean excludePartition(int grpId, int partId) {
+        GroupScanTask grpScanTask = grps.get(grpId);
+
+        if (grpScanTask == null)
+            return false;
+
+        return grpScanTask.excludePartition(partId);
+    }
+
+    /**
+     * Collect current number of pages in the specified cache group.
+     *
+     * @param grp Cache group.
+     * @return Partitions with current page count.
+     * @throws IgniteCheckedException If failed.
+     */
+    public long[] pagesCount(CacheGroupContext grp) throws IgniteCheckedException {
+        // The last element of the array is used to store the status of the index partition.
+        long[] partStates = new long[grp.affinity().partitions() + 1];
+
+        ctx.cache().context().database().checkpointReadLock();
+
+        try {
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) throws IgniteCheckedException {
+                    int pagesCnt = ctx.cache().context().pageStore().pages(grp.groupId(), partId);
+
+                    partStates[Math.min(partId, partStates.length - 1)] = pagesCnt;
+                }
+            });
+        } finally {
+            ctx.cache().context().database().checkpointReadUnlock();
+        }
+
+        return partStates;
+    }
+
+    /**
+     * @param grp Cache group.
+     * @param hnd Partition handler.
+     */
+    private void forEachPageStore(CacheGroupContext grp, IgniteInClosureX<Integer> hnd) throws IgniteCheckedException {
+        int parts = grp.affinity().partitions();
+
+        IgnitePageStoreManager pageStoreMgr = ctx.cache().context().pageStore();
+
+        for (int p = 0; p < parts; p++) {
+            if (!pageStoreMgr.exists(grp.groupId(), p))
+                continue;
+
+            hnd.applyx(p);
+        }
+
+        hnd.applyx(PageIdAllocator.INDEX_PARTITION);
+    }
+
+    /**
+     * Cache group partition scanning task.
+     */
+    private class GroupScanTask extends GridFutureAdapter<Void> implements Runnable {
+        /** Cache group ID. */
+        private final CacheGroupContext grp;
+
+        /** Partition IDs. */
+        private final Set<Integer> parts;
+
+        /** Page memory. */
+        private final PageMemoryEx pageMem;
+
+        /**
+         * @param grp Cache group.
+         */
+        public GroupScanTask(CacheGroupContext grp, Set<Integer> parts) {
+            this.grp = grp;
+            this.parts = new GridConcurrentHashSet<>(parts);
+
+            pageMem = (PageMemoryEx)grp.dataRegion().pageMemory();
+        }
+
+        /** {@inheritDoc} */
+        @Override public synchronized boolean cancel() throws IgniteCheckedException {
+            return onCancelled();
+        }
+
+        /**
+         * Stop reencryption of the specified partition.
+         *
+         * @param partId Partition ID.
+         * @return {@code True} if reencryption was cancelled.
+         */
+        public synchronized boolean excludePartition(int partId) {
+            return parts.remove(partId);
+        }
+
+        /**
+         * @return Cache group ID.
+         */
+        public int groupId() {
+            return grp.groupId();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void run() {
+            try {
+                for (int partId : parts) {
+                    long state = ctx.encryption().getEncryptionState(grp.groupId(), partId);
+
+                    if (state == 0)
+                        continue;
+
+                    scanPartition(partId, ReencryptStateUtils.pageIndex(state), ReencryptStateUtils.pageCount(state));
+
+                    if (isDone())
+                        return;
+                }
+
+                boolean added = cpWaitGrps.add(this);

Review comment:
       I see that we cannot guarantee which pages included in that checkpoint. Because onCheckpointBegin method is executed out of write lock, that means some dirty pages will belong to another checkpoint (next after that).




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GroupKeyChangeProcess.java
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteFeatures;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.managers.encryption.GridEncryptionManager.EmptyResult;
+import org.apache.ignite.internal.managers.encryption.GridEncryptionManager.KeyChangeFuture;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
+import org.apache.ignite.internal.util.distributed.DistributedProcess;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.future.IgniteFinishedFutureImpl;
+import org.apache.ignite.internal.util.future.IgniteFutureImpl;
+import org.apache.ignite.internal.util.typedef.F;
+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.IgniteFuture;
+import org.apache.ignite.lang.IgniteFutureCancelledException;
+
+import static org.apache.ignite.internal.IgniteFeatures.CACHE_GROUP_KEY_CHANGE;
+import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.CACHE_GROUP_KEY_CHANGE_FINISH;
+import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.CACHE_GROUP_KEY_CHANGE_PREPARE;
+
+/**
+ * Two phase distributed process, that performs cache group encryption key rotation.
+ */
+class GroupKeyChangeProcess {
+    /** Grid kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Cache group encyption key change prepare phase. */
+    private final DistributedProcess<ChangeCacheEncryptionRequest, EmptyResult> prepareGKChangeProc;
+
+    /** Cache group encyption key change perform phase. */
+    private final DistributedProcess<ChangeCacheEncryptionRequest, EmptyResult> performGKChangeProc;
+
+    /** Group encryption keys. */
+    private final CacheGroupEncryptionKeys keys;
+
+    /** Cache group key change future. */
+    private volatile GroupKeyChangeFuture fut;
+
+    /** Cache group key change request. */
+    private volatile ChangeCacheEncryptionRequest req;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    GroupKeyChangeProcess(GridKernalContext ctx, CacheGroupEncryptionKeys keys) {
+        this.ctx = ctx;
+        this.keys = keys;
+
+        prepareGKChangeProc =
+            new DistributedProcess<>(ctx, CACHE_GROUP_KEY_CHANGE_PREPARE, this::prepare, this::finishPrepare);
+        performGKChangeProc =
+            new DistributedProcess<>(ctx, CACHE_GROUP_KEY_CHANGE_FINISH, this::perform, this::finishPerform);
+    }
+
+    /**
+     * @return {@code True} if operation is still in progress.
+     */
+    public boolean started() {
+        return req != null;
+    }
+
+    /**
+     * @return {@code True} if operation is not finished.
+     */
+    public boolean finished() {
+        IgniteInternalFuture<Void> fut0 = fut;
+
+        return fut0 == null || fut0.isDone();
+    }
+
+    /**
+     * @param msg Error message.
+     */
+    public void cancel(String msg) {
+        GridFutureAdapter<Void> keyChangeFut = fut;
+
+        if (keyChangeFut != null && !keyChangeFut.isDone())
+            keyChangeFut.onDone(new IgniteFutureCancelledException(msg));
+    }
+
+    /**
+     * Starts cache group encryption key change process.
+     *
+     * @param cacheOrGrpNames Cache or group names.
+     */
+    public IgniteFuture<Void> start(Collection<String> cacheOrGrpNames) {
+        if (ctx.clientNode())
+            throw new UnsupportedOperationException("Client and daemon nodes can not perform this operation.");
+
+        if (!IgniteFeatures.allNodesSupports(ctx.grid().cluster().nodes(), CACHE_GROUP_KEY_CHANGE))
+            throw new IllegalStateException("Not all nodes in the cluster support this operation.");
+
+        if (!ctx.state().clusterState().state().active())
+            throw new IgniteException("Operation was rejected. The cluster is inactive.");
+
+        if (!finished()) {
+            return new IgniteFinishedFutureImpl<>(new IgniteException("Cache group key change was rejected. " +
+                "The previous change was not completed."));
+        }
+
+        int[] grpIds = new int[cacheOrGrpNames.size()];
+        byte[] keyIds = new byte[grpIds.length];
+
+        int n = 0;
+
+        for (String cacheOrGroupName : cacheOrGrpNames) {
+            CacheGroupContext grp = ctx.cache().cacheGroup(CU.cacheId(cacheOrGroupName));

Review comment:
       Changed to descriptors, added test with node filter




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/EncryptedRecord.java
##########
@@ -43,7 +43,7 @@ public EncryptedRecord(int grpId, RecordType plainRecType) {
 
     /** {@inheritDoc} */
     @Override public RecordType type() {
-        return RecordType.ENCRYPTED_RECORD;
+        return RecordType.ENCRYPTED_RECORD_V2;

Review comment:
       I updated this patch and kept the original WAL records (EncryptedRecord and MasterKeyChangeRecord) unchanged.




----------------------------------------------------------------
This is an automated message from the 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] NSAmelchev commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -182,7 +195,7 @@
     private volatile boolean recoveryMasterKeyName;
 
     /** Master key change future. Not {@code null} on request initiator. */

Review comment:
       Javadoc shpuld be fixed too




----------------------------------------------------------------
This is an automated message from the 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] vldpyatkov commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupPageScanner.java
##########
@@ -0,0 +1,479 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId;
+import org.apache.ignite.internal.util.BasicRateLimiter;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+import org.apache.ignite.thread.OomExceptionHandler;
+
+import static org.apache.ignite.internal.util.IgniteUtils.MB;
+
+/**
+ * Cache group page stores scanner.
+ * Scans a range of pages and marks them as dirty to re-encrypt them with the last encryption key on disk.
+ */
+public class CacheGroupPageScanner implements CheckpointListener {
+    /** Thread prefix for scanning tasks. */
+    private static final String REENCRYPT_THREAD_PREFIX = "reencrypt";
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to group scanning task. */
+    private final Map<Integer, GroupScanTask> grps = new ConcurrentHashMap<>();
+
+    /** Collection of groups waiting for a checkpoint. */
+    private final Collection<GroupScanTask> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Page scanning speed limiter. */
+    private final BasicRateLimiter limiter;
+
+    /** Single-threaded executor to run cache group scan task. */
+    private final ThreadPoolExecutor singleExecSvc;
+
+    /** Number of pages that is scanned during reencryption under checkpoint lock. */
+    private final int batchSize;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupPageScanner(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        DataStorageConfiguration dsCfg = ctx.config().getDataStorageConfiguration();
+
+        if (!CU.isPersistenceEnabled(dsCfg)) {
+            batchSize = -1;
+            limiter = null;
+            singleExecSvc = null;
+
+            return;
+        }
+
+        double rateLimit = dsCfg.getEncryptionConfiguration().getReencryptionRateLimit();
+
+        limiter = rateLimit > 0 ? new BasicRateLimiter(rateLimit * MB /
+            (dsCfg.getPageSize() == 0 ? DataStorageConfiguration.DFLT_PAGE_SIZE : dsCfg.getPageSize())) : null;
+
+        batchSize = dsCfg.getEncryptionConfiguration().getReencryptionBatchSize();
+
+        singleExecSvc = new IgniteThreadPoolExecutor(REENCRYPT_THREAD_PREFIX,
+            ctx.igniteInstanceName(),
+            1,
+            1,
+            IgniteConfiguration.DFLT_THREAD_KEEP_ALIVE_TIME,
+            new LinkedBlockingQueue<>(),
+            GridIoPolicy.SYSTEM_POOL,
+            new OomExceptionHandler(ctx));
+
+        singleExecSvc.allowCoreThreadTimeOut(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCheckpointBegin(Context cpCtx) {
+        Set<GroupScanTask> completeCandidates = new HashSet<>();
+
+        cpWaitGrps.removeIf(completeCandidates::add);
+
+        cpCtx.finishedStateFut().listen(
+            f -> {
+                // Retry if error occurs.
+                if (f.error() != null || f.isCancelled()) {
+                    cpWaitGrps.addAll(completeCandidates);
+
+                    return;
+                }
+
+                lock.lock();
+
+                try {
+                    for (GroupScanTask grpScanTask : completeCandidates) {
+                        grps.remove(grpScanTask.groupId());
+
+                        grpScanTask.onDone();
+
+                        if (log.isInfoEnabled())
+                            log.info("Cache group reencryption is finished [grpId=" + grpScanTask.groupId() + "]");
+                    }
+
+                    if (!grps.isEmpty())
+                        return;
+
+                    ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).
+                        removeCheckpointListener(this);
+                }
+                finally {
+                    lock.unlock();
+                }
+            }
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public void beforeCheckpointBegin(Context cpCtx) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onMarkCheckpointBegin(Context ctx) {
+        // No-op.
+    }
+
+    /**
+     * Schedule scanning partitions.
+     *
+     * @param grpId Cache group ID.
+     */
+    public IgniteInternalFuture<Void> schedule(int grpId) throws IgniteCheckedException {
+        CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+        if (grp == null || !grp.affinityNode()) {
+            if (log.isInfoEnabled())
+                log.info("Skip reencryption, cache group doesn't exist on the local node [grp=" + grpId + "]");
+
+            return new GridFinishedFuture<>();
+        }
+
+        lock.lock();
+
+        try {
+            if (stopped)
+                throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
+
+            if (grps.isEmpty())
+                ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).addCheckpointListener(this);
+
+            GroupScanTask prevState = grps.get(grpId);
+
+            if (prevState != null && !prevState.isDone()) {
+                if (log.isDebugEnabled())
+                    log.debug("Reencryption already scheduled [grpId=" + grpId + "]");
+
+                return prevState;
+            }
+
+            Set<Integer> parts = new HashSet<>();
+
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) {
+                    if (ctx.encryption().getEncryptionState(grpId, partId) == 0) {
+                        if (log.isDebugEnabled())
+                            log.debug("Skipping partition reencryption [grp=" + grpId + ", p=" + partId + "]");
+
+                        return;
+                    }
+
+                    parts.add(partId);
+                }
+            });
+
+            GroupScanTask grpScan = new GroupScanTask(grp, parts);
+
+            singleExecSvc.submit(grpScan);
+
+            if (log.isInfoEnabled())
+                log.info("Scheduled reencryption [grpId=" + grpId + "]");
+
+            grps.put(grpId, grpScan);
+
+            return grpScan;
+        }
+        finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @return Future that will be completed when all partitions have been scanned and pages have been written to disk.
+     */
+    public IgniteInternalFuture<Void> statusFuture(int grpId) {
+        GroupScanTask grpScanTask = grps.get(grpId);
+
+        return grpScanTask == null ? new GridFinishedFuture<>() : grpScanTask;
+    }
+
+    /**
+     * Shutdown scanning and disable new tasks scheduling.
+     */
+    public void stop() throws IgniteCheckedException {
+        lock.lock();
+
+        try {
+            stopped = true;
+
+            for (GroupScanTask grpScanTask : grps.values())
+                grpScanTask.cancel();
+
+            if (singleExecSvc != null)
+                singleExecSvc.shutdownNow();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Stop scannig the specified partition.
+     *
+     * @param grpId Cache group ID.
+     * @param partId Partition ID.
+     * @return {@code True} if reencryption was cancelled.
+     */
+    public boolean excludePartition(int grpId, int partId) {
+        GroupScanTask grpScanTask = grps.get(grpId);
+
+        if (grpScanTask == null)
+            return false;
+
+        return grpScanTask.excludePartition(partId);
+    }
+
+    /**
+     * Collect current number of pages in the specified cache group.
+     *
+     * @param grp Cache group.
+     * @return Partitions with current page count.
+     * @throws IgniteCheckedException If failed.
+     */
+    public long[] pagesCount(CacheGroupContext grp) throws IgniteCheckedException {
+        // The last element of the array is used to store the status of the index partition.
+        long[] partStates = new long[grp.affinity().partitions() + 1];
+
+        ctx.cache().context().database().checkpointReadLock();
+
+        try {
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) throws IgniteCheckedException {
+                    int pagesCnt = ctx.cache().context().pageStore().pages(grp.groupId(), partId);
+
+                    partStates[Math.min(partId, partStates.length - 1)] = pagesCnt;
+                }
+            });
+        } finally {
+            ctx.cache().context().database().checkpointReadUnlock();
+        }
+
+        return partStates;
+    }
+
+    /**
+     * @param grp Cache group.
+     * @param hnd Partition handler.
+     */
+    private void forEachPageStore(CacheGroupContext grp, IgniteInClosureX<Integer> hnd) throws IgniteCheckedException {
+        int parts = grp.affinity().partitions();
+
+        IgnitePageStoreManager pageStoreMgr = ctx.cache().context().pageStore();
+
+        for (int p = 0; p < parts; p++) {
+            if (!pageStoreMgr.exists(grp.groupId(), p))
+                continue;
+
+            hnd.applyx(p);
+        }
+
+        hnd.applyx(PageIdAllocator.INDEX_PARTITION);
+    }
+
+    /**
+     * Cache group partition scanning task.
+     */
+    private class GroupScanTask extends GridFutureAdapter<Void> implements Runnable {
+        /** Cache group ID. */
+        private final CacheGroupContext grp;
+
+        /** Partition IDs. */
+        private final Set<Integer> parts;
+
+        /** Page memory. */
+        private final PageMemoryEx pageMem;
+
+        /**
+         * @param grp Cache group.
+         */
+        public GroupScanTask(CacheGroupContext grp, Set<Integer> parts) {
+            this.grp = grp;
+            this.parts = new GridConcurrentHashSet<>(parts);
+
+            pageMem = (PageMemoryEx)grp.dataRegion().pageMemory();
+        }
+
+        /** {@inheritDoc} */
+        @Override public synchronized boolean cancel() throws IgniteCheckedException {
+            return onCancelled();
+        }
+
+        /**
+         * Stop reencryption of the specified partition.
+         *
+         * @param partId Partition ID.
+         * @return {@code True} if reencryption was cancelled.
+         */
+        public synchronized boolean excludePartition(int partId) {
+            return parts.remove(partId);
+        }
+
+        /**
+         * @return Cache group ID.
+         */
+        public int groupId() {
+            return grp.groupId();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void run() {
+            try {
+                for (int partId : parts) {
+                    long state = ctx.encryption().getEncryptionState(grp.groupId(), partId);
+
+                    if (state == 0)
+                        continue;
+
+                    scanPartition(partId, ReencryptStateUtils.pageIndex(state), ReencryptStateUtils.pageCount(state));
+
+                    if (isDone())
+                        return;
+                }
+
+                boolean added = cpWaitGrps.add(this);

Review comment:
       Now, we can get information about a written group after than that happens.
   If it is ok, that is right. I still badly oriented in these patch.




----------------------------------------------------------------
This is an automated message from the 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] vldpyatkov commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GroupKeyChangeProcess.java
##########
@@ -0,0 +1,350 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteFeatures;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.managers.encryption.GridEncryptionManager.EmptyResult;
+import org.apache.ignite.internal.managers.encryption.GridEncryptionManager.KeyChangeFuture;
+import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor;
+import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
+import org.apache.ignite.internal.util.distributed.DistributedProcess;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.future.IgniteFinishedFutureImpl;
+import org.apache.ignite.internal.util.future.IgniteFutureImpl;
+import org.apache.ignite.internal.util.typedef.F;
+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.IgniteFuture;
+import org.apache.ignite.lang.IgniteFutureCancelledException;
+
+import static org.apache.ignite.internal.IgniteFeatures.CACHE_GROUP_KEY_CHANGE;
+import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.CACHE_GROUP_KEY_CHANGE_FINISH;
+import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.CACHE_GROUP_KEY_CHANGE_PREPARE;
+
+/**
+ * A two-phase distributed process that rotates the encryption keys of specified cache groups and initiates
+ * re-encryption of those cache groups.
+ */
+class GroupKeyChangeProcess {
+    /** Grid kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Cache group encyption key change prepare phase. */
+    private final DistributedProcess<ChangeCacheEncryptionRequest, EmptyResult> prepareGKChangeProc;
+
+    /** Cache group encyption key change perform phase. */
+    private final DistributedProcess<ChangeCacheEncryptionRequest, EmptyResult> performGKChangeProc;
+
+    /** Group encryption keys. */
+    private final CacheGroupEncryptionKeys keys;
+
+    /** Cache group key change future. */
+    private volatile GroupKeyChangeFuture fut;
+
+    /** Cache group key change request. */
+    private volatile ChangeCacheEncryptionRequest req;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    GroupKeyChangeProcess(GridKernalContext ctx, CacheGroupEncryptionKeys keys) {
+        this.ctx = ctx;
+        this.keys = keys;
+
+        prepareGKChangeProc =
+            new DistributedProcess<>(ctx, CACHE_GROUP_KEY_CHANGE_PREPARE, this::prepare, this::finishPrepare);
+        performGKChangeProc =
+            new DistributedProcess<>(ctx, CACHE_GROUP_KEY_CHANGE_FINISH, this::perform, this::finishPerform);
+    }
+
+    /**
+     * @return {@code True} if operation is still in progress.
+     */
+    public boolean inProgress() {
+        return req != null;
+    }
+
+    /**
+     * @param msg Error message.
+     */
+    public void cancel(String msg) {
+        GridFutureAdapter<Void> keyChangeFut = fut;
+
+        if (keyChangeFut != null && !keyChangeFut.isDone())
+            keyChangeFut.onDone(new IgniteFutureCancelledException(msg));
+    }
+
+    /**
+     * Starts cache group encryption key change process.
+     *
+     * @param cacheOrGrpNames Cache or group names.
+     */
+    public IgniteFuture<Void> start(Collection<String> cacheOrGrpNames) {
+        if (ctx.clientNode())
+            throw new UnsupportedOperationException("Client and daemon nodes can not perform this operation.");
+
+        if (!IgniteFeatures.allNodesSupports(ctx.grid().cluster().nodes(), CACHE_GROUP_KEY_CHANGE))
+            throw new IllegalStateException("Not all nodes in the cluster support this operation.");
+
+        if (!ctx.state().clusterState().state().active())
+            throw new IgniteException("Operation was rejected. The cluster is inactive.");
+
+        IgniteInternalFuture<Void> fut0 = fut;
+
+        if (fut0 != null && !fut0.isDone()) {
+            return new IgniteFinishedFutureImpl<>(new IgniteException("Cache group key change was rejected. " +
+                "The previous change was not completed."));
+        }
+
+        int[] grpIds = new int[cacheOrGrpNames.size()];
+        byte[] keyIds = new byte[grpIds.length];
+
+        int n = 0;
+
+        for (String cacheOrGroupName : cacheOrGrpNames) {
+            CacheGroupDescriptor grpDesc = ctx.cache().cacheGroupDescriptor(CU.cacheId(cacheOrGroupName));
+
+            if (grpDesc == null) {
+                DynamicCacheDescriptor cacheDesc = ctx.cache().cacheDescriptor(cacheOrGroupName);
+
+                if (cacheDesc == null) {
+                    throw new IgniteException("Cache group key change was rejected. " +
+                        "Cache or group \"" + cacheOrGroupName + "\" doesn't exists");
+                }
+
+                int grpId = cacheDesc.groupId();
+
+                grpDesc = ctx.cache().cacheGroupDescriptor(grpId);
+
+                if (grpDesc.sharedGroup()) {
+                    throw new IgniteException("Cache group key change was rejected. " +
+                        "Cache or group \"" + cacheOrGroupName + "\" is a part of group \"" +
+                        grpDesc.groupName() + "\". Provide group name instead of cache name for shared groups.");
+                }
+            }
+
+            if (!grpDesc.config().isEncryptionEnabled()) {
+                throw new IgniteException("Cache group key change was rejected. " +
+                    "Cache or group \"" + cacheOrGroupName + "\" is not encrypted.");
+            }
+
+            if (ctx.encryption().reencryptionInProgress(grpDesc.groupId())) {
+                throw new IgniteException("Cache group key change was rejected. " +
+                    "Cache group reencryption is in progress [grp=" + cacheOrGroupName + "]");
+            }
+
+            grpIds[n] = grpDesc.groupId();
+            keyIds[n] = (byte)(ctx.encryption().groupKey(grpDesc.groupId()).unsignedId() + 1);
+
+            n += 1;
+        }
+
+        byte[][] keys = ctx.encryption().createKeys(grpIds.length).get1().toArray(new byte[grpIds.length][]);
+
+        ChangeCacheEncryptionRequest req =
+            new ChangeCacheEncryptionRequest(grpIds, keys, keyIds, ctx.config().getEncryptionSpi().getMasterKeyName());
+
+        fut = new GroupKeyChangeFuture(req);
+
+        prepareGKChangeProc.start(req.requestId(), req);
+
+        return new IgniteFutureImpl<>(fut);
+    }
+
+    /**
+     * Validates existing keys.
+     *
+     * @param req Request.
+     * @return Result future.
+     */
+    private IgniteInternalFuture<EmptyResult> prepare(ChangeCacheEncryptionRequest req) {
+        if (ctx.clientNode())
+            return new GridFinishedFuture<>();
+
+        if (inProgress()) {
+            return new GridFinishedFuture<>(new IgniteException("Cache group key change was rejected. " +
+                "The previous change was not completed."));
+        }
+
+        this.req = req;
+
+        try {
+            for (int i = 0; i < req.groupIds().length; i++) {
+                int grpId = req.groupIds()[i];
+                int keyId = req.keyIds()[i] & 0xff;
+
+                if (ctx.encryption().reencryptionInProgress(grpId)) {
+                    return new GridFinishedFuture<>(new IgniteException("Cache group key change was rejected. " +
+                            "Cache group reencryption is in progress [grpId=" + grpId + "]"));
+                }
+
+                List<Integer> keyIds = ctx.encryption().groupKeyIds(grpId);
+
+                if (keyIds == null) {
+                    return new GridFinishedFuture<>(new IgniteException("Cache group key change was rejected." +
+                            "Encrypted cache group not found [grpId=" + grpId + "]"));
+                }
+
+                GroupKey currKey = ctx.encryption().groupKey(grpId);
+
+                for (int locKeyId : keyIds) {
+                    if (locKeyId != keyId)
+                        continue;
+
+                    Long walSegment = keys.reservedSegment(grpId, keyId);
+
+                    // Can overwrite inactive key if it was added during prepare phase.
+                    if (walSegment == null && currKey.id() != (byte)keyId)
+                        continue;
+
+                    return new GridFinishedFuture<>(
+                        new IgniteException("Cache group key change was rejected. Cannot add new key identifier, " +
+                        "it's already present. There existing WAL segments that encrypted with this key [" +
+                        "grpId=" + grpId + ", newId=" + keyId + ", currId=" + currKey.unsignedId() +
+                        ", walSegment=" + walSegment + "]."));
+                }
+            }
+
+            return ctx.encryption().withMasterKeyChangeReadLock(() -> {
+                String curMasterKeyName = ctx.config().getEncryptionSpi().getMasterKeyName();
+
+                if (!curMasterKeyName.equals(req.masterKeyName())) {
+                    return new GridFinishedFuture<>(new IgniteException("Cache group key change was rejected. " +
+                        "Master key has been changed."));
+                }
+
+                for (int i = 0; i < req.groupIds().length; i++) {
+                    // Store new key as inactive.
+                    GroupKeyEncrypted grpKey = new GroupKeyEncrypted(req.keyIds()[i] & 0xff, req.keys()[i]);
+
+                    ctx.encryption().addGroupKey(req.groupIds()[i], grpKey);

Review comment:
       Why we need to store the key here, we will still save it on the perform stage?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/distributed/DistributedProcess.java
##########
@@ -430,6 +430,16 @@ private Process(UUID id) {
          *
          * @see IgniteSnapshotManager
          */
-        END_SNAPSHOT
+        END_SNAPSHOT,
+
+        /**
+         * Cache group encyption key change prepare phase.
+         */
+        CACHE_GROUP_KEY_CHANGE_PREPARE,

Review comment:
       If we often extend this enum, I recommend do not use native ordinal for it.
   Prefer to use enum with strongly fixed ordinal (like ShutdownPolicy)
   It will be able to lead to compatibility problems in the future.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupPageScanner.java
##########
@@ -0,0 +1,479 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId;
+import org.apache.ignite.internal.util.BasicRateLimiter;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+import org.apache.ignite.thread.OomExceptionHandler;
+
+import static org.apache.ignite.internal.util.IgniteUtils.MB;
+
+/**
+ * Cache group page stores scanner.
+ * Scans a range of pages and marks them as dirty to re-encrypt them with the last encryption key on disk.
+ */
+public class CacheGroupPageScanner implements CheckpointListener {
+    /** Thread prefix for scanning tasks. */
+    private static final String REENCRYPT_THREAD_PREFIX = "reencrypt";
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to group scanning task. */
+    private final Map<Integer, GroupScanTask> grps = new ConcurrentHashMap<>();
+
+    /** Collection of groups waiting for a checkpoint. */
+    private final Collection<GroupScanTask> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Page scanning speed limiter. */
+    private final BasicRateLimiter limiter;
+
+    /** Single-threaded executor to run cache group scan task. */
+    private final ThreadPoolExecutor singleExecSvc;
+
+    /** Number of pages that is scanned during reencryption under checkpoint lock. */
+    private final int batchSize;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupPageScanner(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        DataStorageConfiguration dsCfg = ctx.config().getDataStorageConfiguration();
+
+        if (!CU.isPersistenceEnabled(dsCfg)) {
+            batchSize = -1;
+            limiter = null;
+            singleExecSvc = null;
+
+            return;
+        }
+
+        double rateLimit = dsCfg.getEncryptionConfiguration().getReencryptionRateLimit();
+
+        limiter = rateLimit > 0 ? new BasicRateLimiter(rateLimit * MB /
+            (dsCfg.getPageSize() == 0 ? DataStorageConfiguration.DFLT_PAGE_SIZE : dsCfg.getPageSize())) : null;
+
+        batchSize = dsCfg.getEncryptionConfiguration().getReencryptionBatchSize();
+
+        singleExecSvc = new IgniteThreadPoolExecutor(REENCRYPT_THREAD_PREFIX,
+            ctx.igniteInstanceName(),
+            1,
+            1,
+            IgniteConfiguration.DFLT_THREAD_KEEP_ALIVE_TIME,
+            new LinkedBlockingQueue<>(),
+            GridIoPolicy.SYSTEM_POOL,
+            new OomExceptionHandler(ctx));
+
+        singleExecSvc.allowCoreThreadTimeOut(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCheckpointBegin(Context cpCtx) {
+        Set<GroupScanTask> completeCandidates = new HashSet<>();
+
+        cpWaitGrps.removeIf(completeCandidates::add);

Review comment:
       The map (cpWaitGrps) does not synchronous with checkpoint process.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupEncryptionKeys.java
##########
@@ -0,0 +1,376 @@
+/*
+ * 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.managers.encryption;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.CopyOnWriteArrayList;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.encryption.EncryptionSpi;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Serves for managing encryption keys and related datastructure located in the heap.
+ */
+class CacheGroupEncryptionKeys {
+    /** Group encryption keys. */
+    private final Map<Integer, List<GroupKey>> grpKeys = new ConcurrentHashMap<>();
+
+    /**
+     * WAL segments encrypted with previous encryption keys prevent keys from being deleted
+     * until the associated segment is deleted.
+     */
+    private final Collection<TrackedWalSegment> trackedWalSegments = new ConcurrentLinkedQueue<>();

Review comment:
       What kind of WAL segment stored here? All segments which were encrypted or only one for one cache key.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupPageScanner.java
##########
@@ -0,0 +1,479 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId;
+import org.apache.ignite.internal.util.BasicRateLimiter;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+import org.apache.ignite.thread.OomExceptionHandler;
+
+import static org.apache.ignite.internal.util.IgniteUtils.MB;
+
+/**
+ * Cache group page stores scanner.
+ * Scans a range of pages and marks them as dirty to re-encrypt them with the last encryption key on disk.
+ */
+public class CacheGroupPageScanner implements CheckpointListener {
+    /** Thread prefix for scanning tasks. */
+    private static final String REENCRYPT_THREAD_PREFIX = "reencrypt";
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to group scanning task. */
+    private final Map<Integer, GroupScanTask> grps = new ConcurrentHashMap<>();
+
+    /** Collection of groups waiting for a checkpoint. */
+    private final Collection<GroupScanTask> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Page scanning speed limiter. */
+    private final BasicRateLimiter limiter;
+
+    /** Single-threaded executor to run cache group scan task. */
+    private final ThreadPoolExecutor singleExecSvc;
+
+    /** Number of pages that is scanned during reencryption under checkpoint lock. */
+    private final int batchSize;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupPageScanner(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        DataStorageConfiguration dsCfg = ctx.config().getDataStorageConfiguration();
+
+        if (!CU.isPersistenceEnabled(dsCfg)) {
+            batchSize = -1;
+            limiter = null;
+            singleExecSvc = null;
+
+            return;
+        }
+
+        double rateLimit = dsCfg.getEncryptionConfiguration().getReencryptionRateLimit();
+
+        limiter = rateLimit > 0 ? new BasicRateLimiter(rateLimit * MB /
+            (dsCfg.getPageSize() == 0 ? DataStorageConfiguration.DFLT_PAGE_SIZE : dsCfg.getPageSize())) : null;
+
+        batchSize = dsCfg.getEncryptionConfiguration().getReencryptionBatchSize();
+
+        singleExecSvc = new IgniteThreadPoolExecutor(REENCRYPT_THREAD_PREFIX,
+            ctx.igniteInstanceName(),
+            1,
+            1,
+            IgniteConfiguration.DFLT_THREAD_KEEP_ALIVE_TIME,
+            new LinkedBlockingQueue<>(),
+            GridIoPolicy.SYSTEM_POOL,
+            new OomExceptionHandler(ctx));
+
+        singleExecSvc.allowCoreThreadTimeOut(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCheckpointBegin(Context cpCtx) {
+        Set<GroupScanTask> completeCandidates = new HashSet<>();
+
+        cpWaitGrps.removeIf(completeCandidates::add);
+
+        cpCtx.finishedStateFut().listen(
+            f -> {
+                // Retry if error occurs.
+                if (f.error() != null || f.isCancelled()) {
+                    cpWaitGrps.addAll(completeCandidates);
+
+                    return;
+                }
+
+                lock.lock();
+
+                try {
+                    for (GroupScanTask grpScanTask : completeCandidates) {
+                        grps.remove(grpScanTask.groupId());
+
+                        grpScanTask.onDone();
+
+                        if (log.isInfoEnabled())
+                            log.info("Cache group reencryption is finished [grpId=" + grpScanTask.groupId() + "]");
+                    }
+
+                    if (!grps.isEmpty())
+                        return;
+
+                    ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).
+                        removeCheckpointListener(this);
+                }
+                finally {
+                    lock.unlock();
+                }
+            }
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public void beforeCheckpointBegin(Context cpCtx) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onMarkCheckpointBegin(Context ctx) {
+        // No-op.
+    }
+
+    /**
+     * Schedule scanning partitions.
+     *
+     * @param grpId Cache group ID.
+     */
+    public IgniteInternalFuture<Void> schedule(int grpId) throws IgniteCheckedException {
+        CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+        if (grp == null || !grp.affinityNode()) {
+            if (log.isInfoEnabled())
+                log.info("Skip reencryption, cache group doesn't exist on the local node [grp=" + grpId + "]");
+
+            return new GridFinishedFuture<>();
+        }
+
+        lock.lock();
+
+        try {
+            if (stopped)
+                throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
+
+            if (grps.isEmpty())
+                ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).addCheckpointListener(this);
+
+            GroupScanTask prevState = grps.get(grpId);
+
+            if (prevState != null && !prevState.isDone()) {
+                if (log.isDebugEnabled())
+                    log.debug("Reencryption already scheduled [grpId=" + grpId + "]");
+
+                return prevState;
+            }
+
+            Set<Integer> parts = new HashSet<>();
+
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) {
+                    if (ctx.encryption().getEncryptionState(grpId, partId) == 0) {
+                        if (log.isDebugEnabled())
+                            log.debug("Skipping partition reencryption [grp=" + grpId + ", p=" + partId + "]");
+
+                        return;
+                    }
+
+                    parts.add(partId);
+                }
+            });
+
+            GroupScanTask grpScan = new GroupScanTask(grp, parts);
+
+            singleExecSvc.submit(grpScan);
+
+            if (log.isInfoEnabled())
+                log.info("Scheduled reencryption [grpId=" + grpId + "]");
+
+            grps.put(grpId, grpScan);
+
+            return grpScan;
+        }
+        finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @return Future that will be completed when all partitions have been scanned and pages have been written to disk.
+     */
+    public IgniteInternalFuture<Void> statusFuture(int grpId) {
+        GroupScanTask grpScanTask = grps.get(grpId);
+
+        return grpScanTask == null ? new GridFinishedFuture<>() : grpScanTask;
+    }
+
+    /**
+     * Shutdown scanning and disable new tasks scheduling.
+     */
+    public void stop() throws IgniteCheckedException {
+        lock.lock();
+
+        try {
+            stopped = true;
+
+            for (GroupScanTask grpScanTask : grps.values())
+                grpScanTask.cancel();
+
+            if (singleExecSvc != null)
+                singleExecSvc.shutdownNow();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Stop scannig the specified partition.
+     *
+     * @param grpId Cache group ID.
+     * @param partId Partition ID.
+     * @return {@code True} if reencryption was cancelled.
+     */
+    public boolean excludePartition(int grpId, int partId) {
+        GroupScanTask grpScanTask = grps.get(grpId);
+
+        if (grpScanTask == null)
+            return false;
+
+        return grpScanTask.excludePartition(partId);
+    }
+
+    /**
+     * Collect current number of pages in the specified cache group.
+     *
+     * @param grp Cache group.
+     * @return Partitions with current page count.
+     * @throws IgniteCheckedException If failed.
+     */
+    public long[] pagesCount(CacheGroupContext grp) throws IgniteCheckedException {
+        // The last element of the array is used to store the status of the index partition.
+        long[] partStates = new long[grp.affinity().partitions() + 1];
+
+        ctx.cache().context().database().checkpointReadLock();
+
+        try {
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) throws IgniteCheckedException {
+                    int pagesCnt = ctx.cache().context().pageStore().pages(grp.groupId(), partId);
+
+                    partStates[Math.min(partId, partStates.length - 1)] = pagesCnt;
+                }
+            });
+        } finally {
+            ctx.cache().context().database().checkpointReadUnlock();
+        }
+
+        return partStates;
+    }
+
+    /**
+     * @param grp Cache group.
+     * @param hnd Partition handler.
+     */
+    private void forEachPageStore(CacheGroupContext grp, IgniteInClosureX<Integer> hnd) throws IgniteCheckedException {
+        int parts = grp.affinity().partitions();
+
+        IgnitePageStoreManager pageStoreMgr = ctx.cache().context().pageStore();
+
+        for (int p = 0; p < parts; p++) {
+            if (!pageStoreMgr.exists(grp.groupId(), p))
+                continue;
+
+            hnd.applyx(p);
+        }
+
+        hnd.applyx(PageIdAllocator.INDEX_PARTITION);
+    }
+
+    /**
+     * Cache group partition scanning task.
+     */
+    private class GroupScanTask extends GridFutureAdapter<Void> implements Runnable {
+        /** Cache group ID. */
+        private final CacheGroupContext grp;
+
+        /** Partition IDs. */
+        private final Set<Integer> parts;
+
+        /** Page memory. */
+        private final PageMemoryEx pageMem;
+
+        /**
+         * @param grp Cache group.
+         */
+        public GroupScanTask(CacheGroupContext grp, Set<Integer> parts) {
+            this.grp = grp;
+            this.parts = new GridConcurrentHashSet<>(parts);
+
+            pageMem = (PageMemoryEx)grp.dataRegion().pageMemory();
+        }
+
+        /** {@inheritDoc} */
+        @Override public synchronized boolean cancel() throws IgniteCheckedException {
+            return onCancelled();
+        }
+
+        /**
+         * Stop reencryption of the specified partition.
+         *
+         * @param partId Partition ID.
+         * @return {@code True} if reencryption was cancelled.
+         */
+        public synchronized boolean excludePartition(int partId) {
+            return parts.remove(partId);
+        }
+
+        /**
+         * @return Cache group ID.
+         */
+        public int groupId() {
+            return grp.groupId();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void run() {
+            try {
+                for (int partId : parts) {
+                    long state = ctx.encryption().getEncryptionState(grp.groupId(), partId);
+
+                    if (state == 0)
+                        continue;
+
+                    scanPartition(partId, ReencryptStateUtils.pageIndex(state), ReencryptStateUtils.pageCount(state));
+
+                    if (isDone())
+                        return;
+                }
+
+                boolean added = cpWaitGrps.add(this);

Review comment:
       cpWaitGrps is a list of those group which were check pointed.
   How do you guaranty that all pages were written to disk?




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupEncryptionKeys.java
##########
@@ -0,0 +1,376 @@
+/*
+ * 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.managers.encryption;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.CopyOnWriteArrayList;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.encryption.EncryptionSpi;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Serves for managing encryption keys and related datastructure located in the heap.
+ */
+class CacheGroupEncryptionKeys {
+    /** Group encryption keys. */
+    private final Map<Integer, List<GroupKey>> grpKeys = new ConcurrentHashMap<>();
+
+    /**
+     * WAL segments encrypted with previous encryption keys prevent keys from being deleted
+     * until the associated segment is deleted.
+     */
+    private final Collection<TrackedWalSegment> trackedWalSegments = new ConcurrentLinkedQueue<>();

Review comment:
       It is possible that the old key will not be deleted until the next key change.
   
   For example, we start with encryption key with ID = 0.
   
   added new WAL segment 0 (encrypted with key ID = 0)
   ..
   added new WAL segment 10 (encrypted with key ID = 0)
   
   change encryption key ID = 1
   
   added new WAL segment 11 (encrypted with key ID = 1)
   ..
   added new WAL segment 15 (encrypted with key ID = 1)
   
   change encryption key ID = 2
   
   added new WAL segment 16 (encrypted with key ID = 2)
   ...
   
   and so on.
   
   So, in order to read segment 3 of WAL (for example, for delta rebalancing), we need to keep the key with ID = 0 until the segment 10 is removed from the disk.




----------------------------------------------------------------
This is an automated message from the 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 #7941: IGNITE-12843

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


   


----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupEncryptionKeys.java
##########
@@ -0,0 +1,376 @@
+/*
+ * 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.managers.encryption;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.CopyOnWriteArrayList;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.encryption.EncryptionSpi;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Serves for managing encryption keys and related datastructure located in the heap.
+ */
+class CacheGroupEncryptionKeys {
+    /** Group encryption keys. */
+    private final Map<Integer, List<GroupKey>> grpKeys = new ConcurrentHashMap<>();
+
+    /**
+     * WAL segments encrypted with previous encryption keys prevent keys from being deleted
+     * until the associated segment is deleted.
+     */
+    private final Collection<TrackedWalSegment> trackedWalSegments = new ConcurrentLinkedQueue<>();

Review comment:
       @vldpyatkov, we cannot use `NavigableMap<Long, TrackedWalSegment>` here, we can use something like `NavigableMap<Long, Collection<Pair <GrpId, KeyId>>` as in theory we can have multiple cache groups for which the encryption key has been changed multiple times on single WAL segment.
   
   I don't think that NavigableMap will be better than the queue in this case.
   
   In my first approach, I used `ConcurrentSkipListMap` and the code looked more complicated, please check this review comment https://github.com/apache/ignite/pull/7941#discussion_r465759058




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/util/BasicRateLimiterTest.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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;
+
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Test;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Rate limiter tests.
+ */
+public class BasicRateLimiterTest {
+    /**
+     * Check rate limit with multiple threads.
+     */
+    @Test
+    public void checkLimitMultithreaded() throws Exception {
+        int opsPerSec = 1_000;
+        int totalOps = 10_000;
+
+        BasicRateLimiter limiter = new BasicRateLimiter(opsPerSec);
+
+        int threads = Runtime.getRuntime().availableProcessors();
+
+        CyclicBarrier ready = new CyclicBarrier(threads + 1);
+
+        AtomicInteger cntr = new AtomicInteger();
+
+        IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(() -> {
+            ready.await();
+
+            do {
+                limiter.acquire(1);
+            }
+            while (!Thread.currentThread().isInterrupted() && cntr.incrementAndGet() < totalOps);
+
+            return null;
+        }, threads, "worker");
+
+        ready.await();
+
+        long startTime = System.currentTimeMillis();
+
+        fut.get();
+
+        long timeSpent = System.currentTimeMillis() - startTime;
+
+        assertEquals(totalOps / opsPerSec, SECONDS.convert(timeSpent, MILLISECONDS));
+    }
+
+    /**
+     * Check that the average speed is limited correctly even if we are acquiring more permits than allowed per second.
+     */
+    @Test
+    public void checkAcquireWithOverflow() throws IgniteInterruptedCheckedException {
+        double permitsPerSec = 0.5;
+        int permitsPerOp = 1;
+        int totalOps = 5;
+
+        BasicRateLimiter limiter = new BasicRateLimiter(permitsPerSec);
+
+        long startTime = System.currentTimeMillis();
+
+        for (int i = 0; i <= totalOps; i++)

Review comment:
       Ratelimiter targets for an average rate of permits per second, these tests have been reworked. To prevent instability of tests, I added a 50% error probability (in general, the error does not exceed 10%).




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java
##########
@@ -258,6 +258,9 @@
     /** */
     public static final short T_MARKER_PAGE = 33;
 
+    /** */
+    public static final short T_INDEX_META = 34;

Review comment:
       @ibessonov, I want to reach an agreement on naming for this change:
   
   Type: T_META
   Io: MetaPageIOV2 _(extends MetaPageIO)_
   
   Wal record: MetaPageUpdate**Index**DataRecord 
   Wal record type: **INDEX**_META_PAGE_DELTA_RECORD
   
   is it 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] sk0x50 commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/EncryptedRecord.java
##########
@@ -43,7 +43,7 @@ public EncryptedRecord(int grpId, RecordType plainRecType) {
 
     /** {@inheritDoc} */
     @Override public RecordType type() {
-        return RecordType.ENCRYPTED_RECORD;
+        return RecordType.ENCRYPTED_RECORD_V2;

Review comment:
       > (see change in RecordDataV1Serializer.readEncryptedData)
   Yep, I saw this, but it does not explain why you modify the existing WAL record.
   
   In my understanding, WAL records must be immutable in order to avoid compatibility issues in the future.
   I would prefer adding a new WAL record (it can extend an existing one).
   Perhaps, RecordDataV1Serializer implementation will be cleaner in that case.




----------------------------------------------------------------
This is an automated message from the 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] alex-plekhanov commented on a change in pull request #7941: IGNITE-12843

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #7941:
URL: https://github.com/apache/ignite/pull/7941#discussion_r495015657



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/encryption/CacheGroupKeyChangeTest.java
##########
@@ -0,0 +1,988 @@
+/*
+ * 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.encryption;
+
+import java.io.File;
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.IgniteException;
+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.configuration.WALMode;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
+import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
+import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
+import org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType;
+import org.apache.ignite.internal.util.distributed.InitMessage;
+import org.apache.ignite.internal.util.distributed.SingleNodeMessage;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.spi.discovery.tcp.TestTcpDiscoverySpi;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.GridTestUtils.DiscoveryHook;
+import org.junit.Test;
+
+import static org.apache.ignite.configuration.WALMode.LOG_ONLY;
+import static org.apache.ignite.configuration.WALMode.NONE;
+import static org.apache.ignite.internal.managers.encryption.GridEncryptionManager.INITIAL_KEY_ID;
+import static org.apache.ignite.spi.encryption.keystore.KeystoreEncryptionSpi.DEFAULT_MASTER_KEY_NAME;
+import static org.apache.ignite.testframework.GridTestUtils.assertThrowsAnyCause;
+import static org.apache.ignite.testframework.GridTestUtils.assertThrowsWithCause;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+import static org.apache.ignite.testframework.GridTestUtils.waitForCondition;
+
+/**
+ * Cache group key change distributed process tests.
+ */
+public class CacheGroupKeyChangeTest extends AbstractEncryptionTest {
+    /** Timeout. */
+    private static final long MAX_AWAIT_MILLIS = 15_000;
+
+    /** */
+    private static final String GRID_2 = "grid-2";
+
+    /** Discovery hook for distributed process. */
+    private InitMessageDiscoveryHook discoveryHook;
+
+    /** Count of cache backups. */
+    private int backups;
+
+    /** WAL mode. */
+    private WALMode walMode = LOG_ONLY;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        if (discoveryHook != null)
+            ((TestTcpDiscoverySpi)cfg.getDiscoverySpi()).discoveryHook(discoveryHook);
+
+        DataStorageConfiguration memCfg = new DataStorageConfiguration()
+            .setDefaultDataRegionConfiguration(
+                new DataRegionConfiguration()
+                    .setMaxSize(100L * 1024 * 1024)
+                    .setPersistenceEnabled(true))
+            .setPageSize(4 * 1024)
+            .setWalSegmentSize(1024 * 1024)
+            .setWalSegments(10)
+            .setMaxWalArchiveSize(20 * 1024 * 1024)
+            .setCheckpointFrequency(30 * 1000L)
+            .setWalMode(walMode);
+
+        cfg.setDataStorageConfiguration(memCfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected <K, V> CacheConfiguration<K, V> cacheConfiguration(String name, String grp) {
+        CacheConfiguration<K, V> cfg = super.cacheConfiguration(name, grp);
+
+        return cfg.setAffinity(new RendezvousAffinityFunction(false, 8)).setBackups(backups);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    @SuppressWarnings("ThrowableNotThrown")
+    public void testRejectNodeJoinDuringRotation() throws Exception {
+        T2<IgniteEx, IgniteEx> grids = startTestGrids(true);
+
+        createEncryptedCache(grids.get1(), grids.get2(), cacheName(), null);
+
+        int grpId = CU.cacheId(cacheName());
+
+        assertEquals(0, grids.get1().context().encryption().groupKey(grpId).id());
+
+        TestRecordingCommunicationSpi commSpi = TestRecordingCommunicationSpi.spi(grids.get2());
+
+        commSpi.blockMessages((node, msg) -> msg instanceof SingleNodeMessage);
+
+        IgniteFuture<Void> fut = grids.get1().encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+
+        commSpi.waitForBlocked();
+
+        assertThrowsWithCause(() -> startGrid(3), IgniteCheckedException.class);
+
+        commSpi.stopBlock();
+
+        fut.get();
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        checkEncryptedCaches(grids.get1(), grids.get2());
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNotAllBltNodesPresent() throws Exception {
+        startTestGrids(true);
+
+        createEncryptedCache(grid(GRID_0), grid(GRID_1), cacheName(), null);
+
+        stopGrid(GRID_1);
+
+        grid(GRID_0).encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get();
+
+        startGrid(GRID_1);
+
+        checkGroupKey(CU.cacheId(cacheName()), INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNodeFailsBeforePrepare() throws Exception {
+        checkNodeFailsDuringRotation(false, true, true);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNodeFailsBeforePerform() throws Exception {
+        checkNodeFailsDuringRotation(false, false, true);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNodeFailsAfterPrepare() throws Exception {
+        checkNodeFailsDuringRotation(false, true, false);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testCrdFailsAfterPrepare() throws Exception {
+        checkNodeFailsDuringRotation(true, true, false);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNodeFailsAfterPerform() throws Exception {
+        checkNodeFailsDuringRotation(false, false, false);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testCrdFailsAfterPerform() throws Exception {
+        checkNodeFailsDuringRotation(true, false, false);
+    }
+
+    /**
+     * @param stopCrd {@code True} to stop coordinator.
+     * @param prepare {@code True} to stop on the prepare phase. {@code False} to stop on the perform phase.
+     * @param discoBlock  {@code True} to block discovery, {@code False} to block communication SPI.
+     */
+    private void checkNodeFailsDuringRotation(boolean stopCrd, boolean prepare, boolean discoBlock) throws Exception {
+        cleanPersistenceDir();
+
+        DistributedProcessType type = prepare ?
+            DistributedProcessType.CACHE_GROUP_KEY_CHANGE_PREPARE : DistributedProcessType.CACHE_GROUP_KEY_CHANGE_FINISH;
+
+        InitMessageDiscoveryHook locHook = new InitMessageDiscoveryHook(type);
+
+        if (discoBlock && stopCrd)
+            discoveryHook = locHook;
+
+        IgniteEx grid0 = startGrid(GRID_0);
+
+        if (discoBlock && !stopCrd)
+            discoveryHook = locHook;
+
+        IgniteEx grid1 = startGrid(GRID_1);
+
+        grid0.cluster().state(ClusterState.ACTIVE);
+
+        createEncryptedCache(grid0, grid1, cacheName(), null);
+
+        int grpId = CU.cacheId(cacheName());
+
+        checkGroupKey(grpId, INITIAL_KEY_ID, MAX_AWAIT_MILLIS);
+
+        TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid1);
+
+        if (!discoBlock) {
+            AtomicBoolean preparePhase = new AtomicBoolean(true);
+
+            spi.blockMessages((node, msg) -> {
+                if (msg instanceof SingleNodeMessage) {
+                    boolean isPrepare = preparePhase.compareAndSet(true, false);
+
+                    return prepare || !isPrepare;
+                }
+
+                return false;
+            });
+        }
+
+        String alive = stopCrd ? GRID_1 : GRID_0;
+        String stopped = stopCrd ? GRID_0 : GRID_1;
+
+        IgniteFuture<Void> changeFut = grid(alive).encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+
+        IgniteInternalFuture<?> stopFut = new GridFinishedFuture<>();
+
+        if (discoBlock) {
+            locHook.waitForBlocked(MAX_AWAIT_MILLIS);
+
+            stopGrid(stopped, true);
+
+            locHook.stopBlock();
+        }
+        else {
+            spi.waitForBlocked();
+
+            stopFut = runAsync(() -> stopGrid(stopped, true));
+        }
+
+        changeFut.get(MAX_AWAIT_MILLIS);
+        stopFut.get(MAX_AWAIT_MILLIS);
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        IgniteEx stoppedNode = startGrid(stopped);
+
+        stoppedNode.resetLostPartitions(Collections.singleton(ENCRYPTED_CACHE));
+
+        awaitPartitionMapExchange();
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        stoppedNode.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get(MAX_AWAIT_MILLIS);
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 2, MAX_AWAIT_MILLIS);
+    }
+
+    /**
+     * Ensures that we can rotate the key more than 255 times.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testKeyIdentifierOverflow() throws Exception {
+        IgniteEx node = startTestGrids(true).get1();
+
+        createEncryptedCache(node, null, cacheName(), null, false);
+
+        int grpId = CU.cacheId(cacheName());
+
+        byte keyId = INITIAL_KEY_ID;
+
+        do {
+            node.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get();
+
+            // Validates reencryption of index partition.
+            checkGroupKey(grpId, ++keyId & 0xff, MAX_AWAIT_MILLIS);
+        } while (keyId != INITIAL_KEY_ID);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testMasterAndCacheGroupKeySimultaneousChange() throws Exception {
+        startTestGrids(true);
+
+        IgniteEx node0 = grid(GRID_0);
+        IgniteEx node1 = grid(GRID_1);
+
+        createEncryptedCache(node0, node1, cacheName(), null);
+
+        int grpId = CU.cacheId(cacheName());
+
+        assertTrue(checkMasterKeyName(DEFAULT_MASTER_KEY_NAME));
+
+        Random rnd = ThreadLocalRandom.current();
+
+        for (byte keyId = 1; keyId < 50; keyId++) {
+            String currMkName = node0.context().config().getEncryptionSpi().getMasterKeyName();
+            String newMkName = currMkName.equals(MASTER_KEY_NAME_2) ? MASTER_KEY_NAME_3 : MASTER_KEY_NAME_2;
+
+            boolean changeGrpFirst = rnd.nextBoolean();
+
+            IgniteFuture<Void> grpKeyFut;
+            IgniteFuture<Void> masterKeyFut;
+
+            if (changeGrpFirst) {
+                grpKeyFut = node0.encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+                masterKeyFut = node0.encryption().changeMasterKey(newMkName);
+            }
+            else {
+                masterKeyFut = node0.encryption().changeMasterKey(newMkName);
+                grpKeyFut = node0.encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+            }
+
+            masterKeyFut.get(MAX_AWAIT_MILLIS);
+            assertTrue(checkMasterKeyName(newMkName));
+
+            try {
+                grpKeyFut.get(MAX_AWAIT_MILLIS);
+                checkGroupKey(grpId, keyId, MAX_AWAIT_MILLIS);
+            } catch (IgniteException e) {
+                assertTrue(e.getMessage().contains("Cache group key change was rejected. Master key has been changed."));
+
+                keyId -= 1;
+            }
+        }
+    }
+
+    /**
+     * Ensures that after rotation, the node has correct key identifier.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testNodeJoinAfterChange() throws Exception {
+        startTestGrids(true);
+
+        IgniteEx node0 = grid(GRID_0);
+        IgniteEx node1 = grid(GRID_1);
+
+        createEncryptedCache(node0, node1, cacheName(), null);
+
+        node0.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get();
+
+        startGrid(GRID_2);
+
+        resetBaselineTopology();
+
+        awaitPartitionMapExchange();
+
+        checkGroupKey(CU.cacheId(cacheName()), INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testCacheStartDuringRotation() throws Exception {
+        T2<IgniteEx, IgniteEx> grids = startTestGrids(true);
+
+        createEncryptedCache(grids.get1(), grids.get2(), cacheName(), null);
+
+        TestRecordingCommunicationSpi commSpi = TestRecordingCommunicationSpi.spi(grids.get2());
+
+        commSpi.blockMessages((node, msg) -> msg instanceof SingleNodeMessage);
+
+        IgniteFuture<Void> fut = grids.get1().encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+
+        commSpi.waitForBlocked();
+
+        IgniteCache<Integer, Integer> cache = grids.get1().createCache(cacheConfiguration("cache1", null));
+
+        for (int i = 0; i < 100; i++)
+            cache.put(i, i);
+
+        commSpi.stopBlock();
+
+        fut.get();
+
+        checkGroupKey(CU.cacheId(cacheName()), INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        checkGroupKey(CU.cacheId("cache1"), INITIAL_KEY_ID, MAX_AWAIT_MILLIS);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testCacheStartSameGroupDuringRotation() throws Exception {
+        T2<IgniteEx, IgniteEx> grids = startTestGrids(true);
+
+        String grpName = "shared";
+
+        createEncryptedCache(grids.get1(), grids.get2(), cacheName(), grpName);
+
+        TestRecordingCommunicationSpi commSpi = TestRecordingCommunicationSpi.spi(grids.get2());
+
+        commSpi.blockMessages((node, msg) -> msg instanceof SingleNodeMessage);
+
+        IgniteFuture<Void> fut = grids.get1().encryption().changeCacheGroupKey(Collections.singleton(grpName));
+
+        commSpi.waitForBlocked();
+
+        IgniteCache<Integer, Integer> cache =
+            grids.get1().createCache(cacheConfiguration("cache1", grpName));
+
+        commSpi.stopBlock();
+
+        for (int i = 0; i < 100; i++)
+            cache.put(i, i);
+
+        fut.get();
+
+        checkGroupKey(CU.cacheId(grpName), INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testChangeKeyDuringRebalancing() throws Exception {
+        T2<IgniteEx, IgniteEx> grids = startTestGrids(true);
+
+        IgniteEx node0 = grids.get1();
+        IgniteEx node1 = grids.get2();
+
+        createEncryptedCache(node0, node1, cacheName(), null);
+
+        loadData(500_000);
+
+        IgniteEx node2 = startGrid(GRID_2);
+
+        resetBaselineTopology();
+
+        int grpId = CU.cacheId(cacheName());
+
+        IgniteFuture<Void> fut = node2.encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+
+        fut.get(MAX_AWAIT_MILLIS);
+
+        stopAllGrids();
+
+        startGrid(GRID_0);
+        startGrid(GRID_1);
+        startGrid(GRID_2);
+
+        grid(GRID_0).cluster().state(ClusterState.ACTIVE);
+
+        awaitPartitionMapExchange();
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testBasicChange() throws Exception {
+        startTestGrids(true);
+
+        IgniteEx node1 = grid(GRID_0);
+        IgniteEx node2 = grid(GRID_1);
+
+        createEncryptedCache(node1, node2, cacheName(), null);
+
+        forceCheckpoint();
+
+        IgniteInternalCache<Object, Object> cache = node1.cachex(cacheName());
+
+        int grpId = cache.context().groupId();
+
+        node1.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get();
+
+        Set<Integer> keys1 = new TreeSet<>(node1.context().encryption().groupKeyIds(grpId));
+        Set<Integer> keys2 = new TreeSet<>(node2.context().encryption().groupKeyIds(grpId));
+
+        assertEquals(2, keys1.size());
+
+        assertEquals(keys1, keys2);
+
+        info("New key was set on all nodes [grpId=" + grpId + ", keys=" + keys1 + "]");
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        stopAllGrids();
+
+        node1 = startGrid(GRID_0);
+        node2 = startGrid(GRID_1);
+
+        node1.cluster().state(ClusterState.ACTIVE);
+
+        // Previous leys must be deleted when the corresponding WAL segment is deleted.

Review comment:
       leys -> keys

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/encryption/CacheGroupKeyChangeTest.java
##########
@@ -0,0 +1,988 @@
+/*
+ * 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.encryption;
+
+import java.io.File;
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.IgniteException;
+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.configuration.WALMode;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
+import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
+import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
+import org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType;
+import org.apache.ignite.internal.util.distributed.InitMessage;
+import org.apache.ignite.internal.util.distributed.SingleNodeMessage;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.spi.discovery.tcp.TestTcpDiscoverySpi;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.GridTestUtils.DiscoveryHook;
+import org.junit.Test;
+
+import static org.apache.ignite.configuration.WALMode.LOG_ONLY;
+import static org.apache.ignite.configuration.WALMode.NONE;
+import static org.apache.ignite.internal.managers.encryption.GridEncryptionManager.INITIAL_KEY_ID;
+import static org.apache.ignite.spi.encryption.keystore.KeystoreEncryptionSpi.DEFAULT_MASTER_KEY_NAME;
+import static org.apache.ignite.testframework.GridTestUtils.assertThrowsAnyCause;
+import static org.apache.ignite.testframework.GridTestUtils.assertThrowsWithCause;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+import static org.apache.ignite.testframework.GridTestUtils.waitForCondition;
+
+/**
+ * Cache group key change distributed process tests.
+ */
+public class CacheGroupKeyChangeTest extends AbstractEncryptionTest {
+    /** Timeout. */
+    private static final long MAX_AWAIT_MILLIS = 15_000;
+
+    /** */
+    private static final String GRID_2 = "grid-2";
+
+    /** Discovery hook for distributed process. */
+    private InitMessageDiscoveryHook discoveryHook;
+
+    /** Count of cache backups. */
+    private int backups;
+
+    /** WAL mode. */
+    private WALMode walMode = LOG_ONLY;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        if (discoveryHook != null)
+            ((TestTcpDiscoverySpi)cfg.getDiscoverySpi()).discoveryHook(discoveryHook);
+
+        DataStorageConfiguration memCfg = new DataStorageConfiguration()
+            .setDefaultDataRegionConfiguration(
+                new DataRegionConfiguration()
+                    .setMaxSize(100L * 1024 * 1024)
+                    .setPersistenceEnabled(true))
+            .setPageSize(4 * 1024)
+            .setWalSegmentSize(1024 * 1024)
+            .setWalSegments(10)
+            .setMaxWalArchiveSize(20 * 1024 * 1024)
+            .setCheckpointFrequency(30 * 1000L)
+            .setWalMode(walMode);
+
+        cfg.setDataStorageConfiguration(memCfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected <K, V> CacheConfiguration<K, V> cacheConfiguration(String name, String grp) {
+        CacheConfiguration<K, V> cfg = super.cacheConfiguration(name, grp);
+
+        return cfg.setAffinity(new RendezvousAffinityFunction(false, 8)).setBackups(backups);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    @SuppressWarnings("ThrowableNotThrown")
+    public void testRejectNodeJoinDuringRotation() throws Exception {
+        T2<IgniteEx, IgniteEx> grids = startTestGrids(true);
+
+        createEncryptedCache(grids.get1(), grids.get2(), cacheName(), null);
+
+        int grpId = CU.cacheId(cacheName());
+
+        assertEquals(0, grids.get1().context().encryption().groupKey(grpId).id());
+
+        TestRecordingCommunicationSpi commSpi = TestRecordingCommunicationSpi.spi(grids.get2());
+
+        commSpi.blockMessages((node, msg) -> msg instanceof SingleNodeMessage);
+
+        IgniteFuture<Void> fut = grids.get1().encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+
+        commSpi.waitForBlocked();
+
+        assertThrowsWithCause(() -> startGrid(3), IgniteCheckedException.class);
+
+        commSpi.stopBlock();
+
+        fut.get();
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        checkEncryptedCaches(grids.get1(), grids.get2());
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNotAllBltNodesPresent() throws Exception {
+        startTestGrids(true);
+
+        createEncryptedCache(grid(GRID_0), grid(GRID_1), cacheName(), null);
+
+        stopGrid(GRID_1);
+
+        grid(GRID_0).encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get();
+
+        startGrid(GRID_1);
+
+        checkGroupKey(CU.cacheId(cacheName()), INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNodeFailsBeforePrepare() throws Exception {
+        checkNodeFailsDuringRotation(false, true, true);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNodeFailsBeforePerform() throws Exception {
+        checkNodeFailsDuringRotation(false, false, true);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNodeFailsAfterPrepare() throws Exception {
+        checkNodeFailsDuringRotation(false, true, false);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testCrdFailsAfterPrepare() throws Exception {
+        checkNodeFailsDuringRotation(true, true, false);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNodeFailsAfterPerform() throws Exception {
+        checkNodeFailsDuringRotation(false, false, false);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testCrdFailsAfterPerform() throws Exception {
+        checkNodeFailsDuringRotation(true, false, false);
+    }
+
+    /**
+     * @param stopCrd {@code True} to stop coordinator.
+     * @param prepare {@code True} to stop on the prepare phase. {@code False} to stop on the perform phase.
+     * @param discoBlock  {@code True} to block discovery, {@code False} to block communication SPI.
+     */
+    private void checkNodeFailsDuringRotation(boolean stopCrd, boolean prepare, boolean discoBlock) throws Exception {
+        cleanPersistenceDir();
+
+        DistributedProcessType type = prepare ?
+            DistributedProcessType.CACHE_GROUP_KEY_CHANGE_PREPARE : DistributedProcessType.CACHE_GROUP_KEY_CHANGE_FINISH;
+
+        InitMessageDiscoveryHook locHook = new InitMessageDiscoveryHook(type);
+
+        if (discoBlock && stopCrd)
+            discoveryHook = locHook;
+
+        IgniteEx grid0 = startGrid(GRID_0);
+
+        if (discoBlock && !stopCrd)
+            discoveryHook = locHook;
+
+        IgniteEx grid1 = startGrid(GRID_1);
+
+        grid0.cluster().state(ClusterState.ACTIVE);
+
+        createEncryptedCache(grid0, grid1, cacheName(), null);
+
+        int grpId = CU.cacheId(cacheName());
+
+        checkGroupKey(grpId, INITIAL_KEY_ID, MAX_AWAIT_MILLIS);
+
+        TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid1);
+
+        if (!discoBlock) {
+            AtomicBoolean preparePhase = new AtomicBoolean(true);
+
+            spi.blockMessages((node, msg) -> {
+                if (msg instanceof SingleNodeMessage) {
+                    boolean isPrepare = preparePhase.compareAndSet(true, false);
+
+                    return prepare || !isPrepare;
+                }
+
+                return false;
+            });
+        }
+
+        String alive = stopCrd ? GRID_1 : GRID_0;
+        String stopped = stopCrd ? GRID_0 : GRID_1;
+
+        IgniteFuture<Void> changeFut = grid(alive).encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+
+        IgniteInternalFuture<?> stopFut = new GridFinishedFuture<>();
+
+        if (discoBlock) {
+            locHook.waitForBlocked(MAX_AWAIT_MILLIS);
+
+            stopGrid(stopped, true);
+
+            locHook.stopBlock();
+        }
+        else {
+            spi.waitForBlocked();
+
+            stopFut = runAsync(() -> stopGrid(stopped, true));
+        }
+
+        changeFut.get(MAX_AWAIT_MILLIS);
+        stopFut.get(MAX_AWAIT_MILLIS);
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        IgniteEx stoppedNode = startGrid(stopped);
+
+        stoppedNode.resetLostPartitions(Collections.singleton(ENCRYPTED_CACHE));
+
+        awaitPartitionMapExchange();
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        stoppedNode.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get(MAX_AWAIT_MILLIS);
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 2, MAX_AWAIT_MILLIS);
+    }
+
+    /**
+     * Ensures that we can rotate the key more than 255 times.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testKeyIdentifierOverflow() throws Exception {
+        IgniteEx node = startTestGrids(true).get1();
+
+        createEncryptedCache(node, null, cacheName(), null, false);
+
+        int grpId = CU.cacheId(cacheName());
+
+        byte keyId = INITIAL_KEY_ID;
+
+        do {
+            node.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get();
+
+            // Validates reencryption of index partition.
+            checkGroupKey(grpId, ++keyId & 0xff, MAX_AWAIT_MILLIS);
+        } while (keyId != INITIAL_KEY_ID);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testMasterAndCacheGroupKeySimultaneousChange() throws Exception {
+        startTestGrids(true);
+
+        IgniteEx node0 = grid(GRID_0);
+        IgniteEx node1 = grid(GRID_1);
+
+        createEncryptedCache(node0, node1, cacheName(), null);
+
+        int grpId = CU.cacheId(cacheName());
+
+        assertTrue(checkMasterKeyName(DEFAULT_MASTER_KEY_NAME));
+
+        Random rnd = ThreadLocalRandom.current();
+
+        for (byte keyId = 1; keyId < 50; keyId++) {
+            String currMkName = node0.context().config().getEncryptionSpi().getMasterKeyName();
+            String newMkName = currMkName.equals(MASTER_KEY_NAME_2) ? MASTER_KEY_NAME_3 : MASTER_KEY_NAME_2;
+
+            boolean changeGrpFirst = rnd.nextBoolean();
+
+            IgniteFuture<Void> grpKeyFut;
+            IgniteFuture<Void> masterKeyFut;
+
+            if (changeGrpFirst) {
+                grpKeyFut = node0.encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+                masterKeyFut = node0.encryption().changeMasterKey(newMkName);
+            }
+            else {
+                masterKeyFut = node0.encryption().changeMasterKey(newMkName);
+                grpKeyFut = node0.encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+            }
+
+            masterKeyFut.get(MAX_AWAIT_MILLIS);
+            assertTrue(checkMasterKeyName(newMkName));
+
+            try {
+                grpKeyFut.get(MAX_AWAIT_MILLIS);
+                checkGroupKey(grpId, keyId, MAX_AWAIT_MILLIS);
+            } catch (IgniteException e) {
+                assertTrue(e.getMessage().contains("Cache group key change was rejected. Master key has been changed."));
+
+                keyId -= 1;
+            }
+        }
+    }
+
+    /**
+     * Ensures that after rotation, the node has correct key identifier.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testNodeJoinAfterChange() throws Exception {
+        startTestGrids(true);
+
+        IgniteEx node0 = grid(GRID_0);
+        IgniteEx node1 = grid(GRID_1);
+
+        createEncryptedCache(node0, node1, cacheName(), null);
+
+        node0.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get();
+
+        startGrid(GRID_2);
+
+        resetBaselineTopology();
+
+        awaitPartitionMapExchange();
+
+        checkGroupKey(CU.cacheId(cacheName()), INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testCacheStartDuringRotation() throws Exception {
+        T2<IgniteEx, IgniteEx> grids = startTestGrids(true);
+
+        createEncryptedCache(grids.get1(), grids.get2(), cacheName(), null);
+
+        TestRecordingCommunicationSpi commSpi = TestRecordingCommunicationSpi.spi(grids.get2());
+
+        commSpi.blockMessages((node, msg) -> msg instanceof SingleNodeMessage);
+
+        IgniteFuture<Void> fut = grids.get1().encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+
+        commSpi.waitForBlocked();
+
+        IgniteCache<Integer, Integer> cache = grids.get1().createCache(cacheConfiguration("cache1", null));
+
+        for (int i = 0; i < 100; i++)
+            cache.put(i, i);
+
+        commSpi.stopBlock();
+
+        fut.get();
+
+        checkGroupKey(CU.cacheId(cacheName()), INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        checkGroupKey(CU.cacheId("cache1"), INITIAL_KEY_ID, MAX_AWAIT_MILLIS);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testCacheStartSameGroupDuringRotation() throws Exception {
+        T2<IgniteEx, IgniteEx> grids = startTestGrids(true);
+
+        String grpName = "shared";
+
+        createEncryptedCache(grids.get1(), grids.get2(), cacheName(), grpName);
+
+        TestRecordingCommunicationSpi commSpi = TestRecordingCommunicationSpi.spi(grids.get2());
+
+        commSpi.blockMessages((node, msg) -> msg instanceof SingleNodeMessage);
+
+        IgniteFuture<Void> fut = grids.get1().encryption().changeCacheGroupKey(Collections.singleton(grpName));
+
+        commSpi.waitForBlocked();
+
+        IgniteCache<Integer, Integer> cache =
+            grids.get1().createCache(cacheConfiguration("cache1", grpName));
+
+        commSpi.stopBlock();
+
+        for (int i = 0; i < 100; i++)
+            cache.put(i, i);
+
+        fut.get();
+
+        checkGroupKey(CU.cacheId(grpName), INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testChangeKeyDuringRebalancing() throws Exception {
+        T2<IgniteEx, IgniteEx> grids = startTestGrids(true);
+
+        IgniteEx node0 = grids.get1();
+        IgniteEx node1 = grids.get2();
+
+        createEncryptedCache(node0, node1, cacheName(), null);
+
+        loadData(500_000);
+
+        IgniteEx node2 = startGrid(GRID_2);
+
+        resetBaselineTopology();
+
+        int grpId = CU.cacheId(cacheName());
+
+        IgniteFuture<Void> fut = node2.encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+
+        fut.get(MAX_AWAIT_MILLIS);
+
+        stopAllGrids();
+
+        startGrid(GRID_0);
+        startGrid(GRID_1);
+        startGrid(GRID_2);
+
+        grid(GRID_0).cluster().state(ClusterState.ACTIVE);
+
+        awaitPartitionMapExchange();
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testBasicChange() throws Exception {
+        startTestGrids(true);
+
+        IgniteEx node1 = grid(GRID_0);
+        IgniteEx node2 = grid(GRID_1);
+
+        createEncryptedCache(node1, node2, cacheName(), null);
+
+        forceCheckpoint();
+
+        IgniteInternalCache<Object, Object> cache = node1.cachex(cacheName());
+
+        int grpId = cache.context().groupId();
+
+        node1.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get();
+
+        Set<Integer> keys1 = new TreeSet<>(node1.context().encryption().groupKeyIds(grpId));
+        Set<Integer> keys2 = new TreeSet<>(node2.context().encryption().groupKeyIds(grpId));
+
+        assertEquals(2, keys1.size());
+
+        assertEquals(keys1, keys2);
+
+        info("New key was set on all nodes [grpId=" + grpId + ", keys=" + keys1 + "]");
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        stopAllGrids();
+
+        node1 = startGrid(GRID_0);
+        node2 = startGrid(GRID_1);
+
+        node1.cluster().state(ClusterState.ACTIVE);
+
+        // Previous leys must be deleted when the corresponding WAL segment is deleted.
+        try (IgniteDataStreamer<Integer, String> streamer = node1.dataStreamer(cacheName())) {
+            for (int i = node1.cache(cacheName()).size(); i < 500_000; i++) {
+                streamer.addData(i, String.valueOf(i));
+
+                if (i % 1_000 == 0 &&
+                    node1.context().encryption().groupKeyIds(grpId).size() == 1 &&
+                    node2.context().encryption().groupKeyIds(grpId).size() == 1)
+                    break;
+            }
+        }
+
+        assertEquals(1, node1.context().encryption().groupKeyIds(grpId).size());
+        assertEquals(1, node2.context().encryption().groupKeyIds(grpId).size());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testBasicChangeWithConstantLoad() throws Exception {
+        startTestGrids(true);
+
+        IgniteEx node0 = grid(GRID_0);
+        IgniteEx node1 = grid(GRID_1);
+
+        createEncryptedCache(node0, node1, cacheName(), null);
+
+        forceCheckpoint();
+
+        IgniteInternalCache<Object, Object> cache = node0.cachex(cacheName());
+
+        AtomicInteger cntr = new AtomicInteger(cache.size());
+
+        CountDownLatch startLatch = new CountDownLatch(1);
+
+        final Ignite somenode = node0;
+
+        IgniteInternalFuture<?> loadFut = GridTestUtils.runAsync(() -> {
+            try (IgniteDataStreamer<Integer, String> streamer = somenode.dataStreamer(cacheName())) {
+                while (!Thread.currentThread().isInterrupted()) {
+                    int n = cntr.getAndIncrement();
+
+                    streamer.addData(n, String.valueOf(n));
+
+                    if (n == 5000)
+                        startLatch.countDown();
+                }
+            }
+        });
+
+        startLatch.await(MAX_AWAIT_MILLIS, TimeUnit.MILLISECONDS);
+
+        int grpId = cache.context().groupId();
+
+        node0.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get(MAX_AWAIT_MILLIS);
+
+        Set<Integer> keys1 = new TreeSet<>(node0.context().encryption().groupKeyIds(grpId));
+        Set<Integer> keys2 = new TreeSet<>(node1.context().encryption().groupKeyIds(grpId));
+
+        assertEquals(keys1, keys2);
+
+        awaitEncryption(G.allGrids(), grpId, MAX_AWAIT_MILLIS);
+
+        forceCheckpoint();
+
+        loadFut.cancel();
+
+        // Ensure that data is encrypted with the new key.
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        stopAllGrids();
+
+        node0 = startGrid(GRID_0);
+        node1 = startGrid(GRID_1);
+
+        node0.cluster().state(ClusterState.ACTIVE);
+
+        // Wait for WAL segment remove.
+        try (IgniteDataStreamer<Integer, String> streamer = node0.dataStreamer(cacheName())) {
+            int start = cntr.get();
+
+            for (; ; ) {
+                int n = cntr.getAndIncrement();
+
+                streamer.addData(n, String.valueOf(n));
+
+                if (n % 1000 == 0 &&
+                    node0.context().encryption().groupKeyIds(grpId).size() == 1 &&
+                    node1.context().encryption().groupKeyIds(grpId).size() == 1)
+                    break;
+
+                if (n - start == 1_000_000)
+                    break;
+            }
+        }
+
+        assertEquals(1, node0.context().encryption().groupKeyIds(grpId).size());
+        assertEquals(1, node1.context().encryption().groupKeyIds(grpId).size());
+    }
+
+    /**
+     * Ensures that unused key will be removed even if user cleaned wal archive folder manually.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testWalArchiveCleanup() throws Exception {
+        startTestGrids(true);
+
+        IgniteEx node1 = grid(GRID_0);
+        IgniteEx node2 = grid(GRID_1);
+
+        createEncryptedCache(node1, node2, cacheName(), null);
+
+        node1.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get();
+
+        long walIdx = node1.context().cache().context().wal().currentSegment();
+
+        AtomicBoolean stopLoad = new AtomicBoolean();
+
+        IgniteInternalFuture<?> fut = runAsync(() -> {
+            Ignite grid = grid(GRID_0);
+
+            long cntr = grid.cache(cacheName()).size();
+
+            try (IgniteDataStreamer<Long, String> streamer = grid.dataStreamer(cacheName())) {
+                while (!stopLoad.get() && !Thread.currentThread().isInterrupted()) {
+                    streamer.addData(cntr, String.valueOf(cntr));
+
+                    streamer.flush();
+
+                    ++cntr;
+                }
+            }
+        });
+
+        try {
+            IgniteWriteAheadLogManager walMgr = grid(GRID_0).context().cache().context().wal();
+
+            boolean success = waitForCondition(() -> walMgr.lastArchivedSegment() >= walIdx, MAX_AWAIT_MILLIS);
+
+            assertTrue(success);
+        } finally {
+            stopLoad.set(true);
+
+            fut.get(MAX_AWAIT_MILLIS);
+        }
+
+        forceCheckpoint();
+
+        int grpId = CU.cacheId(cacheName());
+
+        assertEquals(2, node1.context().encryption().groupKeyIds(grpId).size());
+        assertEquals(2, node2.context().encryption().groupKeyIds(grpId).size());
+
+        stopAllGrids();
+
+        // Cleanup wal arcive.

Review comment:
       WAL archive

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/encryption/CacheGroupKeyChangeTest.java
##########
@@ -0,0 +1,988 @@
+/*
+ * 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.encryption;
+
+import java.io.File;
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.IgniteException;
+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.configuration.WALMode;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
+import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
+import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
+import org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType;
+import org.apache.ignite.internal.util.distributed.InitMessage;
+import org.apache.ignite.internal.util.distributed.SingleNodeMessage;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.spi.discovery.tcp.TestTcpDiscoverySpi;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.GridTestUtils.DiscoveryHook;
+import org.junit.Test;
+
+import static org.apache.ignite.configuration.WALMode.LOG_ONLY;
+import static org.apache.ignite.configuration.WALMode.NONE;
+import static org.apache.ignite.internal.managers.encryption.GridEncryptionManager.INITIAL_KEY_ID;
+import static org.apache.ignite.spi.encryption.keystore.KeystoreEncryptionSpi.DEFAULT_MASTER_KEY_NAME;
+import static org.apache.ignite.testframework.GridTestUtils.assertThrowsAnyCause;
+import static org.apache.ignite.testframework.GridTestUtils.assertThrowsWithCause;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+import static org.apache.ignite.testframework.GridTestUtils.waitForCondition;
+
+/**
+ * Cache group key change distributed process tests.
+ */
+public class CacheGroupKeyChangeTest extends AbstractEncryptionTest {
+    /** Timeout. */
+    private static final long MAX_AWAIT_MILLIS = 15_000;
+
+    /** */
+    private static final String GRID_2 = "grid-2";
+
+    /** Discovery hook for distributed process. */
+    private InitMessageDiscoveryHook discoveryHook;
+
+    /** Count of cache backups. */
+    private int backups;
+
+    /** WAL mode. */
+    private WALMode walMode = LOG_ONLY;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        if (discoveryHook != null)
+            ((TestTcpDiscoverySpi)cfg.getDiscoverySpi()).discoveryHook(discoveryHook);
+
+        DataStorageConfiguration memCfg = new DataStorageConfiguration()
+            .setDefaultDataRegionConfiguration(
+                new DataRegionConfiguration()
+                    .setMaxSize(100L * 1024 * 1024)
+                    .setPersistenceEnabled(true))
+            .setPageSize(4 * 1024)
+            .setWalSegmentSize(1024 * 1024)
+            .setWalSegments(10)
+            .setMaxWalArchiveSize(20 * 1024 * 1024)
+            .setCheckpointFrequency(30 * 1000L)
+            .setWalMode(walMode);
+
+        cfg.setDataStorageConfiguration(memCfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected <K, V> CacheConfiguration<K, V> cacheConfiguration(String name, String grp) {
+        CacheConfiguration<K, V> cfg = super.cacheConfiguration(name, grp);
+
+        return cfg.setAffinity(new RendezvousAffinityFunction(false, 8)).setBackups(backups);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    @SuppressWarnings("ThrowableNotThrown")
+    public void testRejectNodeJoinDuringRotation() throws Exception {
+        T2<IgniteEx, IgniteEx> grids = startTestGrids(true);
+
+        createEncryptedCache(grids.get1(), grids.get2(), cacheName(), null);
+
+        int grpId = CU.cacheId(cacheName());
+
+        assertEquals(0, grids.get1().context().encryption().groupKey(grpId).id());
+
+        TestRecordingCommunicationSpi commSpi = TestRecordingCommunicationSpi.spi(grids.get2());
+
+        commSpi.blockMessages((node, msg) -> msg instanceof SingleNodeMessage);
+
+        IgniteFuture<Void> fut = grids.get1().encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+
+        commSpi.waitForBlocked();
+
+        assertThrowsWithCause(() -> startGrid(3), IgniteCheckedException.class);
+
+        commSpi.stopBlock();
+
+        fut.get();
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        checkEncryptedCaches(grids.get1(), grids.get2());
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNotAllBltNodesPresent() throws Exception {
+        startTestGrids(true);
+
+        createEncryptedCache(grid(GRID_0), grid(GRID_1), cacheName(), null);
+
+        stopGrid(GRID_1);
+
+        grid(GRID_0).encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get();
+
+        startGrid(GRID_1);
+
+        checkGroupKey(CU.cacheId(cacheName()), INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNodeFailsBeforePrepare() throws Exception {
+        checkNodeFailsDuringRotation(false, true, true);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNodeFailsBeforePerform() throws Exception {
+        checkNodeFailsDuringRotation(false, false, true);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNodeFailsAfterPrepare() throws Exception {
+        checkNodeFailsDuringRotation(false, true, false);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testCrdFailsAfterPrepare() throws Exception {
+        checkNodeFailsDuringRotation(true, true, false);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNodeFailsAfterPerform() throws Exception {
+        checkNodeFailsDuringRotation(false, false, false);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testCrdFailsAfterPerform() throws Exception {
+        checkNodeFailsDuringRotation(true, false, false);
+    }
+
+    /**
+     * @param stopCrd {@code True} to stop coordinator.
+     * @param prepare {@code True} to stop on the prepare phase. {@code False} to stop on the perform phase.
+     * @param discoBlock  {@code True} to block discovery, {@code False} to block communication SPI.
+     */
+    private void checkNodeFailsDuringRotation(boolean stopCrd, boolean prepare, boolean discoBlock) throws Exception {
+        cleanPersistenceDir();
+
+        DistributedProcessType type = prepare ?
+            DistributedProcessType.CACHE_GROUP_KEY_CHANGE_PREPARE : DistributedProcessType.CACHE_GROUP_KEY_CHANGE_FINISH;
+
+        InitMessageDiscoveryHook locHook = new InitMessageDiscoveryHook(type);
+
+        if (discoBlock && stopCrd)
+            discoveryHook = locHook;
+
+        IgniteEx grid0 = startGrid(GRID_0);
+
+        if (discoBlock && !stopCrd)
+            discoveryHook = locHook;
+
+        IgniteEx grid1 = startGrid(GRID_1);
+
+        grid0.cluster().state(ClusterState.ACTIVE);
+
+        createEncryptedCache(grid0, grid1, cacheName(), null);
+
+        int grpId = CU.cacheId(cacheName());
+
+        checkGroupKey(grpId, INITIAL_KEY_ID, MAX_AWAIT_MILLIS);
+
+        TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid1);
+
+        if (!discoBlock) {
+            AtomicBoolean preparePhase = new AtomicBoolean(true);
+
+            spi.blockMessages((node, msg) -> {
+                if (msg instanceof SingleNodeMessage) {
+                    boolean isPrepare = preparePhase.compareAndSet(true, false);
+
+                    return prepare || !isPrepare;
+                }
+
+                return false;
+            });
+        }
+
+        String alive = stopCrd ? GRID_1 : GRID_0;
+        String stopped = stopCrd ? GRID_0 : GRID_1;
+
+        IgniteFuture<Void> changeFut = grid(alive).encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+
+        IgniteInternalFuture<?> stopFut = new GridFinishedFuture<>();
+
+        if (discoBlock) {
+            locHook.waitForBlocked(MAX_AWAIT_MILLIS);
+
+            stopGrid(stopped, true);
+
+            locHook.stopBlock();
+        }
+        else {
+            spi.waitForBlocked();
+
+            stopFut = runAsync(() -> stopGrid(stopped, true));
+        }
+
+        changeFut.get(MAX_AWAIT_MILLIS);
+        stopFut.get(MAX_AWAIT_MILLIS);
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        IgniteEx stoppedNode = startGrid(stopped);
+
+        stoppedNode.resetLostPartitions(Collections.singleton(ENCRYPTED_CACHE));
+
+        awaitPartitionMapExchange();
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        stoppedNode.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get(MAX_AWAIT_MILLIS);
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 2, MAX_AWAIT_MILLIS);
+    }
+
+    /**
+     * Ensures that we can rotate the key more than 255 times.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testKeyIdentifierOverflow() throws Exception {
+        IgniteEx node = startTestGrids(true).get1();
+
+        createEncryptedCache(node, null, cacheName(), null, false);
+
+        int grpId = CU.cacheId(cacheName());
+
+        byte keyId = INITIAL_KEY_ID;
+
+        do {
+            node.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get();
+
+            // Validates reencryption of index partition.
+            checkGroupKey(grpId, ++keyId & 0xff, MAX_AWAIT_MILLIS);
+        } while (keyId != INITIAL_KEY_ID);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testMasterAndCacheGroupKeySimultaneousChange() throws Exception {
+        startTestGrids(true);
+
+        IgniteEx node0 = grid(GRID_0);
+        IgniteEx node1 = grid(GRID_1);
+
+        createEncryptedCache(node0, node1, cacheName(), null);
+
+        int grpId = CU.cacheId(cacheName());
+
+        assertTrue(checkMasterKeyName(DEFAULT_MASTER_KEY_NAME));
+
+        Random rnd = ThreadLocalRandom.current();
+
+        for (byte keyId = 1; keyId < 50; keyId++) {
+            String currMkName = node0.context().config().getEncryptionSpi().getMasterKeyName();
+            String newMkName = currMkName.equals(MASTER_KEY_NAME_2) ? MASTER_KEY_NAME_3 : MASTER_KEY_NAME_2;
+
+            boolean changeGrpFirst = rnd.nextBoolean();
+
+            IgniteFuture<Void> grpKeyFut;
+            IgniteFuture<Void> masterKeyFut;
+
+            if (changeGrpFirst) {
+                grpKeyFut = node0.encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+                masterKeyFut = node0.encryption().changeMasterKey(newMkName);
+            }
+            else {
+                masterKeyFut = node0.encryption().changeMasterKey(newMkName);
+                grpKeyFut = node0.encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+            }
+
+            masterKeyFut.get(MAX_AWAIT_MILLIS);
+            assertTrue(checkMasterKeyName(newMkName));
+
+            try {
+                grpKeyFut.get(MAX_AWAIT_MILLIS);
+                checkGroupKey(grpId, keyId, MAX_AWAIT_MILLIS);
+            } catch (IgniteException e) {
+                assertTrue(e.getMessage().contains("Cache group key change was rejected. Master key has been changed."));
+
+                keyId -= 1;
+            }
+        }
+    }
+
+    /**
+     * Ensures that after rotation, the node has correct key identifier.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testNodeJoinAfterChange() throws Exception {
+        startTestGrids(true);
+
+        IgniteEx node0 = grid(GRID_0);
+        IgniteEx node1 = grid(GRID_1);
+
+        createEncryptedCache(node0, node1, cacheName(), null);
+
+        node0.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get();
+
+        startGrid(GRID_2);
+
+        resetBaselineTopology();
+
+        awaitPartitionMapExchange();
+
+        checkGroupKey(CU.cacheId(cacheName()), INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testCacheStartDuringRotation() throws Exception {
+        T2<IgniteEx, IgniteEx> grids = startTestGrids(true);
+
+        createEncryptedCache(grids.get1(), grids.get2(), cacheName(), null);
+
+        TestRecordingCommunicationSpi commSpi = TestRecordingCommunicationSpi.spi(grids.get2());
+
+        commSpi.blockMessages((node, msg) -> msg instanceof SingleNodeMessage);
+
+        IgniteFuture<Void> fut = grids.get1().encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+
+        commSpi.waitForBlocked();
+
+        IgniteCache<Integer, Integer> cache = grids.get1().createCache(cacheConfiguration("cache1", null));
+
+        for (int i = 0; i < 100; i++)
+            cache.put(i, i);
+
+        commSpi.stopBlock();
+
+        fut.get();
+
+        checkGroupKey(CU.cacheId(cacheName()), INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        checkGroupKey(CU.cacheId("cache1"), INITIAL_KEY_ID, MAX_AWAIT_MILLIS);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testCacheStartSameGroupDuringRotation() throws Exception {
+        T2<IgniteEx, IgniteEx> grids = startTestGrids(true);
+
+        String grpName = "shared";
+
+        createEncryptedCache(grids.get1(), grids.get2(), cacheName(), grpName);
+
+        TestRecordingCommunicationSpi commSpi = TestRecordingCommunicationSpi.spi(grids.get2());
+
+        commSpi.blockMessages((node, msg) -> msg instanceof SingleNodeMessage);
+
+        IgniteFuture<Void> fut = grids.get1().encryption().changeCacheGroupKey(Collections.singleton(grpName));
+
+        commSpi.waitForBlocked();
+
+        IgniteCache<Integer, Integer> cache =
+            grids.get1().createCache(cacheConfiguration("cache1", grpName));
+
+        commSpi.stopBlock();
+
+        for (int i = 0; i < 100; i++)
+            cache.put(i, i);
+
+        fut.get();
+
+        checkGroupKey(CU.cacheId(grpName), INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testChangeKeyDuringRebalancing() throws Exception {
+        T2<IgniteEx, IgniteEx> grids = startTestGrids(true);
+
+        IgniteEx node0 = grids.get1();
+        IgniteEx node1 = grids.get2();
+
+        createEncryptedCache(node0, node1, cacheName(), null);
+
+        loadData(500_000);
+
+        IgniteEx node2 = startGrid(GRID_2);
+
+        resetBaselineTopology();
+
+        int grpId = CU.cacheId(cacheName());
+
+        IgniteFuture<Void> fut = node2.encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+
+        fut.get(MAX_AWAIT_MILLIS);
+
+        stopAllGrids();
+
+        startGrid(GRID_0);
+        startGrid(GRID_1);
+        startGrid(GRID_2);
+
+        grid(GRID_0).cluster().state(ClusterState.ACTIVE);
+
+        awaitPartitionMapExchange();
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testBasicChange() throws Exception {
+        startTestGrids(true);
+
+        IgniteEx node1 = grid(GRID_0);
+        IgniteEx node2 = grid(GRID_1);
+
+        createEncryptedCache(node1, node2, cacheName(), null);
+
+        forceCheckpoint();
+
+        IgniteInternalCache<Object, Object> cache = node1.cachex(cacheName());
+
+        int grpId = cache.context().groupId();
+
+        node1.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get();
+
+        Set<Integer> keys1 = new TreeSet<>(node1.context().encryption().groupKeyIds(grpId));
+        Set<Integer> keys2 = new TreeSet<>(node2.context().encryption().groupKeyIds(grpId));
+
+        assertEquals(2, keys1.size());
+
+        assertEquals(keys1, keys2);
+
+        info("New key was set on all nodes [grpId=" + grpId + ", keys=" + keys1 + "]");
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        stopAllGrids();
+
+        node1 = startGrid(GRID_0);
+        node2 = startGrid(GRID_1);
+
+        node1.cluster().state(ClusterState.ACTIVE);
+
+        // Previous leys must be deleted when the corresponding WAL segment is deleted.
+        try (IgniteDataStreamer<Integer, String> streamer = node1.dataStreamer(cacheName())) {
+            for (int i = node1.cache(cacheName()).size(); i < 500_000; i++) {
+                streamer.addData(i, String.valueOf(i));
+
+                if (i % 1_000 == 0 &&
+                    node1.context().encryption().groupKeyIds(grpId).size() == 1 &&
+                    node2.context().encryption().groupKeyIds(grpId).size() == 1)
+                    break;
+            }
+        }
+
+        assertEquals(1, node1.context().encryption().groupKeyIds(grpId).size());
+        assertEquals(1, node2.context().encryption().groupKeyIds(grpId).size());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testBasicChangeWithConstantLoad() throws Exception {
+        startTestGrids(true);
+
+        IgniteEx node0 = grid(GRID_0);
+        IgniteEx node1 = grid(GRID_1);
+
+        createEncryptedCache(node0, node1, cacheName(), null);
+
+        forceCheckpoint();
+
+        IgniteInternalCache<Object, Object> cache = node0.cachex(cacheName());
+
+        AtomicInteger cntr = new AtomicInteger(cache.size());
+
+        CountDownLatch startLatch = new CountDownLatch(1);
+
+        final Ignite somenode = node0;
+
+        IgniteInternalFuture<?> loadFut = GridTestUtils.runAsync(() -> {
+            try (IgniteDataStreamer<Integer, String> streamer = somenode.dataStreamer(cacheName())) {
+                while (!Thread.currentThread().isInterrupted()) {
+                    int n = cntr.getAndIncrement();
+
+                    streamer.addData(n, String.valueOf(n));
+
+                    if (n == 5000)
+                        startLatch.countDown();
+                }
+            }
+        });
+
+        startLatch.await(MAX_AWAIT_MILLIS, TimeUnit.MILLISECONDS);
+
+        int grpId = cache.context().groupId();
+
+        node0.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get(MAX_AWAIT_MILLIS);
+
+        Set<Integer> keys1 = new TreeSet<>(node0.context().encryption().groupKeyIds(grpId));
+        Set<Integer> keys2 = new TreeSet<>(node1.context().encryption().groupKeyIds(grpId));
+
+        assertEquals(keys1, keys2);

Review comment:
       Load future is still working and one of the nodes can already switch WAL segment, do checkpoint and delete key id in background, so test can become flaky.

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/encryption/CacheGroupKeyChangeTest.java
##########
@@ -0,0 +1,988 @@
+/*
+ * 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.encryption;
+
+import java.io.File;
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.IgniteException;
+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.configuration.WALMode;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
+import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
+import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
+import org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType;
+import org.apache.ignite.internal.util.distributed.InitMessage;
+import org.apache.ignite.internal.util.distributed.SingleNodeMessage;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.spi.discovery.tcp.TestTcpDiscoverySpi;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.GridTestUtils.DiscoveryHook;
+import org.junit.Test;
+
+import static org.apache.ignite.configuration.WALMode.LOG_ONLY;
+import static org.apache.ignite.configuration.WALMode.NONE;
+import static org.apache.ignite.internal.managers.encryption.GridEncryptionManager.INITIAL_KEY_ID;
+import static org.apache.ignite.spi.encryption.keystore.KeystoreEncryptionSpi.DEFAULT_MASTER_KEY_NAME;
+import static org.apache.ignite.testframework.GridTestUtils.assertThrowsAnyCause;
+import static org.apache.ignite.testframework.GridTestUtils.assertThrowsWithCause;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+import static org.apache.ignite.testframework.GridTestUtils.waitForCondition;
+
+/**
+ * Cache group key change distributed process tests.
+ */
+public class CacheGroupKeyChangeTest extends AbstractEncryptionTest {
+    /** Timeout. */
+    private static final long MAX_AWAIT_MILLIS = 15_000;
+
+    /** */
+    private static final String GRID_2 = "grid-2";
+
+    /** Discovery hook for distributed process. */
+    private InitMessageDiscoveryHook discoveryHook;
+
+    /** Count of cache backups. */
+    private int backups;
+
+    /** WAL mode. */
+    private WALMode walMode = LOG_ONLY;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        if (discoveryHook != null)
+            ((TestTcpDiscoverySpi)cfg.getDiscoverySpi()).discoveryHook(discoveryHook);
+
+        DataStorageConfiguration memCfg = new DataStorageConfiguration()
+            .setDefaultDataRegionConfiguration(
+                new DataRegionConfiguration()
+                    .setMaxSize(100L * 1024 * 1024)
+                    .setPersistenceEnabled(true))
+            .setPageSize(4 * 1024)
+            .setWalSegmentSize(1024 * 1024)
+            .setWalSegments(10)
+            .setMaxWalArchiveSize(20 * 1024 * 1024)
+            .setCheckpointFrequency(30 * 1000L)
+            .setWalMode(walMode);
+
+        cfg.setDataStorageConfiguration(memCfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected <K, V> CacheConfiguration<K, V> cacheConfiguration(String name, String grp) {
+        CacheConfiguration<K, V> cfg = super.cacheConfiguration(name, grp);
+
+        return cfg.setAffinity(new RendezvousAffinityFunction(false, 8)).setBackups(backups);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    @SuppressWarnings("ThrowableNotThrown")
+    public void testRejectNodeJoinDuringRotation() throws Exception {
+        T2<IgniteEx, IgniteEx> grids = startTestGrids(true);
+
+        createEncryptedCache(grids.get1(), grids.get2(), cacheName(), null);
+
+        int grpId = CU.cacheId(cacheName());
+
+        assertEquals(0, grids.get1().context().encryption().groupKey(grpId).id());
+
+        TestRecordingCommunicationSpi commSpi = TestRecordingCommunicationSpi.spi(grids.get2());
+
+        commSpi.blockMessages((node, msg) -> msg instanceof SingleNodeMessage);
+
+        IgniteFuture<Void> fut = grids.get1().encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+
+        commSpi.waitForBlocked();
+
+        assertThrowsWithCause(() -> startGrid(3), IgniteCheckedException.class);
+
+        commSpi.stopBlock();
+
+        fut.get();
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        checkEncryptedCaches(grids.get1(), grids.get2());
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNotAllBltNodesPresent() throws Exception {
+        startTestGrids(true);
+
+        createEncryptedCache(grid(GRID_0), grid(GRID_1), cacheName(), null);
+
+        stopGrid(GRID_1);
+
+        grid(GRID_0).encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get();
+
+        startGrid(GRID_1);
+
+        checkGroupKey(CU.cacheId(cacheName()), INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNodeFailsBeforePrepare() throws Exception {
+        checkNodeFailsDuringRotation(false, true, true);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNodeFailsBeforePerform() throws Exception {
+        checkNodeFailsDuringRotation(false, false, true);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNodeFailsAfterPrepare() throws Exception {
+        checkNodeFailsDuringRotation(false, true, false);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testCrdFailsAfterPrepare() throws Exception {
+        checkNodeFailsDuringRotation(true, true, false);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNodeFailsAfterPerform() throws Exception {
+        checkNodeFailsDuringRotation(false, false, false);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testCrdFailsAfterPerform() throws Exception {
+        checkNodeFailsDuringRotation(true, false, false);
+    }
+
+    /**
+     * @param stopCrd {@code True} to stop coordinator.
+     * @param prepare {@code True} to stop on the prepare phase. {@code False} to stop on the perform phase.
+     * @param discoBlock  {@code True} to block discovery, {@code False} to block communication SPI.
+     */
+    private void checkNodeFailsDuringRotation(boolean stopCrd, boolean prepare, boolean discoBlock) throws Exception {
+        cleanPersistenceDir();
+
+        DistributedProcessType type = prepare ?
+            DistributedProcessType.CACHE_GROUP_KEY_CHANGE_PREPARE : DistributedProcessType.CACHE_GROUP_KEY_CHANGE_FINISH;
+
+        InitMessageDiscoveryHook locHook = new InitMessageDiscoveryHook(type);
+
+        if (discoBlock && stopCrd)
+            discoveryHook = locHook;
+
+        IgniteEx grid0 = startGrid(GRID_0);
+
+        if (discoBlock && !stopCrd)
+            discoveryHook = locHook;
+
+        IgniteEx grid1 = startGrid(GRID_1);
+
+        grid0.cluster().state(ClusterState.ACTIVE);
+
+        createEncryptedCache(grid0, grid1, cacheName(), null);
+
+        int grpId = CU.cacheId(cacheName());
+
+        checkGroupKey(grpId, INITIAL_KEY_ID, MAX_AWAIT_MILLIS);
+
+        TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid1);
+
+        if (!discoBlock) {
+            AtomicBoolean preparePhase = new AtomicBoolean(true);
+
+            spi.blockMessages((node, msg) -> {
+                if (msg instanceof SingleNodeMessage) {
+                    boolean isPrepare = preparePhase.compareAndSet(true, false);
+
+                    return prepare || !isPrepare;
+                }
+
+                return false;
+            });
+        }
+
+        String alive = stopCrd ? GRID_1 : GRID_0;
+        String stopped = stopCrd ? GRID_0 : GRID_1;
+
+        IgniteFuture<Void> changeFut = grid(alive).encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+
+        IgniteInternalFuture<?> stopFut = new GridFinishedFuture<>();
+
+        if (discoBlock) {
+            locHook.waitForBlocked(MAX_AWAIT_MILLIS);
+
+            stopGrid(stopped, true);
+
+            locHook.stopBlock();
+        }
+        else {
+            spi.waitForBlocked();
+
+            stopFut = runAsync(() -> stopGrid(stopped, true));
+        }
+
+        changeFut.get(MAX_AWAIT_MILLIS);
+        stopFut.get(MAX_AWAIT_MILLIS);
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        IgniteEx stoppedNode = startGrid(stopped);
+
+        stoppedNode.resetLostPartitions(Collections.singleton(ENCRYPTED_CACHE));
+
+        awaitPartitionMapExchange();
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        stoppedNode.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get(MAX_AWAIT_MILLIS);
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 2, MAX_AWAIT_MILLIS);
+    }
+
+    /**
+     * Ensures that we can rotate the key more than 255 times.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testKeyIdentifierOverflow() throws Exception {
+        IgniteEx node = startTestGrids(true).get1();
+
+        createEncryptedCache(node, null, cacheName(), null, false);
+
+        int grpId = CU.cacheId(cacheName());
+
+        byte keyId = INITIAL_KEY_ID;
+
+        do {
+            node.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get();
+
+            // Validates reencryption of index partition.
+            checkGroupKey(grpId, ++keyId & 0xff, MAX_AWAIT_MILLIS);
+        } while (keyId != INITIAL_KEY_ID);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testMasterAndCacheGroupKeySimultaneousChange() throws Exception {
+        startTestGrids(true);
+
+        IgniteEx node0 = grid(GRID_0);
+        IgniteEx node1 = grid(GRID_1);
+
+        createEncryptedCache(node0, node1, cacheName(), null);
+
+        int grpId = CU.cacheId(cacheName());
+
+        assertTrue(checkMasterKeyName(DEFAULT_MASTER_KEY_NAME));
+
+        Random rnd = ThreadLocalRandom.current();
+
+        for (byte keyId = 1; keyId < 50; keyId++) {
+            String currMkName = node0.context().config().getEncryptionSpi().getMasterKeyName();
+            String newMkName = currMkName.equals(MASTER_KEY_NAME_2) ? MASTER_KEY_NAME_3 : MASTER_KEY_NAME_2;
+
+            boolean changeGrpFirst = rnd.nextBoolean();
+
+            IgniteFuture<Void> grpKeyFut;
+            IgniteFuture<Void> masterKeyFut;
+
+            if (changeGrpFirst) {
+                grpKeyFut = node0.encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+                masterKeyFut = node0.encryption().changeMasterKey(newMkName);
+            }
+            else {
+                masterKeyFut = node0.encryption().changeMasterKey(newMkName);
+                grpKeyFut = node0.encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+            }
+
+            masterKeyFut.get(MAX_AWAIT_MILLIS);
+            assertTrue(checkMasterKeyName(newMkName));
+
+            try {
+                grpKeyFut.get(MAX_AWAIT_MILLIS);
+                checkGroupKey(grpId, keyId, MAX_AWAIT_MILLIS);
+            } catch (IgniteException e) {
+                assertTrue(e.getMessage().contains("Cache group key change was rejected. Master key has been changed."));
+
+                keyId -= 1;
+            }
+        }
+    }
+
+    /**
+     * Ensures that after rotation, the node has correct key identifier.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testNodeJoinAfterChange() throws Exception {
+        startTestGrids(true);
+
+        IgniteEx node0 = grid(GRID_0);
+        IgniteEx node1 = grid(GRID_1);
+
+        createEncryptedCache(node0, node1, cacheName(), null);
+
+        node0.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get();
+
+        startGrid(GRID_2);
+
+        resetBaselineTopology();
+
+        awaitPartitionMapExchange();
+
+        checkGroupKey(CU.cacheId(cacheName()), INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testCacheStartDuringRotation() throws Exception {
+        T2<IgniteEx, IgniteEx> grids = startTestGrids(true);
+
+        createEncryptedCache(grids.get1(), grids.get2(), cacheName(), null);
+
+        TestRecordingCommunicationSpi commSpi = TestRecordingCommunicationSpi.spi(grids.get2());
+
+        commSpi.blockMessages((node, msg) -> msg instanceof SingleNodeMessage);
+
+        IgniteFuture<Void> fut = grids.get1().encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+
+        commSpi.waitForBlocked();
+
+        IgniteCache<Integer, Integer> cache = grids.get1().createCache(cacheConfiguration("cache1", null));
+
+        for (int i = 0; i < 100; i++)
+            cache.put(i, i);
+
+        commSpi.stopBlock();
+
+        fut.get();
+
+        checkGroupKey(CU.cacheId(cacheName()), INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        checkGroupKey(CU.cacheId("cache1"), INITIAL_KEY_ID, MAX_AWAIT_MILLIS);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testCacheStartSameGroupDuringRotation() throws Exception {
+        T2<IgniteEx, IgniteEx> grids = startTestGrids(true);
+
+        String grpName = "shared";
+
+        createEncryptedCache(grids.get1(), grids.get2(), cacheName(), grpName);
+
+        TestRecordingCommunicationSpi commSpi = TestRecordingCommunicationSpi.spi(grids.get2());
+
+        commSpi.blockMessages((node, msg) -> msg instanceof SingleNodeMessage);
+
+        IgniteFuture<Void> fut = grids.get1().encryption().changeCacheGroupKey(Collections.singleton(grpName));
+
+        commSpi.waitForBlocked();
+
+        IgniteCache<Integer, Integer> cache =
+            grids.get1().createCache(cacheConfiguration("cache1", grpName));
+
+        commSpi.stopBlock();
+
+        for (int i = 0; i < 100; i++)
+            cache.put(i, i);
+
+        fut.get();
+
+        checkGroupKey(CU.cacheId(grpName), INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testChangeKeyDuringRebalancing() throws Exception {
+        T2<IgniteEx, IgniteEx> grids = startTestGrids(true);
+
+        IgniteEx node0 = grids.get1();
+        IgniteEx node1 = grids.get2();
+
+        createEncryptedCache(node0, node1, cacheName(), null);
+
+        loadData(500_000);
+
+        IgniteEx node2 = startGrid(GRID_2);
+
+        resetBaselineTopology();
+
+        int grpId = CU.cacheId(cacheName());
+
+        IgniteFuture<Void> fut = node2.encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+
+        fut.get(MAX_AWAIT_MILLIS);
+
+        stopAllGrids();
+
+        startGrid(GRID_0);
+        startGrid(GRID_1);
+        startGrid(GRID_2);
+
+        grid(GRID_0).cluster().state(ClusterState.ACTIVE);
+
+        awaitPartitionMapExchange();
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testBasicChange() throws Exception {
+        startTestGrids(true);
+
+        IgniteEx node1 = grid(GRID_0);
+        IgniteEx node2 = grid(GRID_1);
+
+        createEncryptedCache(node1, node2, cacheName(), null);
+
+        forceCheckpoint();
+
+        IgniteInternalCache<Object, Object> cache = node1.cachex(cacheName());
+
+        int grpId = cache.context().groupId();
+
+        node1.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get();
+
+        Set<Integer> keys1 = new TreeSet<>(node1.context().encryption().groupKeyIds(grpId));
+        Set<Integer> keys2 = new TreeSet<>(node2.context().encryption().groupKeyIds(grpId));
+
+        assertEquals(2, keys1.size());
+
+        assertEquals(keys1, keys2);
+
+        info("New key was set on all nodes [grpId=" + grpId + ", keys=" + keys1 + "]");
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        stopAllGrids();
+
+        node1 = startGrid(GRID_0);
+        node2 = startGrid(GRID_1);
+
+        node1.cluster().state(ClusterState.ACTIVE);
+
+        // Previous leys must be deleted when the corresponding WAL segment is deleted.
+        try (IgniteDataStreamer<Integer, String> streamer = node1.dataStreamer(cacheName())) {
+            for (int i = node1.cache(cacheName()).size(); i < 500_000; i++) {
+                streamer.addData(i, String.valueOf(i));
+
+                if (i % 1_000 == 0 &&
+                    node1.context().encryption().groupKeyIds(grpId).size() == 1 &&
+                    node2.context().encryption().groupKeyIds(grpId).size() == 1)
+                    break;
+            }
+        }
+
+        assertEquals(1, node1.context().encryption().groupKeyIds(grpId).size());
+        assertEquals(1, node2.context().encryption().groupKeyIds(grpId).size());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testBasicChangeWithConstantLoad() throws Exception {
+        startTestGrids(true);
+
+        IgniteEx node0 = grid(GRID_0);
+        IgniteEx node1 = grid(GRID_1);
+
+        createEncryptedCache(node0, node1, cacheName(), null);
+
+        forceCheckpoint();
+
+        IgniteInternalCache<Object, Object> cache = node0.cachex(cacheName());
+
+        AtomicInteger cntr = new AtomicInteger(cache.size());
+
+        CountDownLatch startLatch = new CountDownLatch(1);
+
+        final Ignite somenode = node0;
+
+        IgniteInternalFuture<?> loadFut = GridTestUtils.runAsync(() -> {
+            try (IgniteDataStreamer<Integer, String> streamer = somenode.dataStreamer(cacheName())) {
+                while (!Thread.currentThread().isInterrupted()) {
+                    int n = cntr.getAndIncrement();
+
+                    streamer.addData(n, String.valueOf(n));
+
+                    if (n == 5000)
+                        startLatch.countDown();
+                }
+            }
+        });
+
+        startLatch.await(MAX_AWAIT_MILLIS, TimeUnit.MILLISECONDS);
+
+        int grpId = cache.context().groupId();
+
+        node0.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get(MAX_AWAIT_MILLIS);
+
+        Set<Integer> keys1 = new TreeSet<>(node0.context().encryption().groupKeyIds(grpId));
+        Set<Integer> keys2 = new TreeSet<>(node1.context().encryption().groupKeyIds(grpId));
+
+        assertEquals(keys1, keys2);
+
+        awaitEncryption(G.allGrids(), grpId, MAX_AWAIT_MILLIS);
+
+        forceCheckpoint();
+
+        loadFut.cancel();
+
+        // Ensure that data is encrypted with the new key.
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        stopAllGrids();
+
+        node0 = startGrid(GRID_0);
+        node1 = startGrid(GRID_1);
+
+        node0.cluster().state(ClusterState.ACTIVE);
+
+        // Wait for WAL segment remove.
+        try (IgniteDataStreamer<Integer, String> streamer = node0.dataStreamer(cacheName())) {
+            int start = cntr.get();
+
+            for (; ; ) {
+                int n = cntr.getAndIncrement();
+
+                streamer.addData(n, String.valueOf(n));
+
+                if (n % 1000 == 0 &&
+                    node0.context().encryption().groupKeyIds(grpId).size() == 1 &&
+                    node1.context().encryption().groupKeyIds(grpId).size() == 1)
+                    break;
+
+                if (n - start == 1_000_000)
+                    break;
+            }
+        }
+
+        assertEquals(1, node0.context().encryption().groupKeyIds(grpId).size());
+        assertEquals(1, node1.context().encryption().groupKeyIds(grpId).size());
+    }
+
+    /**
+     * Ensures that unused key will be removed even if user cleaned wal archive folder manually.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testWalArchiveCleanup() throws Exception {
+        startTestGrids(true);
+
+        IgniteEx node1 = grid(GRID_0);
+        IgniteEx node2 = grid(GRID_1);
+
+        createEncryptedCache(node1, node2, cacheName(), null);
+
+        node1.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get();
+
+        long walIdx = node1.context().cache().context().wal().currentSegment();
+
+        AtomicBoolean stopLoad = new AtomicBoolean();
+
+        IgniteInternalFuture<?> fut = runAsync(() -> {
+            Ignite grid = grid(GRID_0);
+
+            long cntr = grid.cache(cacheName()).size();
+
+            try (IgniteDataStreamer<Long, String> streamer = grid.dataStreamer(cacheName())) {
+                while (!stopLoad.get() && !Thread.currentThread().isInterrupted()) {
+                    streamer.addData(cntr, String.valueOf(cntr));
+
+                    streamer.flush();
+
+                    ++cntr;
+                }
+            }
+        });
+
+        try {
+            IgniteWriteAheadLogManager walMgr = grid(GRID_0).context().cache().context().wal();
+
+            boolean success = waitForCondition(() -> walMgr.lastArchivedSegment() >= walIdx, MAX_AWAIT_MILLIS);
+
+            assertTrue(success);
+        } finally {
+            stopLoad.set(true);
+
+            fut.get(MAX_AWAIT_MILLIS);
+        }
+
+        forceCheckpoint();
+
+        int grpId = CU.cacheId(cacheName());
+
+        assertEquals(2, node1.context().encryption().groupKeyIds(grpId).size());

Review comment:
       Can we temporary restrict deletions? Or here we can get size 1 in some circumstances

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/encryption/CacheGroupKeyChangeTest.java
##########
@@ -0,0 +1,988 @@
+/*
+ * 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.encryption;
+
+import java.io.File;
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.IgniteException;
+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.configuration.WALMode;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
+import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
+import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
+import org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType;
+import org.apache.ignite.internal.util.distributed.InitMessage;
+import org.apache.ignite.internal.util.distributed.SingleNodeMessage;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.spi.discovery.tcp.TestTcpDiscoverySpi;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.GridTestUtils.DiscoveryHook;
+import org.junit.Test;
+
+import static org.apache.ignite.configuration.WALMode.LOG_ONLY;
+import static org.apache.ignite.configuration.WALMode.NONE;
+import static org.apache.ignite.internal.managers.encryption.GridEncryptionManager.INITIAL_KEY_ID;
+import static org.apache.ignite.spi.encryption.keystore.KeystoreEncryptionSpi.DEFAULT_MASTER_KEY_NAME;
+import static org.apache.ignite.testframework.GridTestUtils.assertThrowsAnyCause;
+import static org.apache.ignite.testframework.GridTestUtils.assertThrowsWithCause;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+import static org.apache.ignite.testframework.GridTestUtils.waitForCondition;
+
+/**
+ * Cache group key change distributed process tests.
+ */
+public class CacheGroupKeyChangeTest extends AbstractEncryptionTest {
+    /** Timeout. */
+    private static final long MAX_AWAIT_MILLIS = 15_000;
+
+    /** */
+    private static final String GRID_2 = "grid-2";
+
+    /** Discovery hook for distributed process. */
+    private InitMessageDiscoveryHook discoveryHook;
+
+    /** Count of cache backups. */
+    private int backups;
+
+    /** WAL mode. */
+    private WALMode walMode = LOG_ONLY;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setConsistentId(name);
+        cfg.setCommunicationSpi(new TestRecordingCommunicationSpi());
+
+        if (discoveryHook != null)
+            ((TestTcpDiscoverySpi)cfg.getDiscoverySpi()).discoveryHook(discoveryHook);
+
+        DataStorageConfiguration memCfg = new DataStorageConfiguration()
+            .setDefaultDataRegionConfiguration(
+                new DataRegionConfiguration()
+                    .setMaxSize(100L * 1024 * 1024)
+                    .setPersistenceEnabled(true))
+            .setPageSize(4 * 1024)
+            .setWalSegmentSize(1024 * 1024)
+            .setWalSegments(10)
+            .setMaxWalArchiveSize(20 * 1024 * 1024)
+            .setCheckpointFrequency(30 * 1000L)
+            .setWalMode(walMode);
+
+        cfg.setDataStorageConfiguration(memCfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected <K, V> CacheConfiguration<K, V> cacheConfiguration(String name, String grp) {
+        CacheConfiguration<K, V> cfg = super.cacheConfiguration(name, grp);
+
+        return cfg.setAffinity(new RendezvousAffinityFunction(false, 8)).setBackups(backups);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    @SuppressWarnings("ThrowableNotThrown")
+    public void testRejectNodeJoinDuringRotation() throws Exception {
+        T2<IgniteEx, IgniteEx> grids = startTestGrids(true);
+
+        createEncryptedCache(grids.get1(), grids.get2(), cacheName(), null);
+
+        int grpId = CU.cacheId(cacheName());
+
+        assertEquals(0, grids.get1().context().encryption().groupKey(grpId).id());
+
+        TestRecordingCommunicationSpi commSpi = TestRecordingCommunicationSpi.spi(grids.get2());
+
+        commSpi.blockMessages((node, msg) -> msg instanceof SingleNodeMessage);
+
+        IgniteFuture<Void> fut = grids.get1().encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+
+        commSpi.waitForBlocked();
+
+        assertThrowsWithCause(() -> startGrid(3), IgniteCheckedException.class);
+
+        commSpi.stopBlock();
+
+        fut.get();
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        checkEncryptedCaches(grids.get1(), grids.get2());
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNotAllBltNodesPresent() throws Exception {
+        startTestGrids(true);
+
+        createEncryptedCache(grid(GRID_0), grid(GRID_1), cacheName(), null);
+
+        stopGrid(GRID_1);
+
+        grid(GRID_0).encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get();
+
+        startGrid(GRID_1);
+
+        checkGroupKey(CU.cacheId(cacheName()), INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNodeFailsBeforePrepare() throws Exception {
+        checkNodeFailsDuringRotation(false, true, true);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNodeFailsBeforePerform() throws Exception {
+        checkNodeFailsDuringRotation(false, false, true);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNodeFailsAfterPrepare() throws Exception {
+        checkNodeFailsDuringRotation(false, true, false);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testCrdFailsAfterPrepare() throws Exception {
+        checkNodeFailsDuringRotation(true, true, false);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNodeFailsAfterPerform() throws Exception {
+        checkNodeFailsDuringRotation(false, false, false);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testCrdFailsAfterPerform() throws Exception {
+        checkNodeFailsDuringRotation(true, false, false);
+    }
+
+    /**
+     * @param stopCrd {@code True} to stop coordinator.
+     * @param prepare {@code True} to stop on the prepare phase. {@code False} to stop on the perform phase.
+     * @param discoBlock  {@code True} to block discovery, {@code False} to block communication SPI.
+     */
+    private void checkNodeFailsDuringRotation(boolean stopCrd, boolean prepare, boolean discoBlock) throws Exception {
+        cleanPersistenceDir();
+
+        DistributedProcessType type = prepare ?
+            DistributedProcessType.CACHE_GROUP_KEY_CHANGE_PREPARE : DistributedProcessType.CACHE_GROUP_KEY_CHANGE_FINISH;
+
+        InitMessageDiscoveryHook locHook = new InitMessageDiscoveryHook(type);
+
+        if (discoBlock && stopCrd)
+            discoveryHook = locHook;
+
+        IgniteEx grid0 = startGrid(GRID_0);
+
+        if (discoBlock && !stopCrd)
+            discoveryHook = locHook;
+
+        IgniteEx grid1 = startGrid(GRID_1);
+
+        grid0.cluster().state(ClusterState.ACTIVE);
+
+        createEncryptedCache(grid0, grid1, cacheName(), null);
+
+        int grpId = CU.cacheId(cacheName());
+
+        checkGroupKey(grpId, INITIAL_KEY_ID, MAX_AWAIT_MILLIS);
+
+        TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid1);
+
+        if (!discoBlock) {
+            AtomicBoolean preparePhase = new AtomicBoolean(true);
+
+            spi.blockMessages((node, msg) -> {
+                if (msg instanceof SingleNodeMessage) {
+                    boolean isPrepare = preparePhase.compareAndSet(true, false);
+
+                    return prepare || !isPrepare;
+                }
+
+                return false;
+            });
+        }
+
+        String alive = stopCrd ? GRID_1 : GRID_0;
+        String stopped = stopCrd ? GRID_0 : GRID_1;
+
+        IgniteFuture<Void> changeFut = grid(alive).encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+
+        IgniteInternalFuture<?> stopFut = new GridFinishedFuture<>();
+
+        if (discoBlock) {
+            locHook.waitForBlocked(MAX_AWAIT_MILLIS);
+
+            stopGrid(stopped, true);
+
+            locHook.stopBlock();
+        }
+        else {
+            spi.waitForBlocked();
+
+            stopFut = runAsync(() -> stopGrid(stopped, true));
+        }
+
+        changeFut.get(MAX_AWAIT_MILLIS);
+        stopFut.get(MAX_AWAIT_MILLIS);
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        IgniteEx stoppedNode = startGrid(stopped);
+
+        stoppedNode.resetLostPartitions(Collections.singleton(ENCRYPTED_CACHE));
+
+        awaitPartitionMapExchange();
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        stoppedNode.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get(MAX_AWAIT_MILLIS);
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 2, MAX_AWAIT_MILLIS);
+    }
+
+    /**
+     * Ensures that we can rotate the key more than 255 times.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testKeyIdentifierOverflow() throws Exception {
+        IgniteEx node = startTestGrids(true).get1();
+
+        createEncryptedCache(node, null, cacheName(), null, false);
+
+        int grpId = CU.cacheId(cacheName());
+
+        byte keyId = INITIAL_KEY_ID;
+
+        do {
+            node.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get();
+
+            // Validates reencryption of index partition.
+            checkGroupKey(grpId, ++keyId & 0xff, MAX_AWAIT_MILLIS);
+        } while (keyId != INITIAL_KEY_ID);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testMasterAndCacheGroupKeySimultaneousChange() throws Exception {
+        startTestGrids(true);
+
+        IgniteEx node0 = grid(GRID_0);
+        IgniteEx node1 = grid(GRID_1);
+
+        createEncryptedCache(node0, node1, cacheName(), null);
+
+        int grpId = CU.cacheId(cacheName());
+
+        assertTrue(checkMasterKeyName(DEFAULT_MASTER_KEY_NAME));
+
+        Random rnd = ThreadLocalRandom.current();
+
+        for (byte keyId = 1; keyId < 50; keyId++) {
+            String currMkName = node0.context().config().getEncryptionSpi().getMasterKeyName();
+            String newMkName = currMkName.equals(MASTER_KEY_NAME_2) ? MASTER_KEY_NAME_3 : MASTER_KEY_NAME_2;
+
+            boolean changeGrpFirst = rnd.nextBoolean();
+
+            IgniteFuture<Void> grpKeyFut;
+            IgniteFuture<Void> masterKeyFut;
+
+            if (changeGrpFirst) {
+                grpKeyFut = node0.encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+                masterKeyFut = node0.encryption().changeMasterKey(newMkName);
+            }
+            else {
+                masterKeyFut = node0.encryption().changeMasterKey(newMkName);
+                grpKeyFut = node0.encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+            }
+
+            masterKeyFut.get(MAX_AWAIT_MILLIS);
+            assertTrue(checkMasterKeyName(newMkName));
+
+            try {
+                grpKeyFut.get(MAX_AWAIT_MILLIS);
+                checkGroupKey(grpId, keyId, MAX_AWAIT_MILLIS);
+            } catch (IgniteException e) {
+                assertTrue(e.getMessage().contains("Cache group key change was rejected. Master key has been changed."));
+
+                keyId -= 1;
+            }
+        }
+    }
+
+    /**
+     * Ensures that after rotation, the node has correct key identifier.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testNodeJoinAfterChange() throws Exception {
+        startTestGrids(true);
+
+        IgniteEx node0 = grid(GRID_0);
+        IgniteEx node1 = grid(GRID_1);
+
+        createEncryptedCache(node0, node1, cacheName(), null);
+
+        node0.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get();
+
+        startGrid(GRID_2);
+
+        resetBaselineTopology();
+
+        awaitPartitionMapExchange();
+
+        checkGroupKey(CU.cacheId(cacheName()), INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testCacheStartDuringRotation() throws Exception {
+        T2<IgniteEx, IgniteEx> grids = startTestGrids(true);
+
+        createEncryptedCache(grids.get1(), grids.get2(), cacheName(), null);
+
+        TestRecordingCommunicationSpi commSpi = TestRecordingCommunicationSpi.spi(grids.get2());
+
+        commSpi.blockMessages((node, msg) -> msg instanceof SingleNodeMessage);
+
+        IgniteFuture<Void> fut = grids.get1().encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+
+        commSpi.waitForBlocked();
+
+        IgniteCache<Integer, Integer> cache = grids.get1().createCache(cacheConfiguration("cache1", null));
+
+        for (int i = 0; i < 100; i++)
+            cache.put(i, i);
+
+        commSpi.stopBlock();
+
+        fut.get();
+
+        checkGroupKey(CU.cacheId(cacheName()), INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        checkGroupKey(CU.cacheId("cache1"), INITIAL_KEY_ID, MAX_AWAIT_MILLIS);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testCacheStartSameGroupDuringRotation() throws Exception {
+        T2<IgniteEx, IgniteEx> grids = startTestGrids(true);
+
+        String grpName = "shared";
+
+        createEncryptedCache(grids.get1(), grids.get2(), cacheName(), grpName);
+
+        TestRecordingCommunicationSpi commSpi = TestRecordingCommunicationSpi.spi(grids.get2());
+
+        commSpi.blockMessages((node, msg) -> msg instanceof SingleNodeMessage);
+
+        IgniteFuture<Void> fut = grids.get1().encryption().changeCacheGroupKey(Collections.singleton(grpName));
+
+        commSpi.waitForBlocked();
+
+        IgniteCache<Integer, Integer> cache =
+            grids.get1().createCache(cacheConfiguration("cache1", grpName));
+
+        commSpi.stopBlock();
+
+        for (int i = 0; i < 100; i++)
+            cache.put(i, i);
+
+        fut.get();
+
+        checkGroupKey(CU.cacheId(grpName), INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testChangeKeyDuringRebalancing() throws Exception {
+        T2<IgniteEx, IgniteEx> grids = startTestGrids(true);
+
+        IgniteEx node0 = grids.get1();
+        IgniteEx node1 = grids.get2();
+
+        createEncryptedCache(node0, node1, cacheName(), null);
+
+        loadData(500_000);
+
+        IgniteEx node2 = startGrid(GRID_2);
+
+        resetBaselineTopology();
+
+        int grpId = CU.cacheId(cacheName());
+
+        IgniteFuture<Void> fut = node2.encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+
+        fut.get(MAX_AWAIT_MILLIS);
+
+        stopAllGrids();
+
+        startGrid(GRID_0);
+        startGrid(GRID_1);
+        startGrid(GRID_2);
+
+        grid(GRID_0).cluster().state(ClusterState.ACTIVE);
+
+        awaitPartitionMapExchange();
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testBasicChange() throws Exception {
+        startTestGrids(true);
+
+        IgniteEx node1 = grid(GRID_0);
+        IgniteEx node2 = grid(GRID_1);
+
+        createEncryptedCache(node1, node2, cacheName(), null);
+
+        forceCheckpoint();
+
+        IgniteInternalCache<Object, Object> cache = node1.cachex(cacheName());
+
+        int grpId = cache.context().groupId();
+
+        node1.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get();
+
+        Set<Integer> keys1 = new TreeSet<>(node1.context().encryption().groupKeyIds(grpId));
+        Set<Integer> keys2 = new TreeSet<>(node2.context().encryption().groupKeyIds(grpId));
+
+        assertEquals(2, keys1.size());
+
+        assertEquals(keys1, keys2);
+
+        info("New key was set on all nodes [grpId=" + grpId + ", keys=" + keys1 + "]");
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        stopAllGrids();
+
+        node1 = startGrid(GRID_0);
+        node2 = startGrid(GRID_1);
+
+        node1.cluster().state(ClusterState.ACTIVE);
+
+        // Previous leys must be deleted when the corresponding WAL segment is deleted.
+        try (IgniteDataStreamer<Integer, String> streamer = node1.dataStreamer(cacheName())) {
+            for (int i = node1.cache(cacheName()).size(); i < 500_000; i++) {
+                streamer.addData(i, String.valueOf(i));
+
+                if (i % 1_000 == 0 &&
+                    node1.context().encryption().groupKeyIds(grpId).size() == 1 &&
+                    node2.context().encryption().groupKeyIds(grpId).size() == 1)
+                    break;
+            }
+        }
+
+        assertEquals(1, node1.context().encryption().groupKeyIds(grpId).size());
+        assertEquals(1, node2.context().encryption().groupKeyIds(grpId).size());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testBasicChangeWithConstantLoad() throws Exception {
+        startTestGrids(true);
+
+        IgniteEx node0 = grid(GRID_0);
+        IgniteEx node1 = grid(GRID_1);
+
+        createEncryptedCache(node0, node1, cacheName(), null);
+
+        forceCheckpoint();
+
+        IgniteInternalCache<Object, Object> cache = node0.cachex(cacheName());
+
+        AtomicInteger cntr = new AtomicInteger(cache.size());
+
+        CountDownLatch startLatch = new CountDownLatch(1);
+
+        final Ignite somenode = node0;
+
+        IgniteInternalFuture<?> loadFut = GridTestUtils.runAsync(() -> {
+            try (IgniteDataStreamer<Integer, String> streamer = somenode.dataStreamer(cacheName())) {
+                while (!Thread.currentThread().isInterrupted()) {
+                    int n = cntr.getAndIncrement();
+
+                    streamer.addData(n, String.valueOf(n));
+
+                    if (n == 5000)
+                        startLatch.countDown();
+                }
+            }
+        });
+
+        startLatch.await(MAX_AWAIT_MILLIS, TimeUnit.MILLISECONDS);
+
+        int grpId = cache.context().groupId();
+
+        node0.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get(MAX_AWAIT_MILLIS);
+
+        Set<Integer> keys1 = new TreeSet<>(node0.context().encryption().groupKeyIds(grpId));
+        Set<Integer> keys2 = new TreeSet<>(node1.context().encryption().groupKeyIds(grpId));
+
+        assertEquals(keys1, keys2);
+
+        awaitEncryption(G.allGrids(), grpId, MAX_AWAIT_MILLIS);
+
+        forceCheckpoint();
+
+        loadFut.cancel();
+
+        // Ensure that data is encrypted with the new key.
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        stopAllGrids();
+
+        node0 = startGrid(GRID_0);
+        node1 = startGrid(GRID_1);
+
+        node0.cluster().state(ClusterState.ACTIVE);
+
+        // Wait for WAL segment remove.
+        try (IgniteDataStreamer<Integer, String> streamer = node0.dataStreamer(cacheName())) {
+            int start = cntr.get();
+
+            for (; ; ) {
+                int n = cntr.getAndIncrement();
+
+                streamer.addData(n, String.valueOf(n));
+
+                if (n % 1000 == 0 &&
+                    node0.context().encryption().groupKeyIds(grpId).size() == 1 &&
+                    node1.context().encryption().groupKeyIds(grpId).size() == 1)
+                    break;
+
+                if (n - start == 1_000_000)
+                    break;
+            }
+        }
+
+        assertEquals(1, node0.context().encryption().groupKeyIds(grpId).size());
+        assertEquals(1, node1.context().encryption().groupKeyIds(grpId).size());
+    }
+
+    /**
+     * Ensures that unused key will be removed even if user cleaned wal archive folder manually.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testWalArchiveCleanup() throws Exception {
+        startTestGrids(true);
+
+        IgniteEx node1 = grid(GRID_0);
+        IgniteEx node2 = grid(GRID_1);
+
+        createEncryptedCache(node1, node2, cacheName(), null);
+
+        node1.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get();
+
+        long walIdx = node1.context().cache().context().wal().currentSegment();
+
+        AtomicBoolean stopLoad = new AtomicBoolean();
+
+        IgniteInternalFuture<?> fut = runAsync(() -> {
+            Ignite grid = grid(GRID_0);
+
+            long cntr = grid.cache(cacheName()).size();
+
+            try (IgniteDataStreamer<Long, String> streamer = grid.dataStreamer(cacheName())) {
+                while (!stopLoad.get() && !Thread.currentThread().isInterrupted()) {
+                    streamer.addData(cntr, String.valueOf(cntr));
+
+                    streamer.flush();
+
+                    ++cntr;
+                }
+            }
+        });
+
+        try {
+            IgniteWriteAheadLogManager walMgr = grid(GRID_0).context().cache().context().wal();
+
+            boolean success = waitForCondition(() -> walMgr.lastArchivedSegment() >= walIdx, MAX_AWAIT_MILLIS);
+
+            assertTrue(success);
+        } finally {
+            stopLoad.set(true);
+
+            fut.get(MAX_AWAIT_MILLIS);
+        }
+
+        forceCheckpoint();
+
+        int grpId = CU.cacheId(cacheName());
+
+        assertEquals(2, node1.context().encryption().groupKeyIds(grpId).size());
+        assertEquals(2, node2.context().encryption().groupKeyIds(grpId).size());
+
+        stopAllGrids();
+
+        // Cleanup wal arcive.
+        File dbDir = U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false);
+
+        boolean rmvd = U.delete(new File(dbDir, "wal/archive"));
+
+        assertTrue(rmvd);
+
+        startTestGrids(false);
+
+        node1 = grid(GRID_0);
+        node2 = grid(GRID_1);
+
+        assertEquals(2, node1.context().encryption().groupKeyIds(grpId).size());
+        assertEquals(2, node2.context().encryption().groupKeyIds(grpId).size());
+
+        stopLoad.set(false);
+
+        fut = runAsync(() -> {
+            Ignite grid = grid(GRID_0);
+
+            long cntr = grid.cache(cacheName()).size();
+
+            try (IgniteDataStreamer<Long, String> streamer = grid.dataStreamer(cacheName())) {
+                while (!stopLoad.get() && !Thread.currentThread().isInterrupted()) {
+                    streamer.addData(cntr, String.valueOf(cntr));
+
+                    ++cntr;
+                }
+            }
+        });
+
+        try {
+            waitForCondition(() -> {
+                List<Integer> keys1 = grid(GRID_0).context().encryption().groupKeyIds(grpId);
+                List<Integer> keys2 = grid(GRID_1).context().encryption().groupKeyIds(grpId);
+
+                return keys1.size() == 1 && keys2.size() == 1;
+            }, MAX_AWAIT_MILLIS);
+
+            assertEquals(1, node1.context().encryption().groupKeyIds(grpId).size());
+            assertEquals(1, node2.context().encryption().groupKeyIds(grpId).size());
+        } finally {
+            stopLoad.set(true);
+
+            fut.get(MAX_AWAIT_MILLIS);
+        }
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testCacheStartOnClientDuringRotation() throws Exception {
+        T2<IgniteEx, IgniteEx> nodes = startTestGrids(true);
+
+        IgniteEx node0 = nodes.get1();
+        IgniteEx node1 = nodes.get2();
+
+        IgniteEx client = startClientGrid(getConfiguration("client"));
+
+        node0.cluster().state(ClusterState.ACTIVE);
+
+        String grpName = "shared";
+
+        createEncryptedCache(client, null, cacheName(), grpName);
+
+        awaitPartitionMapExchange();
+
+        TestRecordingCommunicationSpi commSpi = TestRecordingCommunicationSpi.spi(node1);
+
+        commSpi.blockMessages((node, message) -> message instanceof SingleNodeMessage);
+
+        IgniteFuture<Void> changeKeyFut = node0.encryption().changeCacheGroupKey(Collections.singleton(grpName));
+
+        commSpi.waitForBlocked();
+
+        String cacheName = "userCache";
+
+        IgniteInternalFuture<?> cacheStartFut = runAsync(() -> {
+            client.getOrCreateCache(cacheConfiguration(cacheName, grpName));
+        });
+
+        commSpi.stopBlock();
+
+        changeKeyFut.get(MAX_AWAIT_MILLIS);
+        cacheStartFut.get(MAX_AWAIT_MILLIS);
+
+        IgniteCache<Integer, String> cache = client.cache(cacheName);
+
+        for (int i = 0; i < 200; i++)
+            cache.put(i, String.valueOf(i));
+
+        checkEncryptedCaches(node0, client);
+
+        checkGroupKey(CU.cacheId(grpName), INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testClientJoinDuringRotation() throws Exception {
+        T2<IgniteEx, IgniteEx> nodes = startTestGrids(true);
+
+        IgniteEx node0 = nodes.get1();
+        IgniteEx node1 = nodes.get2();
+
+        node0.cluster().state(ClusterState.ACTIVE);
+
+        createEncryptedCache(node0, node1, cacheName(), null);
+
+        awaitPartitionMapExchange();
+
+        TestRecordingCommunicationSpi commSpi = TestRecordingCommunicationSpi.spi(node1);
+
+        commSpi.blockMessages((node, message) -> message instanceof SingleNodeMessage);
+
+        IgniteFuture<Void> changeKeyFut = node0.encryption().changeCacheGroupKey(Collections.singleton(cacheName()));
+
+        commSpi.waitForBlocked();
+
+        IgniteEx client = startClientGrid(getConfiguration("client"));
+
+        assertTrue(!changeKeyFut.isDone());
+
+        commSpi.stopBlock();
+
+        changeKeyFut.get(MAX_AWAIT_MILLIS);
+
+        checkEncryptedCaches(node0, client);
+
+        checkGroupKey(CU.cacheId(cacheName()), INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testNotBltNodeJoin() throws Exception {
+        backups = 1;
+
+        T2<IgniteEx, IgniteEx> nodes = startTestGrids(true);
+
+        createEncryptedCache(nodes.get1(), nodes.get2(), cacheName(), null);
+
+        forceCheckpoint();
+
+        long startIdx = nodes.get2().context().cache().context().wal().currentSegment();
+
+        stopGrid(GRID_1);
+
+        resetBaselineTopology();
+
+        nodes.get1().encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get();
+
+        startGrid(GRID_1);
+
+        resetBaselineTopology();
+
+        awaitPartitionMapExchange();
+
+        long endIdx = nodes.get2().context().cache().context().wal().currentSegment();
+
+        int grpId = CU.cacheId(cacheName());
+
+        checkGroupKey(grpId, INITIAL_KEY_ID + 1, MAX_AWAIT_MILLIS);
+
+        checkEncryptedCaches(grid(GRID_0), grid(GRID_1));
+
+        for (long segment = startIdx; segment <= endIdx; segment++)
+            grid(GRID_1).context().encryption().onWalSegmentRemoved(segment);
+
+        assertEquals(1, grid(GRID_1).context().encryption().groupKeyIds(grpId).size());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testWrongCacheGroupSpecified() throws Exception {
+        T2<IgniteEx, IgniteEx> grids = startTestGrids(true);
+
+        IgniteEx node0 = grids.get1();
+        IgniteEx node1 = grids.get2();
+
+        assertThrowsAnyCause(log,
+            () -> node0.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get(MAX_AWAIT_MILLIS),
+            IgniteException.class,
+            "Cache group key change was rejected. Cache or group \"" + cacheName() + "\" doesn't exists");
+
+        node0.createCache(new CacheConfiguration<>(cacheName()).setNodeFilter(node -> node.equals(node0.localNode())));
+
+        assertThrowsAnyCause(log,
+            () -> node1.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get(MAX_AWAIT_MILLIS),
+            IgniteException.class,
+            "Cache group key change was rejected. Cache or group \"" + cacheName() + "\" is not encrypted.");
+
+        node0.destroyCache(cacheName());
+
+        awaitPartitionMapExchange();
+
+        String grpName = "cacheGroup1";
+
+        createEncryptedCache(node0, node1, cacheName(), grpName);
+
+        assertThrowsAnyCause(log,
+            () -> node0.encryption().changeCacheGroupKey(Collections.singleton(cacheName())).get(MAX_AWAIT_MILLIS),
+            IgniteException.class,
+            "Cache group key change was rejected. Cache or group \"" + cacheName() + "\" is a part of group \"" +
+            grpName + "\". Provide group name instead of cache name for shared groups.");
+    }
+
+    /**
+     * Custom discovery hook to block distributed process.
+     */
+    private static class InitMessageDiscoveryHook extends DiscoveryHook {
+        /**
+         * Latch to sync execution.
+         */
+        private final CountDownLatch unlockLatch = new CountDownLatch(1);
+
+        /**
+         * Latch to sync execution.
+         */
+        private final CountDownLatch blockedLatch = new CountDownLatch(1);
+
+        /**
+         * Distributed process type.
+         */
+        private final DistributedProcessType type;
+
+        /**
+         * @param type Distributed process type.
+         */
+        private InitMessageDiscoveryHook(DistributedProcessType type) {
+            this.type = type;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void beforeDiscovery(DiscoveryCustomMessage customMsg) {
+            if (!(customMsg instanceof InitMessage))
+                return;
+
+            InitMessage<Serializable> msg = (InitMessage<Serializable>)customMsg;
+
+            if (msg.type() != type.ordinal())
+                return;
+
+            try {
+                blockedLatch.countDown();
+
+                unlockLatch.await(MAX_AWAIT_MILLIS, TimeUnit.MILLISECONDS);
+            }
+            catch (InterruptedException ignore) {
+                Thread.currentThread().interrupt();
+            }
+        }
+
+        /**
+         * @param timeout Timeout in milliseconds.
+         * @throws InterruptedException If interrupted.
+         */
+        public void waitForBlocked(long timeout) throws InterruptedException {
+            blockedLatch.await(timeout, TimeUnit.MILLISECONDS);
+        }
+
+        /** */
+        public void stopBlock() {
+            unlockLatch.countDown();
+        }
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testChangeCacheGroupKeyWithoutWAL() throws Exception {

Review comment:
       Let's move this test before InitMessageDiscoveryHook class




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

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



[GitHub] [ignite] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupReencryption.java
##########
@@ -0,0 +1,583 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+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.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+import org.apache.ignite.thread.OomExceptionHandler;
+
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_REENCRYPTION_BATCH_SIZE;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_REENCRYPTION_DISABLED;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_REENCRYPTION_THREAD_POOL_SIZE;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_REENCRYPTION_THROTTLE;
+
+/**
+ * Cache group reencryption manager.
+ */
+public class CacheGroupReencryption implements DbCheckpointListener {

Review comment:
       Now it's called  CacheGroupPageScanner )




----------------------------------------------------------------
This is an automated message from the 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] NSAmelchev commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GroupKeyChangeProcess.java
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteFeatures;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.managers.encryption.GridEncryptionManager.EmptyResult;
+import org.apache.ignite.internal.managers.encryption.GridEncryptionManager.KeyChangeFuture;
+import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor;
+import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
+import org.apache.ignite.internal.util.distributed.DistributedProcess;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.future.IgniteFinishedFutureImpl;
+import org.apache.ignite.internal.util.future.IgniteFutureImpl;
+import org.apache.ignite.internal.util.typedef.F;
+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.IgniteFuture;
+import org.apache.ignite.lang.IgniteFutureCancelledException;
+
+import static org.apache.ignite.internal.IgniteFeatures.CACHE_GROUP_KEY_CHANGE;
+import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.CACHE_GROUP_KEY_CHANGE_FINISH;
+import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.CACHE_GROUP_KEY_CHANGE_PREPARE;
+
+/**
+ * A two-phase distributed process that rotates the encryption keys of specified cache groups and initiates
+ * re-encryption of those cache groups.
+ */
+class GroupKeyChangeProcess {
+    /** Grid kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Cache group encyption key change prepare phase. */
+    private final DistributedProcess<ChangeCacheEncryptionRequest, EmptyResult> prepareGKChangeProc;
+
+    /** Cache group encyption key change perform phase. */
+    private final DistributedProcess<ChangeCacheEncryptionRequest, EmptyResult> performGKChangeProc;
+
+    /** Group encryption keys. */
+    private final CacheGroupEncryptionKeys keys;
+
+    /** Cache group key change future. */
+    private volatile GroupKeyChangeFuture fut;
+
+    /** Cache group key change request. */
+    private volatile ChangeCacheEncryptionRequest req;
+
+    /**
+     * @param ctx Grid kernal context.
+     * @param keys Cache group encryption keys.
+     */
+    GroupKeyChangeProcess(GridKernalContext ctx, CacheGroupEncryptionKeys keys) {
+        this.ctx = ctx;
+        this.keys = keys;
+
+        prepareGKChangeProc =
+            new DistributedProcess<>(ctx, CACHE_GROUP_KEY_CHANGE_PREPARE, this::prepare, this::finishPrepare);
+        performGKChangeProc =
+            new DistributedProcess<>(ctx, CACHE_GROUP_KEY_CHANGE_FINISH, this::perform, this::finishPerform);
+    }
+
+    /**
+     * @return {@code True} if operation is still in progress.
+     */
+    public boolean inProgress() {
+        return req != null;
+    }
+
+    /**
+     * @param msg Error message.
+     */
+    public void cancel(String msg) {
+        GridFutureAdapter<Void> keyChangeFut = fut;
+
+        if (keyChangeFut != null && !keyChangeFut.isDone())
+            keyChangeFut.onDone(new IgniteFutureCancelledException(msg));
+    }
+
+    /**
+     * Starts cache group encryption key change process.
+     *
+     * @param cacheOrGrpNames Cache or group names.
+     */
+    public IgniteFuture<Void> start(Collection<String> cacheOrGrpNames) {
+        if (ctx.clientNode())
+            throw new UnsupportedOperationException("Client and daemon nodes can not perform this operation.");
+
+        if (!IgniteFeatures.allNodesSupports(ctx.grid().cluster().nodes(), CACHE_GROUP_KEY_CHANGE))
+            throw new IllegalStateException("Not all nodes in the cluster support this operation.");
+
+        if (!ctx.state().clusterState().state().active())
+            throw new IgniteException("Operation was rejected. The cluster is inactive.");
+
+        IgniteInternalFuture<Void> fut0 = fut;
+
+        if (fut0 != null && !fut0.isDone()) {
+            return new IgniteFinishedFutureImpl<>(new IgniteException("Cache group key change was rejected. " +
+                "The previous change was not completed."));
+        }
+
+        int[] grpIds = new int[cacheOrGrpNames.size()];
+        byte[] keyIds = new byte[grpIds.length];
+
+        int n = 0;
+
+        for (String cacheOrGroupName : cacheOrGrpNames) {
+            CacheGroupDescriptor grpDesc = ctx.cache().cacheGroupDescriptor(CU.cacheId(cacheOrGroupName));
+
+            if (grpDesc == null) {
+                DynamicCacheDescriptor cacheDesc = ctx.cache().cacheDescriptor(cacheOrGroupName);
+
+                if (cacheDesc == null) {
+                    throw new IgniteException("Cache group key change was rejected. " +
+                        "Cache or group \"" + cacheOrGroupName + "\" doesn't exists");
+                }
+
+                int grpId = cacheDesc.groupId();
+
+                grpDesc = ctx.cache().cacheGroupDescriptor(grpId);
+
+                if (grpDesc.sharedGroup()) {
+                    throw new IgniteException("Cache group key change was rejected. " +
+                        "Cache or group \"" + cacheOrGroupName + "\" is a part of group \"" +
+                        grpDesc.groupName() + "\". Provide group name instead of cache name for shared groups.");
+                }
+            }
+
+            if (!grpDesc.config().isEncryptionEnabled()) {
+                throw new IgniteException("Cache group key change was rejected. " +
+                    "Cache or group \"" + cacheOrGroupName + "\" is not encrypted.");
+            }
+
+            if (ctx.encryption().reencryptionInProgress(grpDesc.groupId())) {
+                throw new IgniteException("Cache group key change was rejected. " +
+                    "Cache group reencryption is in progress [grp=" + cacheOrGroupName + "]");
+            }
+
+            grpIds[n] = grpDesc.groupId();
+            keyIds[n] = (byte)(ctx.encryption().groupKey(grpDesc.groupId()).unsignedId() + 1);
+
+            n += 1;
+        }
+
+        byte[][] keys = ctx.encryption().createKeys(grpIds.length).get1().toArray(new byte[grpIds.length][]);
+
+        ChangeCacheEncryptionRequest req =
+            new ChangeCacheEncryptionRequest(grpIds, keys, keyIds, ctx.config().getEncryptionSpi().getMasterKeyName());

Review comment:
       Lets send a MK digest instead of a name




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

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



[GitHub] [ignite] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -627,22 +716,104 @@ public void groupKey(int grpId, byte[] encGrpKey) {
         return withMasterKeyChangeReadLock(() -> getSpi().getMasterKeyName());
     }
 
+    /** {@inheritDoc} */
+    @Override public IgniteFuture<Void> changeCacheGroupKey(Collection<String> cacheOrGrpNames) {
+        A.notEmpty(cacheOrGrpNames, "cacheOrGrpNames");
+
+        synchronized (opsMux) {
+            if (stopped) {
+                return new IgniteFinishedFutureImpl<>(new IgniteException("Cache group key change was rejected. " +
+                    "Node is stopping."));
+            }
+
+            if (masterKeyChangeFut != null && !masterKeyChangeFut.isDone()) {
+                return new IgniteFinishedFutureImpl<>(new IgniteException("Cache group key change was rejected. " +
+                    "The master key change is in progress."));
+            }
+
+            return grpKeyChangeProc.start(cacheOrGrpNames);
+        }
+    }
+
+    /**
+     * @param grpIds Cache group IDs.
+     * @param keyIds Encryption key IDs.
+     * @param keys Encryption keys.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected void changeCacheGroupKeyLocal(int[] grpIds, byte[] keyIds, byte[][] keys) throws IgniteCheckedException {
+        Map<Integer, Byte> encryptionStatus = U.newHashMap(grpIds.length);
+
+        for (int i = 0; i < grpIds.length; i++)
+            encryptionStatus.put(grpIds[i], keyIds[i]);
+
+        WALPointer ptr = ctx.cache().context().wal().log(new ReencryptionStatusRecord(encryptionStatus));
+
+        if (ptr != null)
+            ctx.cache().context().wal().flush(ptr, false);
+
+        for (int i = 0; i < grpIds.length; i++) {
+            int grpId = grpIds[i];
+
+            CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+            if (grp == null)
+                continue;
+
+            GroupKeyEncrypted key = new GroupKeyEncrypted(keyIds[i] & 0xff, keys[i]);
+
+            synchronized (metaStorageMux) {
+                // Store new key as inactive for recovery.
+                grpKeys.addKey(grpId, key);
+
+                writeToMetaStore(grpId, true, false);
+
+                // Set new key as key for writing.
+                GroupKey prevGrpKey = grpKeys.changeActiveKey(grpId, key);
+
+                assert prevGrpKey != null && prevGrpKey.id() != key.id() : "prev=" + prevGrpKey + ", currId=" + key.id();
+
+                grpKeys.reserveWalKey(grpId, prevGrpKey.unsignedId(), ctx.cache().context().wal().currentSegment());
+
+                writeToMetaStore(grpId, true, true);
+            }
+
+            reencryptGroups.put(grpId, pageScanner.pagesCount(grp));
+
+            if (log.isInfoEnabled())
+                log.info("New encryption key for group was added [grpId=" + grpId + ", keyId=" + key.id() + "]");
+        }
+
+        startReencryption(encryptionStatus.keySet());
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @return Future that will be completed when reencryption of the specified group is finished.
+     */
+    public IgniteInternalFuture<Void> reencryptionFuture(int grpId) {
+        if (pageScanner.disabled() && reencryptGroups.containsKey(grpId))
+            return new GridFutureAdapter<>();

Review comment:
       Changed to throw 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] vldpyatkov commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupPageScanner.java
##########
@@ -0,0 +1,479 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId;
+import org.apache.ignite.internal.util.BasicRateLimiter;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+import org.apache.ignite.thread.OomExceptionHandler;
+
+import static org.apache.ignite.internal.util.IgniteUtils.MB;
+
+/**
+ * Cache group page stores scanner.
+ * Scans a range of pages and marks them as dirty to re-encrypt them with the last encryption key on disk.
+ */
+public class CacheGroupPageScanner implements CheckpointListener {
+    /** Thread prefix for scanning tasks. */
+    private static final String REENCRYPT_THREAD_PREFIX = "reencrypt";
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to group scanning task. */
+    private final Map<Integer, GroupScanTask> grps = new ConcurrentHashMap<>();
+
+    /** Collection of groups waiting for a checkpoint. */
+    private final Collection<GroupScanTask> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Page scanning speed limiter. */
+    private final BasicRateLimiter limiter;
+
+    /** Single-threaded executor to run cache group scan task. */
+    private final ThreadPoolExecutor singleExecSvc;
+
+    /** Number of pages that is scanned during reencryption under checkpoint lock. */
+    private final int batchSize;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupPageScanner(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        DataStorageConfiguration dsCfg = ctx.config().getDataStorageConfiguration();
+
+        if (!CU.isPersistenceEnabled(dsCfg)) {
+            batchSize = -1;
+            limiter = null;
+            singleExecSvc = null;
+
+            return;
+        }
+
+        double rateLimit = dsCfg.getEncryptionConfiguration().getReencryptionRateLimit();
+
+        limiter = rateLimit > 0 ? new BasicRateLimiter(rateLimit * MB /
+            (dsCfg.getPageSize() == 0 ? DataStorageConfiguration.DFLT_PAGE_SIZE : dsCfg.getPageSize())) : null;
+
+        batchSize = dsCfg.getEncryptionConfiguration().getReencryptionBatchSize();
+
+        singleExecSvc = new IgniteThreadPoolExecutor(REENCRYPT_THREAD_PREFIX,
+            ctx.igniteInstanceName(),
+            1,
+            1,
+            IgniteConfiguration.DFLT_THREAD_KEEP_ALIVE_TIME,
+            new LinkedBlockingQueue<>(),
+            GridIoPolicy.SYSTEM_POOL,
+            new OomExceptionHandler(ctx));
+
+        singleExecSvc.allowCoreThreadTimeOut(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCheckpointBegin(Context cpCtx) {
+        Set<GroupScanTask> completeCandidates = new HashSet<>();
+
+        cpWaitGrps.removeIf(completeCandidates::add);
+
+        cpCtx.finishedStateFut().listen(
+            f -> {
+                // Retry if error occurs.
+                if (f.error() != null || f.isCancelled()) {
+                    cpWaitGrps.addAll(completeCandidates);
+
+                    return;
+                }
+
+                lock.lock();
+
+                try {
+                    for (GroupScanTask grpScanTask : completeCandidates) {
+                        grps.remove(grpScanTask.groupId());
+
+                        grpScanTask.onDone();
+
+                        if (log.isInfoEnabled())
+                            log.info("Cache group reencryption is finished [grpId=" + grpScanTask.groupId() + "]");
+                    }
+
+                    if (!grps.isEmpty())
+                        return;
+
+                    ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).
+                        removeCheckpointListener(this);
+                }
+                finally {
+                    lock.unlock();
+                }
+            }
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public void beforeCheckpointBegin(Context cpCtx) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onMarkCheckpointBegin(Context ctx) {
+        // No-op.
+    }
+
+    /**
+     * Schedule scanning partitions.
+     *
+     * @param grpId Cache group ID.
+     */
+    public IgniteInternalFuture<Void> schedule(int grpId) throws IgniteCheckedException {
+        CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+        if (grp == null || !grp.affinityNode()) {
+            if (log.isInfoEnabled())
+                log.info("Skip reencryption, cache group doesn't exist on the local node [grp=" + grpId + "]");
+
+            return new GridFinishedFuture<>();
+        }
+
+        lock.lock();
+
+        try {
+            if (stopped)
+                throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
+
+            if (grps.isEmpty())
+                ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).addCheckpointListener(this);
+
+            GroupScanTask prevState = grps.get(grpId);
+
+            if (prevState != null && !prevState.isDone()) {
+                if (log.isDebugEnabled())
+                    log.debug("Reencryption already scheduled [grpId=" + grpId + "]");
+
+                return prevState;
+            }
+
+            Set<Integer> parts = new HashSet<>();
+
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) {
+                    if (ctx.encryption().getEncryptionState(grpId, partId) == 0) {
+                        if (log.isDebugEnabled())
+                            log.debug("Skipping partition reencryption [grp=" + grpId + ", p=" + partId + "]");
+
+                        return;
+                    }
+
+                    parts.add(partId);
+                }
+            });
+
+            GroupScanTask grpScan = new GroupScanTask(grp, parts);
+
+            singleExecSvc.submit(grpScan);
+
+            if (log.isInfoEnabled())
+                log.info("Scheduled reencryption [grpId=" + grpId + "]");
+
+            grps.put(grpId, grpScan);
+
+            return grpScan;
+        }
+        finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @return Future that will be completed when all partitions have been scanned and pages have been written to disk.
+     */
+    public IgniteInternalFuture<Void> statusFuture(int grpId) {
+        GroupScanTask grpScanTask = grps.get(grpId);
+
+        return grpScanTask == null ? new GridFinishedFuture<>() : grpScanTask;
+    }
+
+    /**
+     * Shutdown scanning and disable new tasks scheduling.
+     */
+    public void stop() throws IgniteCheckedException {
+        lock.lock();
+
+        try {
+            stopped = true;
+
+            for (GroupScanTask grpScanTask : grps.values())
+                grpScanTask.cancel();
+
+            if (singleExecSvc != null)
+                singleExecSvc.shutdownNow();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Stop scannig the specified partition.
+     *
+     * @param grpId Cache group ID.
+     * @param partId Partition ID.
+     * @return {@code True} if reencryption was cancelled.
+     */
+    public boolean excludePartition(int grpId, int partId) {
+        GroupScanTask grpScanTask = grps.get(grpId);
+
+        if (grpScanTask == null)
+            return false;
+
+        return grpScanTask.excludePartition(partId);
+    }
+
+    /**
+     * Collect current number of pages in the specified cache group.
+     *
+     * @param grp Cache group.
+     * @return Partitions with current page count.
+     * @throws IgniteCheckedException If failed.
+     */
+    public long[] pagesCount(CacheGroupContext grp) throws IgniteCheckedException {
+        // The last element of the array is used to store the status of the index partition.
+        long[] partStates = new long[grp.affinity().partitions() + 1];
+
+        ctx.cache().context().database().checkpointReadLock();
+
+        try {
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) throws IgniteCheckedException {
+                    int pagesCnt = ctx.cache().context().pageStore().pages(grp.groupId(), partId);
+
+                    partStates[Math.min(partId, partStates.length - 1)] = pagesCnt;
+                }
+            });
+        } finally {
+            ctx.cache().context().database().checkpointReadUnlock();
+        }
+
+        return partStates;
+    }
+
+    /**
+     * @param grp Cache group.
+     * @param hnd Partition handler.
+     */
+    private void forEachPageStore(CacheGroupContext grp, IgniteInClosureX<Integer> hnd) throws IgniteCheckedException {
+        int parts = grp.affinity().partitions();
+
+        IgnitePageStoreManager pageStoreMgr = ctx.cache().context().pageStore();
+
+        for (int p = 0; p < parts; p++) {
+            if (!pageStoreMgr.exists(grp.groupId(), p))
+                continue;
+
+            hnd.applyx(p);
+        }
+
+        hnd.applyx(PageIdAllocator.INDEX_PARTITION);
+    }
+
+    /**
+     * Cache group partition scanning task.
+     */
+    private class GroupScanTask extends GridFutureAdapter<Void> implements Runnable {
+        /** Cache group ID. */
+        private final CacheGroupContext grp;
+
+        /** Partition IDs. */
+        private final Set<Integer> parts;
+
+        /** Page memory. */
+        private final PageMemoryEx pageMem;
+
+        /**
+         * @param grp Cache group.
+         */
+        public GroupScanTask(CacheGroupContext grp, Set<Integer> parts) {
+            this.grp = grp;
+            this.parts = new GridConcurrentHashSet<>(parts);
+
+            pageMem = (PageMemoryEx)grp.dataRegion().pageMemory();
+        }
+
+        /** {@inheritDoc} */
+        @Override public synchronized boolean cancel() throws IgniteCheckedException {
+            return onCancelled();
+        }
+
+        /**
+         * Stop reencryption of the specified partition.
+         *
+         * @param partId Partition ID.
+         * @return {@code True} if reencryption was cancelled.
+         */
+        public synchronized boolean excludePartition(int partId) {
+            return parts.remove(partId);
+        }
+
+        /**
+         * @return Cache group ID.
+         */
+        public int groupId() {
+            return grp.groupId();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void run() {
+            try {
+                for (int partId : parts) {
+                    long state = ctx.encryption().getEncryptionState(grp.groupId(), partId);
+
+                    if (state == 0)
+                        continue;
+
+                    scanPartition(partId, ReencryptStateUtils.pageIndex(state), ReencryptStateUtils.pageCount(state));
+
+                    if (isDone())
+                        return;
+                }
+
+                boolean added = cpWaitGrps.add(this);

Review comment:
       Now, we can get information about a written group after than that happens.
   If it is ok, that is right. I still badly oriented in this patch.




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupReencryption.java
##########
@@ -0,0 +1,583 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+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.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+import org.apache.ignite.thread.OomExceptionHandler;
+
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_REENCRYPTION_BATCH_SIZE;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_REENCRYPTION_DISABLED;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_REENCRYPTION_THREAD_POOL_SIZE;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_REENCRYPTION_THROTTLE;
+
+/**
+ * Cache group reencryption manager.
+ */
+public class CacheGroupReencryption implements DbCheckpointListener {
+    /** Thread prefix for reencryption tasks. */
+    private static final String REENCRYPT_THREAD_PREFIX = "reencrypt";
+
+    /** Max amount of pages that will be read into memory under checkpoint lock. */
+    private final int batchSize = IgniteSystemProperties.getInteger(IGNITE_REENCRYPTION_BATCH_SIZE, 1_000);
+
+    /** Timeout between batches. */
+    private final long timeoutBetweenBatches = IgniteSystemProperties.getLong(IGNITE_REENCRYPTION_THROTTLE, 0);
+
+    /** Disable background reencryption. */
+    private final boolean disabled = IgniteSystemProperties.getBoolean(IGNITE_REENCRYPTION_DISABLED, false);
+
+    /** Number of threads for partition scanning. */
+    private final int threadsCnt = IgniteSystemProperties.getInteger(IGNITE_REENCRYPTION_THREAD_POOL_SIZE,
+        Runtime.getRuntime().availableProcessors());
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to reencryption context. */
+    private final Map<Integer, GroupReencryptionContext> grps = new ConcurrentHashMap<>();
+
+    /** Queue of groups waiting for a checkpoint. */
+    private final Queue<Integer> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Executor to start partition scan tasks. */
+    private final IgniteThreadPoolExecutor execSvc;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupReencryption(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        execSvc = new IgniteThreadPoolExecutor(REENCRYPT_THREAD_PREFIX,
+            ctx.igniteInstanceName(),
+            threadsCnt,
+            threadsCnt,
+            IgniteConfiguration.DFLT_THREAD_KEEP_ALIVE_TIME,
+            new LinkedBlockingQueue<>(),
+            GridIoPolicy.SYSTEM_POOL,
+            new OomExceptionHandler(ctx));
+
+        execSvc.allowCoreThreadTimeOut(true);
+    }
+
+    /**
+     * Shutdown reencryption and disable new tasks scheduling.
+     */
+    public void stop() throws IgniteCheckedException {
+        lock.lock();
+
+        try {
+            stopped = true;
+
+            for (GroupReencryptionContext ctx0 : grps.values())
+                ctx0.finishFuture().cancel();
+
+            execSvc.shutdown();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCheckpointBegin(Context cpCtx) {
+        Set<Integer> completeCandidates = new HashSet<>();
+
+        Integer grpId;
+
+        while ((grpId = cpWaitGrps.poll()) != null)
+            completeCandidates.add(grpId);
+
+        cpCtx.finishedStateFut().listen(
+            f -> {
+                if (f.error() != null || f.isCancelled()) {
+                    cpWaitGrps.addAll(completeCandidates);
+
+                    return;
+                }
+
+                lock.lock();
+
+                try {
+                    for (int groupId : completeCandidates) {
+                        GroupReencryptionContext encrCtx = grps.remove(groupId);
+
+                        boolean finished = encrCtx.finish();
+
+                        assert finished : groupId;
+
+                        if (log.isInfoEnabled())
+                            log.info("Cache group reencryption is finished [grpId=" + groupId + "]");
+                    }
+
+                    if (!grps.isEmpty())
+                        return;
+
+                    ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).
+                        removeCheckpointListener(this);
+                }
+                finally {
+                    lock.unlock();
+                }
+            }
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public void beforeCheckpointBegin(Context cpCtx) {
+        for (GroupReencryptionContext encrCtx : grps.values()) {
+            if (encrCtx.skipDirty())
+                return;
+
+            cpCtx.finishedStateFut().listen(f -> {
+                if (f.error() == null && !f.isCancelled())
+                    encrCtx.skipDirty(true);
+            });
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onMarkCheckpointBegin(Context ctx) {
+        // No-op.
+    }
+
+    /**
+     * @return {@code True} If reencryption is disabled.
+     */
+    public boolean disabled() {
+        return disabled;
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @param skipDirty Dirty page skip flag.
+     */
+    public IgniteInternalFuture schedule(int grpId, boolean skipDirty) throws IgniteCheckedException {
+        if (disabled)
+            throw new IgniteCheckedException("Reencryption is disabled.");
+
+        CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+        if (grp == null) {
+            if (log.isDebugEnabled())
+                log.debug("Skip reencryption, group was destroyed [grp=" + grpId + "]");
+
+            return new GridFinishedFuture();
+        }
+
+        lock.lock();
+
+        try {
+            if (stopped)
+                throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
+
+            if (grps.isEmpty())
+                ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).addCheckpointListener(this);
+
+            GroupReencryptionContext prevState = grps.get(grpId);
+
+            if (prevState != null) {
+                if (log.isDebugEnabled())
+                    log.debug("Reencryption already scheduled [grpId=" + grpId + "]");
+
+                return prevState.finishFuture();
+            }
+
+            GroupReencryptionContext ctx0 = new GroupReencryptionContext(grpId, skipDirty);
+
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) {
+                    if (ctx.encryption().getEncryptionState(grpId, partId) == 0) {
+                        if (log.isDebugEnabled())
+                            log.debug("Skipping partition reencryption [grp=" + grpId + ", p=" + partId + "]");
+
+                        return;
+                    }
+
+                    PageStoreScanTask scan = new PageStoreScanTask(ctx0, partId);
+
+                    ctx0.add(partId, scan);
+
+                    execSvc.submit(scan);
+                }
+            });
+
+            ctx0.initialize().listen(f -> {
+                Throwable t = f.error();
+
+                if (t != null) {
+                    log.error("Reencryption is failed [grpId=" + grpId + "]", t);
+
+                    ctx0.fail(t);
+
+                    return;
+                }
+
+                boolean added = cpWaitGrps.offer(grpId);
+
+                assert added;
+            });
+
+            if (log.isInfoEnabled())
+                log.info("Scheduled reencryption [grpId=" + grpId + "]");
+
+            grps.put(grpId, ctx0);
+
+            return ctx0.finishFuture();
+        }
+        finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @return Future that will be completed when the reencryption of the specified group ends.
+     */
+    public IgniteInternalFuture<Void> statusFuture(int grpId) {
+        GroupReencryptionContext ctx0 = grps.get(grpId);
+
+        return ctx0 == null ? new GridFinishedFuture<>() : ctx0.finishFuture();
+    }
+
+    /**
+     * Stop reencryption of the specified partition.
+     *
+     * @param grpId Cache group ID.
+     * @param partId Partition ID.
+     * @return {@code True} if reencryption was cancelled.
+     * @throws IgniteCheckedException If failed.
+     */
+    public boolean cancel(int grpId, int partId) throws IgniteCheckedException {
+        GroupReencryptionContext ctx = grps.get(grpId);
+
+        if (ctx == null)
+            return false;
+
+        return ctx.cancel(partId);
+    }
+
+    /**
+     * Save current pages count for reencryption.
+     *
+     * @param grp Cache group.
+     * @return Map of partitions with current page count.
+     * @throws IgniteCheckedException If failed.
+     */
+    public Map<Integer, Long> storePagesCount(CacheGroupContext grp) throws IgniteCheckedException {
+        Map<Integer, Long> offsets = new HashMap<>();
+
+        ctx.cache().context().database().checkpointReadLock();
+
+        try {
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) throws IgniteCheckedException {
+                    int pagesCnt = ctx.cache().context().pageStore().pages(grp.groupId(), partId);
+
+                    offsets.put(partId, (long)pagesCnt);
+                }
+            });
+        } finally {
+            ctx.cache().context().database().checkpointReadUnlock();
+        }
+
+        return offsets;
+    }
+
+    /**
+     * @param grp Cache group context.
+     * @param hnd Page store handler.
+     */
+    private void forEachPageStore(CacheGroupContext grp, IgniteInClosureX<Integer> hnd) throws IgniteCheckedException {
+        int parts = grp.affinity().partitions();
+
+        IgnitePageStoreManager pageStoreMgr = ctx.cache().context().pageStore();
+
+        for (int p = 0; p < parts; p++) {
+            if (!pageStoreMgr.exists(grp.groupId(), p))
+                continue;
+
+            hnd.applyx(p);
+        }
+
+        hnd.applyx(PageIdAllocator.INDEX_PARTITION);
+    }
+
+    /**
+     * Cache group reencryption context.
+     */
+    private static class GroupReencryptionContext {
+        /** Partition scanning futures. */
+        private final Map<Integer, IgniteInternalFuture<Void>> futMap = new ConcurrentHashMap<>();
+
+        /** Compound future, that will be completed when all partitions scanned. */
+        private final GridCompoundFuture<Void, Void> compFut = new GridCompoundFuture<>();
+
+        /** Cache group ID. */
+        private final int grpId;
+
+        /** Dirty page skip flag. */
+        private volatile boolean skipDirty;
+
+        /** Future that ends after all partitions are done and a checkpoint is finished. */
+        private final GridFutureAdapter<Void> cpFut = new GridFutureAdapter<Void>() {
+            @Override public boolean cancel() throws IgniteCheckedException {
+                compFut.cancel();
+
+                return onDone(null, null, true);
+            }
+        };
+
+        /**
+         * @param grpId Cache group ID.
+         * @param skipDirty Dirty page skip flag.
+         */
+        public GroupReencryptionContext(int grpId, boolean skipDirty) {
+            this.grpId = grpId;
+            this.skipDirty = skipDirty;
+        }
+
+        /**
+         * @return Cache group ID.
+         */
+        public int groupId() {
+            return grpId;
+        }
+
+        /**
+         * @return Dirty page skip flag.
+         */
+        public boolean skipDirty() {
+            return skipDirty;
+        }
+
+        /**
+         * @param skipDirty Dirty page skip flag.
+         */
+        public void skipDirty(boolean skipDirty) {
+            this.skipDirty = skipDirty;
+        }
+
+        /**
+         * @param partId Partition ID.
+         * @param fut Partition scanning future.
+         */
+        public void add(int partId, IgniteInternalFuture<Void> fut) {
+            compFut.add(fut);
+
+            futMap.put(partId, fut);
+        }
+
+        /**
+         * @return Compound future, that will be completed when all partitions scanned.
+         */
+        public IgniteInternalFuture<Void> initialize() {
+            return compFut.markInitialized();
+        }
+
+        /**
+         * @return Future that ends after all partitions are done and a checkpoint is finished.
+         */
+        public IgniteInternalFuture<Void> finishFuture() {
+            return cpFut;
+        }
+
+        /**
+         * Finish reencryption future.
+         *
+         * @return {@code True} if the future was finished by this call.
+         */
+        public boolean finish() {
+            return cpFut.onDone(compFut.result());
+        }
+
+        /**
+         * Stop reencryption of the specified partition.
+         *
+         * @param partId Partition ID.
+         * @return {@code True} if reencryption was cancelled.
+         * @throws IgniteCheckedException If failed.
+         */
+        public boolean cancel(int partId) throws IgniteCheckedException {
+            IgniteInternalFuture<Void> fut = futMap.get(partId);
+
+            if (fut == null)
+                return false;
+
+            return fut.cancel();
+        }
+
+        /**
+         * @param t Throwable.
+         */
+        public void fail(Throwable t) {
+            cpFut.onDone(t);
+        }
+    }
+
+    /**
+     * Page store scanning task.
+     */
+    private class PageStoreScanTask extends GridFutureAdapter<Void> implements Runnable {
+        /** Partiion ID. */
+        private final int partId;
+
+        /** Cache group reencryption context. */
+        private final GroupReencryptionContext encrCtx;
+
+        /**
+         * @param encrCtx Cache group reencryption context.
+         * @param partId Partition ID.
+         */
+        public PageStoreScanTask(GroupReencryptionContext encrCtx, int partId) {
+            this.encrCtx = encrCtx;
+            this.partId = partId;
+        }
+
+        /** {@inheritDoc} */
+        @Override public synchronized boolean cancel() throws IgniteCheckedException {
+            return onDone();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void run() {
+            try {
+                int grpId = encrCtx.groupId();
+
+                CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+                if (grp == null) {
+                    onDone();
+
+                    return;
+                }
+
+                PageMemoryEx pageMem = (PageMemoryEx)grp.dataRegion().pageMemory();
+                long metaPageId = pageMem.partitionMetaPageId(grpId, partId);
+                long state = ctx.encryption().getEncryptionState(grpId, partId);
+
+                int pageIdx = (int)(state >> 32);
+                int cnt = (int)state;
+
+                if (log.isDebugEnabled()) {
+                    log.debug("Partition reencryption is started [grpId=" + grpId +
+                        ", p=" + partId + ", remain=" + (cnt - pageIdx) + ", total=" + cnt + "]");
+                }
+
+                while (pageIdx < cnt) {
+                    synchronized (this) {
+                        ctx.cache().context().database().checkpointReadLock();
+
+                        try {
+                            if (isDone() || ctx.encryption().getEncryptionState(grpId, partId) == 0)
+                                break;
+
+                            int end = Math.min(pageIdx + batchSize, cnt);
+
+                            do {
+                                long pageId = metaPageId + pageIdx;
+
+                                pageIdx += 1;
+
+                                long page = pageMem.acquirePage(grpId, pageId);
+
+                                try {
+                                    // Can skip rewriting a dirty page if the checkpoint has been completed.
+                                    if (encrCtx.skipDirty() && pageMem.isDirty(grpId, pageId, page))
+                                        continue;
+
+                                    pageMem.writeLock(grpId, pageId, page, true);

Review comment:
       Could you describe the question in more detail?




----------------------------------------------------------------
This is an automated message from the 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] alex-plekhanov commented on a change in pull request #7941: IGNITE-12843

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #7941:
URL: https://github.com/apache/ignite/pull/7941#discussion_r465676384



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupEncryptionKeys.java
##########
@@ -0,0 +1,387 @@
+/*
+ * 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.managers.encryption;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import org.apache.ignite.internal.pagemem.wal.record.MasterKeyChangeRecord;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T3;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.encryption.EncryptionSpi;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Serves for managing encryption keys and related datastructure located in the heap.
+ */
+class CacheGroupEncryptionKeys {
+    /** Group encryption keys. */
+    private final Map<Integer, List<GroupKey>> grpKeys = new ConcurrentHashMap<>();
+
+    /** WAL segments encrypted with previous encrypted keys, mapped to cache group encryption key identifiers. */
+    private final Map<Long, Map<Integer, Set<Integer>>> trackedWalSegments = new ConcurrentSkipListMap<>();
+
+    /** Encryption spi. */
+    private final EncryptionSpi encSpi;
+
+    /**
+     * @param encSpi Encryption spi.
+     */
+    CacheGroupEncryptionKeys(EncryptionSpi encSpi) {
+        this.encSpi = encSpi;
+    }
+
+    /**
+     * Returns group encryption key.
+     *
+     * @param grpId Cache group ID.
+     * @return Group encryption key with identifier, that was set for writing.
+     */
+    GroupKey get(int grpId) {
+        List<GroupKey> keys = grpKeys.get(grpId);
+
+        if (F.isEmpty(keys))
+            return null;
+
+        return keys.get(0);
+    }
+
+    /**
+     * Returns group encryption key with specified identifier.
+     *
+     * @param grpId Cache group ID.
+     * @param keyId Encryption key ID.
+     * @return Group encryption key.
+     */
+    GroupKey get(int grpId, int keyId) {
+        List<GroupKey> keys = grpKeys.get(grpId);
+
+        if (keys == null)
+            return null;
+
+        for (GroupKey groupKey : keys) {
+            if (groupKey.unsignedId() == keyId)
+                return groupKey;
+        }
+
+        return null;
+    }
+
+    /**
+     * Gets information about existing encryption keys for the specified cache group.
+     *
+     * @param grpId Cache group ID.
+     * @return Map of the key identifier with hash code of encryption key.
+     */
+    Map<Integer, Integer> info(int grpId) {
+        List<GroupKey> keys = grpKeys.get(grpId);
+
+        if (keys == null)
+            return null;
+
+        Map<Integer, Integer> keysInfo = new LinkedHashMap<>();
+
+        for (GroupKey groupKey : keys)
+            keysInfo.put(groupKey.unsignedId(), Arrays.hashCode(U.toBytes(groupKey.key())));
+
+        return keysInfo;
+    }
+
+    /**
+     * @return Local encryption keys.
+     */
+    @Nullable HashMap<Integer, GroupKeyEncrypted> getAll() {
+        if (F.isEmpty(grpKeys))
+            return null;
+
+        HashMap<Integer, GroupKeyEncrypted> Keys = new HashMap<>();

Review comment:
       `Keys` -> `keys`

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupEncryptionKeys.java
##########
@@ -0,0 +1,387 @@
+/*
+ * 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.managers.encryption;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import org.apache.ignite.internal.pagemem.wal.record.MasterKeyChangeRecord;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T3;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.encryption.EncryptionSpi;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Serves for managing encryption keys and related datastructure located in the heap.
+ */
+class CacheGroupEncryptionKeys {
+    /** Group encryption keys. */
+    private final Map<Integer, List<GroupKey>> grpKeys = new ConcurrentHashMap<>();
+
+    /** WAL segments encrypted with previous encrypted keys, mapped to cache group encryption key identifiers. */
+    private final Map<Long, Map<Integer, Set<Integer>>> trackedWalSegments = new ConcurrentSkipListMap<>();
+
+    /** Encryption spi. */
+    private final EncryptionSpi encSpi;
+
+    /**
+     * @param encSpi Encryption spi.
+     */
+    CacheGroupEncryptionKeys(EncryptionSpi encSpi) {
+        this.encSpi = encSpi;
+    }
+
+    /**
+     * Returns group encryption key.
+     *
+     * @param grpId Cache group ID.
+     * @return Group encryption key with identifier, that was set for writing.
+     */
+    GroupKey get(int grpId) {
+        List<GroupKey> keys = grpKeys.get(grpId);
+
+        if (F.isEmpty(keys))
+            return null;
+
+        return keys.get(0);
+    }
+
+    /**
+     * Returns group encryption key with specified identifier.
+     *
+     * @param grpId Cache group ID.
+     * @param keyId Encryption key ID.
+     * @return Group encryption key.
+     */
+    GroupKey get(int grpId, int keyId) {
+        List<GroupKey> keys = grpKeys.get(grpId);
+
+        if (keys == null)
+            return null;
+
+        for (GroupKey groupKey : keys) {
+            if (groupKey.unsignedId() == keyId)
+                return groupKey;
+        }
+
+        return null;
+    }
+
+    /**
+     * Gets information about existing encryption keys for the specified cache group.
+     *
+     * @param grpId Cache group ID.
+     * @return Map of the key identifier with hash code of encryption key.
+     */
+    Map<Integer, Integer> info(int grpId) {
+        List<GroupKey> keys = grpKeys.get(grpId);
+
+        if (keys == null)
+            return null;
+
+        Map<Integer, Integer> keysInfo = new LinkedHashMap<>();
+
+        for (GroupKey groupKey : keys)
+            keysInfo.put(groupKey.unsignedId(), Arrays.hashCode(U.toBytes(groupKey.key())));
+
+        return keysInfo;
+    }
+
+    /**
+     * @return Local encryption keys.
+     */
+    @Nullable HashMap<Integer, GroupKeyEncrypted> getAll() {
+        if (F.isEmpty(grpKeys))
+            return null;
+
+        HashMap<Integer, GroupKeyEncrypted> Keys = new HashMap<>();
+
+        for (Map.Entry<Integer, List<GroupKey>> entry : grpKeys.entrySet()) {
+            int grpId = entry.getKey();
+            GroupKey grpKey = entry.getValue().get(0);
+
+            Keys.put(grpId, new GroupKeyEncrypted(grpKey.unsignedId(), encSpi.encryptKey(grpKey.key())));
+        }
+
+        return Keys;
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     *
+     * @return Local encryption keys used for specified cache group.
+     */
+    List<GroupKeyEncrypted> getAll(int grpId) {
+        List<GroupKey> grpKeys = this.grpKeys.get(grpId);
+
+        if (F.isEmpty(grpKeys))
+            return null;
+
+        List<GroupKeyEncrypted> encryptedKeys = new ArrayList<>(grpKeys.size());
+
+        for (GroupKey grpKey : grpKeys)
+            encryptedKeys.add(new GroupKeyEncrypted(grpKey.unsignedId(), encSpi.encryptKey(grpKey.key())));
+
+        return encryptedKeys;
+    }
+
+    /**
+     * Put new encryption key and set it for writing.
+     *
+     * @param grpId Cache group ID.
+     * @param newEncKey New encrypted key for writing.
+     * @return Previous encryption key for writing.
+     */
+    GroupKey put(int grpId, GroupKeyEncrypted newEncKey) {
+        assert newEncKey != null;
+
+        List<GroupKey> keys = grpKeys.computeIfAbsent(grpId, list -> new CopyOnWriteArrayList<>());
+
+        if (keys == null)

Review comment:
       Is it possible?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupEncryptionKeys.java
##########
@@ -0,0 +1,387 @@
+/*
+ * 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.managers.encryption;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import org.apache.ignite.internal.pagemem.wal.record.MasterKeyChangeRecord;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T3;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.encryption.EncryptionSpi;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Serves for managing encryption keys and related datastructure located in the heap.
+ */
+class CacheGroupEncryptionKeys {
+    /** Group encryption keys. */
+    private final Map<Integer, List<GroupKey>> grpKeys = new ConcurrentHashMap<>();
+
+    /** WAL segments encrypted with previous encrypted keys, mapped to cache group encryption key identifiers. */
+    private final Map<Long, Map<Integer, Set<Integer>>> trackedWalSegments = new ConcurrentSkipListMap<>();

Review comment:
       You don't need Map of Maps here. You can use plain structure with flat (walIdx, grpId, keyId) tuple (ConcurrentLinkedQueue for example). This structure will be naturally ordered, since you call reserveWalKey under the lock and ctx.cache().context().wal().currentSegment() always increasing.
   This will simplify a lot all methods where trackedWalSegments is used.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupEncryptionKeys.java
##########
@@ -0,0 +1,387 @@
+/*
+ * 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.managers.encryption;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import org.apache.ignite.internal.pagemem.wal.record.MasterKeyChangeRecord;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T3;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.encryption.EncryptionSpi;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Serves for managing encryption keys and related datastructure located in the heap.
+ */
+class CacheGroupEncryptionKeys {
+    /** Group encryption keys. */
+    private final Map<Integer, List<GroupKey>> grpKeys = new ConcurrentHashMap<>();
+
+    /** WAL segments encrypted with previous encrypted keys, mapped to cache group encryption key identifiers. */
+    private final Map<Long, Map<Integer, Set<Integer>>> trackedWalSegments = new ConcurrentSkipListMap<>();
+
+    /** Encryption spi. */
+    private final EncryptionSpi encSpi;
+
+    /**
+     * @param encSpi Encryption spi.
+     */
+    CacheGroupEncryptionKeys(EncryptionSpi encSpi) {
+        this.encSpi = encSpi;
+    }
+
+    /**
+     * Returns group encryption key.
+     *
+     * @param grpId Cache group ID.
+     * @return Group encryption key with identifier, that was set for writing.
+     */
+    GroupKey get(int grpId) {
+        List<GroupKey> keys = grpKeys.get(grpId);
+
+        if (F.isEmpty(keys))
+            return null;
+
+        return keys.get(0);
+    }
+
+    /**
+     * Returns group encryption key with specified identifier.
+     *
+     * @param grpId Cache group ID.
+     * @param keyId Encryption key ID.
+     * @return Group encryption key.
+     */
+    GroupKey get(int grpId, int keyId) {
+        List<GroupKey> keys = grpKeys.get(grpId);
+
+        if (keys == null)
+            return null;
+
+        for (GroupKey groupKey : keys) {
+            if (groupKey.unsignedId() == keyId)
+                return groupKey;
+        }
+
+        return null;
+    }
+
+    /**
+     * Gets information about existing encryption keys for the specified cache group.
+     *
+     * @param grpId Cache group ID.
+     * @return Map of the key identifier with hash code of encryption key.
+     */
+    Map<Integer, Integer> info(int grpId) {
+        List<GroupKey> keys = grpKeys.get(grpId);
+
+        if (keys == null)
+            return null;
+
+        Map<Integer, Integer> keysInfo = new LinkedHashMap<>();
+
+        for (GroupKey groupKey : keys)
+            keysInfo.put(groupKey.unsignedId(), Arrays.hashCode(U.toBytes(groupKey.key())));
+
+        return keysInfo;
+    }
+
+    /**
+     * @return Local encryption keys.
+     */
+    @Nullable HashMap<Integer, GroupKeyEncrypted> getAll() {
+        if (F.isEmpty(grpKeys))
+            return null;
+
+        HashMap<Integer, GroupKeyEncrypted> Keys = new HashMap<>();
+
+        for (Map.Entry<Integer, List<GroupKey>> entry : grpKeys.entrySet()) {
+            int grpId = entry.getKey();
+            GroupKey grpKey = entry.getValue().get(0);
+
+            Keys.put(grpId, new GroupKeyEncrypted(grpKey.unsignedId(), encSpi.encryptKey(grpKey.key())));
+        }
+
+        return Keys;
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     *
+     * @return Local encryption keys used for specified cache group.
+     */
+    List<GroupKeyEncrypted> getAll(int grpId) {
+        List<GroupKey> grpKeys = this.grpKeys.get(grpId);
+
+        if (F.isEmpty(grpKeys))
+            return null;
+
+        List<GroupKeyEncrypted> encryptedKeys = new ArrayList<>(grpKeys.size());
+
+        for (GroupKey grpKey : grpKeys)
+            encryptedKeys.add(new GroupKeyEncrypted(grpKey.unsignedId(), encSpi.encryptKey(grpKey.key())));
+
+        return encryptedKeys;
+    }
+
+    /**
+     * Put new encryption key and set it for writing.
+     *
+     * @param grpId Cache group ID.
+     * @param newEncKey New encrypted key for writing.
+     * @return Previous encryption key for writing.
+     */
+    GroupKey put(int grpId, GroupKeyEncrypted newEncKey) {
+        assert newEncKey != null;
+
+        List<GroupKey> keys = grpKeys.computeIfAbsent(grpId, list -> new CopyOnWriteArrayList<>());
+
+        if (keys == null)
+            return null;
+
+        GroupKey prevKey = F.first(keys);
+
+        GroupKey newKey = new GroupKey(newEncKey.id(), encSpi.decryptKey(newEncKey.key()));
+
+        keys.add(0, newKey);
+
+        // Remove the duplicate key from the tail of the list if exists.
+        keys.subList(1, keys.size()).remove(newKey);
+
+        return prevKey;
+    }
+
+    /**
+     * Put new unused key.
+     *
+     * @param grpId Cache group ID.
+     * @param newEncKey New encrypted key for writing.
+     */
+    void putUnused(int grpId, GroupKeyEncrypted newEncKey) {
+        grpKeys.get(grpId).add(new GroupKey(newEncKey.id(), encSpi.decryptKey(newEncKey.key())));
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @param encryptedKeys Encrypted keys.
+     */
+    void put(int grpId, List<GroupKeyEncrypted> encryptedKeys) {
+        List<GroupKey> keys = new CopyOnWriteArrayList<>();
+
+        for (GroupKeyEncrypted encrKey : encryptedKeys)
+            keys.add(new GroupKey(encrKey.id(), encSpi.decryptKey(encrKey.key())));
+
+        grpKeys.put(grpId, keys);
+    }
+
+    /**
+     * @return Cache group identifiers for which encryption keys are stored.
+     */
+    Set<Integer> groups() {
+        return grpKeys.keySet();
+    }
+
+    /**
+     * Remove encrytion keys associated with the specified cache group.
+     *
+     * @param grpId Cache group ID.
+     */
+    void remove(int grpId) {
+        grpKeys.remove(grpId);
+    }
+
+    /**
+     * Convert encryption keys to WAL logical record that stores encryption keys.
+     */
+    MasterKeyChangeRecord toMasterKeyChangeRecord() {

Review comment:
       I think you should not work with WAL records in this class. Just return reencryptedKeys and wrap into MasterKeyChangeRecord in encryption manager.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupEncryptionKeys.java
##########
@@ -0,0 +1,387 @@
+/*
+ * 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.managers.encryption;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import org.apache.ignite.internal.pagemem.wal.record.MasterKeyChangeRecord;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T3;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.encryption.EncryptionSpi;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Serves for managing encryption keys and related datastructure located in the heap.
+ */
+class CacheGroupEncryptionKeys {
+    /** Group encryption keys. */
+    private final Map<Integer, List<GroupKey>> grpKeys = new ConcurrentHashMap<>();
+
+    /** WAL segments encrypted with previous encrypted keys, mapped to cache group encryption key identifiers. */
+    private final Map<Long, Map<Integer, Set<Integer>>> trackedWalSegments = new ConcurrentSkipListMap<>();
+
+    /** Encryption spi. */
+    private final EncryptionSpi encSpi;
+
+    /**
+     * @param encSpi Encryption spi.
+     */
+    CacheGroupEncryptionKeys(EncryptionSpi encSpi) {
+        this.encSpi = encSpi;
+    }
+
+    /**
+     * Returns group encryption key.
+     *
+     * @param grpId Cache group ID.
+     * @return Group encryption key with identifier, that was set for writing.
+     */
+    GroupKey get(int grpId) {

Review comment:
       Perhaps should be renamed `getDefault` or `getCurrent` or `getActive`

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupEncryptionKeys.java
##########
@@ -0,0 +1,387 @@
+/*
+ * 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.managers.encryption;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import org.apache.ignite.internal.pagemem.wal.record.MasterKeyChangeRecord;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T3;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.encryption.EncryptionSpi;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Serves for managing encryption keys and related datastructure located in the heap.
+ */
+class CacheGroupEncryptionKeys {
+    /** Group encryption keys. */
+    private final Map<Integer, List<GroupKey>> grpKeys = new ConcurrentHashMap<>();
+
+    /** WAL segments encrypted with previous encrypted keys, mapped to cache group encryption key identifiers. */
+    private final Map<Long, Map<Integer, Set<Integer>>> trackedWalSegments = new ConcurrentSkipListMap<>();
+
+    /** Encryption spi. */
+    private final EncryptionSpi encSpi;
+
+    /**
+     * @param encSpi Encryption spi.
+     */
+    CacheGroupEncryptionKeys(EncryptionSpi encSpi) {
+        this.encSpi = encSpi;
+    }
+
+    /**
+     * Returns group encryption key.
+     *
+     * @param grpId Cache group ID.
+     * @return Group encryption key with identifier, that was set for writing.
+     */
+    GroupKey get(int grpId) {
+        List<GroupKey> keys = grpKeys.get(grpId);
+
+        if (F.isEmpty(keys))
+            return null;
+
+        return keys.get(0);
+    }
+
+    /**
+     * Returns group encryption key with specified identifier.
+     *
+     * @param grpId Cache group ID.
+     * @param keyId Encryption key ID.
+     * @return Group encryption key.
+     */
+    GroupKey get(int grpId, int keyId) {
+        List<GroupKey> keys = grpKeys.get(grpId);
+
+        if (keys == null)
+            return null;
+
+        for (GroupKey groupKey : keys) {
+            if (groupKey.unsignedId() == keyId)
+                return groupKey;
+        }
+
+        return null;
+    }
+
+    /**
+     * Gets information about existing encryption keys for the specified cache group.
+     *
+     * @param grpId Cache group ID.
+     * @return Map of the key identifier with hash code of encryption key.
+     */
+    Map<Integer, Integer> info(int grpId) {
+        List<GroupKey> keys = grpKeys.get(grpId);
+
+        if (keys == null)
+            return null;
+
+        Map<Integer, Integer> keysInfo = new LinkedHashMap<>();
+
+        for (GroupKey groupKey : keys)
+            keysInfo.put(groupKey.unsignedId(), Arrays.hashCode(U.toBytes(groupKey.key())));
+
+        return keysInfo;
+    }
+
+    /**
+     * @return Local encryption keys.
+     */
+    @Nullable HashMap<Integer, GroupKeyEncrypted> getAll() {
+        if (F.isEmpty(grpKeys))
+            return null;
+
+        HashMap<Integer, GroupKeyEncrypted> Keys = new HashMap<>();
+
+        for (Map.Entry<Integer, List<GroupKey>> entry : grpKeys.entrySet()) {
+            int grpId = entry.getKey();
+            GroupKey grpKey = entry.getValue().get(0);
+
+            Keys.put(grpId, new GroupKeyEncrypted(grpKey.unsignedId(), encSpi.encryptKey(grpKey.key())));
+        }
+
+        return Keys;
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     *
+     * @return Local encryption keys used for specified cache group.
+     */
+    List<GroupKeyEncrypted> getAll(int grpId) {
+        List<GroupKey> grpKeys = this.grpKeys.get(grpId);
+
+        if (F.isEmpty(grpKeys))
+            return null;
+
+        List<GroupKeyEncrypted> encryptedKeys = new ArrayList<>(grpKeys.size());
+
+        for (GroupKey grpKey : grpKeys)
+            encryptedKeys.add(new GroupKeyEncrypted(grpKey.unsignedId(), encSpi.encryptKey(grpKey.key())));
+
+        return encryptedKeys;
+    }
+
+    /**
+     * Put new encryption key and set it for writing.
+     *
+     * @param grpId Cache group ID.
+     * @param newEncKey New encrypted key for writing.
+     * @return Previous encryption key for writing.
+     */
+    GroupKey put(int grpId, GroupKeyEncrypted newEncKey) {

Review comment:
       Perhaps method name should be changed to something like `changeDefaultKey` or `setDefaultKey` or `setCurrentKey` or `setActiveKey`, etc

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupEncryptionKeys.java
##########
@@ -0,0 +1,387 @@
+/*
+ * 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.managers.encryption;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import org.apache.ignite.internal.pagemem.wal.record.MasterKeyChangeRecord;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T3;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.encryption.EncryptionSpi;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Serves for managing encryption keys and related datastructure located in the heap.
+ */
+class CacheGroupEncryptionKeys {
+    /** Group encryption keys. */
+    private final Map<Integer, List<GroupKey>> grpKeys = new ConcurrentHashMap<>();
+
+    /** WAL segments encrypted with previous encrypted keys, mapped to cache group encryption key identifiers. */
+    private final Map<Long, Map<Integer, Set<Integer>>> trackedWalSegments = new ConcurrentSkipListMap<>();
+
+    /** Encryption spi. */
+    private final EncryptionSpi encSpi;
+
+    /**
+     * @param encSpi Encryption spi.
+     */
+    CacheGroupEncryptionKeys(EncryptionSpi encSpi) {
+        this.encSpi = encSpi;
+    }
+
+    /**
+     * Returns group encryption key.
+     *
+     * @param grpId Cache group ID.
+     * @return Group encryption key with identifier, that was set for writing.
+     */
+    GroupKey get(int grpId) {
+        List<GroupKey> keys = grpKeys.get(grpId);
+
+        if (F.isEmpty(keys))
+            return null;
+
+        return keys.get(0);
+    }
+
+    /**
+     * Returns group encryption key with specified identifier.
+     *
+     * @param grpId Cache group ID.
+     * @param keyId Encryption key ID.
+     * @return Group encryption key.
+     */
+    GroupKey get(int grpId, int keyId) {
+        List<GroupKey> keys = grpKeys.get(grpId);
+
+        if (keys == null)
+            return null;
+
+        for (GroupKey groupKey : keys) {
+            if (groupKey.unsignedId() == keyId)
+                return groupKey;
+        }
+
+        return null;
+    }
+
+    /**
+     * Gets information about existing encryption keys for the specified cache group.
+     *
+     * @param grpId Cache group ID.
+     * @return Map of the key identifier with hash code of encryption key.
+     */
+    Map<Integer, Integer> info(int grpId) {
+        List<GroupKey> keys = grpKeys.get(grpId);
+
+        if (keys == null)
+            return null;
+
+        Map<Integer, Integer> keysInfo = new LinkedHashMap<>();
+
+        for (GroupKey groupKey : keys)
+            keysInfo.put(groupKey.unsignedId(), Arrays.hashCode(U.toBytes(groupKey.key())));
+
+        return keysInfo;
+    }
+
+    /**
+     * @return Local encryption keys.
+     */
+    @Nullable HashMap<Integer, GroupKeyEncrypted> getAll() {
+        if (F.isEmpty(grpKeys))
+            return null;
+
+        HashMap<Integer, GroupKeyEncrypted> Keys = new HashMap<>();
+
+        for (Map.Entry<Integer, List<GroupKey>> entry : grpKeys.entrySet()) {
+            int grpId = entry.getKey();
+            GroupKey grpKey = entry.getValue().get(0);
+
+            Keys.put(grpId, new GroupKeyEncrypted(grpKey.unsignedId(), encSpi.encryptKey(grpKey.key())));
+        }
+
+        return Keys;
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     *
+     * @return Local encryption keys used for specified cache group.
+     */
+    List<GroupKeyEncrypted> getAll(int grpId) {
+        List<GroupKey> grpKeys = this.grpKeys.get(grpId);
+
+        if (F.isEmpty(grpKeys))
+            return null;
+
+        List<GroupKeyEncrypted> encryptedKeys = new ArrayList<>(grpKeys.size());
+
+        for (GroupKey grpKey : grpKeys)
+            encryptedKeys.add(new GroupKeyEncrypted(grpKey.unsignedId(), encSpi.encryptKey(grpKey.key())));
+
+        return encryptedKeys;
+    }
+
+    /**
+     * Put new encryption key and set it for writing.
+     *
+     * @param grpId Cache group ID.
+     * @param newEncKey New encrypted key for writing.
+     * @return Previous encryption key for writing.
+     */
+    GroupKey put(int grpId, GroupKeyEncrypted newEncKey) {
+        assert newEncKey != null;
+
+        List<GroupKey> keys = grpKeys.computeIfAbsent(grpId, list -> new CopyOnWriteArrayList<>());
+
+        if (keys == null)
+            return null;
+
+        GroupKey prevKey = F.first(keys);
+
+        GroupKey newKey = new GroupKey(newEncKey.id(), encSpi.decryptKey(newEncKey.key()));
+
+        keys.add(0, newKey);
+
+        // Remove the duplicate key from the tail of the list if exists.
+        keys.subList(1, keys.size()).remove(newKey);
+
+        return prevKey;
+    }
+
+    /**
+     * Put new unused key.
+     *
+     * @param grpId Cache group ID.
+     * @param newEncKey New encrypted key for writing.
+     */
+    void putUnused(int grpId, GroupKeyEncrypted newEncKey) {
+        grpKeys.get(grpId).add(new GroupKey(newEncKey.id(), encSpi.decryptKey(newEncKey.key())));
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @param encryptedKeys Encrypted keys.
+     */
+    void put(int grpId, List<GroupKeyEncrypted> encryptedKeys) {
+        List<GroupKey> keys = new CopyOnWriteArrayList<>();
+
+        for (GroupKeyEncrypted encrKey : encryptedKeys)
+            keys.add(new GroupKey(encrKey.id(), encSpi.decryptKey(encrKey.key())));
+
+        grpKeys.put(grpId, keys);
+    }
+
+    /**
+     * @return Cache group identifiers for which encryption keys are stored.
+     */
+    Set<Integer> groups() {
+        return grpKeys.keySet();
+    }
+
+    /**
+     * Remove encrytion keys associated with the specified cache group.
+     *
+     * @param grpId Cache group ID.
+     */
+    void remove(int grpId) {
+        grpKeys.remove(grpId);
+    }
+
+    /**
+     * Convert encryption keys to WAL logical record that stores encryption keys.
+     */
+    MasterKeyChangeRecord toMasterKeyChangeRecord() {
+        List<T3<Integer, Byte, byte[]>> reencryptedKeys = new ArrayList<>();
+
+        for (Map.Entry<Integer, List<GroupKey>> entry : grpKeys.entrySet()) {
+            int grpId = entry.getKey();
+
+            for (GroupKey grpKey : entry.getValue()) {
+                byte keyId = grpKey.id();
+                byte[] encryptedKey = encSpi.encryptKey(grpKey.key());
+
+                reencryptedKeys.add(new T3<>(grpId, keyId, encryptedKey));
+            }
+        }
+
+        return new MasterKeyChangeRecord(encSpi.getMasterKeyName(), reencryptedKeys);
+    }
+
+    /**
+     * Load encryption keys from WAL logical record that stores encryption keys.
+     *
+     * @param rec Logical record that stores encryption keys.
+     */
+    void fromMasterKeyChangeRecord(MasterKeyChangeRecord rec) {
+        for (T3<Integer, Byte, byte[]> entry : rec.getGrpKeys()) {
+            int grpId = entry.get1();
+            int keyId = entry.get2() & 0xff;
+            byte[] key = entry.get3();
+
+            grpKeys.computeIfAbsent(grpId, list ->
+                new CopyOnWriteArrayList<>()).add(new GroupKey(keyId, encSpi.decryptKey(key)));
+        }
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @param ids Key identifiers for deletion.
+     * @return {@code True} if the keys have been deleted.
+     */
+    boolean removeKeysById(int grpId, Set<Integer> ids) {
+        return removeKeysById(grpKeys.get(grpId), ids);

Review comment:
       `removeKeysById(List<GroupKey> keys, Set<Integer> ids)` is redundant, can be replaced by `removeIf(key -> ids.contains(key.unsignedId()));`
   Here for example:
   `grpKeys.get(grpId).removeIf(key -> ids.contains(key.unsignedId()));`
   Also, do we need to check `grpKeys.get(grpId)` for `null`?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupEncryptionKeys.java
##########
@@ -0,0 +1,387 @@
+/*
+ * 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.managers.encryption;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import org.apache.ignite.internal.pagemem.wal.record.MasterKeyChangeRecord;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T3;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.encryption.EncryptionSpi;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Serves for managing encryption keys and related datastructure located in the heap.
+ */
+class CacheGroupEncryptionKeys {
+    /** Group encryption keys. */
+    private final Map<Integer, List<GroupKey>> grpKeys = new ConcurrentHashMap<>();
+
+    /** WAL segments encrypted with previous encrypted keys, mapped to cache group encryption key identifiers. */
+    private final Map<Long, Map<Integer, Set<Integer>>> trackedWalSegments = new ConcurrentSkipListMap<>();
+
+    /** Encryption spi. */
+    private final EncryptionSpi encSpi;
+
+    /**
+     * @param encSpi Encryption spi.
+     */
+    CacheGroupEncryptionKeys(EncryptionSpi encSpi) {
+        this.encSpi = encSpi;
+    }
+
+    /**
+     * Returns group encryption key.
+     *
+     * @param grpId Cache group ID.
+     * @return Group encryption key with identifier, that was set for writing.
+     */
+    GroupKey get(int grpId) {
+        List<GroupKey> keys = grpKeys.get(grpId);
+
+        if (F.isEmpty(keys))
+            return null;
+
+        return keys.get(0);
+    }
+
+    /**
+     * Returns group encryption key with specified identifier.
+     *
+     * @param grpId Cache group ID.
+     * @param keyId Encryption key ID.
+     * @return Group encryption key.
+     */
+    GroupKey get(int grpId, int keyId) {
+        List<GroupKey> keys = grpKeys.get(grpId);
+
+        if (keys == null)
+            return null;
+
+        for (GroupKey groupKey : keys) {
+            if (groupKey.unsignedId() == keyId)
+                return groupKey;
+        }
+
+        return null;
+    }
+
+    /**
+     * Gets information about existing encryption keys for the specified cache group.
+     *
+     * @param grpId Cache group ID.
+     * @return Map of the key identifier with hash code of encryption key.
+     */
+    Map<Integer, Integer> info(int grpId) {
+        List<GroupKey> keys = grpKeys.get(grpId);
+
+        if (keys == null)
+            return null;
+
+        Map<Integer, Integer> keysInfo = new LinkedHashMap<>();
+
+        for (GroupKey groupKey : keys)
+            keysInfo.put(groupKey.unsignedId(), Arrays.hashCode(U.toBytes(groupKey.key())));
+
+        return keysInfo;
+    }
+
+    /**
+     * @return Local encryption keys.
+     */
+    @Nullable HashMap<Integer, GroupKeyEncrypted> getAll() {
+        if (F.isEmpty(grpKeys))
+            return null;
+
+        HashMap<Integer, GroupKeyEncrypted> Keys = new HashMap<>();
+
+        for (Map.Entry<Integer, List<GroupKey>> entry : grpKeys.entrySet()) {
+            int grpId = entry.getKey();
+            GroupKey grpKey = entry.getValue().get(0);
+
+            Keys.put(grpId, new GroupKeyEncrypted(grpKey.unsignedId(), encSpi.encryptKey(grpKey.key())));
+        }
+
+        return Keys;
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     *
+     * @return Local encryption keys used for specified cache group.
+     */
+    List<GroupKeyEncrypted> getAll(int grpId) {
+        List<GroupKey> grpKeys = this.grpKeys.get(grpId);
+
+        if (F.isEmpty(grpKeys))
+            return null;
+
+        List<GroupKeyEncrypted> encryptedKeys = new ArrayList<>(grpKeys.size());
+
+        for (GroupKey grpKey : grpKeys)
+            encryptedKeys.add(new GroupKeyEncrypted(grpKey.unsignedId(), encSpi.encryptKey(grpKey.key())));
+
+        return encryptedKeys;
+    }
+
+    /**
+     * Put new encryption key and set it for writing.
+     *
+     * @param grpId Cache group ID.
+     * @param newEncKey New encrypted key for writing.
+     * @return Previous encryption key for writing.
+     */
+    GroupKey put(int grpId, GroupKeyEncrypted newEncKey) {
+        assert newEncKey != null;
+
+        List<GroupKey> keys = grpKeys.computeIfAbsent(grpId, list -> new CopyOnWriteArrayList<>());
+
+        if (keys == null)
+            return null;
+
+        GroupKey prevKey = F.first(keys);
+
+        GroupKey newKey = new GroupKey(newEncKey.id(), encSpi.decryptKey(newEncKey.key()));
+
+        keys.add(0, newKey);
+
+        // Remove the duplicate key from the tail of the list if exists.
+        keys.subList(1, keys.size()).remove(newKey);
+
+        return prevKey;
+    }
+
+    /**
+     * Put new unused key.
+     *
+     * @param grpId Cache group ID.
+     * @param newEncKey New encrypted key for writing.
+     */
+    void putUnused(int grpId, GroupKeyEncrypted newEncKey) {

Review comment:
       `addKey`?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupEncryptionKeys.java
##########
@@ -0,0 +1,387 @@
+/*
+ * 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.managers.encryption;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import org.apache.ignite.internal.pagemem.wal.record.MasterKeyChangeRecord;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T3;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.encryption.EncryptionSpi;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Serves for managing encryption keys and related datastructure located in the heap.
+ */
+class CacheGroupEncryptionKeys {
+    /** Group encryption keys. */
+    private final Map<Integer, List<GroupKey>> grpKeys = new ConcurrentHashMap<>();
+
+    /** WAL segments encrypted with previous encrypted keys, mapped to cache group encryption key identifiers. */
+    private final Map<Long, Map<Integer, Set<Integer>>> trackedWalSegments = new ConcurrentSkipListMap<>();
+
+    /** Encryption spi. */
+    private final EncryptionSpi encSpi;
+
+    /**
+     * @param encSpi Encryption spi.
+     */
+    CacheGroupEncryptionKeys(EncryptionSpi encSpi) {
+        this.encSpi = encSpi;
+    }
+
+    /**
+     * Returns group encryption key.
+     *
+     * @param grpId Cache group ID.
+     * @return Group encryption key with identifier, that was set for writing.
+     */
+    GroupKey get(int grpId) {
+        List<GroupKey> keys = grpKeys.get(grpId);
+
+        if (F.isEmpty(keys))
+            return null;
+
+        return keys.get(0);
+    }
+
+    /**
+     * Returns group encryption key with specified identifier.
+     *
+     * @param grpId Cache group ID.
+     * @param keyId Encryption key ID.
+     * @return Group encryption key.
+     */
+    GroupKey get(int grpId, int keyId) {
+        List<GroupKey> keys = grpKeys.get(grpId);
+
+        if (keys == null)
+            return null;
+
+        for (GroupKey groupKey : keys) {
+            if (groupKey.unsignedId() == keyId)
+                return groupKey;
+        }
+
+        return null;
+    }
+
+    /**
+     * Gets information about existing encryption keys for the specified cache group.
+     *
+     * @param grpId Cache group ID.
+     * @return Map of the key identifier with hash code of encryption key.
+     */
+    Map<Integer, Integer> info(int grpId) {
+        List<GroupKey> keys = grpKeys.get(grpId);
+
+        if (keys == null)
+            return null;
+
+        Map<Integer, Integer> keysInfo = new LinkedHashMap<>();
+
+        for (GroupKey groupKey : keys)
+            keysInfo.put(groupKey.unsignedId(), Arrays.hashCode(U.toBytes(groupKey.key())));
+
+        return keysInfo;
+    }
+
+    /**
+     * @return Local encryption keys.
+     */
+    @Nullable HashMap<Integer, GroupKeyEncrypted> getAll() {
+        if (F.isEmpty(grpKeys))
+            return null;
+
+        HashMap<Integer, GroupKeyEncrypted> Keys = new HashMap<>();
+
+        for (Map.Entry<Integer, List<GroupKey>> entry : grpKeys.entrySet()) {
+            int grpId = entry.getKey();
+            GroupKey grpKey = entry.getValue().get(0);
+
+            Keys.put(grpId, new GroupKeyEncrypted(grpKey.unsignedId(), encSpi.encryptKey(grpKey.key())));
+        }
+
+        return Keys;
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     *
+     * @return Local encryption keys used for specified cache group.
+     */
+    List<GroupKeyEncrypted> getAll(int grpId) {
+        List<GroupKey> grpKeys = this.grpKeys.get(grpId);
+
+        if (F.isEmpty(grpKeys))
+            return null;
+
+        List<GroupKeyEncrypted> encryptedKeys = new ArrayList<>(grpKeys.size());
+
+        for (GroupKey grpKey : grpKeys)
+            encryptedKeys.add(new GroupKeyEncrypted(grpKey.unsignedId(), encSpi.encryptKey(grpKey.key())));
+
+        return encryptedKeys;
+    }
+
+    /**
+     * Put new encryption key and set it for writing.
+     *
+     * @param grpId Cache group ID.
+     * @param newEncKey New encrypted key for writing.
+     * @return Previous encryption key for writing.
+     */
+    GroupKey put(int grpId, GroupKeyEncrypted newEncKey) {
+        assert newEncKey != null;
+
+        List<GroupKey> keys = grpKeys.computeIfAbsent(grpId, list -> new CopyOnWriteArrayList<>());
+
+        if (keys == null)
+            return null;
+
+        GroupKey prevKey = F.first(keys);
+
+        GroupKey newKey = new GroupKey(newEncKey.id(), encSpi.decryptKey(newEncKey.key()));
+
+        keys.add(0, newKey);
+
+        // Remove the duplicate key from the tail of the list if exists.
+        keys.subList(1, keys.size()).remove(newKey);
+
+        return prevKey;
+    }
+
+    /**
+     * Put new unused key.
+     *
+     * @param grpId Cache group ID.
+     * @param newEncKey New encrypted key for writing.
+     */
+    void putUnused(int grpId, GroupKeyEncrypted newEncKey) {
+        grpKeys.get(grpId).add(new GroupKey(newEncKey.id(), encSpi.decryptKey(newEncKey.key())));
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @param encryptedKeys Encrypted keys.
+     */
+    void put(int grpId, List<GroupKeyEncrypted> encryptedKeys) {

Review comment:
       `setGroupKeys`?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupEncryptionKeys.java
##########
@@ -0,0 +1,387 @@
+/*
+ * 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.managers.encryption;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import org.apache.ignite.internal.pagemem.wal.record.MasterKeyChangeRecord;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T3;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.encryption.EncryptionSpi;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Serves for managing encryption keys and related datastructure located in the heap.
+ */
+class CacheGroupEncryptionKeys {
+    /** Group encryption keys. */
+    private final Map<Integer, List<GroupKey>> grpKeys = new ConcurrentHashMap<>();
+
+    /** WAL segments encrypted with previous encrypted keys, mapped to cache group encryption key identifiers. */
+    private final Map<Long, Map<Integer, Set<Integer>>> trackedWalSegments = new ConcurrentSkipListMap<>();
+
+    /** Encryption spi. */
+    private final EncryptionSpi encSpi;
+
+    /**
+     * @param encSpi Encryption spi.
+     */
+    CacheGroupEncryptionKeys(EncryptionSpi encSpi) {
+        this.encSpi = encSpi;
+    }
+
+    /**
+     * Returns group encryption key.
+     *
+     * @param grpId Cache group ID.
+     * @return Group encryption key with identifier, that was set for writing.
+     */
+    GroupKey get(int grpId) {
+        List<GroupKey> keys = grpKeys.get(grpId);
+
+        if (F.isEmpty(keys))
+            return null;
+
+        return keys.get(0);
+    }
+
+    /**
+     * Returns group encryption key with specified identifier.
+     *
+     * @param grpId Cache group ID.
+     * @param keyId Encryption key ID.
+     * @return Group encryption key.
+     */
+    GroupKey get(int grpId, int keyId) {
+        List<GroupKey> keys = grpKeys.get(grpId);
+
+        if (keys == null)
+            return null;
+
+        for (GroupKey groupKey : keys) {
+            if (groupKey.unsignedId() == keyId)
+                return groupKey;
+        }
+
+        return null;
+    }
+
+    /**
+     * Gets information about existing encryption keys for the specified cache group.
+     *
+     * @param grpId Cache group ID.
+     * @return Map of the key identifier with hash code of encryption key.
+     */
+    Map<Integer, Integer> info(int grpId) {
+        List<GroupKey> keys = grpKeys.get(grpId);
+
+        if (keys == null)
+            return null;
+
+        Map<Integer, Integer> keysInfo = new LinkedHashMap<>();
+
+        for (GroupKey groupKey : keys)
+            keysInfo.put(groupKey.unsignedId(), Arrays.hashCode(U.toBytes(groupKey.key())));
+
+        return keysInfo;
+    }
+
+    /**
+     * @return Local encryption keys.
+     */
+    @Nullable HashMap<Integer, GroupKeyEncrypted> getAll() {
+        if (F.isEmpty(grpKeys))
+            return null;
+
+        HashMap<Integer, GroupKeyEncrypted> Keys = new HashMap<>();
+
+        for (Map.Entry<Integer, List<GroupKey>> entry : grpKeys.entrySet()) {
+            int grpId = entry.getKey();
+            GroupKey grpKey = entry.getValue().get(0);
+
+            Keys.put(grpId, new GroupKeyEncrypted(grpKey.unsignedId(), encSpi.encryptKey(grpKey.key())));
+        }
+
+        return Keys;
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     *
+     * @return Local encryption keys used for specified cache group.
+     */
+    List<GroupKeyEncrypted> getAll(int grpId) {
+        List<GroupKey> grpKeys = this.grpKeys.get(grpId);
+
+        if (F.isEmpty(grpKeys))
+            return null;
+
+        List<GroupKeyEncrypted> encryptedKeys = new ArrayList<>(grpKeys.size());
+
+        for (GroupKey grpKey : grpKeys)
+            encryptedKeys.add(new GroupKeyEncrypted(grpKey.unsignedId(), encSpi.encryptKey(grpKey.key())));
+
+        return encryptedKeys;
+    }
+
+    /**
+     * Put new encryption key and set it for writing.
+     *
+     * @param grpId Cache group ID.
+     * @param newEncKey New encrypted key for writing.
+     * @return Previous encryption key for writing.
+     */
+    GroupKey put(int grpId, GroupKeyEncrypted newEncKey) {
+        assert newEncKey != null;
+
+        List<GroupKey> keys = grpKeys.computeIfAbsent(grpId, list -> new CopyOnWriteArrayList<>());
+
+        if (keys == null)
+            return null;
+
+        GroupKey prevKey = F.first(keys);
+
+        GroupKey newKey = new GroupKey(newEncKey.id(), encSpi.decryptKey(newEncKey.key()));
+
+        keys.add(0, newKey);
+
+        // Remove the duplicate key from the tail of the list if exists.
+        keys.subList(1, keys.size()).remove(newKey);
+
+        return prevKey;
+    }
+
+    /**
+     * Put new unused key.
+     *
+     * @param grpId Cache group ID.
+     * @param newEncKey New encrypted key for writing.
+     */
+    void putUnused(int grpId, GroupKeyEncrypted newEncKey) {
+        grpKeys.get(grpId).add(new GroupKey(newEncKey.id(), encSpi.decryptKey(newEncKey.key())));
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @param encryptedKeys Encrypted keys.
+     */
+    void put(int grpId, List<GroupKeyEncrypted> encryptedKeys) {
+        List<GroupKey> keys = new CopyOnWriteArrayList<>();
+
+        for (GroupKeyEncrypted encrKey : encryptedKeys)
+            keys.add(new GroupKey(encrKey.id(), encSpi.decryptKey(encrKey.key())));
+
+        grpKeys.put(grpId, keys);
+    }
+
+    /**
+     * @return Cache group identifiers for which encryption keys are stored.
+     */
+    Set<Integer> groups() {
+        return grpKeys.keySet();
+    }
+
+    /**
+     * Remove encrytion keys associated with the specified cache group.
+     *
+     * @param grpId Cache group ID.
+     */
+    void remove(int grpId) {
+        grpKeys.remove(grpId);
+    }
+
+    /**
+     * Convert encryption keys to WAL logical record that stores encryption keys.
+     */
+    MasterKeyChangeRecord toMasterKeyChangeRecord() {
+        List<T3<Integer, Byte, byte[]>> reencryptedKeys = new ArrayList<>();
+
+        for (Map.Entry<Integer, List<GroupKey>> entry : grpKeys.entrySet()) {
+            int grpId = entry.getKey();
+
+            for (GroupKey grpKey : entry.getValue()) {
+                byte keyId = grpKey.id();
+                byte[] encryptedKey = encSpi.encryptKey(grpKey.key());
+
+                reencryptedKeys.add(new T3<>(grpId, keyId, encryptedKey));
+            }
+        }
+
+        return new MasterKeyChangeRecord(encSpi.getMasterKeyName(), reencryptedKeys);
+    }
+
+    /**
+     * Load encryption keys from WAL logical record that stores encryption keys.
+     *
+     * @param rec Logical record that stores encryption keys.
+     */
+    void fromMasterKeyChangeRecord(MasterKeyChangeRecord rec) {

Review comment:
       Pass rec.getGrpKeys() to this method without WAL record (method also should be renamed of course).




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -194,10 +227,28 @@
      * Master key change prepare process. Checks that all server nodes have the same new master key and then starts
      * finish process.
      */
-    private DistributedProcess<MasterKeyChangeRequest, MasterKeyChangeResult> prepareMKChangeProc;
+    private DistributedProcess<MasterKeyChangeRequest, EmptyResult> prepareMKChangeProc;
 
     /** Process to perform the master key change. Changes master key and reencrypt group keys. */
-    private DistributedProcess<MasterKeyChangeRequest, MasterKeyChangeResult> performMKChangeProc;
+    private DistributedProcess<MasterKeyChangeRequest, EmptyResult> performMKChangeProc;
+
+    /** Two phase distributed process, that performs cache group encryption key rotation. */
+    private GroupKeyChangeProcess grpKeyChangeProc;
+
+    /** Cache groups for which encryption key was changed, and they must be re-encrypted. */
+    private final Map<Integer, Map<Integer, Long>> reencryptGroups = new ConcurrentHashMap<>();

Review comment:
       submap was replaced with array of longs




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/util/BasicRateLimiterTest.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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;
+
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Test;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Rate limiter tests.
+ */
+public class BasicRateLimiterTest {

Review comment:
       FIxed (included in "Basic 1")




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupPageScanner.java
##########
@@ -0,0 +1,567 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.EncryptionConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.util.BasicRateLimiter;
+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.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+import org.apache.ignite.thread.OomExceptionHandler;
+
+import static org.apache.ignite.internal.util.IgniteUtils.MB;
+
+/**
+ * Cache group page stores scanner.
+ * Scans a range of pages and marks them as dirty to re-encrypt them with the last encryption key on disk.
+ */
+public class CacheGroupPageScanner implements DbCheckpointListener {
+    /** Thread prefix for scanning tasks. */
+    private static final String REENCRYPT_THREAD_PREFIX = "reencrypt";
+
+    /** Encryption configuration. */
+    private final EncryptionConfiguration encrCfg;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to group scanning context. */
+    private final Map<Integer, GroupScanContext> grps = new ConcurrentHashMap<>();
+
+    /** Queue of groups waiting for a checkpoint. */
+    private final Queue<Integer> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Executor to start partition scan tasks. */
+    private final IgniteThreadPoolExecutor execSvc;
+
+    /** Page scanning speed limiter. */
+    private final BasicRateLimiter limiter;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupPageScanner(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        encrCfg = ctx.config().getDataStorageConfiguration().getEncryptionConfiguration();
+
+        execSvc = new IgniteThreadPoolExecutor(REENCRYPT_THREAD_PREFIX,
+            ctx.igniteInstanceName(),
+            encrCfg.getReencryptionThreadCnt(),
+            encrCfg.getReencryptionThreadCnt(),
+            IgniteConfiguration.DFLT_THREAD_KEEP_ALIVE_TIME,
+            new LinkedBlockingQueue<>(),
+            GridIoPolicy.SYSTEM_POOL,
+            new OomExceptionHandler(ctx));
+
+        execSvc.allowCoreThreadTimeOut(true);
+
+        DataStorageConfiguration dsCfg = ctx.config().getDataStorageConfiguration();
+
+        limiter = CU.isPersistenceEnabled(dsCfg) && encrCfg.getReencryptionRateLimit() > 0 ?
+            new BasicRateLimiter(encrCfg.getReencryptionRateLimit() * MB / dsCfg.getPageSize()) : null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCheckpointBegin(Context cpCtx) {
+        Set<Integer> completeCandidates = new HashSet<>();
+
+        Integer grpId;
+
+        while ((grpId = cpWaitGrps.poll()) != null)
+            completeCandidates.add(grpId);
+
+        cpCtx.finishedStateFut().listen(
+            f -> {
+                if (f.error() != null || f.isCancelled()) {
+                    cpWaitGrps.addAll(completeCandidates);
+
+                    return;
+                }
+
+                lock.lock();
+
+                try {
+                    for (int groupId : completeCandidates) {
+                        GroupScanContext scanCtx = grps.remove(groupId);
+
+                        boolean finished = scanCtx.finish();
+
+                        assert finished : groupId;
+
+                        if (log.isInfoEnabled())
+                            log.info("Cache group reencryption is finished [grpId=" + groupId + "]");
+                    }
+
+                    if (!grps.isEmpty())
+                        return;
+
+                    ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).
+                        removeCheckpointListener(this);
+                }
+                finally {
+                    lock.unlock();
+                }
+            }
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public void beforeCheckpointBegin(Context cpCtx) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onMarkCheckpointBegin(Context ctx) {
+        // No-op.
+    }
+
+    /**
+     * @return {@code True} If reencryption is disabled.
+     */
+    public boolean disabled() {
+        return encrCfg.isReencryptionDisabled();
+    }
+
+    /**
+     * Schedule scanning partitions.
+     *
+     * @param grpId Cache group ID.
+     */
+    public IgniteInternalFuture<Void> schedule(int grpId) throws IgniteCheckedException {
+        if (disabled())
+            throw new IgniteCheckedException("Reencryption is disabled.");
+
+        CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+        if (grp == null) {
+            if (log.isDebugEnabled())
+                log.debug("Skip reencryption, cache group was destroyed [grp=" + grpId + "]");
+
+            return new GridFinishedFuture<>();
+        }
+
+        lock.lock();
+
+        try {
+            if (stopped)
+                throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
+
+            if (grps.isEmpty())
+                ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).addCheckpointListener(this);
+
+            GroupScanContext prevState = grps.get(grpId);
+
+            if (prevState != null) {
+                if (log.isDebugEnabled())
+                    log.debug("Reencryption already scheduled [grpId=" + grpId + "]");
+
+                return prevState.finishFuture();
+            }
+
+            GroupScanContext ctx0 = new GroupScanContext(grpId);
+
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) {
+                    if (ctx.encryption().getEncryptionState(grpId, partId) == 0) {
+                        if (log.isDebugEnabled())
+                            log.debug("Skipping partition reencryption [grp=" + grpId + ", p=" + partId + "]");
+
+                        return;
+                    }
+
+                    PageStoreScanTask scanTask = new PageStoreScanTask(ctx0, partId);
+
+                    ctx0.add(partId, scanTask);
+
+                    execSvc.submit(scanTask);
+                }
+            });
+
+            ctx0.initialize().listen(f -> {
+                Throwable t = f.error();
+
+                if (t != null) {
+                    log.error("Reencryption is failed [grpId=" + grpId + "]", t);
+
+                    ctx0.fail(t);
+
+                    return;
+                }
+
+                boolean added = cpWaitGrps.offer(grpId);
+
+                assert added;
+            });
+
+            if (log.isInfoEnabled())
+                log.info("Scheduled reencryption [grpId=" + grpId + "]");
+
+            grps.put(grpId, ctx0);
+
+            return ctx0.finishFuture();
+        }
+        finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @return Future that will be completed when all partitions have been scanned and pages have been written to disk.
+     */
+    public IgniteInternalFuture<Void> statusFuture(int grpId) {
+        GroupScanContext ctx0 = grps.get(grpId);
+
+        return ctx0 == null ? new GridFinishedFuture<>() : ctx0.finishFuture();
+    }
+
+    /**
+     * Shutdown scanning and disable new tasks scheduling.
+     */
+    public void stop() throws IgniteCheckedException {
+        lock.lock();
+
+        try {
+            stopped = true;
+
+            for (GroupScanContext ctx0 : grps.values())
+                ctx0.finishFuture().cancel();
+
+            execSvc.shutdown();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Stop scannig the specified partition.
+     *
+     * @param grpId Cache group ID.
+     * @param partId Partition ID.
+     * @return {@code True} if reencryption was cancelled.
+     * @throws IgniteCheckedException If failed.
+     */
+    public boolean cancel(int grpId, int partId) throws IgniteCheckedException {
+        GroupScanContext ctx = grps.get(grpId);
+
+        if (ctx == null)
+            return false;
+
+        return ctx.cancel(partId);
+    }
+
+    /**
+     * Collect current number of pages in the specified cache group.
+     *
+     * @param grp Cache group.
+     * @return Map of partitions with current page count.
+     * @throws IgniteCheckedException If failed.
+     */
+    public Map<Integer, Long> pagesCount(CacheGroupContext grp) throws IgniteCheckedException {
+        Map<Integer, Long> partStates = new HashMap<>();

Review comment:
       Map<Integer, ReencryptState> replaced with long[grp.affinity().partitions() + 1] (+1 - for index partition)




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -283,11 +334,17 @@ public GridEncryptionManager(GridKernalContext ctx) {
 
         performMKChangeProc = new DistributedProcess<>(ctx, MASTER_KEY_CHANGE_FINISH, this::performMasterKeyChange,
             this::finishPerformMasterKeyChange);
+
+        reencryption = new CacheGroupReencryption(ctx);

Review comment:
       Could you describe in more detail what should we do 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupPageScanner.java
##########
@@ -0,0 +1,479 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId;
+import org.apache.ignite.internal.util.BasicRateLimiter;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+import org.apache.ignite.thread.OomExceptionHandler;
+
+import static org.apache.ignite.internal.util.IgniteUtils.MB;
+
+/**
+ * Cache group page stores scanner.
+ * Scans a range of pages and marks them as dirty to re-encrypt them with the last encryption key on disk.
+ */
+public class CacheGroupPageScanner implements CheckpointListener {
+    /** Thread prefix for scanning tasks. */
+    private static final String REENCRYPT_THREAD_PREFIX = "reencrypt";
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to group scanning task. */
+    private final Map<Integer, GroupScanTask> grps = new ConcurrentHashMap<>();
+
+    /** Collection of groups waiting for a checkpoint. */
+    private final Collection<GroupScanTask> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Page scanning speed limiter. */
+    private final BasicRateLimiter limiter;
+
+    /** Single-threaded executor to run cache group scan task. */
+    private final ThreadPoolExecutor singleExecSvc;
+
+    /** Number of pages that is scanned during reencryption under checkpoint lock. */
+    private final int batchSize;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupPageScanner(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        DataStorageConfiguration dsCfg = ctx.config().getDataStorageConfiguration();
+
+        if (!CU.isPersistenceEnabled(dsCfg)) {
+            batchSize = -1;
+            limiter = null;
+            singleExecSvc = null;
+
+            return;
+        }
+
+        double rateLimit = dsCfg.getEncryptionConfiguration().getReencryptionRateLimit();
+
+        limiter = rateLimit > 0 ? new BasicRateLimiter(rateLimit * MB /
+            (dsCfg.getPageSize() == 0 ? DataStorageConfiguration.DFLT_PAGE_SIZE : dsCfg.getPageSize())) : null;
+
+        batchSize = dsCfg.getEncryptionConfiguration().getReencryptionBatchSize();
+
+        singleExecSvc = new IgniteThreadPoolExecutor(REENCRYPT_THREAD_PREFIX,
+            ctx.igniteInstanceName(),
+            1,
+            1,
+            IgniteConfiguration.DFLT_THREAD_KEEP_ALIVE_TIME,
+            new LinkedBlockingQueue<>(),
+            GridIoPolicy.SYSTEM_POOL,
+            new OomExceptionHandler(ctx));
+
+        singleExecSvc.allowCoreThreadTimeOut(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCheckpointBegin(Context cpCtx) {
+        Set<GroupScanTask> completeCandidates = new HashSet<>();
+
+        cpWaitGrps.removeIf(completeCandidates::add);

Review comment:
       As mentioned in the previous comment, we now use beforeCheckpointBegin to wait for a checkpoint, which is called before the checkpoint mark-phase is executed (and before the write lock is acquired), so we have now synchronization between releasing the read lock (while scanning) and acquiring a write lock (on mark phase).




----------------------------------------------------------------
This is an automated message from the 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] NSAmelchev commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -627,22 +716,104 @@ public void groupKey(int grpId, byte[] encGrpKey) {
         return withMasterKeyChangeReadLock(() -> getSpi().getMasterKeyName());
     }
 
+    /** {@inheritDoc} */
+    @Override public IgniteFuture<Void> changeCacheGroupKey(Collection<String> cacheOrGrpNames) {
+        A.notEmpty(cacheOrGrpNames, "cacheOrGrpNames");
+
+        synchronized (opsMux) {
+            if (stopped) {

Review comment:
       We should check `disconnected` too




----------------------------------------------------------------
This is an automated message from the 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] alex-plekhanov commented on a change in pull request #7941: IGNITE-12843

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #7941:
URL: https://github.com/apache/ignite/pull/7941#discussion_r486411518



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupPageScanner.java
##########
@@ -0,0 +1,453 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId;
+import org.apache.ignite.internal.util.BasicRateLimiter;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+
+import static org.apache.ignite.internal.util.IgniteUtils.MB;
+
+/**
+ * Cache group page stores scanner.
+ * Scans a range of pages and marks them as dirty to re-encrypt them with the last encryption key on disk.
+ */
+public class CacheGroupPageScanner implements DbCheckpointListener {
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to group scanning task. */
+    private final Map<Integer, GroupScanTask> grps = new ConcurrentHashMap<>();
+
+    /** Collection of groups waiting for a checkpoint. */
+    private final Collection<GroupScanTask> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Page scanning speed limiter. */
+    private final BasicRateLimiter limiter;
+
+    /** Number of pages that is scanned during reencryption under checkpoint lock. */
+    private final int batchSize;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupPageScanner(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        DataStorageConfiguration dsCfg = ctx.config().getDataStorageConfiguration();
+
+        if (!CU.isPersistenceEnabled(dsCfg)) {
+            batchSize = -1;
+            limiter = null;
+
+            return;
+        }
+
+        batchSize = dsCfg.getEncryptionConfiguration().getReencryptionBatchSize();
+
+        double rateLimit = dsCfg.getEncryptionConfiguration().getReencryptionRateLimit();
+
+        limiter = rateLimit > 0 ? new BasicRateLimiter(rateLimit * MB /
+            (dsCfg.getPageSize() == 0 ? DataStorageConfiguration.DFLT_PAGE_SIZE : dsCfg.getPageSize())) : null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCheckpointBegin(Context cpCtx) {
+        Set<GroupScanTask> completeCandidates = new HashSet<>();
+
+        cpWaitGrps.removeIf(completeCandidates::add);
+
+        cpCtx.finishedStateFut().listen(
+            f -> {
+                // Retry if error occurs.
+                if (f.error() != null || f.isCancelled()) {
+                    cpWaitGrps.addAll(completeCandidates);
+
+                    return;
+                }
+
+                lock.lock();
+
+                try {
+                    for (GroupScanTask grpScanTask : completeCandidates) {
+                        grps.remove(grpScanTask.groupId());
+
+                        grpScanTask.onDone();
+
+                        if (log.isInfoEnabled())
+                            log.info("Cache group reencryption is finished [grpId=" + grpScanTask.groupId() + "]");
+                    }
+
+                    if (!grps.isEmpty())
+                        return;
+
+                    ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).
+                        removeCheckpointListener(this);
+                }
+                finally {
+                    lock.unlock();
+                }
+            }
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public void beforeCheckpointBegin(Context cpCtx) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onMarkCheckpointBegin(Context ctx) {
+        // No-op.
+    }
+
+    /**
+     * Schedule scanning partitions.
+     *
+     * @param grpId Cache group ID.
+     */
+    public IgniteInternalFuture<Void> schedule(int grpId) throws IgniteCheckedException {
+        CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+        if (grp == null) {
+            if (log.isDebugEnabled())
+                log.debug("Skip reencryption, cache group was destroyed [grp=" + grpId + "]");
+
+            return new GridFinishedFuture<>();
+        }
+
+        lock.lock();
+
+        try {
+            if (stopped)
+                throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
+
+            if (grps.isEmpty())
+                ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).addCheckpointListener(this);
+
+            GroupScanTask prevState = grps.get(grpId);
+
+            if (prevState != null && !prevState.isDone()) {
+                if (log.isDebugEnabled())
+                    log.debug("Reencryption already scheduled [grpId=" + grpId + "]");
+
+                return prevState;
+            }
+
+            Set<Integer> parts = new HashSet<>();
+
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) {
+                    if (ctx.encryption().getEncryptionState(grpId, partId) == 0) {
+                        if (log.isDebugEnabled())
+                            log.debug("Skipping partition reencryption [grp=" + grpId + ", p=" + partId + "]");
+
+                        return;
+                    }
+
+                    parts.add(partId);
+                }
+            });
+
+            GroupScanTask grpScan = new GroupScanTask(grp, parts);
+
+            ctx.getSystemExecutorService().submit(grpScan);

Review comment:
       Here task for each cache group can be executed by different threads. Let's use single-threaded execution 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] NSAmelchev commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GroupKeyChangeProcess.java
##########
@@ -0,0 +1,333 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteFeatures;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.managers.encryption.GridEncryptionManager.EmptyResult;
+import org.apache.ignite.internal.managers.encryption.GridEncryptionManager.KeyChangeFuture;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
+import org.apache.ignite.internal.util.distributed.DistributedProcess;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.future.IgniteFinishedFutureImpl;
+import org.apache.ignite.internal.util.future.IgniteFutureImpl;
+import org.apache.ignite.internal.util.typedef.F;
+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.IgniteFuture;
+import org.apache.ignite.lang.IgniteFutureCancelledException;
+
+import static org.apache.ignite.internal.IgniteFeatures.CACHE_GROUP_KEY_CHANGE;
+import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.CACHE_GROUP_KEY_CHANGE_FINISH;
+import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.CACHE_GROUP_KEY_CHANGE_PREPARE;
+
+/**
+ * Two phase distributed process, that performs cache group encryption key rotation.
+ */
+class GroupKeyChangeProcess {
+    /** Grid kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Cache group encyption key change prepare phase. */
+    private final DistributedProcess<ChangeCacheEncryptionRequest, EmptyResult> prepareGKChangeProc;
+
+    /** Cache group encyption key change perform phase. */
+    private final DistributedProcess<ChangeCacheEncryptionRequest, EmptyResult> performGKChangeProc;
+
+    /** Group encryption keys. */
+    private final CacheGroupEncryptionKeys keys;
+
+    /** Cache group key change future. */
+    private volatile GroupKeyChangeFuture fut;
+
+    /** Cache group key change request. */
+    private volatile ChangeCacheEncryptionRequest req;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    GroupKeyChangeProcess(GridKernalContext ctx, CacheGroupEncryptionKeys keys) {
+        this.ctx = ctx;
+        this.keys = keys;
+
+        prepareGKChangeProc =
+            new DistributedProcess<>(ctx, CACHE_GROUP_KEY_CHANGE_PREPARE, this::prepare, this::finishPrepare);
+        performGKChangeProc =
+            new DistributedProcess<>(ctx, CACHE_GROUP_KEY_CHANGE_FINISH, this::perform, this::finishPerform);
+    }
+
+    /**
+     * @return {@code True} if operation is still in progress.
+     */
+    public boolean started() {
+        return req != null;
+    }
+
+    /**
+     * @return {@code True} if operation is not finished.
+     */
+    public boolean finished() {
+        IgniteInternalFuture<Void> fut0 = fut;
+
+        return fut0 == null || fut0.isDone();
+    }
+
+    /**
+     * @param msg Error message.
+     */
+    public void cancel(String msg) {
+        GridFutureAdapter<Void> keyChangeFut = fut;
+
+        if (keyChangeFut != null && !keyChangeFut.isDone())
+            keyChangeFut.onDone(new IgniteFutureCancelledException(msg));
+    }
+
+    /**
+     * Starts cache group encryption key change process.
+     *
+     * @param cacheOrGrpNames Cache or group names.
+     */
+    public IgniteFuture<Void> start(Collection<String> cacheOrGrpNames) {
+        if (ctx.clientNode())
+            throw new UnsupportedOperationException("Client and daemon nodes can not perform this operation.");
+
+        if (!IgniteFeatures.allNodesSupports(ctx.grid().cluster().nodes(), CACHE_GROUP_KEY_CHANGE))
+            throw new IllegalStateException("Not all nodes in the cluster support this operation.");
+
+        if (ctx.state().clusterState().state() != ClusterState.ACTIVE)

Review comment:
       if read_only mode is supported than `state().active()` can be used 




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupPageScanner.java
##########
@@ -0,0 +1,453 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId;
+import org.apache.ignite.internal.util.BasicRateLimiter;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+
+import static org.apache.ignite.internal.util.IgniteUtils.MB;
+
+/**
+ * Cache group page stores scanner.
+ * Scans a range of pages and marks them as dirty to re-encrypt them with the last encryption key on disk.
+ */
+public class CacheGroupPageScanner implements DbCheckpointListener {
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to group scanning task. */
+    private final Map<Integer, GroupScanTask> grps = new ConcurrentHashMap<>();
+
+    /** Collection of groups waiting for a checkpoint. */
+    private final Collection<GroupScanTask> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Page scanning speed limiter. */
+    private final BasicRateLimiter limiter;
+
+    /** Number of pages that is scanned during reencryption under checkpoint lock. */
+    private final int batchSize;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupPageScanner(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        DataStorageConfiguration dsCfg = ctx.config().getDataStorageConfiguration();
+
+        if (!CU.isPersistenceEnabled(dsCfg)) {
+            batchSize = -1;
+            limiter = null;
+
+            return;
+        }
+
+        batchSize = dsCfg.getEncryptionConfiguration().getReencryptionBatchSize();
+
+        double rateLimit = dsCfg.getEncryptionConfiguration().getReencryptionRateLimit();
+
+        limiter = rateLimit > 0 ? new BasicRateLimiter(rateLimit * MB /
+            (dsCfg.getPageSize() == 0 ? DataStorageConfiguration.DFLT_PAGE_SIZE : dsCfg.getPageSize())) : null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCheckpointBegin(Context cpCtx) {
+        Set<GroupScanTask> completeCandidates = new HashSet<>();
+
+        cpWaitGrps.removeIf(completeCandidates::add);
+
+        cpCtx.finishedStateFut().listen(
+            f -> {
+                // Retry if error occurs.
+                if (f.error() != null || f.isCancelled()) {
+                    cpWaitGrps.addAll(completeCandidates);
+
+                    return;
+                }
+
+                lock.lock();
+
+                try {
+                    for (GroupScanTask grpScanTask : completeCandidates) {
+                        grps.remove(grpScanTask.groupId());
+
+                        grpScanTask.onDone();
+
+                        if (log.isInfoEnabled())
+                            log.info("Cache group reencryption is finished [grpId=" + grpScanTask.groupId() + "]");
+                    }
+
+                    if (!grps.isEmpty())
+                        return;
+
+                    ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).
+                        removeCheckpointListener(this);
+                }
+                finally {
+                    lock.unlock();
+                }
+            }
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public void beforeCheckpointBegin(Context cpCtx) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onMarkCheckpointBegin(Context ctx) {
+        // No-op.
+    }
+
+    /**
+     * Schedule scanning partitions.
+     *
+     * @param grpId Cache group ID.
+     */
+    public IgniteInternalFuture<Void> schedule(int grpId) throws IgniteCheckedException {
+        CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+        if (grp == null) {
+            if (log.isDebugEnabled())
+                log.debug("Skip reencryption, cache group was destroyed [grp=" + grpId + "]");
+
+            return new GridFinishedFuture<>();
+        }
+
+        lock.lock();
+
+        try {
+            if (stopped)
+                throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
+
+            if (grps.isEmpty())
+                ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).addCheckpointListener(this);
+
+            GroupScanTask prevState = grps.get(grpId);
+
+            if (prevState != null && !prevState.isDone()) {
+                if (log.isDebugEnabled())
+                    log.debug("Reencryption already scheduled [grpId=" + grpId + "]");
+
+                return prevState;
+            }
+
+            Set<Integer> parts = new HashSet<>();
+
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) {
+                    if (ctx.encryption().getEncryptionState(grpId, partId) == 0) {
+                        if (log.isDebugEnabled())
+                            log.debug("Skipping partition reencryption [grp=" + grpId + ", p=" + partId + "]");
+
+                        return;
+                    }
+
+                    parts.add(partId);
+                }
+            });
+
+            GroupScanTask grpScan = new GroupScanTask(grp, parts);
+
+            ctx.getSystemExecutorService().submit(grpScan);

Review comment:
       Added single threaded executor




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/distributed/DistributedProcess.java
##########
@@ -430,6 +430,16 @@ private Process(UUID id) {
          *
          * @see IgniteSnapshotManager
          */
-        END_SNAPSHOT
+        END_SNAPSHOT,
+
+        /**
+         * Cache group encyption key change prepare phase.
+         */
+        CACHE_GROUP_KEY_CHANGE_PREPARE,

Review comment:
       Thanks, good point, but this patch is big enough and this change is not related to it, so I think it would be better to do it in a separate task.




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
##########
@@ -2863,11 +2869,16 @@ private RestoreLogicalState applyLogicalUpdates(
 
                         break;
 
-                    case MASTER_KEY_CHANGE_RECORD:
+                    case MASTER_KEY_CHANGE_RECORD_V2:

Review comment:
       There is no compatibility issue. If we read "old" (MASTER_KEY_CHANGE_RECORD) in the serializer, we will have a new WALRecord with type V2 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -194,10 +207,22 @@
      * Master key change prepare process. Checks that all server nodes have the same new master key and then starts
      * finish process.
      */
-    private DistributedProcess<MasterKeyChangeRequest, MasterKeyChangeResult> prepareMKChangeProc;
+    private DistributedProcess<MasterKeyChangeRequest, EmptyResult> prepareMKChangeProc;
 
     /** Process to perform the master key change. Changes master key and reencrypt group keys. */
-    private DistributedProcess<MasterKeyChangeRequest, MasterKeyChangeResult> performMKChangeProc;
+    private DistributedProcess<MasterKeyChangeRequest, EmptyResult> performMKChangeProc;
+
+    /** Two phase distributed process, that performs cache group encryption key rotation. */
+    private GroupKeyChangeProcess grpKeyChangeProc;
+
+    /** Cache groups for which encryption key was changed, and they must be re-encrypted. */
+    private final Map<Integer, long[]> reencryptGroups = new ConcurrentHashMap<>();
+
+    /** Cache groups for which encryption key was changed on node join. */
+    private final Map<Integer, Integer> reencryptGroupsForced = new ConcurrentHashMap<>();

Review comment:
       fixed




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java
##########
@@ -258,6 +258,9 @@
     /** */
     public static final short T_MARKER_PAGE = 33;
 
+    /** */
+    public static final short T_INDEX_META = 34;

Review comment:
       Yes, we adding two int fields into two pages.
   
   We have 2 types of memory pages:
   T_META (index partition)
   T_PART_META (other partition)
   
   We want to add 2 ints into each of them, but don't forget that IO for T_PART_META extends IO for T_META.
   
   In my first approach I added 2 ints into T_META, and added PagePartitionMetaIOV3 for T_PART_META.
   But since PagePartitionMetaIO "extends" PageMetaIO all offsets in PagePartitionMetaIO was shifted and there was a problem with inheritance (for example in PagePartitionMetaIO/V2 new methods were visible, all offsets was redefined in PagePartitionMetaIOV3), you can imagine what it looked like in commit with the second approach https://github.com/apache/ignite/pull/7941/commits/036b931bd40e2ffbd519779fde86fae7343da23e
   
   With current approach we simply create the new IO that is only used for the "index" partition.
   
   If you can suggest a better approach for storing 2 extra int's in the index meta page without losing binary compatibility, please describe.
   
   p.s. I explained this change in wiki https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=95652384#TDE.Phase3.Cachekeyrotation.-PageMetaIOandPagePartitionMetaIO




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PagePartitionMetaIOV3.java
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.tree.io;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageUtils;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState;
+import org.apache.ignite.internal.util.GridStringBuilder;
+
+/**
+ * IO for partition metadata pages.
+ * Persistent partition contains it's own PendingTree.
+ */
+public class PagePartitionMetaIOV3 extends PagePartitionMetaIOV2 {
+    /** Last reencrypted page index offset. */
+    private static final int ENCRYPT_PAGE_IDX_OFF = END_OF_PARTITION_PAGE_META_V2;
+
+    /** Total pages to be reencrypted offset. */
+    private static final int ENCRYPT_PAGE_MAX_OFF = ENCRYPT_PAGE_IDX_OFF + 4;
+
+    /**
+     * @param ver Version.
+     */
+    public PagePartitionMetaIOV3(int ver) {
+        super(ver);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void initNewPage(long pageAddr, long pageId, int pageSize) {
+        super.initNewPage(pageAddr, pageId, pageSize);
+
+        setEncryptedPageIndex(pageAddr, 0);
+        setEncryptedPageCount(pageAddr, 0);
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @return Index of the last reencrypted page.
+     */
+    public int getEncryptedPageIndex(long pageAddr) {
+        return PageUtils.getInt(pageAddr, ENCRYPT_PAGE_IDX_OFF);
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @param pageIdx Index of the last reencrypted page.
+     *
+     * @return {@code true} if value has changed as a result of this method's invocation.
+     */
+    public boolean setEncryptedPageIndex(long pageAddr, int pageIdx) {
+        if (getEncryptedPageIndex(pageAddr) == pageIdx)
+            return false;
+
+        PageUtils.putLong(pageAddr, ENCRYPT_PAGE_IDX_OFF, pageIdx);
+
+        return true;
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @return Total pages to be reencrypted.
+     */
+    public int getEncryptedPageCount(long pageAddr) {
+        return PageUtils.getInt(pageAddr, ENCRYPT_PAGE_MAX_OFF);
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @param pagesCnt Total pages to be reencrypted.
+     *
+     * @return {@code true} if value has changed as a result of this method's invocation.
+     */
+    public boolean setEncryptedPageCount(long pageAddr, int pagesCnt) {
+        if (getEncryptedPageCount(pageAddr) == pagesCnt)
+            return false;
+
+        PageUtils.putInt(pageAddr, ENCRYPT_PAGE_MAX_OFF, pagesCnt);
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void printPage(long pageAddr, int pageSize, GridStringBuilder sb) throws IgniteCheckedException {
+        byte state = getPartitionState(pageAddr);
+
+        sb.a("PagePartitionMeta[\n\ttreeRoot=").a(getReuseListRoot(pageAddr));
+        sb.a(",\n\tpendingTreeRoot=").a(getLastSuccessfulFullSnapshotId(pageAddr));
+        sb.a(",\n\tlastSuccessfulFullSnapshotId=").a(getLastSuccessfulFullSnapshotId(pageAddr));
+        sb.a(",\n\tlastSuccessfulSnapshotId=").a(getLastSuccessfulSnapshotId(pageAddr));
+        sb.a(",\n\tnextSnapshotTag=").a(getNextSnapshotTag(pageAddr));
+        sb.a(",\n\tlastSuccessfulSnapshotTag=").a(getLastSuccessfulSnapshotTag(pageAddr));
+        sb.a(",\n\tlastAllocatedPageCount=").a(getLastAllocatedPageCount(pageAddr));
+        sb.a(",\n\tcandidatePageCount=").a(getCandidatePageCount(pageAddr));
+        sb.a(",\n\tencryptedPageIndex=").a(getEncryptedPageIndex(pageAddr));
+        sb.a(",\n\tencryptedPageCount=").a(getEncryptedPageCount(pageAddr));

Review comment:
       Fixed, added additional method to  print fields.




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupPageScanner.java
##########
@@ -0,0 +1,479 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId;
+import org.apache.ignite.internal.util.BasicRateLimiter;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+import org.apache.ignite.thread.OomExceptionHandler;
+
+import static org.apache.ignite.internal.util.IgniteUtils.MB;
+
+/**
+ * Cache group page stores scanner.
+ * Scans a range of pages and marks them as dirty to re-encrypt them with the last encryption key on disk.
+ */
+public class CacheGroupPageScanner implements CheckpointListener {
+    /** Thread prefix for scanning tasks. */
+    private static final String REENCRYPT_THREAD_PREFIX = "reencrypt";
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to group scanning task. */
+    private final Map<Integer, GroupScanTask> grps = new ConcurrentHashMap<>();
+
+    /** Collection of groups waiting for a checkpoint. */
+    private final Collection<GroupScanTask> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Page scanning speed limiter. */
+    private final BasicRateLimiter limiter;
+
+    /** Single-threaded executor to run cache group scan task. */
+    private final ThreadPoolExecutor singleExecSvc;
+
+    /** Number of pages that is scanned during reencryption under checkpoint lock. */
+    private final int batchSize;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupPageScanner(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        DataStorageConfiguration dsCfg = ctx.config().getDataStorageConfiguration();
+
+        if (!CU.isPersistenceEnabled(dsCfg)) {
+            batchSize = -1;
+            limiter = null;
+            singleExecSvc = null;
+
+            return;
+        }
+
+        double rateLimit = dsCfg.getEncryptionConfiguration().getReencryptionRateLimit();
+
+        limiter = rateLimit > 0 ? new BasicRateLimiter(rateLimit * MB /
+            (dsCfg.getPageSize() == 0 ? DataStorageConfiguration.DFLT_PAGE_SIZE : dsCfg.getPageSize())) : null;
+
+        batchSize = dsCfg.getEncryptionConfiguration().getReencryptionBatchSize();
+
+        singleExecSvc = new IgniteThreadPoolExecutor(REENCRYPT_THREAD_PREFIX,
+            ctx.igniteInstanceName(),
+            1,
+            1,
+            IgniteConfiguration.DFLT_THREAD_KEEP_ALIVE_TIME,
+            new LinkedBlockingQueue<>(),
+            GridIoPolicy.SYSTEM_POOL,
+            new OomExceptionHandler(ctx));
+
+        singleExecSvc.allowCoreThreadTimeOut(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCheckpointBegin(Context cpCtx) {
+        Set<GroupScanTask> completeCandidates = new HashSet<>();
+
+        cpWaitGrps.removeIf(completeCandidates::add);
+
+        cpCtx.finishedStateFut().listen(
+            f -> {
+                // Retry if error occurs.
+                if (f.error() != null || f.isCancelled()) {
+                    cpWaitGrps.addAll(completeCandidates);
+
+                    return;
+                }
+
+                lock.lock();
+
+                try {
+                    for (GroupScanTask grpScanTask : completeCandidates) {
+                        grps.remove(grpScanTask.groupId());
+
+                        grpScanTask.onDone();
+
+                        if (log.isInfoEnabled())
+                            log.info("Cache group reencryption is finished [grpId=" + grpScanTask.groupId() + "]");
+                    }
+
+                    if (!grps.isEmpty())
+                        return;
+
+                    ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).
+                        removeCheckpointListener(this);
+                }
+                finally {
+                    lock.unlock();
+                }
+            }
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public void beforeCheckpointBegin(Context cpCtx) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onMarkCheckpointBegin(Context ctx) {
+        // No-op.
+    }
+
+    /**
+     * Schedule scanning partitions.
+     *
+     * @param grpId Cache group ID.
+     */
+    public IgniteInternalFuture<Void> schedule(int grpId) throws IgniteCheckedException {
+        CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+        if (grp == null || !grp.affinityNode()) {
+            if (log.isInfoEnabled())
+                log.info("Skip reencryption, cache group doesn't exist on the local node [grp=" + grpId + "]");
+
+            return new GridFinishedFuture<>();
+        }
+
+        lock.lock();
+
+        try {
+            if (stopped)
+                throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
+
+            if (grps.isEmpty())
+                ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).addCheckpointListener(this);
+
+            GroupScanTask prevState = grps.get(grpId);
+
+            if (prevState != null && !prevState.isDone()) {
+                if (log.isDebugEnabled())
+                    log.debug("Reencryption already scheduled [grpId=" + grpId + "]");
+
+                return prevState;
+            }
+
+            Set<Integer> parts = new HashSet<>();
+
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) {
+                    if (ctx.encryption().getEncryptionState(grpId, partId) == 0) {
+                        if (log.isDebugEnabled())
+                            log.debug("Skipping partition reencryption [grp=" + grpId + ", p=" + partId + "]");
+
+                        return;
+                    }
+
+                    parts.add(partId);
+                }
+            });
+
+            GroupScanTask grpScan = new GroupScanTask(grp, parts);
+
+            singleExecSvc.submit(grpScan);
+
+            if (log.isInfoEnabled())
+                log.info("Scheduled reencryption [grpId=" + grpId + "]");
+
+            grps.put(grpId, grpScan);
+
+            return grpScan;
+        }
+        finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @return Future that will be completed when all partitions have been scanned and pages have been written to disk.
+     */
+    public IgniteInternalFuture<Void> statusFuture(int grpId) {
+        GroupScanTask grpScanTask = grps.get(grpId);
+
+        return grpScanTask == null ? new GridFinishedFuture<>() : grpScanTask;
+    }
+
+    /**
+     * Shutdown scanning and disable new tasks scheduling.
+     */
+    public void stop() throws IgniteCheckedException {
+        lock.lock();
+
+        try {
+            stopped = true;
+
+            for (GroupScanTask grpScanTask : grps.values())
+                grpScanTask.cancel();
+
+            if (singleExecSvc != null)
+                singleExecSvc.shutdownNow();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Stop scannig the specified partition.
+     *
+     * @param grpId Cache group ID.
+     * @param partId Partition ID.
+     * @return {@code True} if reencryption was cancelled.
+     */
+    public boolean excludePartition(int grpId, int partId) {
+        GroupScanTask grpScanTask = grps.get(grpId);
+
+        if (grpScanTask == null)
+            return false;
+
+        return grpScanTask.excludePartition(partId);
+    }
+
+    /**
+     * Collect current number of pages in the specified cache group.
+     *
+     * @param grp Cache group.
+     * @return Partitions with current page count.
+     * @throws IgniteCheckedException If failed.
+     */
+    public long[] pagesCount(CacheGroupContext grp) throws IgniteCheckedException {
+        // The last element of the array is used to store the status of the index partition.
+        long[] partStates = new long[grp.affinity().partitions() + 1];
+
+        ctx.cache().context().database().checkpointReadLock();
+
+        try {
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) throws IgniteCheckedException {
+                    int pagesCnt = ctx.cache().context().pageStore().pages(grp.groupId(), partId);
+
+                    partStates[Math.min(partId, partStates.length - 1)] = pagesCnt;
+                }
+            });
+        } finally {
+            ctx.cache().context().database().checkpointReadUnlock();
+        }
+
+        return partStates;
+    }
+
+    /**
+     * @param grp Cache group.
+     * @param hnd Partition handler.
+     */
+    private void forEachPageStore(CacheGroupContext grp, IgniteInClosureX<Integer> hnd) throws IgniteCheckedException {
+        int parts = grp.affinity().partitions();
+
+        IgnitePageStoreManager pageStoreMgr = ctx.cache().context().pageStore();
+
+        for (int p = 0; p < parts; p++) {
+            if (!pageStoreMgr.exists(grp.groupId(), p))
+                continue;
+
+            hnd.applyx(p);
+        }
+
+        hnd.applyx(PageIdAllocator.INDEX_PARTITION);
+    }
+
+    /**
+     * Cache group partition scanning task.
+     */
+    private class GroupScanTask extends GridFutureAdapter<Void> implements Runnable {
+        /** Cache group ID. */
+        private final CacheGroupContext grp;
+
+        /** Partition IDs. */
+        private final Set<Integer> parts;
+
+        /** Page memory. */
+        private final PageMemoryEx pageMem;
+
+        /**
+         * @param grp Cache group.
+         */
+        public GroupScanTask(CacheGroupContext grp, Set<Integer> parts) {
+            this.grp = grp;
+            this.parts = new GridConcurrentHashSet<>(parts);
+
+            pageMem = (PageMemoryEx)grp.dataRegion().pageMemory();
+        }
+
+        /** {@inheritDoc} */
+        @Override public synchronized boolean cancel() throws IgniteCheckedException {
+            return onCancelled();
+        }
+
+        /**
+         * Stop reencryption of the specified partition.
+         *
+         * @param partId Partition ID.
+         * @return {@code True} if reencryption was cancelled.
+         */
+        public synchronized boolean excludePartition(int partId) {
+            return parts.remove(partId);
+        }
+
+        /**
+         * @return Cache group ID.
+         */
+        public int groupId() {
+            return grp.groupId();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void run() {
+            try {
+                for (int partId : parts) {
+                    long state = ctx.encryption().getEncryptionState(grp.groupId(), partId);
+
+                    if (state == 0)
+                        continue;
+
+                    scanPartition(partId, ReencryptStateUtils.pageIndex(state), ReencryptStateUtils.pageCount(state));
+
+                    if (isDone())
+                        return;
+                }
+
+                boolean added = cpWaitGrps.add(this);

Review comment:
       cpWaitGrps is a collection of groups waiting for a checkpoint to start.
   
   PageScanner performs the following steps:
   1. mark all existing (at the moment) pages in the group as dirty
   2. wait for the start of the checkpoint (onCheckpointBegin)
   3. wait for finishedStateFut() of the current checkpoint to complete
   
   I though that after finishedStateFut finishes, all pages (dirty at the time of starting the checkpoint) are written to disk, am I missing something?




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupPageScanner.java
##########
@@ -0,0 +1,567 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.EncryptionConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.util.BasicRateLimiter;
+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.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+import org.apache.ignite.thread.OomExceptionHandler;
+
+import static org.apache.ignite.internal.util.IgniteUtils.MB;
+
+/**
+ * Cache group page stores scanner.
+ * Scans a range of pages and marks them as dirty to re-encrypt them with the last encryption key on disk.
+ */
+public class CacheGroupPageScanner implements DbCheckpointListener {
+    /** Thread prefix for scanning tasks. */
+    private static final String REENCRYPT_THREAD_PREFIX = "reencrypt";
+
+    /** Encryption configuration. */
+    private final EncryptionConfiguration encrCfg;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to group scanning context. */
+    private final Map<Integer, GroupScanContext> grps = new ConcurrentHashMap<>();
+
+    /** Queue of groups waiting for a checkpoint. */
+    private final Queue<Integer> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Executor to start partition scan tasks. */
+    private final IgniteThreadPoolExecutor execSvc;
+
+    /** Page scanning speed limiter. */
+    private final BasicRateLimiter limiter;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupPageScanner(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        encrCfg = ctx.config().getDataStorageConfiguration().getEncryptionConfiguration();
+
+        execSvc = new IgniteThreadPoolExecutor(REENCRYPT_THREAD_PREFIX,
+            ctx.igniteInstanceName(),
+            encrCfg.getReencryptionThreadCnt(),
+            encrCfg.getReencryptionThreadCnt(),
+            IgniteConfiguration.DFLT_THREAD_KEEP_ALIVE_TIME,
+            new LinkedBlockingQueue<>(),
+            GridIoPolicy.SYSTEM_POOL,
+            new OomExceptionHandler(ctx));
+
+        execSvc.allowCoreThreadTimeOut(true);
+
+        DataStorageConfiguration dsCfg = ctx.config().getDataStorageConfiguration();
+
+        limiter = CU.isPersistenceEnabled(dsCfg) && encrCfg.getReencryptionRateLimit() > 0 ?
+            new BasicRateLimiter(encrCfg.getReencryptionRateLimit() * MB / dsCfg.getPageSize()) : null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCheckpointBegin(Context cpCtx) {
+        Set<Integer> completeCandidates = new HashSet<>();
+
+        Integer grpId;
+
+        while ((grpId = cpWaitGrps.poll()) != null)
+            completeCandidates.add(grpId);
+
+        cpCtx.finishedStateFut().listen(
+            f -> {
+                if (f.error() != null || f.isCancelled()) {
+                    cpWaitGrps.addAll(completeCandidates);
+
+                    return;
+                }
+
+                lock.lock();
+
+                try {
+                    for (int groupId : completeCandidates) {
+                        GroupScanContext scanCtx = grps.remove(groupId);
+
+                        boolean finished = scanCtx.finish();
+
+                        assert finished : groupId;
+
+                        if (log.isInfoEnabled())
+                            log.info("Cache group reencryption is finished [grpId=" + groupId + "]");
+                    }
+
+                    if (!grps.isEmpty())
+                        return;
+
+                    ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).
+                        removeCheckpointListener(this);
+                }
+                finally {
+                    lock.unlock();
+                }
+            }
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public void beforeCheckpointBegin(Context cpCtx) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onMarkCheckpointBegin(Context ctx) {
+        // No-op.
+    }
+
+    /**
+     * @return {@code True} If reencryption is disabled.
+     */
+    public boolean disabled() {
+        return encrCfg.isReencryptionDisabled();
+    }
+
+    /**
+     * Schedule scanning partitions.
+     *
+     * @param grpId Cache group ID.
+     */
+    public IgniteInternalFuture<Void> schedule(int grpId) throws IgniteCheckedException {
+        if (disabled())
+            throw new IgniteCheckedException("Reencryption is disabled.");
+
+        CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+        if (grp == null) {
+            if (log.isDebugEnabled())
+                log.debug("Skip reencryption, cache group was destroyed [grp=" + grpId + "]");
+
+            return new GridFinishedFuture<>();
+        }
+
+        lock.lock();
+
+        try {
+            if (stopped)
+                throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
+
+            if (grps.isEmpty())
+                ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).addCheckpointListener(this);
+
+            GroupScanContext prevState = grps.get(grpId);
+
+            if (prevState != null) {
+                if (log.isDebugEnabled())
+                    log.debug("Reencryption already scheduled [grpId=" + grpId + "]");
+
+                return prevState.finishFuture();
+            }
+
+            GroupScanContext ctx0 = new GroupScanContext(grpId);
+
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) {
+                    if (ctx.encryption().getEncryptionState(grpId, partId) == 0) {
+                        if (log.isDebugEnabled())
+                            log.debug("Skipping partition reencryption [grp=" + grpId + ", p=" + partId + "]");
+
+                        return;
+                    }
+
+                    PageStoreScanTask scanTask = new PageStoreScanTask(ctx0, partId);
+
+                    ctx0.add(partId, scanTask);
+
+                    execSvc.submit(scanTask);
+                }
+            });
+
+            ctx0.initialize().listen(f -> {
+                Throwable t = f.error();
+
+                if (t != null) {
+                    log.error("Reencryption is failed [grpId=" + grpId + "]", t);
+
+                    ctx0.fail(t);
+
+                    return;
+                }
+
+                boolean added = cpWaitGrps.offer(grpId);
+
+                assert added;
+            });
+
+            if (log.isInfoEnabled())
+                log.info("Scheduled reencryption [grpId=" + grpId + "]");
+
+            grps.put(grpId, ctx0);
+
+            return ctx0.finishFuture();
+        }
+        finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @return Future that will be completed when all partitions have been scanned and pages have been written to disk.
+     */
+    public IgniteInternalFuture<Void> statusFuture(int grpId) {
+        GroupScanContext ctx0 = grps.get(grpId);
+
+        return ctx0 == null ? new GridFinishedFuture<>() : ctx0.finishFuture();
+    }
+
+    /**
+     * Shutdown scanning and disable new tasks scheduling.
+     */
+    public void stop() throws IgniteCheckedException {
+        lock.lock();
+
+        try {
+            stopped = true;
+
+            for (GroupScanContext ctx0 : grps.values())
+                ctx0.finishFuture().cancel();
+
+            execSvc.shutdown();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Stop scannig the specified partition.
+     *
+     * @param grpId Cache group ID.
+     * @param partId Partition ID.
+     * @return {@code True} if reencryption was cancelled.
+     * @throws IgniteCheckedException If failed.
+     */
+    public boolean cancel(int grpId, int partId) throws IgniteCheckedException {
+        GroupScanContext ctx = grps.get(grpId);
+
+        if (ctx == null)
+            return false;
+
+        return ctx.cancel(partId);
+    }
+
+    /**
+     * Collect current number of pages in the specified cache group.
+     *
+     * @param grp Cache group.
+     * @return Map of partitions with current page count.
+     * @throws IgniteCheckedException If failed.
+     */
+    public Map<Integer, Long> pagesCount(CacheGroupContext grp) throws IgniteCheckedException {
+        Map<Integer, Long> partStates = new HashMap<>();
+
+        ctx.cache().context().database().checkpointReadLock();
+
+        try {
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) throws IgniteCheckedException {
+                    int pagesCnt = ctx.cache().context().pageStore().pages(grp.groupId(), partId);
+
+                    partStates.put(partId, (long)pagesCnt);
+                }
+            });
+        } finally {
+            ctx.cache().context().database().checkpointReadUnlock();
+        }
+
+        return partStates;
+    }
+
+    /**
+     * @param grp Cache group.
+     * @param hnd Partition handler.
+     */
+    private void forEachPageStore(CacheGroupContext grp, IgniteInClosureX<Integer> hnd) throws IgniteCheckedException {
+        int parts = grp.affinity().partitions();
+
+        IgnitePageStoreManager pageStoreMgr = ctx.cache().context().pageStore();
+
+        for (int p = 0; p < parts; p++) {
+            if (!pageStoreMgr.exists(grp.groupId(), p))
+                continue;
+
+            hnd.applyx(p);
+        }
+
+        hnd.applyx(PageIdAllocator.INDEX_PARTITION);
+    }
+
+    /**
+     * Cache group scanning context.
+     */
+    private static class GroupScanContext {
+        /** Partition scanning futures. */
+        private final Map<Integer, IgniteInternalFuture<Void>> futMap = new ConcurrentHashMap<>();
+
+        /** Compound future, that will be completed when all partitions scanned. */
+        private final GridCompoundFuture<Void, Void> compFut = new GridCompoundFuture<>();
+
+        /** Cache group ID. */
+        private final int grpId;
+
+        /** Future that ends after all partitions are done and a checkpoint is finished. */
+        private final GridFutureAdapter<Void> cpFut = new GridFutureAdapter<Void>() {
+            @Override public boolean cancel() throws IgniteCheckedException {
+                compFut.cancel();
+
+                return onDone(null, null, true);
+            }
+        };
+
+        /**
+         * @param grpId Cache group ID.
+         */
+        public GroupScanContext(int grpId) {
+            this.grpId = grpId;
+        }
+
+        /**
+         * @return Cache group ID.
+         */
+        public int groupId() {
+            return grpId;
+        }
+
+        /**
+         * @param partId Partition ID.
+         * @param fut Partition scanning future.
+         */
+        public void add(int partId, IgniteInternalFuture<Void> fut) {
+            compFut.add(fut);
+
+            futMap.put(partId, fut);
+        }
+
+        /**
+         * @return Compound future, that will be completed when all partitions scanned.
+         */
+        public IgniteInternalFuture<Void> initialize() {
+            return compFut.markInitialized();
+        }
+
+        /**
+         * @return Future that ends after all partitions are done and a checkpoint is finished.
+         */
+        public IgniteInternalFuture<Void> finishFuture() {
+            return cpFut;
+        }
+
+        /**
+         * Finish reencryption future.
+         *
+         * @return {@code True} if the future was finished by this call.
+         */
+        public boolean finish() {
+            return cpFut.onDone(compFut.result());
+        }
+
+        /**
+         * Stop reencryption of the specified partition.
+         *
+         * @param partId Partition ID.
+         * @return {@code True} if reencryption was cancelled.
+         * @throws IgniteCheckedException If failed.
+         */
+        public boolean cancel(int partId) throws IgniteCheckedException {
+            IgniteInternalFuture<Void> fut = futMap.get(partId);
+
+            if (fut == null)
+                return false;
+
+            return fut.cancel();
+        }
+
+        /**
+         * @param t Throwable.
+         */
+        public void fail(Throwable t) {
+            cpFut.onDone(t);
+        }
+    }
+
+    /**
+     * Page store scanning task.
+     */
+    private class PageStoreScanTask extends GridFutureAdapter<Void> implements Runnable {
+        /** Partiion ID. */
+        private final int partId;
+
+        /** Cache group scan context. */
+        private final GroupScanContext scanCtx;
+
+        /**
+         * @param scanCtx Cache group scanning context.
+         * @param partId Partition ID.
+         */
+        public PageStoreScanTask(GroupScanContext scanCtx, int partId) {
+            this.scanCtx = scanCtx;
+            this.partId = partId;
+        }
+
+        /** {@inheritDoc} */
+        @Override public synchronized boolean cancel() throws IgniteCheckedException {
+            return onDone();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void run() {
+            try {
+                int grpId = scanCtx.groupId();
+
+                CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+                if (grp == null) {
+                    onDone();
+
+                    return;
+                }
+
+                PageMemoryEx pageMem = (PageMemoryEx)grp.dataRegion().pageMemory();
+                long metaPageId = pageMem.partitionMetaPageId(grpId, partId);
+                long state = ctx.encryption().getEncryptionState(grpId, partId);
+                int batchSize = encrCfg.getReencryptionBatchSize();
+
+                int off = (int)(state >> Integer.SIZE);
+                int cnt = (int)state;

Review comment:
       Added utility class ReencryptStateUtils 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java
##########
@@ -258,6 +258,9 @@
     /** */
     public static final short T_MARKER_PAGE = 33;
 
+    /** */
+    public static final short T_INDEX_META = 34;

Review comment:
       Yes, we adding two int fields into two pages.
   
   We have 2 types of memory pages:
   T_META (index partition)
   T_PART_META (other partition)
   
   We want to add 2 ints into each of them, but don't forget that IO for T_PART_META extends IO for T_META.
   
   In my first approach I added 2 ints into T_META, and added PagePartitionMetaIOV3 for T_PART_META.
   But since PagePartitionMetaIO "extends" PageMetaIO all offsets in PagePartitionMetaIO was shifted and there was a problem with inheritance (for example in PagePartitionMetaIO/V2 new methods were visible, all offsets was redefined in PagePartitionMetaIOV3), you can imagine what it looked like in commit with the second approach https://github.com/apache/ignite/pull/7941/commits/036b931bd40e2ffbd519779fde86fae7343da23e
   
   With current approach we simply create the new IO that is only used for the "index" partition.
   
   If you can suggest a better approach how for storing 2 extra int's itn the index meta page without losing binary compatibility, please describe.
   
   p.s. I explained this change in wiki https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=95652384#TDE.Phase3.Cachekeyrotation.-PageMetaIOandPagePartitionMetaIO

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java
##########
@@ -258,6 +258,9 @@
     /** */
     public static final short T_MARKER_PAGE = 33;
 
+    /** */
+    public static final short T_INDEX_META = 34;

Review comment:
       Yes, we adding two int fields into two pages.
   
   We have 2 types of memory pages:
   T_META (index partition)
   T_PART_META (other partition)
   
   We want to add 2 ints into each of them, but don't forget that IO for T_PART_META extends IO for T_META.
   
   In my first approach I added 2 ints into T_META, and added PagePartitionMetaIOV3 for T_PART_META.
   But since PagePartitionMetaIO "extends" PageMetaIO all offsets in PagePartitionMetaIO was shifted and there was a problem with inheritance (for example in PagePartitionMetaIO/V2 new methods were visible, all offsets was redefined in PagePartitionMetaIOV3), you can imagine what it looked like in commit with the second approach https://github.com/apache/ignite/pull/7941/commits/036b931bd40e2ffbd519779fde86fae7343da23e
   
   With current approach we simply create the new IO that is only used for the "index" partition.
   
   If you can suggest a better approach for storing 2 extra int's in the index meta page without losing binary compatibility, please describe.
   
   p.s. I explained this change in wiki https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=95652384#TDE.Phase3.Cachekeyrotation.-PageMetaIOandPagePartitionMetaIO

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/EncryptedRecord.java
##########
@@ -43,7 +43,7 @@ public EncryptedRecord(int grpId, RecordType plainRecType) {
 
     /** {@inheritDoc} */
     @Override public RecordType type() {
-        return RecordType.ENCRYPTED_RECORD;
+        return RecordType.ENCRYPTED_RECORD_V2;

Review comment:
       For binary compatibility.
   For ENCRYPTED_RECORD_V2 we reading 1 extra byte (encryption key identifier).
   (see change in RecordDataV1Serializer.readEncryptedData)

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java
##########
@@ -258,6 +258,9 @@
     /** */
     public static final short T_MARKER_PAGE = 33;
 
+    /** */
+    public static final short T_INDEX_META = 34;

Review comment:
       If I understood you correctly, you suggest to add PageMetaIOV2 (instead of PageIndexMetaIO)?
   PagePartitionMeta will continue to extend PageMetaIO(not v2), is this correct?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java
##########
@@ -258,6 +258,9 @@
     /** */
     public static final short T_MARKER_PAGE = 33;
 
+    /** */
+    public static final short T_INDEX_META = 34;

Review comment:
       If I understood you correctly, you suggest to add PageMetaIOV2 (instead of PageIndexMetaIO) and keep page type T_META?
   PagePartitionMeta will continue to extend PageMetaIO(not v2), is this correct?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/EncryptedRecord.java
##########
@@ -43,7 +43,7 @@ public EncryptedRecord(int grpId, RecordType plainRecType) {
 
     /** {@inheritDoc} */
     @Override public RecordType type() {
-        return RecordType.ENCRYPTED_RECORD;
+        return RecordType.ENCRYPTED_RECORD_V2;

Review comment:
       Thanks for the suggestion, as far as I understand you are suggesting not to modify any existing WAL record classes.
   Could you please give an example of a compatibility issue that we may face in the future (in theory)?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/EncryptedRecord.java
##########
@@ -43,7 +43,7 @@ public EncryptedRecord(int grpId, RecordType plainRecType) {
 
     /** {@inheritDoc} */
     @Override public RecordType type() {
-        return RecordType.ENCRYPTED_RECORD;
+        return RecordType.ENCRYPTED_RECORD_V2;

Review comment:
       Thanks for the suggestion, as far as I understand you are suggesting not to modify any existing WAL record classes.
   Could you please give an example of a compatibility issue that we may face in the future (in theory) that can be avoided by keeping the old "unused" classes?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java
##########
@@ -258,6 +258,9 @@
     /** */
     public static final short T_MARKER_PAGE = 33;
 
+    /** */
+    public static final short T_INDEX_META = 34;

Review comment:
       Thanks, I believe I've fixed it.




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

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



[GitHub] [ignite] alex-plekhanov commented on a change in pull request #7941: IGNITE-12843

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #7941:
URL: https://github.com/apache/ignite/pull/7941#discussion_r494888623



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/encryption/AbstractEncryptionTest.java
##########
@@ -271,4 +313,185 @@ protected boolean checkMasterKeyName(String name) {
 
         return true;
     }
+
+    /**
+     * Load data into cache "{@link #cacheName()}" using node "{@link #GRID_0}".
+     *
+     * @param cnt Count of entries.
+     */
+    protected void loadData(int cnt) {
+        loadData(cacheName(), cnt);
+    }
+
+    /**
+     * Load data into cache using node "{@link #GRID_0}".
+     *
+     * @param cnt Count of entries.
+     * @param cacheName Cache name.
+     */
+    protected void loadData(String cacheName, int cnt) {
+        info("Loading " + cnt + " entries into " + cacheName);
+
+        int start = grid(GRID_0).cache(cacheName).size();
+
+        try (IgniteDataStreamer<Long, Object> streamer = grid(GRID_0).dataStreamer(cacheName)) {
+            for (long i = start; i < (cnt + start); i++)
+                streamer.addData(i, generateValue(i));
+        }
+
+        info("Load data finished");
+    }
+
+    /**
+     * Ensures that all pages of page store have expected encryption key identifier.
+     *
+     * @param grpId Cache group ID.
+     * @param keyId Encryption key ID.
+     * @param timeout Timeout to wait for encryption to complete.
+     * @throws Exception If failed.
+     */
+    protected void checkGroupKey(int grpId, int keyId, long timeout) throws Exception {
+        awaitEncryption(G.allGrids(), grpId, timeout);
+
+        for (Ignite g : G.allGrids()) {
+            info("Validating encryption key [node=" + g.cluster().localNode().id() + ", grp=" + grpId + "]");
+
+            IgniteEx grid = (IgniteEx)g;
+
+            if (grid.context().clientNode())
+                continue;
+
+            GridEncryptionManager encryption = grid.context().encryption();
+
+            assertEquals(grid.localNode().id().toString(), (byte)keyId, encryption.groupKey(grpId).id());
+
+            IgniteInternalFuture<Void> fut = encryption.reencryptionFuture(grpId);
+
+            // The future will be completed after the checkpoint, forcecheckpoint does nothing
+            // if the checkpoint has already been scheduled.
+            GridTestUtils.waitForCondition(() -> {
+                if (fut.isDone())
+                    return true;
+
+                try {
+                    forceCheckpoint(g);
+                }
+                catch (IgniteCheckedException e) {
+                    throw new RuntimeException(e);
+                }
+
+                return fut.isDone();
+            }, timeout);
+
+            assertTrue(fut.isDone());
+
+            CacheGroupContext grp = grid.context().cache().cacheGroup(grpId);
+
+            List<Integer> parts = IntStream.range(0, grp.shared().affinity().affinity(grpId).partitions())
+                .boxed().collect(Collectors.toList());
+
+            parts.add(INDEX_PARTITION);
+
+            int realPageSize = grp.dataRegion().pageMemory().realPageSize(grpId);
+            int encryptionBlockSize = grp.shared().kernalContext().config().getEncryptionSpi().blockSize();
+
+            for (int p : parts) {
+                FilePageStore pageStore =
+                    (FilePageStore)((FilePageStoreManager)grp.shared().pageStore()).getStore(grpId, p);
+
+                if (!pageStore.exists())
+                    continue;
+
+                long state = grid.context().encryption().getEncryptionState(grpId, p);
+
+                String msg = String.format("p=%d, off=%d, total=%d",
+                    p, ReencryptStateUtils.pageIndex(state), ReencryptStateUtils.pageCount(state));
+
+                assertEquals(msg, 0, ReencryptStateUtils.pageCount(state));
+                assertEquals(msg, 0, ReencryptStateUtils.pageIndex(state));
+
+                long startPageId = PageIdUtils.pageId(p, PageIdAllocator.FLAG_DATA, 0);
+
+                int pagesCnt = pageStore.pages();
+                int pageSize = pageStore.getPageSize();
+
+                ByteBuffer pageBuf = ByteBuffer.allocate(pageSize);
+
+                Path path = new File(pageStore.getFileAbsolutePath()).toPath();
+
+                try (FileChannel ch = FileChannel.open(path, StandardOpenOption.READ)) {
+                    for (int n = 0; n < pagesCnt; n++) {
+                        long pageId = startPageId + n;
+                        long pageOff = pageStore.pageOffset(pageId);
+
+                        pageBuf.position(0);
+
+                        ch.position(pageOff);
+                        ch.read(pageBuf);
+
+                        pageBuf.position(realPageSize + encryptionBlockSize);
+
+                        // If crc present

Review comment:
       Point at the end.
   Why there can be pages without CRC?




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GroupKeyChangeProcess.java
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteFeatures;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.managers.encryption.GridEncryptionManager.EmptyResult;
+import org.apache.ignite.internal.managers.encryption.GridEncryptionManager.KeyChangeFuture;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
+import org.apache.ignite.internal.util.distributed.DistributedProcess;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.future.IgniteFinishedFutureImpl;
+import org.apache.ignite.internal.util.future.IgniteFutureImpl;
+import org.apache.ignite.internal.util.typedef.F;
+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.IgniteFuture;
+import org.apache.ignite.lang.IgniteFutureCancelledException;
+
+import static org.apache.ignite.internal.IgniteFeatures.CACHE_GROUP_KEY_CHANGE;
+import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.CACHE_GROUP_KEY_CHANGE_FINISH;
+import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.CACHE_GROUP_KEY_CHANGE_PREPARE;
+
+/**
+ * Two phase distributed process, that performs cache group encryption key rotation.
+ */
+class GroupKeyChangeProcess {

Review comment:
       javadoc updated to
   `
   A two-phase distributed process that rotates the encryption keys of specified cache groups and initiates
   re-encryption of those cache groups.
   `




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GroupKeyChangeProcess.java
##########
@@ -0,0 +1,350 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteFeatures;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.managers.encryption.GridEncryptionManager.EmptyResult;
+import org.apache.ignite.internal.managers.encryption.GridEncryptionManager.KeyChangeFuture;
+import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor;
+import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
+import org.apache.ignite.internal.util.distributed.DistributedProcess;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.future.IgniteFinishedFutureImpl;
+import org.apache.ignite.internal.util.future.IgniteFutureImpl;
+import org.apache.ignite.internal.util.typedef.F;
+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.IgniteFuture;
+import org.apache.ignite.lang.IgniteFutureCancelledException;
+
+import static org.apache.ignite.internal.IgniteFeatures.CACHE_GROUP_KEY_CHANGE;
+import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.CACHE_GROUP_KEY_CHANGE_FINISH;
+import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.CACHE_GROUP_KEY_CHANGE_PREPARE;
+
+/**
+ * A two-phase distributed process that rotates the encryption keys of specified cache groups and initiates
+ * re-encryption of those cache groups.
+ */
+class GroupKeyChangeProcess {
+    /** Grid kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Cache group encyption key change prepare phase. */
+    private final DistributedProcess<ChangeCacheEncryptionRequest, EmptyResult> prepareGKChangeProc;
+
+    /** Cache group encyption key change perform phase. */
+    private final DistributedProcess<ChangeCacheEncryptionRequest, EmptyResult> performGKChangeProc;
+
+    /** Group encryption keys. */
+    private final CacheGroupEncryptionKeys keys;
+
+    /** Cache group key change future. */
+    private volatile GroupKeyChangeFuture fut;
+
+    /** Cache group key change request. */
+    private volatile ChangeCacheEncryptionRequest req;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    GroupKeyChangeProcess(GridKernalContext ctx, CacheGroupEncryptionKeys keys) {
+        this.ctx = ctx;
+        this.keys = keys;
+
+        prepareGKChangeProc =
+            new DistributedProcess<>(ctx, CACHE_GROUP_KEY_CHANGE_PREPARE, this::prepare, this::finishPrepare);
+        performGKChangeProc =
+            new DistributedProcess<>(ctx, CACHE_GROUP_KEY_CHANGE_FINISH, this::perform, this::finishPerform);
+    }
+
+    /**
+     * @return {@code True} if operation is still in progress.
+     */
+    public boolean inProgress() {
+        return req != null;
+    }
+
+    /**
+     * @param msg Error message.
+     */
+    public void cancel(String msg) {
+        GridFutureAdapter<Void> keyChangeFut = fut;
+
+        if (keyChangeFut != null && !keyChangeFut.isDone())
+            keyChangeFut.onDone(new IgniteFutureCancelledException(msg));
+    }
+
+    /**
+     * Starts cache group encryption key change process.
+     *
+     * @param cacheOrGrpNames Cache or group names.
+     */
+    public IgniteFuture<Void> start(Collection<String> cacheOrGrpNames) {
+        if (ctx.clientNode())
+            throw new UnsupportedOperationException("Client and daemon nodes can not perform this operation.");
+
+        if (!IgniteFeatures.allNodesSupports(ctx.grid().cluster().nodes(), CACHE_GROUP_KEY_CHANGE))
+            throw new IllegalStateException("Not all nodes in the cluster support this operation.");
+
+        if (!ctx.state().clusterState().state().active())
+            throw new IgniteException("Operation was rejected. The cluster is inactive.");
+
+        IgniteInternalFuture<Void> fut0 = fut;
+
+        if (fut0 != null && !fut0.isDone()) {
+            return new IgniteFinishedFutureImpl<>(new IgniteException("Cache group key change was rejected. " +
+                "The previous change was not completed."));
+        }
+
+        int[] grpIds = new int[cacheOrGrpNames.size()];
+        byte[] keyIds = new byte[grpIds.length];
+
+        int n = 0;
+
+        for (String cacheOrGroupName : cacheOrGrpNames) {
+            CacheGroupDescriptor grpDesc = ctx.cache().cacheGroupDescriptor(CU.cacheId(cacheOrGroupName));
+
+            if (grpDesc == null) {
+                DynamicCacheDescriptor cacheDesc = ctx.cache().cacheDescriptor(cacheOrGroupName);
+
+                if (cacheDesc == null) {
+                    throw new IgniteException("Cache group key change was rejected. " +
+                        "Cache or group \"" + cacheOrGroupName + "\" doesn't exists");
+                }
+
+                int grpId = cacheDesc.groupId();
+
+                grpDesc = ctx.cache().cacheGroupDescriptor(grpId);
+
+                if (grpDesc.sharedGroup()) {
+                    throw new IgniteException("Cache group key change was rejected. " +
+                        "Cache or group \"" + cacheOrGroupName + "\" is a part of group \"" +
+                        grpDesc.groupName() + "\". Provide group name instead of cache name for shared groups.");
+                }
+            }
+
+            if (!grpDesc.config().isEncryptionEnabled()) {
+                throw new IgniteException("Cache group key change was rejected. " +
+                    "Cache or group \"" + cacheOrGroupName + "\" is not encrypted.");
+            }
+
+            if (ctx.encryption().reencryptionInProgress(grpDesc.groupId())) {
+                throw new IgniteException("Cache group key change was rejected. " +
+                    "Cache group reencryption is in progress [grp=" + cacheOrGroupName + "]");
+            }
+
+            grpIds[n] = grpDesc.groupId();
+            keyIds[n] = (byte)(ctx.encryption().groupKey(grpDesc.groupId()).unsignedId() + 1);
+
+            n += 1;
+        }
+
+        byte[][] keys = ctx.encryption().createKeys(grpIds.length).get1().toArray(new byte[grpIds.length][]);
+
+        ChangeCacheEncryptionRequest req =
+            new ChangeCacheEncryptionRequest(grpIds, keys, keyIds, ctx.config().getEncryptionSpi().getMasterKeyName());
+
+        fut = new GroupKeyChangeFuture(req);
+
+        prepareGKChangeProc.start(req.requestId(), req);
+
+        return new IgniteFutureImpl<>(fut);
+    }
+
+    /**
+     * Validates existing keys.
+     *
+     * @param req Request.
+     * @return Result future.
+     */
+    private IgniteInternalFuture<EmptyResult> prepare(ChangeCacheEncryptionRequest req) {
+        if (ctx.clientNode())
+            return new GridFinishedFuture<>();
+
+        if (inProgress()) {
+            return new GridFinishedFuture<>(new IgniteException("Cache group key change was rejected. " +
+                "The previous change was not completed."));
+        }
+
+        this.req = req;
+
+        try {
+            for (int i = 0; i < req.groupIds().length; i++) {
+                int grpId = req.groupIds()[i];
+                int keyId = req.keyIds()[i] & 0xff;
+
+                if (ctx.encryption().reencryptionInProgress(grpId)) {
+                    return new GridFinishedFuture<>(new IgniteException("Cache group key change was rejected. " +
+                            "Cache group reencryption is in progress [grpId=" + grpId + "]"));
+                }
+
+                List<Integer> keyIds = ctx.encryption().groupKeyIds(grpId);
+
+                if (keyIds == null) {
+                    return new GridFinishedFuture<>(new IgniteException("Cache group key change was rejected." +
+                            "Encrypted cache group not found [grpId=" + grpId + "]"));
+                }
+
+                GroupKey currKey = ctx.encryption().groupKey(grpId);
+
+                for (int locKeyId : keyIds) {
+                    if (locKeyId != keyId)
+                        continue;
+
+                    Long walSegment = keys.reservedSegment(grpId, keyId);
+
+                    // Can overwrite inactive key if it was added during prepare phase.
+                    if (walSegment == null && currKey.id() != (byte)keyId)
+                        continue;
+
+                    return new GridFinishedFuture<>(
+                        new IgniteException("Cache group key change was rejected. Cannot add new key identifier, " +
+                        "it's already present. There existing WAL segments that encrypted with this key [" +
+                        "grpId=" + grpId + ", newId=" + keyId + ", currId=" + currKey.unsignedId() +
+                        ", walSegment=" + walSegment + "]."));
+                }
+            }
+
+            return ctx.encryption().withMasterKeyChangeReadLock(() -> {
+                String curMasterKeyName = ctx.config().getEncryptionSpi().getMasterKeyName();
+
+                if (!curMasterKeyName.equals(req.masterKeyName())) {
+                    return new GridFinishedFuture<>(new IgniteException("Cache group key change was rejected. " +
+                        "Master key has been changed."));
+                }
+
+                for (int i = 0; i < req.groupIds().length; i++) {
+                    // Store new key as inactive.
+                    GroupKeyEncrypted grpKey = new GroupKeyEncrypted(req.keyIds()[i] & 0xff, req.keys()[i]);
+
+                    ctx.encryption().addGroupKey(req.groupIds()[i], grpKey);

Review comment:
       Because the master key may changes between prepare/perform stages and we'll not be able to decrypt received keys.
   I think I should add a comment about this, thanks.




----------------------------------------------------------------
This is an automated message from the 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] sk0x50 commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/EncryptedRecord.java
##########
@@ -43,7 +43,7 @@ public EncryptedRecord(int grpId, RecordType plainRecType) {
 
     /** {@inheritDoc} */
     @Override public RecordType type() {
-        return RecordType.ENCRYPTED_RECORD;
+        return RecordType.ENCRYPTED_RECORD_V2;

Review comment:
       Could you please describe in short why you need to change the type of this WAL record?
   WAL record was not changed itself, so it looks weird to me.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/EncryptedRecord.java
##########
@@ -43,7 +43,7 @@ public EncryptedRecord(int grpId, RecordType plainRecType) {
 
     /** {@inheritDoc} */
     @Override public RecordType type() {
-        return RecordType.ENCRYPTED_RECORD;
+        return RecordType.ENCRYPTED_RECORD_V2;

Review comment:
       > (see change in RecordDataV1Serializer.readEncryptedData)
   Yep, I saw this, but it does not explain why you modify the existing WAL record.
   
   In my understanding, WAL records must be immutable in order to avoid compatibility issues in the future.
   I would prefer adding a new WAL record (it can extend an existing one).
   Perhaps, RecordDataV1Serializer implementation will be cleaner in that case.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/EncryptedRecord.java
##########
@@ -43,7 +43,7 @@ public EncryptedRecord(int grpId, RecordType plainRecType) {
 
     /** {@inheritDoc} */
     @Override public RecordType type() {
-        return RecordType.ENCRYPTED_RECORD;
+        return RecordType.ENCRYPTED_RECORD_V2;

Review comment:
       > (see change in RecordDataV1Serializer.readEncryptedData)
   
   Yep, I saw this, but it does not explain why you modify the existing WAL record.
   
   In my understanding, WAL records must be immutable in order to avoid compatibility issues in the future.
   I would prefer adding a new WAL record (it can extend an existing one).
   Perhaps, RecordDataV1Serializer implementation will be cleaner in that case.




----------------------------------------------------------------
This is an automated message from the 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] NSAmelchev commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -627,10 +820,83 @@ public void groupKey(int grpId, byte[] encGrpKey) {
         return withMasterKeyChangeReadLock(() -> getSpi().getMasterKeyName());
     }
 
+    /** {@inheritDoc} */
+    @Override public IgniteFuture<Void> changeCacheGroupKey(Collection<String> cacheOrGrpNames) {
+        A.notEmpty(cacheOrGrpNames, "cacheOrGrpNames");
+
+        if (ctx.clientNode())
+            throw new UnsupportedOperationException("Client and daemon nodes can not perform this operation.");
+
+        if (!IgniteFeatures.allNodesSupports(ctx.grid().cluster().nodes(), CACHE_GROUP_KEY_CHANGE))
+            throw new IllegalStateException("Not all nodes in the cluster support this operation.");
+
+        if (!ctx.state().clusterState().active())
+            throw new IgniteException("Operation was rejected. The cluster is inactive.");
+
+        DiscoCache discoCache = ctx.discovery().discoCache();
+
+        int bltSize = discoCache.baselineNodes().size();
+        int bltOnline = discoCache.aliveBaselineNodes().size();
+
+        if (bltSize != bltOnline)
+            throw new IgniteException("Not all baseline nodes online [total=" + bltSize + ", online=" + bltOnline + "]");
+
+        int[] grpIds = new int[cacheOrGrpNames.size()];
+        byte[] keyIds = new byte[grpIds.length];
+        byte[][] keys = new byte[grpIds.length][];
+
+        int n = 0;
+
+        for (String cacheOrGroupName : cacheOrGrpNames) {
+            CacheGroupContext grp = ctx.cache().cacheGroup(CU.cacheId(cacheOrGroupName));
+
+            if (grp == null) {
+                IgniteInternalCache cache = ctx.cache().cache(cacheOrGroupName);
+
+                if (cache == null)
+                    throw new IgniteException("Cache or group \"" + cacheOrGroupName + "\" doesn't exists");
+
+                grp = cache.context().group();
+
+                if (grp.sharedGroup()) {
+                    throw new IgniteException("Cache or group \"" + cacheOrGroupName + "\" is a part of group " +
+                        grp.name() + ". Provide group name instead of cache name for shared groups.");
+                }
+            }
+
+            if (!grp.config().isEncryptionEnabled())
+                throw new IgniteException("Cache or group \"" + cacheOrGroupName + "\" is not encrypted.");
+
+            if (reencryptGroups.containsKey(grp.groupId()))
+                throw new IgniteException("Reencryption is in progress [grp=" + cacheOrGroupName + "]");
+
+            grpIds[n] = grp.groupId();
+            keyIds[n] = (byte)(groupKey(grp.groupId()).unsignedId() + 1);
+            keys[n] = getSpi().encryptKey(getSpi().create());

Review comment:
       Must be under master key change read lock




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupMetricsImpl.java
##########
@@ -175,6 +181,16 @@ public void onTopologyInitialized() {
         mreg.register("TotalAllocatedSize",
             this::getTotalAllocatedSize,
             "Total size of memory allocated for group, in bytes.");
+
+        if (ctx.config().isEncryptionEnabled()) {
+            mreg.register("ReencryptionFinished",

Review comment:
       I think it is better to move metrics to a separate issue.




----------------------------------------------------------------
This is an automated message from the 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] xtern commented on a change in pull request #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupEncryptionKeys.java
##########
@@ -0,0 +1,376 @@
+/*
+ * 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.managers.encryption;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.CopyOnWriteArrayList;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.encryption.EncryptionSpi;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Serves for managing encryption keys and related datastructure located in the heap.
+ */
+class CacheGroupEncryptionKeys {
+    /** Group encryption keys. */
+    private final Map<Integer, List<GroupKey>> grpKeys = new ConcurrentHashMap<>();
+
+    /**
+     * WAL segments encrypted with previous encryption keys prevent keys from being deleted
+     * until the associated segment is deleted.
+     */
+    private final Collection<TrackedWalSegment> trackedWalSegments = new ConcurrentLinkedQueue<>();

Review comment:
       It is possible that the old key will not be deleted until the next key change.
   
   For example, we start with encryption key with ID = 0.
   
   added new WAL segment 0 (encrypted with key ID = 0)
   ..
   added new WAL segment 10 (encrypted with key ID = 0)
   
   change encryption key ID = 1
   
   added new WAL segment 11 (encrypted with key ID = 1)
   ..
   added new WAL segment 15 (encrypted with key ID = 1)
   
   change encryption key ID = 2
   
   added new WAL segment 16 (encrypted with key ID = 2)
   ...
   
   and so on.
   
   So, in order to read segment 3 of WAL (for example, for delta rebalancing), we need to keep the key with ID = 0 until the segment 10 is removed from the disk.




----------------------------------------------------------------
This is an automated message from the 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 #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java
##########
@@ -258,6 +258,9 @@
     /** */
     public static final short T_MARKER_PAGE = 33;
 
+    /** */
+    public static final short T_INDEX_META = 34;

Review comment:
       Can you please explain why you needed new IO type instead of new version for existing type? You only add two fields, right? I don't get it then.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java
##########
@@ -258,6 +258,9 @@
     /** */
     public static final short T_MARKER_PAGE = 33;
 
+    /** */
+    public static final short T_INDEX_META = 34;

Review comment:
       I suggest class PageIndexIO being just T_META with version 2, see no problems with this approach. Just as now, two different classes will add two independent sets of new integers, that's fine. The only thing that's changed is that PageIndexIO will have the same type as its base class.
   
   Types should be immutable while pages are alive. This saves us from a lot of troubles, I wouldn't abandon this tradition. That's exactly what versions are for.
   
   BTW your current code has bugs, you don't mark meta page as dirty in "getOrAllocateCacheMetas" while upgrading, please fix it.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java
##########
@@ -258,6 +258,9 @@
     /** */
     public static final short T_MARKER_PAGE = 33;
 
+    /** */
+    public static final short T_INDEX_META = 34;

Review comment:
       I suggest class PageIndexIO being just T_META with version 2, see no problems with this approach. Just as now, two different classes will add two independent sets of new integers, that's fine. The only thing that's changed is that PageIndexIO will have the same type as its base class.
   
   Types should be immutable while pages are alive. This saves us from a lot of troubles, I wouldn't abandon this tradition. That's exactly what versions are for.
   
   BTW your current code has bugs, you don't mark meta page as dirty in "getOrAllocateCacheMetas" while upgrading, please fix it.
   
   EDIT: I guess you can add getters to base class and implement them in inheritors, while IOs that don't support them could just return 0 or throw UnsupportedOperationException

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java
##########
@@ -258,6 +258,9 @@
     /** */
     public static final short T_MARKER_PAGE = 33;
 
+    /** */
+    public static final short T_INDEX_META = 34;

Review comment:
       Yes, this is correct. Naming of the classes doesn't bother me too much so I was fine with "Indexing" instead of V2 (although that page is also referred as "cache group meta page" sometimes, like in PageMemoryEx#metaPageId, for example).
   
   I think I understand the core problem, inheritance between MetaIO and PartitionMetaIO should have been replaced with extension of the same abstract superclass back in the day. But with current faulty design people just get confused :(




----------------------------------------------------------------
This is an automated message from the 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] alex-plekhanov commented on a change in pull request #7941: IGNITE-12843

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #7941:
URL: https://github.com/apache/ignite/pull/7941#discussion_r494384542



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -821,11 +1181,57 @@ private void writeKeysToMetaStore(boolean writeAll) throws IgniteCheckedExceptio
         if (writeAll)
             metaStorage.write(MASTER_KEY_NAME_PREFIX, getSpi().getMasterKeyName());
 
-        for (Map.Entry<Integer, Serializable> entry : grpEncKeys.entrySet()) {
-            if (!writeAll && metaStorage.read(ENCRYPTION_KEY_PREFIX + entry.getKey()) != null)
+        if (!reencryptGroupsForced.isEmpty())
+            writeTrackedWalIdxsToMetaStore();
+
+        for (Integer grpId : grpKeys.groupIds()) {
+            if (!writeAll && !reencryptGroupsForced.containsKey(grpId) &&
+                metaStorage.read(ENCRYPTION_KEYS_PREFIX + grpId) != null)
                 continue;
 
-            writeToMetaStore(entry.getKey(), getSpi().encryptKey(entry.getValue()));
+            writeGroupKeysToMetaStore(grpId);
+        }
+    }
+
+    /**
+     * Writes cache group encryption keys to metastore.
+     *
+     * @param grpId Cache group ID.
+     */
+    private void writeGroupKeysToMetaStore(int grpId) throws IgniteCheckedException {
+        assert Thread.holdsLock(metaStorageMux);
+
+        if (metaStorage == null || !writeToMetaStoreEnabled || stopped)
+            return;
+
+        List<GroupKeyEncrypted> keysEncrypted = withMasterKeyChangeReadLock(() -> grpKeys.getAll(grpId));

Review comment:
       Different locks order, deadlock is possible. This method is invoked under metaStorageMux and then master key change log acquired, but in doChangeMasterKey() master key change lock acquired first and then metaStorageMux.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/WALRecord.java
##########
@@ -233,7 +233,25 @@
         TRACKING_PAGE_REPAIR_DELTA(61, PHYSICAL),
 
         /** Atomic out-of-order update. */
-        OUT_OF_ORDER_UPDATE(62, LOGICAL);
+        OUT_OF_ORDER_UPDATE(62, LOGICAL),
+
+        /** Encrypted WAL-record. */
+        ENCRYPTED_RECORD_V2(63, PHYSICAL),
+
+        /** Ecnrypted data record. */
+        ENCRYPTED_DATA_RECORD_V2(64, LOGICAL),
+
+        /** Master key change record containing multiple keys for single cache group. */
+        MASTER_KEY_CHANGE_RECORD_V2(65, LOGICAL),
+
+        /** Logical record to restart reencryption with the latest encryption key. */
+        REENCRYPTION_START_RECORD(66, LOGICAL),
+
+        /** Partition meta page delta record includes encryption status data. */
+        PARTITION_META_PAGE_UPDATE_COUNTERS_V3(67, PHYSICAL),

Review comment:
       Maybe `PARTITION_META_PAGE_DELTA_RECORD_V3`? WDYT?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -483,16 +534,18 @@ public void onLocalJoin() {
         if (dataBag.isJoiningNodeClient() || dataBag.commonDataCollectedFor(ENCRYPTION_MGR.ordinal()))
             return;
 
-        HashMap<Integer, byte[]> knownEncKeys = knownEncryptionKeys();
+        HashMap<Integer, GroupKeyEncrypted> knownEncKeys = grpKeys.getAll();

Review comment:
       As far as I understand master key can be changed concurrently with this method and getAll() encrypt keys with master key, so we should use  grpKeys.getAll() under master key change read lock (please pay attention to locks order to avoid deadlocks). At least in this method, perhaps in other methods too (please review also usages of getAll(grpId) and setGroupKeys methods, perhaps these methods should be invoked under lock too)   

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
##########
@@ -2863,11 +2869,16 @@ private RestoreLogicalState applyLogicalUpdates(
 
                         break;
 
-                    case MASTER_KEY_CHANGE_RECORD:
+                    case MASTER_KEY_CHANGE_RECORD_V2:

Review comment:
       Let's also keep `case MASTER_KEY_CHANGE_RECORD:`

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -811,6 +1124,53 @@ private void sendGenerateEncryptionKeyRequest(GenerateEncryptionKeyFuture fut) t
         ctx.io().sendToGridTopic(rndNode.id(), TOPIC_GEN_ENC_KEY, req, SYSTEM_POOL);
     }
 
+    /**
+     * @param grpIds Cache group IDs.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void startReencryption(Collection<Integer> grpIds) throws IgniteCheckedException {
+        if (pageScanner.disabled())
+            return;
+
+        for (int grpId : grpIds) {
+            IgniteInternalFuture<?> fut = pageScanner.schedule(grpId);
+
+            fut.listen(f -> {
+                try {
+                    f.get();

Review comment:
       Because exception driven flow control is an anti-pattern

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateIndexDataRecord.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.pagemem.wal.record.delta;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIndexMetaIO;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * Meta page delta record, includes encryption status data.
+ */
+public class MetaPageUpdateIndexDataRecord extends PageDeltaRecord {
+    /** Index of the last reencrypted page. */
+    private int encryptPageIdx;
+
+    /** Total pages to be reencrypted. */
+    private int encryptPageCnt;
+
+    /**
+     * @param grpId  Cache group ID.
+     * @param pageId Page ID.
+     * @param encryptPageIdx Index of the last reencrypted page.
+     * @param encryptPageCnt Total pages to be reencrypted.
+     */
+    public MetaPageUpdateIndexDataRecord(int grpId, long pageId, int encryptPageIdx, int encryptPageCnt) {
+        super(grpId, pageId);
+
+        this.encryptPageIdx = encryptPageIdx;
+        this.encryptPageCnt = encryptPageCnt;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        PageIndexMetaIO io = PageIndexMetaIO.VERSIONS.forPage(pageAddr);

Review comment:
       Perhaps here we can face with a not upgraded page (in case node failed after page upgrade and before first checkpoint)

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/util/BasicRateLimiterTest.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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;
+
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Test;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Rate limiter tests.
+ */
+public class BasicRateLimiterTest {

Review comment:
       Not included into any test suite

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -627,22 +720,104 @@ public void groupKey(int grpId, byte[] encGrpKey) {
         return withMasterKeyChangeReadLock(() -> getSpi().getMasterKeyName());
     }
 
+    /** {@inheritDoc} */
+    @Override public IgniteFuture<Void> changeCacheGroupKey(Collection<String> cacheOrGrpNames) {
+        A.notEmpty(cacheOrGrpNames, "cacheOrGrpNames");
+
+        synchronized (opsMux) {
+            if (stopped) {
+                return new IgniteFinishedFutureImpl<>(new IgniteException("Cache group key change was rejected. " +
+                    "Node is stopping."));
+            }
+
+            return grpKeyChangeProc.start(cacheOrGrpNames);
+        }
+    }
+
+    /**
+     * @param grpIds Cache group IDs.
+     * @param keyIds Encryption key IDs.
+     * @param keys Encryption keys.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected void changeCacheGroupKeyLocal(int[] grpIds, byte[] keyIds, byte[][] keys) throws IgniteCheckedException {
+        Map<Integer, Byte> encryptionStatus = U.newHashMap(grpIds.length);
+
+        for (int i = 0; i < grpIds.length; i++)
+            encryptionStatus.put(grpIds[i], keyIds[i]);
+
+        WALPointer ptr = ctx.cache().context().wal().log(new ReencryptionStartRecord(encryptionStatus));
+
+        if (ptr != null)
+            ctx.cache().context().wal().flush(ptr, false);
+
+        for (int i = 0; i < grpIds.length; i++) {
+            int grpId = grpIds[i];
+
+            CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+            if (grp == null)
+                continue;
+
+            int newKeyId = keyIds[i] & 0xff;
+
+            synchronized (metaStorageMux) {
+                // Set new key as key for writing.
+                GroupKey prevGrpKey = grpKeys.changeActiveKey(grpId, newKeyId);
+
+                writeGroupKeysToMetaStore(grpId);
+
+                if (ptr != null) {
+                    grpKeys.reserveWalKey(grpId, prevGrpKey.unsignedId(), ctx.cache().context().wal().currentSegment());
+
+                    writeTrackedWalIdxsToMetaStore();
+                }
+            }
+
+            reencryptGroups.put(grpId, pageScanner.pagesCount(grp));
+
+            if (log.isInfoEnabled())
+                log.info("New encryption key for group was added [grpId=" + grpId + ", keyId=" + newKeyId + "]");
+        }
+
+        startReencryption(encryptionStatus.keySet());
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @return Future that will be completed when reencryption of the specified group is finished.
+     */
+    public IgniteInternalFuture<Void> reencryptionFuture(int grpId) {
+        return pageScanner.statusFuture(grpId);
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @return {@code True} If the specified cache group should be re-encrypted.
+     */
+    public boolean reencryptionRequired(int grpId) {

Review comment:
       All usages of this method want to check that reencryption in progress. Let's rename it to something like `reencryptionInProgress`

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/util/BasicRateLimiterTest.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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;
+
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Test;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Rate limiter tests.
+ */
+public class BasicRateLimiterTest {
+    /**
+     * Check rate limit with multiple threads.
+     */
+    @Test
+    public void checkLimitMultithreaded() throws Exception {
+        int opsPerSec = 1_000;
+        int totalOps = 10_000;
+
+        BasicRateLimiter limiter = new BasicRateLimiter(opsPerSec);
+
+        int threads = Runtime.getRuntime().availableProcessors();
+
+        CyclicBarrier ready = new CyclicBarrier(threads + 1);
+
+        AtomicInteger cntr = new AtomicInteger();
+
+        IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(() -> {
+            ready.await();
+
+            do {
+                limiter.acquire(1);
+            }
+            while (!Thread.currentThread().isInterrupted() && cntr.incrementAndGet() < totalOps);
+
+            return null;
+        }, threads, "worker");
+
+        ready.await();
+
+        long startTime = System.currentTimeMillis();
+
+        fut.get();
+
+        long timeSpent = System.currentTimeMillis() - startTime;
+
+        assertEquals(totalOps / opsPerSec, SECONDS.convert(timeSpent, MILLISECONDS));
+    }
+
+    /**
+     * Check that the average speed is limited correctly even if we are acquiring more permits than allowed per second.
+     */
+    @Test
+    public void checkAcquireWithOverflow() throws IgniteInterruptedCheckedException {
+        double permitsPerSec = 0.5;
+        int permitsPerOp = 1;
+        int totalOps = 5;
+
+        BasicRateLimiter limiter = new BasicRateLimiter(permitsPerSec);
+
+        long startTime = System.currentTimeMillis();
+
+        for (int i = 0; i <= totalOps; i++)

Review comment:
       Actually, you check `totalOps + 1` operations here

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/util/BasicRateLimiterTest.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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;
+
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Test;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Rate limiter tests.
+ */
+public class BasicRateLimiterTest {
+    /**
+     * Check rate limit with multiple threads.
+     */
+    @Test
+    public void checkLimitMultithreaded() throws Exception {
+        int opsPerSec = 1_000;
+        int totalOps = 10_000;
+
+        BasicRateLimiter limiter = new BasicRateLimiter(opsPerSec);
+
+        int threads = Runtime.getRuntime().availableProcessors();
+
+        CyclicBarrier ready = new CyclicBarrier(threads + 1);
+
+        AtomicInteger cntr = new AtomicInteger();
+
+        IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(() -> {
+            ready.await();
+
+            do {
+                limiter.acquire(1);
+            }
+            while (!Thread.currentThread().isInterrupted() && cntr.incrementAndGet() < totalOps);
+
+            return null;
+        }, threads, "worker");
+
+        ready.await();
+
+        long startTime = System.currentTimeMillis();
+
+        fut.get();
+
+        long timeSpent = System.currentTimeMillis() - startTime;
+
+        assertEquals(totalOps / opsPerSec, SECONDS.convert(timeSpent, MILLISECONDS));

Review comment:
       In some circumstances, I think `timeSpent` can be a little bit less than 10 seconds

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/encryption/AbstractEncryptionTest.java
##########
@@ -271,4 +313,185 @@ protected boolean checkMasterKeyName(String name) {
 
         return true;
     }
+
+    /**
+     * Load data into cache "{@link #cacheName()}" using node "{@link #GRID_0}".
+     *
+     * @param cnt Count of entries.
+     */
+    protected void loadData(int cnt) {
+        loadData(cacheName(), cnt);
+    }
+
+    /**
+     * Load data into cache using node "{@link #GRID_0}".
+     *
+     * @param cnt Count of entries.
+     * @param cacheName Cache name.
+     */
+    protected void loadData(String cacheName, int cnt) {
+        info("Loading " + cnt + " entries into " + cacheName);
+
+        int start = grid(GRID_0).cache(cacheName).size();
+
+        try (IgniteDataStreamer<Long, Object> streamer = grid(GRID_0).dataStreamer(cacheName)) {
+            for (long i = start; i < (cnt + start); i++)
+                streamer.addData(i, generateValue(i));
+        }
+
+        info("Load data finished");
+    }
+
+    /**
+     * Ensures that all pages of page store have expected encryption key identifier.
+     *
+     * @param grpId Cache group ID.
+     * @param keyId Encryption key ID.
+     * @param timeout Timeout to wait for encryption to complete.
+     * @throws Exception If failed.
+     */
+    protected void checkGroupKey(int grpId, int keyId, long timeout) throws Exception {
+        awaitEncryption(G.allGrids(), grpId, timeout);
+
+        for (Ignite g : G.allGrids()) {
+            info("Validating encryption key [node=" + g.cluster().localNode().id() + ", grp=" + grpId + "]");
+
+            IgniteEx grid = (IgniteEx)g;
+
+            if (grid.context().clientNode())
+                continue;
+
+            GridEncryptionManager encryption = grid.context().encryption();
+
+            assertEquals(grid.localNode().id().toString(), (byte)keyId, encryption.groupKey(grpId).id());
+
+            IgniteInternalFuture<Void> fut = encryption.reencryptionFuture(grpId);
+
+            // The future will be completed after the checkpoint, forcecheckpoint does nothing
+            // if the checkpoint has already been scheduled.
+            GridTestUtils.waitForCondition(() -> {
+                if (fut.isDone())
+                    return true;
+
+                try {
+                    forceCheckpoint(g);
+                }
+                catch (IgniteCheckedException e) {
+                    throw new RuntimeException(e);
+                }
+
+                return fut.isDone();
+            }, timeout);
+
+            assertTrue(fut.isDone());
+
+            CacheGroupContext grp = grid.context().cache().cacheGroup(grpId);
+
+            List<Integer> parts = IntStream.range(0, grp.shared().affinity().affinity(grpId).partitions())
+                .boxed().collect(Collectors.toList());
+
+            parts.add(INDEX_PARTITION);
+
+            int realPageSize = grp.dataRegion().pageMemory().realPageSize(grpId);
+            int encryptionBlockSize = grp.shared().kernalContext().config().getEncryptionSpi().blockSize();
+
+            for (int p : parts) {
+                FilePageStore pageStore =
+                    (FilePageStore)((FilePageStoreManager)grp.shared().pageStore()).getStore(grpId, p);
+
+                if (!pageStore.exists())
+                    continue;
+
+                long state = grid.context().encryption().getEncryptionState(grpId, p);
+
+                String msg = String.format("p=%d, off=%d, total=%d",
+                    p, ReencryptStateUtils.pageIndex(state), ReencryptStateUtils.pageCount(state));
+
+                assertEquals(msg, 0, ReencryptStateUtils.pageCount(state));
+                assertEquals(msg, 0, ReencryptStateUtils.pageIndex(state));
+
+                long startPageId = PageIdUtils.pageId(p, PageIdAllocator.FLAG_DATA, 0);
+
+                int pagesCnt = pageStore.pages();
+                int pageSize = pageStore.getPageSize();
+
+                ByteBuffer pageBuf = ByteBuffer.allocate(pageSize);
+
+                Path path = new File(pageStore.getFileAbsolutePath()).toPath();
+
+                try (FileChannel ch = FileChannel.open(path, StandardOpenOption.READ)) {
+                    for (int n = 0; n < pagesCnt; n++) {
+                        long pageId = startPageId + n;
+                        long pageOff = pageStore.pageOffset(pageId);
+
+                        pageBuf.position(0);
+
+                        ch.position(pageOff);
+                        ch.read(pageBuf);
+
+                        pageBuf.position(realPageSize + encryptionBlockSize);
+
+                        // If crc present

Review comment:
       Point at the end.
   Why there can be pages without CRC?




----------------------------------------------------------------
This is an automated message from the 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 #7941: IGNITE-12843

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java
##########
@@ -258,6 +258,9 @@
     /** */
     public static final short T_MARKER_PAGE = 33;
 
+    /** */
+    public static final short T_INDEX_META = 34;

Review comment:
       Can you please explain why you needed new IO type instead of new version for existing type? You only add two fields, right? I don't get it then.




----------------------------------------------------------------
This is an automated message from the 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] alex-plekhanov commented on a change in pull request #7941: IGNITE-12843

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #7941:
URL: https://github.com/apache/ignite/pull/7941#discussion_r475452890



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupPageScanner.java
##########
@@ -0,0 +1,453 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.EncryptionConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId;
+import org.apache.ignite.internal.util.BasicRateLimiter;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+
+import static org.apache.ignite.internal.util.IgniteUtils.MB;
+
+/**
+ * Cache group page stores scanner.
+ * Scans a range of pages and marks them as dirty to re-encrypt them with the last encryption key on disk.
+ */
+public class CacheGroupPageScanner implements DbCheckpointListener {
+    /** Encryption configuration. */
+    private final EncryptionConfiguration encrCfg;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to group scanning task. */
+    private final Map<Integer, GroupScanTask> grps = new ConcurrentHashMap<>();
+
+    /** Collection of groups waiting for a checkpoint. */
+    private final Collection<GroupScanTask> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Page scanning speed limiter. */
+    private final BasicRateLimiter limiter;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupPageScanner(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        encrCfg = ctx.config().getDataStorageConfiguration().getEncryptionConfiguration();
+
+        DataStorageConfiguration dsCfg = ctx.config().getDataStorageConfiguration();
+
+        limiter = CU.isPersistenceEnabled(dsCfg) && encrCfg.getReencryptionRateLimit() > 0 ?
+            new BasicRateLimiter(encrCfg.getReencryptionRateLimit() * MB / dsCfg.getPageSize()) : null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCheckpointBegin(Context cpCtx) {
+        Set<GroupScanTask> completeCandidates = new HashSet<>();
+
+        cpWaitGrps.removeIf(completeCandidates::add);
+
+        cpCtx.finishedStateFut().listen(
+            f -> {
+                // Retry if error occurs.
+                if (f.error() != null || f.isCancelled()) {
+                    cpWaitGrps.addAll(completeCandidates);
+
+                    return;
+                }
+
+                lock.lock();
+
+                try {
+                    for (GroupScanTask grpScanTask : completeCandidates) {
+                        grps.remove(grpScanTask.groupId());
+
+                        grpScanTask.onDone();
+
+                        if (log.isInfoEnabled())
+                            log.info("Cache group reencryption is finished [grpId=" + grpScanTask.groupId() + "]");
+                    }
+
+                    if (!grps.isEmpty())
+                        return;
+
+                    ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).
+                        removeCheckpointListener(this);
+                }
+                finally {
+                    lock.unlock();
+                }
+            }
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public void beforeCheckpointBegin(Context cpCtx) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onMarkCheckpointBegin(Context ctx) {
+        // No-op.
+    }
+
+    /**
+     * @return {@code True} If reencryption is disabled.
+     */
+    public boolean disabled() {
+        return encrCfg.isReencryptionDisabled();
+    }
+
+    /**
+     * Schedule scanning partitions.
+     *
+     * @param grpId Cache group ID.
+     */
+    public IgniteInternalFuture<Void> schedule(int grpId) throws IgniteCheckedException {
+        if (disabled())
+            throw new IgniteCheckedException("Reencryption is disabled.");
+
+        CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+        if (grp == null) {
+            if (log.isDebugEnabled())
+                log.debug("Skip reencryption, cache group was destroyed [grp=" + grpId + "]");
+
+            return new GridFinishedFuture<>();
+        }
+
+        lock.lock();
+
+        try {
+            if (stopped)
+                throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
+
+            if (grps.isEmpty())
+                ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).addCheckpointListener(this);
+
+            GroupScanTask prevState = grps.get(grpId);
+
+            if (prevState != null) {
+                if (log.isDebugEnabled())
+                    log.debug("Reencryption already scheduled [grpId=" + grpId + "]");
+
+                return prevState;
+            }
+
+            Set<Integer> parts = new HashSet<>();
+
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) {
+                    if (ctx.encryption().getEncryptionState(grpId, partId) == 0) {
+                        if (log.isDebugEnabled())
+                            log.debug("Skipping partition reencryption [grp=" + grpId + ", p=" + partId + "]");
+
+                        return;
+                    }
+
+                    parts.add(partId);
+                }
+            });
+
+            GroupScanTask grpScan = new GroupScanTask(grp, parts);
+
+            ctx.getSystemExecutorService().submit(grpScan);
+
+            if (log.isInfoEnabled())
+                log.info("Scheduled reencryption [grpId=" + grpId + "]");
+
+            grps.put(grpId, grpScan);
+
+            return grpScan;
+        }
+        finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @return Future that will be completed when all partitions have been scanned and pages have been written to disk.
+     */
+    public IgniteInternalFuture<Void> statusFuture(int grpId) {
+        GroupScanTask ctx0 = grps.get(grpId);
+
+        return ctx0 == null ? new GridFinishedFuture<>() : ctx0;
+    }
+
+    /**
+     * Shutdown scanning and disable new tasks scheduling.
+     */
+    public void stop() throws IgniteCheckedException {
+        lock.lock();
+
+        try {
+            stopped = true;
+
+            for (GroupScanTask grpScanTask : grps.values())
+                grpScanTask.cancel();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Stop scannig the specified partition.
+     *
+     * @param grpId Cache group ID.
+     * @param partId Partition ID.
+     * @return {@code True} if reencryption was cancelled.
+     */
+    public boolean cancel(int grpId, int partId) {
+        GroupScanTask grpScanTask = grps.get(grpId);
+
+        if (grpScanTask == null)
+            return false;
+
+        return grpScanTask.cancel(partId);
+    }
+
+    /**
+     * Collect current number of pages in the specified cache group.
+     *
+     * @param grp Cache group.
+     * @return Partitions with current page count.
+     * @throws IgniteCheckedException If failed.
+     */
+    public long[] pagesCount(CacheGroupContext grp) throws IgniteCheckedException {
+        long[] partStates = new long[grp.affinity().partitions() + 1];
+
+        ctx.cache().context().database().checkpointReadLock();
+
+        try {
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) throws IgniteCheckedException {
+                    int pagesCnt = ctx.cache().context().pageStore().pages(grp.groupId(), partId);
+
+                    partStates[Math.min(partId, partStates.length - 1)] = pagesCnt;
+                }
+            });
+        } finally {
+            ctx.cache().context().database().checkpointReadUnlock();
+        }
+
+        return partStates;
+    }
+
+    /**
+     * @param grp Cache group.
+     * @param hnd Partition handler.
+     */
+    private void forEachPageStore(CacheGroupContext grp, IgniteInClosureX<Integer> hnd) throws IgniteCheckedException {
+        int parts = grp.affinity().partitions();
+
+        IgnitePageStoreManager pageStoreMgr = ctx.cache().context().pageStore();
+
+        for (int p = 0; p < parts; p++) {
+            if (!pageStoreMgr.exists(grp.groupId(), p))
+                continue;
+
+            hnd.applyx(p);
+        }
+
+        hnd.applyx(PageIdAllocator.INDEX_PARTITION);
+    }
+
+    /**
+     * Cache group partition scanning task.
+     */
+    private class GroupScanTask extends GridFutureAdapter<Void> implements Runnable {
+        /** Cache group ID. */
+        private final CacheGroupContext grp;
+
+        /** Partition IDs. */
+        private final Set<Integer> parts;
+
+        /** Page memory. */
+        private final PageMemoryEx pageMem;
+
+        /**
+         * @param grp Cache group.
+         */
+        public GroupScanTask(CacheGroupContext grp, Set<Integer> parts) {
+            this.grp = grp;
+            this.parts = new GridConcurrentHashSet<>(parts);
+
+            pageMem = (PageMemoryEx)grp.dataRegion().pageMemory();
+        }
+
+        /** {@inheritDoc} */
+        @Override public synchronized boolean cancel() throws IgniteCheckedException {
+            return onDone(null, null, true);
+        }
+
+        /**
+         * Stop reencryption of the specified partition.
+         *
+         * @param partId Partition ID.
+         * @return {@code True} if reencryption was cancelled.
+         */
+        public synchronized boolean cancel(int partId) {

Review comment:
       Perhaps `cancel` is not a good method name for `page scanner`. Maybe something like `excludePartition`?
   Also, I think `synchronized` is redundant here

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupPageScanner.java
##########
@@ -0,0 +1,453 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.EncryptionConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId;
+import org.apache.ignite.internal.util.BasicRateLimiter;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+
+import static org.apache.ignite.internal.util.IgniteUtils.MB;
+
+/**
+ * Cache group page stores scanner.
+ * Scans a range of pages and marks them as dirty to re-encrypt them with the last encryption key on disk.
+ */
+public class CacheGroupPageScanner implements DbCheckpointListener {
+    /** Encryption configuration. */
+    private final EncryptionConfiguration encrCfg;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to group scanning task. */
+    private final Map<Integer, GroupScanTask> grps = new ConcurrentHashMap<>();
+
+    /** Collection of groups waiting for a checkpoint. */
+    private final Collection<GroupScanTask> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Page scanning speed limiter. */
+    private final BasicRateLimiter limiter;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupPageScanner(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        encrCfg = ctx.config().getDataStorageConfiguration().getEncryptionConfiguration();
+
+        DataStorageConfiguration dsCfg = ctx.config().getDataStorageConfiguration();
+
+        limiter = CU.isPersistenceEnabled(dsCfg) && encrCfg.getReencryptionRateLimit() > 0 ?
+            new BasicRateLimiter(encrCfg.getReencryptionRateLimit() * MB / dsCfg.getPageSize()) : null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCheckpointBegin(Context cpCtx) {
+        Set<GroupScanTask> completeCandidates = new HashSet<>();
+
+        cpWaitGrps.removeIf(completeCandidates::add);
+
+        cpCtx.finishedStateFut().listen(
+            f -> {
+                // Retry if error occurs.
+                if (f.error() != null || f.isCancelled()) {
+                    cpWaitGrps.addAll(completeCandidates);
+
+                    return;
+                }
+
+                lock.lock();
+
+                try {
+                    for (GroupScanTask grpScanTask : completeCandidates) {
+                        grps.remove(grpScanTask.groupId());
+
+                        grpScanTask.onDone();
+
+                        if (log.isInfoEnabled())
+                            log.info("Cache group reencryption is finished [grpId=" + grpScanTask.groupId() + "]");
+                    }
+
+                    if (!grps.isEmpty())
+                        return;
+
+                    ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).
+                        removeCheckpointListener(this);
+                }
+                finally {
+                    lock.unlock();
+                }
+            }
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public void beforeCheckpointBegin(Context cpCtx) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onMarkCheckpointBegin(Context ctx) {
+        // No-op.
+    }
+
+    /**
+     * @return {@code True} If reencryption is disabled.
+     */
+    public boolean disabled() {
+        return encrCfg.isReencryptionDisabled();
+    }
+
+    /**
+     * Schedule scanning partitions.
+     *
+     * @param grpId Cache group ID.
+     */
+    public IgniteInternalFuture<Void> schedule(int grpId) throws IgniteCheckedException {
+        if (disabled())
+            throw new IgniteCheckedException("Reencryption is disabled.");
+
+        CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+        if (grp == null) {
+            if (log.isDebugEnabled())
+                log.debug("Skip reencryption, cache group was destroyed [grp=" + grpId + "]");
+
+            return new GridFinishedFuture<>();
+        }
+
+        lock.lock();
+
+        try {
+            if (stopped)
+                throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
+
+            if (grps.isEmpty())
+                ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).addCheckpointListener(this);
+
+            GroupScanTask prevState = grps.get(grpId);
+
+            if (prevState != null) {
+                if (log.isDebugEnabled())
+                    log.debug("Reencryption already scheduled [grpId=" + grpId + "]");
+
+                return prevState;
+            }
+
+            Set<Integer> parts = new HashSet<>();
+
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) {
+                    if (ctx.encryption().getEncryptionState(grpId, partId) == 0) {
+                        if (log.isDebugEnabled())
+                            log.debug("Skipping partition reencryption [grp=" + grpId + ", p=" + partId + "]");
+
+                        return;
+                    }
+
+                    parts.add(partId);
+                }
+            });
+
+            GroupScanTask grpScan = new GroupScanTask(grp, parts);
+
+            ctx.getSystemExecutorService().submit(grpScan);
+
+            if (log.isInfoEnabled())
+                log.info("Scheduled reencryption [grpId=" + grpId + "]");
+
+            grps.put(grpId, grpScan);
+
+            return grpScan;
+        }
+        finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @return Future that will be completed when all partitions have been scanned and pages have been written to disk.
+     */
+    public IgniteInternalFuture<Void> statusFuture(int grpId) {
+        GroupScanTask ctx0 = grps.get(grpId);
+
+        return ctx0 == null ? new GridFinishedFuture<>() : ctx0;
+    }
+
+    /**
+     * Shutdown scanning and disable new tasks scheduling.
+     */
+    public void stop() throws IgniteCheckedException {
+        lock.lock();
+
+        try {
+            stopped = true;
+
+            for (GroupScanTask grpScanTask : grps.values())
+                grpScanTask.cancel();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Stop scannig the specified partition.
+     *
+     * @param grpId Cache group ID.
+     * @param partId Partition ID.
+     * @return {@code True} if reencryption was cancelled.
+     */
+    public boolean cancel(int grpId, int partId) {
+        GroupScanTask grpScanTask = grps.get(grpId);
+
+        if (grpScanTask == null)
+            return false;
+
+        return grpScanTask.cancel(partId);
+    }
+
+    /**
+     * Collect current number of pages in the specified cache group.
+     *
+     * @param grp Cache group.
+     * @return Partitions with current page count.
+     * @throws IgniteCheckedException If failed.
+     */
+    public long[] pagesCount(CacheGroupContext grp) throws IgniteCheckedException {
+        long[] partStates = new long[grp.affinity().partitions() + 1];

Review comment:
       Lets add a comment, that we need 1 extra item for index partition

##########
File path: modules/core/src/main/java/org/apache/ignite/mxbean/EncryptionMXBean.java
##########
@@ -43,4 +44,15 @@
     public void changeMasterKey(
         @MXBeanParameter(name = "masterKeyName", description = "Master key name.") String masterKeyName
     );
+
+    /**
+     * Starts cache group encryption key change process.
+     *
+     * @param cacheOrGrpName Cache or group name.
+     * @see IgniteEncryption#changeCacheGroupKey(Collection)
+     */
+    @MXBeanDescription("Change cache group key.")
+    public void changeCacheGroupKey(
+        @MXBeanParameter(name = "cacheOrGroupName", description = "Cache or group name.") String cacheOrGrpName

Review comment:
       Let's rename parameter `cacheOrGroupName` to just `cacheGroupName`, otherwise it's confusing and someone can decide that it's possible to change keys for single cache within cache group. (I know that method CacheGroupContext.cacheOrGroupName exists, but it's internal API)

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupPageScanner.java
##########
@@ -0,0 +1,453 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.EncryptionConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId;
+import org.apache.ignite.internal.util.BasicRateLimiter;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+
+import static org.apache.ignite.internal.util.IgniteUtils.MB;
+
+/**
+ * Cache group page stores scanner.
+ * Scans a range of pages and marks them as dirty to re-encrypt them with the last encryption key on disk.
+ */
+public class CacheGroupPageScanner implements DbCheckpointListener {
+    /** Encryption configuration. */
+    private final EncryptionConfiguration encrCfg;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to group scanning task. */
+    private final Map<Integer, GroupScanTask> grps = new ConcurrentHashMap<>();
+
+    /** Collection of groups waiting for a checkpoint. */
+    private final Collection<GroupScanTask> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Page scanning speed limiter. */
+    private final BasicRateLimiter limiter;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupPageScanner(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        encrCfg = ctx.config().getDataStorageConfiguration().getEncryptionConfiguration();
+
+        DataStorageConfiguration dsCfg = ctx.config().getDataStorageConfiguration();
+
+        limiter = CU.isPersistenceEnabled(dsCfg) && encrCfg.getReencryptionRateLimit() > 0 ?
+            new BasicRateLimiter(encrCfg.getReencryptionRateLimit() * MB / dsCfg.getPageSize()) : null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCheckpointBegin(Context cpCtx) {
+        Set<GroupScanTask> completeCandidates = new HashSet<>();
+
+        cpWaitGrps.removeIf(completeCandidates::add);
+
+        cpCtx.finishedStateFut().listen(
+            f -> {
+                // Retry if error occurs.
+                if (f.error() != null || f.isCancelled()) {
+                    cpWaitGrps.addAll(completeCandidates);
+
+                    return;
+                }
+
+                lock.lock();
+
+                try {
+                    for (GroupScanTask grpScanTask : completeCandidates) {
+                        grps.remove(grpScanTask.groupId());
+
+                        grpScanTask.onDone();
+
+                        if (log.isInfoEnabled())
+                            log.info("Cache group reencryption is finished [grpId=" + grpScanTask.groupId() + "]");
+                    }
+
+                    if (!grps.isEmpty())
+                        return;
+
+                    ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).
+                        removeCheckpointListener(this);
+                }
+                finally {
+                    lock.unlock();
+                }
+            }
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public void beforeCheckpointBegin(Context cpCtx) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onMarkCheckpointBegin(Context ctx) {
+        // No-op.
+    }
+
+    /**
+     * @return {@code True} If reencryption is disabled.
+     */
+    public boolean disabled() {
+        return encrCfg.isReencryptionDisabled();
+    }
+
+    /**
+     * Schedule scanning partitions.
+     *
+     * @param grpId Cache group ID.
+     */
+    public IgniteInternalFuture<Void> schedule(int grpId) throws IgniteCheckedException {
+        if (disabled())
+            throw new IgniteCheckedException("Reencryption is disabled.");
+
+        CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+        if (grp == null) {
+            if (log.isDebugEnabled())
+                log.debug("Skip reencryption, cache group was destroyed [grp=" + grpId + "]");
+
+            return new GridFinishedFuture<>();
+        }
+
+        lock.lock();
+
+        try {
+            if (stopped)
+                throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
+
+            if (grps.isEmpty())
+                ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).addCheckpointListener(this);
+
+            GroupScanTask prevState = grps.get(grpId);
+
+            if (prevState != null) {
+                if (log.isDebugEnabled())
+                    log.debug("Reencryption already scheduled [grpId=" + grpId + "]");
+
+                return prevState;
+            }
+
+            Set<Integer> parts = new HashSet<>();
+
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) {
+                    if (ctx.encryption().getEncryptionState(grpId, partId) == 0) {
+                        if (log.isDebugEnabled())
+                            log.debug("Skipping partition reencryption [grp=" + grpId + ", p=" + partId + "]");
+
+                        return;
+                    }
+
+                    parts.add(partId);
+                }
+            });
+
+            GroupScanTask grpScan = new GroupScanTask(grp, parts);
+
+            ctx.getSystemExecutorService().submit(grpScan);
+
+            if (log.isInfoEnabled())
+                log.info("Scheduled reencryption [grpId=" + grpId + "]");
+
+            grps.put(grpId, grpScan);
+
+            return grpScan;
+        }
+        finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @return Future that will be completed when all partitions have been scanned and pages have been written to disk.
+     */
+    public IgniteInternalFuture<Void> statusFuture(int grpId) {
+        GroupScanTask ctx0 = grps.get(grpId);
+
+        return ctx0 == null ? new GridFinishedFuture<>() : ctx0;
+    }
+
+    /**
+     * Shutdown scanning and disable new tasks scheduling.
+     */
+    public void stop() throws IgniteCheckedException {
+        lock.lock();
+
+        try {
+            stopped = true;
+
+            for (GroupScanTask grpScanTask : grps.values())
+                grpScanTask.cancel();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Stop scannig the specified partition.
+     *
+     * @param grpId Cache group ID.
+     * @param partId Partition ID.
+     * @return {@code True} if reencryption was cancelled.
+     */
+    public boolean cancel(int grpId, int partId) {
+        GroupScanTask grpScanTask = grps.get(grpId);
+
+        if (grpScanTask == null)
+            return false;
+
+        return grpScanTask.cancel(partId);
+    }
+
+    /**
+     * Collect current number of pages in the specified cache group.
+     *
+     * @param grp Cache group.
+     * @return Partitions with current page count.
+     * @throws IgniteCheckedException If failed.
+     */
+    public long[] pagesCount(CacheGroupContext grp) throws IgniteCheckedException {
+        long[] partStates = new long[grp.affinity().partitions() + 1];
+
+        ctx.cache().context().database().checkpointReadLock();
+
+        try {
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) throws IgniteCheckedException {
+                    int pagesCnt = ctx.cache().context().pageStore().pages(grp.groupId(), partId);
+
+                    partStates[Math.min(partId, partStates.length - 1)] = pagesCnt;
+                }
+            });
+        } finally {
+            ctx.cache().context().database().checkpointReadUnlock();
+        }
+
+        return partStates;
+    }
+
+    /**
+     * @param grp Cache group.
+     * @param hnd Partition handler.
+     */
+    private void forEachPageStore(CacheGroupContext grp, IgniteInClosureX<Integer> hnd) throws IgniteCheckedException {
+        int parts = grp.affinity().partitions();
+
+        IgnitePageStoreManager pageStoreMgr = ctx.cache().context().pageStore();
+
+        for (int p = 0; p < parts; p++) {
+            if (!pageStoreMgr.exists(grp.groupId(), p))
+                continue;
+
+            hnd.applyx(p);
+        }
+
+        hnd.applyx(PageIdAllocator.INDEX_PARTITION);
+    }
+
+    /**
+     * Cache group partition scanning task.
+     */
+    private class GroupScanTask extends GridFutureAdapter<Void> implements Runnable {
+        /** Cache group ID. */
+        private final CacheGroupContext grp;
+
+        /** Partition IDs. */
+        private final Set<Integer> parts;
+
+        /** Page memory. */
+        private final PageMemoryEx pageMem;
+
+        /**
+         * @param grp Cache group.
+         */
+        public GroupScanTask(CacheGroupContext grp, Set<Integer> parts) {
+            this.grp = grp;
+            this.parts = new GridConcurrentHashSet<>(parts);
+
+            pageMem = (PageMemoryEx)grp.dataRegion().pageMemory();
+        }
+
+        /** {@inheritDoc} */
+        @Override public synchronized boolean cancel() throws IgniteCheckedException {

Review comment:
       synchronized is redundant here.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -611,7 +695,12 @@ public void groupKey(int grpId, byte[] encGrpKey) {
                     "The previous change was not completed."));
             }
 
-            masterKeyChangeFut = new MasterKeyChangeFuture(request.requestId());
+            if (!grpKeyChangeProc.finished()) {

Review comment:
       `finished()` can be `false` only for node where key change process was initiated. Why we don't restrict master key change on other nodes? Do we need `finished` method at all? (Perhaps can be replaced with `started()`) 

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/CacheGroupPageScanner.java
##########
@@ -0,0 +1,453 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.EncryptionConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId;
+import org.apache.ignite.internal.util.BasicRateLimiter;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.lang.IgniteInClosureX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+
+import static org.apache.ignite.internal.util.IgniteUtils.MB;
+
+/**
+ * Cache group page stores scanner.
+ * Scans a range of pages and marks them as dirty to re-encrypt them with the last encryption key on disk.
+ */
+public class CacheGroupPageScanner implements DbCheckpointListener {
+    /** Encryption configuration. */
+    private final EncryptionConfiguration encrCfg;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /** Mapping of cache group ID to group scanning task. */
+    private final Map<Integer, GroupScanTask> grps = new ConcurrentHashMap<>();
+
+    /** Collection of groups waiting for a checkpoint. */
+    private final Collection<GroupScanTask> cpWaitGrps = new ConcurrentLinkedQueue<>();
+
+    /** Page scanning speed limiter. */
+    private final BasicRateLimiter limiter;
+
+    /** Stop flag. */
+    private boolean stopped;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    public CacheGroupPageScanner(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(getClass());
+
+        encrCfg = ctx.config().getDataStorageConfiguration().getEncryptionConfiguration();
+
+        DataStorageConfiguration dsCfg = ctx.config().getDataStorageConfiguration();
+
+        limiter = CU.isPersistenceEnabled(dsCfg) && encrCfg.getReencryptionRateLimit() > 0 ?
+            new BasicRateLimiter(encrCfg.getReencryptionRateLimit() * MB / dsCfg.getPageSize()) : null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCheckpointBegin(Context cpCtx) {
+        Set<GroupScanTask> completeCandidates = new HashSet<>();
+
+        cpWaitGrps.removeIf(completeCandidates::add);
+
+        cpCtx.finishedStateFut().listen(
+            f -> {
+                // Retry if error occurs.
+                if (f.error() != null || f.isCancelled()) {
+                    cpWaitGrps.addAll(completeCandidates);
+
+                    return;
+                }
+
+                lock.lock();
+
+                try {
+                    for (GroupScanTask grpScanTask : completeCandidates) {
+                        grps.remove(grpScanTask.groupId());
+
+                        grpScanTask.onDone();
+
+                        if (log.isInfoEnabled())
+                            log.info("Cache group reencryption is finished [grpId=" + grpScanTask.groupId() + "]");
+                    }
+
+                    if (!grps.isEmpty())
+                        return;
+
+                    ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).
+                        removeCheckpointListener(this);
+                }
+                finally {
+                    lock.unlock();
+                }
+            }
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public void beforeCheckpointBegin(Context cpCtx) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onMarkCheckpointBegin(Context ctx) {
+        // No-op.
+    }
+
+    /**
+     * @return {@code True} If reencryption is disabled.
+     */
+    public boolean disabled() {
+        return encrCfg.isReencryptionDisabled();
+    }
+
+    /**
+     * Schedule scanning partitions.
+     *
+     * @param grpId Cache group ID.
+     */
+    public IgniteInternalFuture<Void> schedule(int grpId) throws IgniteCheckedException {
+        if (disabled())
+            throw new IgniteCheckedException("Reencryption is disabled.");
+
+        CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+        if (grp == null) {
+            if (log.isDebugEnabled())
+                log.debug("Skip reencryption, cache group was destroyed [grp=" + grpId + "]");
+
+            return new GridFinishedFuture<>();
+        }
+
+        lock.lock();
+
+        try {
+            if (stopped)
+                throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
+
+            if (grps.isEmpty())
+                ((GridCacheDatabaseSharedManager)ctx.cache().context().database()).addCheckpointListener(this);
+
+            GroupScanTask prevState = grps.get(grpId);
+
+            if (prevState != null) {
+                if (log.isDebugEnabled())
+                    log.debug("Reencryption already scheduled [grpId=" + grpId + "]");
+
+                return prevState;
+            }
+
+            Set<Integer> parts = new HashSet<>();
+
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) {
+                    if (ctx.encryption().getEncryptionState(grpId, partId) == 0) {
+                        if (log.isDebugEnabled())
+                            log.debug("Skipping partition reencryption [grp=" + grpId + ", p=" + partId + "]");
+
+                        return;
+                    }
+
+                    parts.add(partId);
+                }
+            });
+
+            GroupScanTask grpScan = new GroupScanTask(grp, parts);
+
+            ctx.getSystemExecutorService().submit(grpScan);
+
+            if (log.isInfoEnabled())
+                log.info("Scheduled reencryption [grpId=" + grpId + "]");
+
+            grps.put(grpId, grpScan);
+
+            return grpScan;
+        }
+        finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @return Future that will be completed when all partitions have been scanned and pages have been written to disk.
+     */
+    public IgniteInternalFuture<Void> statusFuture(int grpId) {
+        GroupScanTask ctx0 = grps.get(grpId);
+
+        return ctx0 == null ? new GridFinishedFuture<>() : ctx0;
+    }
+
+    /**
+     * Shutdown scanning and disable new tasks scheduling.
+     */
+    public void stop() throws IgniteCheckedException {
+        lock.lock();
+
+        try {
+            stopped = true;
+
+            for (GroupScanTask grpScanTask : grps.values())
+                grpScanTask.cancel();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Stop scannig the specified partition.
+     *
+     * @param grpId Cache group ID.
+     * @param partId Partition ID.
+     * @return {@code True} if reencryption was cancelled.
+     */
+    public boolean cancel(int grpId, int partId) {
+        GroupScanTask grpScanTask = grps.get(grpId);
+
+        if (grpScanTask == null)
+            return false;
+
+        return grpScanTask.cancel(partId);
+    }
+
+    /**
+     * Collect current number of pages in the specified cache group.
+     *
+     * @param grp Cache group.
+     * @return Partitions with current page count.
+     * @throws IgniteCheckedException If failed.
+     */
+    public long[] pagesCount(CacheGroupContext grp) throws IgniteCheckedException {
+        long[] partStates = new long[grp.affinity().partitions() + 1];
+
+        ctx.cache().context().database().checkpointReadLock();
+
+        try {
+            forEachPageStore(grp, new IgniteInClosureX<Integer>() {
+                @Override public void applyx(Integer partId) throws IgniteCheckedException {
+                    int pagesCnt = ctx.cache().context().pageStore().pages(grp.groupId(), partId);
+
+                    partStates[Math.min(partId, partStates.length - 1)] = pagesCnt;
+                }
+            });
+        } finally {
+            ctx.cache().context().database().checkpointReadUnlock();
+        }
+
+        return partStates;
+    }
+
+    /**
+     * @param grp Cache group.
+     * @param hnd Partition handler.
+     */
+    private void forEachPageStore(CacheGroupContext grp, IgniteInClosureX<Integer> hnd) throws IgniteCheckedException {
+        int parts = grp.affinity().partitions();
+
+        IgnitePageStoreManager pageStoreMgr = ctx.cache().context().pageStore();
+
+        for (int p = 0; p < parts; p++) {
+            if (!pageStoreMgr.exists(grp.groupId(), p))
+                continue;
+
+            hnd.applyx(p);
+        }
+
+        hnd.applyx(PageIdAllocator.INDEX_PARTITION);
+    }
+
+    /**
+     * Cache group partition scanning task.
+     */
+    private class GroupScanTask extends GridFutureAdapter<Void> implements Runnable {
+        /** Cache group ID. */
+        private final CacheGroupContext grp;
+
+        /** Partition IDs. */
+        private final Set<Integer> parts;
+
+        /** Page memory. */
+        private final PageMemoryEx pageMem;
+
+        /**
+         * @param grp Cache group.
+         */
+        public GroupScanTask(CacheGroupContext grp, Set<Integer> parts) {
+            this.grp = grp;
+            this.parts = new GridConcurrentHashSet<>(parts);
+
+            pageMem = (PageMemoryEx)grp.dataRegion().pageMemory();
+        }
+
+        /** {@inheritDoc} */
+        @Override public synchronized boolean cancel() throws IgniteCheckedException {
+            return onDone(null, null, true);

Review comment:
       `return onCancelled()`?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -503,58 +549,96 @@ else if (newKeys != null) {
 
     /** {@inheritDoc} */
     @Override public void onGridDataReceived(GridDiscoveryData data) {
+        assert !writeToMetaStoreEnabled;
+
         if (ctx.clientNode())
             return;
 
-        Map<Integer, byte[]> encKeysFromCluster = (Map<Integer, byte[]>)data.commonData();
+        Map<Integer, Object> encKeysFromCluster = (Map<Integer, Object>)data.commonData();
 
         if (F.isEmpty(encKeysFromCluster))
             return;
 
-        for (Map.Entry<Integer, byte[]> entry : encKeysFromCluster.entrySet()) {
-            if (groupKey(entry.getKey()) == null) {
-                U.quietAndInfo(log, "Store group key received from coordinator [grp=" + entry.getKey() + "]");
+        for (Map.Entry<Integer, Object> entry : encKeysFromCluster.entrySet()) {
+            int grpId = entry.getKey();
 
-                groupKey(entry.getKey(), entry.getValue());
-            }
-            else {
+            GroupKeyEncrypted rmtKey;
+
+            if (entry.getValue() instanceof GroupKeyEncrypted)
+                rmtKey = (GroupKeyEncrypted)entry.getValue();
+            else
+                rmtKey = new GroupKeyEncrypted(INITIAL_KEY_ID, (byte[])entry.getValue());
+
+            GroupKey locGrpKey = groupKey(grpId);
+
+            if (locGrpKey != null && locGrpKey.unsignedId() == rmtKey.id()) {
                 U.quietAndInfo(log, "Skip group key received from coordinator. Already exists. [grp=" +
-                    entry.getKey() + "]");
+                    grpId + ", keyId=" + rmtKey.id() + "]");
+
+                continue;
             }
+
+            U.quietAndInfo(log, "Store group key received from coordinator [grp=" + grpId +
+                ", keyId=" + rmtKey.id() + "]");
+
+            //changeActiveKey

Review comment:
       ?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GroupKeyChangeProcess.java
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteFeatures;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.managers.encryption.GridEncryptionManager.EmptyResult;
+import org.apache.ignite.internal.managers.encryption.GridEncryptionManager.KeyChangeFuture;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
+import org.apache.ignite.internal.util.distributed.DistributedProcess;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.future.IgniteFinishedFutureImpl;
+import org.apache.ignite.internal.util.future.IgniteFutureImpl;
+import org.apache.ignite.internal.util.typedef.F;
+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.IgniteFuture;
+import org.apache.ignite.lang.IgniteFutureCancelledException;
+
+import static org.apache.ignite.internal.IgniteFeatures.CACHE_GROUP_KEY_CHANGE;
+import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.CACHE_GROUP_KEY_CHANGE_FINISH;
+import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.CACHE_GROUP_KEY_CHANGE_PREPARE;
+
+/**
+ * Two phase distributed process, that performs cache group encryption key rotation.
+ */
+class GroupKeyChangeProcess {
+    /** Grid kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Cache group encyption key change prepare phase. */
+    private final DistributedProcess<ChangeCacheEncryptionRequest, EmptyResult> prepareGKChangeProc;
+
+    /** Cache group encyption key change perform phase. */
+    private final DistributedProcess<ChangeCacheEncryptionRequest, EmptyResult> performGKChangeProc;
+
+    /** Group encryption keys. */
+    private final CacheGroupEncryptionKeys keys;
+
+    /** Cache group key change future. */
+    private volatile GroupKeyChangeFuture fut;
+
+    /** Cache group key change request. */
+    private volatile ChangeCacheEncryptionRequest req;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    GroupKeyChangeProcess(GridKernalContext ctx, CacheGroupEncryptionKeys keys) {
+        this.ctx = ctx;
+        this.keys = keys;
+
+        prepareGKChangeProc =
+            new DistributedProcess<>(ctx, CACHE_GROUP_KEY_CHANGE_PREPARE, this::prepare, this::finishPrepare);
+        performGKChangeProc =
+            new DistributedProcess<>(ctx, CACHE_GROUP_KEY_CHANGE_FINISH, this::perform, this::finishPerform);
+    }
+
+    /**
+     * @return {@code True} if operation is still in progress.
+     */
+    public boolean started() {
+        return req != null;
+    }
+
+    /**
+     * @return {@code True} if operation is not finished.
+     */
+    public boolean finished() {
+        IgniteInternalFuture<Void> fut0 = fut;
+
+        return fut0 == null || fut0.isDone();
+    }
+
+    /**
+     * @param msg Error message.
+     */
+    public void cancel(String msg) {
+        GridFutureAdapter<Void> keyChangeFut = fut;
+
+        if (keyChangeFut != null && !keyChangeFut.isDone())
+            keyChangeFut.onDone(new IgniteFutureCancelledException(msg));
+    }
+
+    /**
+     * Starts cache group encryption key change process.
+     *
+     * @param cacheOrGrpNames Cache or group names.
+     */
+    public IgniteFuture<Void> start(Collection<String> cacheOrGrpNames) {
+        if (ctx.clientNode())
+            throw new UnsupportedOperationException("Client and daemon nodes can not perform this operation.");
+
+        if (!IgniteFeatures.allNodesSupports(ctx.grid().cluster().nodes(), CACHE_GROUP_KEY_CHANGE))
+            throw new IllegalStateException("Not all nodes in the cluster support this operation.");
+
+        if (!ctx.state().clusterState().state().active())
+            throw new IgniteException("Operation was rejected. The cluster is inactive.");
+
+        if (!finished()) {
+            return new IgniteFinishedFutureImpl<>(new IgniteException("Cache group key change was rejected. " +
+                "The previous change was not completed."));
+        }
+
+        int[] grpIds = new int[cacheOrGrpNames.size()];
+        byte[] keyIds = new byte[grpIds.length];
+
+        int n = 0;
+
+        for (String cacheOrGroupName : cacheOrGrpNames) {
+            CacheGroupContext grp = ctx.cache().cacheGroup(CU.cacheId(cacheOrGroupName));

Review comment:
       Cache group context not always exists on each node (for example, node filter can be used). Use here cache group descriptors instead of the cache group context.
   Also, I think cache check is redundant, you should use cacheGroupName parameter and fail if cache group doesn't exist.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -483,16 +530,15 @@ public void onLocalJoin() {
         if (dataBag.isJoiningNodeClient() || dataBag.commonDataCollectedFor(ENCRYPTION_MGR.ordinal()))
             return;
 
-        HashMap<Integer, byte[]> knownEncKeys = knownEncryptionKeys();
+        HashMap<Integer, GroupKeyEncrypted> knownEncKeys = grpKeys.getAll();

Review comment:
       Can `knownEncKeys` be `null` here? If yes, why `null` check was removed? You can get NPE on `knownEncKeys.putIfAbsent`. If no, there is a redundant check in `newEncryptionKeys`.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -811,6 +1124,53 @@ private void sendGenerateEncryptionKeyRequest(GenerateEncryptionKeyFuture fut) t
         ctx.io().sendToGridTopic(rndNode.id(), TOPIC_GEN_ENC_KEY, req, SYSTEM_POOL);
     }
 
+    /**
+     * @param grpIds Cache group IDs.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void startReencryption(Collection<Integer> grpIds) throws IgniteCheckedException {
+        if (pageScanner.disabled())
+            return;
+
+        for (int grpId : grpIds) {
+            IgniteInternalFuture<?> fut = pageScanner.schedule(grpId);
+
+            fut.listen(f -> {
+                try {
+                    f.get();

Review comment:
       `if (f.isCancelled()) {...} else if (f.error() != null) {...} else {...}`

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GroupKeyChangeProcess.java
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteFeatures;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.managers.encryption.GridEncryptionManager.EmptyResult;
+import org.apache.ignite.internal.managers.encryption.GridEncryptionManager.KeyChangeFuture;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
+import org.apache.ignite.internal.util.distributed.DistributedProcess;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.future.IgniteFinishedFutureImpl;
+import org.apache.ignite.internal.util.future.IgniteFutureImpl;
+import org.apache.ignite.internal.util.typedef.F;
+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.IgniteFuture;
+import org.apache.ignite.lang.IgniteFutureCancelledException;
+
+import static org.apache.ignite.internal.IgniteFeatures.CACHE_GROUP_KEY_CHANGE;
+import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.CACHE_GROUP_KEY_CHANGE_FINISH;
+import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.CACHE_GROUP_KEY_CHANGE_PREPARE;
+
+/**
+ * Two phase distributed process, that performs cache group encryption key rotation.
+ */
+class GroupKeyChangeProcess {
+    /** Grid kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Cache group encyption key change prepare phase. */
+    private final DistributedProcess<ChangeCacheEncryptionRequest, EmptyResult> prepareGKChangeProc;
+
+    /** Cache group encyption key change perform phase. */
+    private final DistributedProcess<ChangeCacheEncryptionRequest, EmptyResult> performGKChangeProc;
+
+    /** Group encryption keys. */
+    private final CacheGroupEncryptionKeys keys;
+
+    /** Cache group key change future. */
+    private volatile GroupKeyChangeFuture fut;
+
+    /** Cache group key change request. */
+    private volatile ChangeCacheEncryptionRequest req;
+
+    /**
+     * @param ctx Grid kernal context.
+     */
+    GroupKeyChangeProcess(GridKernalContext ctx, CacheGroupEncryptionKeys keys) {
+        this.ctx = ctx;
+        this.keys = keys;
+
+        prepareGKChangeProc =
+            new DistributedProcess<>(ctx, CACHE_GROUP_KEY_CHANGE_PREPARE, this::prepare, this::finishPrepare);
+        performGKChangeProc =
+            new DistributedProcess<>(ctx, CACHE_GROUP_KEY_CHANGE_FINISH, this::perform, this::finishPerform);
+    }
+
+    /**
+     * @return {@code True} if operation is still in progress.
+     */
+    public boolean started() {

Review comment:
       `started` -> `inProgress`?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GroupKeyChangeProcess.java
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.managers.encryption;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteFeatures;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.managers.encryption.GridEncryptionManager.EmptyResult;
+import org.apache.ignite.internal.managers.encryption.GridEncryptionManager.KeyChangeFuture;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
+import org.apache.ignite.internal.util.distributed.DistributedProcess;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.future.IgniteFinishedFutureImpl;
+import org.apache.ignite.internal.util.future.IgniteFutureImpl;
+import org.apache.ignite.internal.util.typedef.F;
+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.IgniteFuture;
+import org.apache.ignite.lang.IgniteFutureCancelledException;
+
+import static org.apache.ignite.internal.IgniteFeatures.CACHE_GROUP_KEY_CHANGE;
+import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.CACHE_GROUP_KEY_CHANGE_FINISH;
+import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.CACHE_GROUP_KEY_CHANGE_PREPARE;
+
+/**
+ * Two phase distributed process, that performs cache group encryption key rotation.
+ */
+class GroupKeyChangeProcess {

Review comment:
       This process changes active cache group key and initiates re-encryption process, perhaps it's better to change process name, since for example `GroupKeyChangeProcess.finished()` looks confusing and can be treated as "re-encryption process finished". At least javadoc should be added.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -627,22 +716,104 @@ public void groupKey(int grpId, byte[] encGrpKey) {
         return withMasterKeyChangeReadLock(() -> getSpi().getMasterKeyName());
     }
 
+    /** {@inheritDoc} */
+    @Override public IgniteFuture<Void> changeCacheGroupKey(Collection<String> cacheOrGrpNames) {
+        A.notEmpty(cacheOrGrpNames, "cacheOrGrpNames");
+
+        synchronized (opsMux) {
+            if (stopped) {
+                return new IgniteFinishedFutureImpl<>(new IgniteException("Cache group key change was rejected. " +
+                    "Node is stopping."));
+            }
+
+            if (masterKeyChangeFut != null && !masterKeyChangeFut.isDone()) {
+                return new IgniteFinishedFutureImpl<>(new IgniteException("Cache group key change was rejected. " +
+                    "The master key change is in progress."));
+            }
+
+            return grpKeyChangeProc.start(cacheOrGrpNames);
+        }
+    }
+
+    /**
+     * @param grpIds Cache group IDs.
+     * @param keyIds Encryption key IDs.
+     * @param keys Encryption keys.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected void changeCacheGroupKeyLocal(int[] grpIds, byte[] keyIds, byte[][] keys) throws IgniteCheckedException {
+        Map<Integer, Byte> encryptionStatus = U.newHashMap(grpIds.length);
+
+        for (int i = 0; i < grpIds.length; i++)
+            encryptionStatus.put(grpIds[i], keyIds[i]);
+
+        WALPointer ptr = ctx.cache().context().wal().log(new ReencryptionStatusRecord(encryptionStatus));
+
+        if (ptr != null)
+            ctx.cache().context().wal().flush(ptr, false);
+
+        for (int i = 0; i < grpIds.length; i++) {
+            int grpId = grpIds[i];
+
+            CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+            if (grp == null)
+                continue;
+
+            GroupKeyEncrypted key = new GroupKeyEncrypted(keyIds[i] & 0xff, keys[i]);
+
+            synchronized (metaStorageMux) {
+                // Store new key as inactive for recovery.
+                grpKeys.addKey(grpId, key);
+
+                writeToMetaStore(grpId, true, false);
+
+                // Set new key as key for writing.
+                GroupKey prevGrpKey = grpKeys.changeActiveKey(grpId, key);
+
+                assert prevGrpKey != null && prevGrpKey.id() != key.id() : "prev=" + prevGrpKey + ", currId=" + key.id();
+
+                grpKeys.reserveWalKey(grpId, prevGrpKey.unsignedId(), ctx.cache().context().wal().currentSegment());
+
+                writeToMetaStore(grpId, true, true);

Review comment:
       Why can't we write to metastore once? What problem do we solve by these two separate calls?
   
   Also, it's the only place where `writeToMetaStore(grp, true, true)` is used, I think writeToMetaStore can be splitted to two methods writeKeysToMetaStore and writeTrackedWalSegmentsToMetaStore (or something like that) for better readability.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -676,6 +869,57 @@ public void onCacheGroupDestroyed(int grpId) {
         removeGroupKey(grpId);
     }
 
+    /**
+     * @param grp Cache group.
+     * @param partId Partition ID.
+     */
+    public void onDestroyPartitionStore(CacheGroupContext grp, int partId) {
+        pageScanner.cancel(grp.groupId(), partId);
+
+        setEncryptionState(grp, partId, 0, 0);
+    }
+
+    /**
+     * Callabck when WAL segment is removed.
+     *
+     * @param segmentIdx WAL segment index.
+     */
+    public void onWalSegmentRemoved(long segmentIdx) {
+        Map<Integer, Set<Integer>> rmvKeys = grpKeys.releaseWalKeys(segmentIdx);
+
+        if (F.isEmpty(rmvKeys))
+            return;
+
+        synchronized (metaStorageMux) {
+            try {
+                writeToMetaStore(0, false, true);
+
+                for (Map.Entry<Integer, Set<Integer>> entry : rmvKeys.entrySet()) {
+                    int grpId = entry.getKey();

Review comment:
       `int` -> `Integer` to avoid boxing on `containsKey`

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -767,6 +1050,36 @@ public void onCacheGroupDestroyed(int grpId) {
         }
     }
 
+    /**
+     * Set reencryption status for partition.
+     *
+     * @param grp Cache group.
+     * @param partId Partition ID.
+     * @param idx Index of the last reencrypted page.
+     * @param total Total pages to be reencrypted.
+     */
+    public void setEncryptionState(CacheGroupContext grp, int partId, int idx, int total) {
+        long[] states = reencryptGroups.computeIfAbsent(grp.groupId(), v -> new long[grp.affinity().partitions() + 1]);
+
+        states[Math.min(partId, states.length - 1)] = ReencryptStateUtils.state(idx, total);

Review comment:
       Let's add a comment about index partition and `Math.min(partId, states.length - 1)` hack

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -627,22 +716,104 @@ public void groupKey(int grpId, byte[] encGrpKey) {
         return withMasterKeyChangeReadLock(() -> getSpi().getMasterKeyName());
     }
 
+    /** {@inheritDoc} */
+    @Override public IgniteFuture<Void> changeCacheGroupKey(Collection<String> cacheOrGrpNames) {
+        A.notEmpty(cacheOrGrpNames, "cacheOrGrpNames");
+
+        synchronized (opsMux) {
+            if (stopped) {
+                return new IgniteFinishedFutureImpl<>(new IgniteException("Cache group key change was rejected. " +
+                    "Node is stopping."));
+            }
+
+            if (masterKeyChangeFut != null && !masterKeyChangeFut.isDone()) {
+                return new IgniteFinishedFutureImpl<>(new IgniteException("Cache group key change was rejected. " +
+                    "The master key change is in progress."));
+            }
+
+            return grpKeyChangeProc.start(cacheOrGrpNames);
+        }
+    }
+
+    /**
+     * @param grpIds Cache group IDs.
+     * @param keyIds Encryption key IDs.
+     * @param keys Encryption keys.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected void changeCacheGroupKeyLocal(int[] grpIds, byte[] keyIds, byte[][] keys) throws IgniteCheckedException {
+        Map<Integer, Byte> encryptionStatus = U.newHashMap(grpIds.length);
+
+        for (int i = 0; i < grpIds.length; i++)
+            encryptionStatus.put(grpIds[i], keyIds[i]);
+
+        WALPointer ptr = ctx.cache().context().wal().log(new ReencryptionStatusRecord(encryptionStatus));
+
+        if (ptr != null)
+            ctx.cache().context().wal().flush(ptr, false);
+
+        for (int i = 0; i < grpIds.length; i++) {
+            int grpId = grpIds[i];
+
+            CacheGroupContext grp = ctx.cache().cacheGroup(grpId);
+
+            if (grp == null)
+                continue;
+
+            GroupKeyEncrypted key = new GroupKeyEncrypted(keyIds[i] & 0xff, keys[i]);
+
+            synchronized (metaStorageMux) {
+                // Store new key as inactive for recovery.
+                grpKeys.addKey(grpId, key);
+
+                writeToMetaStore(grpId, true, false);
+
+                // Set new key as key for writing.
+                GroupKey prevGrpKey = grpKeys.changeActiveKey(grpId, key);
+
+                assert prevGrpKey != null && prevGrpKey.id() != key.id() : "prev=" + prevGrpKey + ", currId=" + key.id();
+
+                grpKeys.reserveWalKey(grpId, prevGrpKey.unsignedId(), ctx.cache().context().wal().currentSegment());
+
+                writeToMetaStore(grpId, true, true);
+            }
+
+            reencryptGroups.put(grpId, pageScanner.pagesCount(grp));
+
+            if (log.isInfoEnabled())
+                log.info("New encryption key for group was added [grpId=" + grpId + ", keyId=" + key.id() + "]");
+        }
+
+        startReencryption(encryptionStatus.keySet());
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @return Future that will be completed when reencryption of the specified group is finished.
+     */
+    public IgniteInternalFuture<Void> reencryptionFuture(int grpId) {
+        if (pageScanner.disabled() && reencryptGroups.containsKey(grpId))
+            return new GridFutureAdapter<>();

Review comment:
       This future will never be finished, do we really need unfinished future here?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -194,10 +207,22 @@
      * Master key change prepare process. Checks that all server nodes have the same new master key and then starts
      * finish process.
      */
-    private DistributedProcess<MasterKeyChangeRequest, MasterKeyChangeResult> prepareMKChangeProc;
+    private DistributedProcess<MasterKeyChangeRequest, EmptyResult> prepareMKChangeProc;
 
     /** Process to perform the master key change. Changes master key and reencrypt group keys. */
-    private DistributedProcess<MasterKeyChangeRequest, MasterKeyChangeResult> performMKChangeProc;
+    private DistributedProcess<MasterKeyChangeRequest, EmptyResult> performMKChangeProc;
+
+    /** Two phase distributed process, that performs cache group encryption key rotation. */
+    private GroupKeyChangeProcess grpKeyChangeProc;
+
+    /** Cache groups for which encryption key was changed, and they must be re-encrypted. */
+    private final Map<Integer, long[]> reencryptGroups = new ConcurrentHashMap<>();
+
+    /** Cache groups for which encryption key was changed on node join. */
+    private final Map<Integer, Integer> reencryptGroupsForced = new ConcurrentHashMap<>();

Review comment:
       Why do we only add entries to this map and never clean it up?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -655,19 +826,41 @@ private void removeGroupKey(int grpId) {
 
     /**
      * Callback for cache group start event.
-     * @param grpId Group id.
+     *
+     * @param grpId Cache group ID.
      * @param encKey Encryption key
      */
     public void beforeCacheGroupStart(int grpId, @Nullable byte[] encKey) {
         if (encKey == null || ctx.clientNode())
             return;
 
-        groupKey(grpId, encKey);
+        addGroupKey(grpId, new GroupKeyEncrypted(INITIAL_KEY_ID, encKey));
+    }
+
+    /**
+     * Callback is called before invalidate page memory.
+     *
+     * @param grpId Cache group ID.
+     */
+    public void onCacheGroupStop(int grpId) {
+        IgniteInternalFuture<Void> fut = reencryptionFuture(grpId);
+
+        if (!fut.isDone()) {

Review comment:
       `fut.isDone()` - redundant




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

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



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

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java
##########
@@ -258,6 +258,9 @@
     /** */
     public static final short T_MARKER_PAGE = 33;
 
+    /** */
+    public static final short T_INDEX_META = 34;

Review comment:
       I suggest class PageIndexIO being just T_META with version 2, see no problems with this approach. Just as now, two different classes will add two independent sets of new integers, that's fine. The only thing that's changed is that PageIndexIO will have the same type as its base class.
   
   Types should be immutable while pages are alive. This saves us from a lot of troubles, I wouldn't abandon this tradition. That's exactly what versions are for.
   
   BTW your current code has bugs, you don't mark meta page as dirty in "getOrAllocateCacheMetas" while upgrading, please fix it.
   
   EDIT: I guess you can add getters to base class and implement them in inheritors, while IOs that don't support them could just return 0 or throw UnsupportedOperationException




----------------------------------------------------------------
This is an automated message from the 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