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 2021/08/10 19:01:43 UTC

[GitHub] [ignite] Vladsz83 opened a new pull request #9313: IGNITE-14999v3

Vladsz83 opened a new pull request #9313:
URL: https://github.com/apache/ignite/pull/9313


   After manual replacements of encrypted snapshots works, let's enable dynamic snapshot restoration (`SnapshotRestoreProcess`)


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

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

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



[GitHub] [ignite] Vladsz83 commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java
##########
@@ -170,6 +178,54 @@ public boolean sameSnapshot(SnapshotMetadata compare) {
             Objects.equals(baselineNodes(), compare.baselineNodes());
     }
 
+    /**
+     * @param grpId Cache id or cache group id.
+     * @return {@code True} if cache group is encrypted. {@code False} otherwise.
+     */
+    public boolean isCacheGroupEncrypted(int grpId) {
+        Set<Integer> encrGrpIds = this.encrGrpIds;

Review comment:
       Exactly. Local variable which is not affected by other thred. It guaranties the reference won't change during several reads. Please check https://github.com/apache/ignite/pull/9377 , `EncryptedSnapshotTest.testNPE()`. Try uncomment `//        Set<Integer> encrGrpIds = this.encrGrpIds;` in `SnapshotMetadata.isCacheGroupEncrypted()`




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

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

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



[GitHub] [ignite] Vladsz83 commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -1065,24 +1080,46 @@ public void cancelLocalSnapshotTask(String name) {
 
         GridKernalContext kctx0 = cctx.kernalContext();
 
-        Collection<ClusterNode> bltNodes = F.view(cctx.discovery().serverNodes(AffinityTopologyVersion.NONE),
-            (node) -> CU.baselineNode(node, kctx0.state().clusterState()));
-
-        kctx0.task().setThreadContext(TC_SKIP_AUTH, true);
-        kctx0.task().setThreadContext(TC_SUBGRID, bltNodes);
-
         kctx0.task().execute(SnapshotMetadataCollectorTask.class, name).listen(f0 -> {
             if (f0.error() == null) {
                 Map<ClusterNode, List<SnapshotMetadata>> metas = f0.result();
 
                 Map<Integer, String> grpIds = grps == null ? Collections.emptyMap() :
                     grps.stream().collect(Collectors.toMap(CU::cacheId, v -> v));
 
+                byte[] currentMasterKeyDigest = kctx0.config().getEncryptionSpi().masterKeyDigest();
+
                 for (List<SnapshotMetadata> nodeMetas : metas.values()) {
-                    for (SnapshotMetadata meta : nodeMetas)
+                    for (SnapshotMetadata meta : nodeMetas) {
+                        if (meta.masterKeyDigest() == null)
+                            continue;
+
+                        if (currentMasterKeyDigest == null) {
+                            res.onDone(new SnapshotPartitionsVerifyTaskResult(metas, new IdleVerifyResultV2(
+                                Collections.singletonMap(cctx.localNode(), new IllegalArgumentException("Snapshot '" + meta.snapshotName() +
+                                    "' has encrypted caches while encryption is disabled. No keys exist to decrypt data to validate.")))));
+
+                            return;
+                        }
+
+                        if (!Arrays.equals(meta.masterKeyDigest(), currentMasterKeyDigest)) {
+                            res.onDone(new SnapshotPartitionsVerifyTaskResult(metas, new IdleVerifyResultV2(
+                                Collections.singletonMap(cctx.localNode(), new IllegalArgumentException("Snapshot '" + meta.snapshotName() +
+                                    "' has different signature of the master key. Unable to decrypt data to validate.")))));
+
+                            return;
+                        }
+
                         grpIds.keySet().removeAll(meta.partitions().keySet());
+                    }
                 }
 
+                Collection<ClusterNode> bltNodes = F.view(cctx.discovery().serverNodes(AffinityTopologyVersion.NONE),
+                    (node) -> CU.baselineNode(node, kctx0.state().clusterState()));
+
+                kctx0.task().setThreadContext(TC_SKIP_AUTH, true);

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.

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

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



[GitHub] [ignite] Vladsz83 commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -1065,24 +1080,46 @@ public void cancelLocalSnapshotTask(String name) {
 
         GridKernalContext kctx0 = cctx.kernalContext();
 
-        Collection<ClusterNode> bltNodes = F.view(cctx.discovery().serverNodes(AffinityTopologyVersion.NONE),
-            (node) -> CU.baselineNode(node, kctx0.state().clusterState()));
-
-        kctx0.task().setThreadContext(TC_SKIP_AUTH, true);
-        kctx0.task().setThreadContext(TC_SUBGRID, bltNodes);
-
         kctx0.task().execute(SnapshotMetadataCollectorTask.class, name).listen(f0 -> {
             if (f0.error() == null) {
                 Map<ClusterNode, List<SnapshotMetadata>> metas = f0.result();
 
                 Map<Integer, String> grpIds = grps == null ? Collections.emptyMap() :
                     grps.stream().collect(Collectors.toMap(CU::cacheId, v -> v));
 
+                byte[] currentMasterKeyDigest = kctx0.config().getEncryptionSpi().masterKeyDigest();
+
                 for (List<SnapshotMetadata> nodeMetas : metas.values()) {
-                    for (SnapshotMetadata meta : nodeMetas)
+                    for (SnapshotMetadata meta : nodeMetas) {
+                        if (meta.masterKeyDigest() == null)
+                            continue;

Review comment:
       Fixed

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandler.java
##########
@@ -249,4 +261,66 @@ public SnapshotPartitionsVerifyHandler(GridCacheSharedContext<?, ?> cctx) {
 
         throw new IgniteCheckedException(buf.toString());
     }
+
+    /**
+     * Provides encryption keys stored within snapshot.
+     */
+    private static class SnapshotEncrKeyProvider implements EncryptionCacheKeyProvider {
+        /** Kernal context */
+        private GridKernalContext ctx;

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.

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

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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/StoredCacheData.java
##########
@@ -118,6 +122,20 @@ public StoredCacheData sql(boolean sql) {
         return this;
     }
 
+    /**
+     * @return Chipered encryption key for this cache or cache group. {@code Null} if not encrypted.

Review comment:
       Here and below, replace "Chipered" with "Ciphered"

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java
##########
@@ -170,6 +178,54 @@ public boolean sameSnapshot(SnapshotMetadata compare) {
             Objects.equals(baselineNodes(), compare.baselineNodes());
     }
 
+    /**
+     * @param grpId Cache id or cache group id.
+     * @return {@code True} if cache group is encrypted. {@code False} otherwise.
+     */
+    public boolean isCacheGroupEncrypted(int grpId) {
+        Set<Integer> encrGrpIds = this.encrGrpIds;

Review comment:
       What is a purpose of this local var?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -597,6 +606,12 @@ public File snapshotTmpDir() {
                 "prior to snapshot operation start: " + leftNodes));
         }
 
+        if (!cctx.localNode().isClient() && cctx.kernalContext().encryption().isMasterKeyChangeInProgress()

Review comment:
       In the construction `A && B || C` the C operation will be invoked despite of whether the A is true of false. So it may lead to NPE, as cctx is `null` for client nodes.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -993,8 +1008,19 @@ public void cancelLocalSnapshotTask(String name) {
                     grps.stream().collect(Collectors.toMap(CU::cacheId, v -> v));
 
                 for (List<SnapshotMetadata> nodeMetas : metas.values()) {
-                    for (SnapshotMetadata meta : nodeMetas)
+                    for (SnapshotMetadata meta : nodeMetas) {
+                        if (meta.masterKeyDigest() != null && !Arrays.equals(meta.masterKeyDigest(),
+                            kctx0.config().getEncryptionSpi().masterKeyDigest())) {

Review comment:
       It's possible to get NPE for `kctx0.config().getEncryptionSpi().masterKeyDigest()` for config with disabled encryption.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java
##########
@@ -170,6 +178,54 @@ public boolean sameSnapshot(SnapshotMetadata compare) {
             Objects.equals(baselineNodes(), compare.baselineNodes());
     }
 
+    /**
+     * @param grpId Cache id or cache group id.
+     * @return {@code True} if cache group is encrypted. {@code False} otherwise.
+     */
+    public boolean isCacheGroupEncrypted(int grpId) {
+        Set<Integer> encrGrpIds = this.encrGrpIds;
+
+        return encrGrpIds != null && encrGrpIds.contains(grpId);
+    }
+
+    /**
+     * @param masterKeyDigest Master key digest for encrypted caches.
+     * @return this meta.
+     */
+    public SnapshotMetadata masterKeyDigest(@Nullable byte[] masterKeyDigest) {
+        this.masterKeyDigest = masterKeyDigest == null ? null : masterKeyDigest.clone();
+
+        return this;
+    }
+
+    /**
+     * @return Master key digest for encrypted caches.
+     */
+    public byte[] masterKeyDigest() {
+        byte[] masterKeyDigest = this.masterKeyDigest;
+
+        return masterKeyDigest == null ? null : masterKeyDigest.clone();
+    }
+
+    /**
+     * Stores ids of encrypted cache groups.
+     *
+     * @return this meta.
+     */
+    public SnapshotMetadata encrGrpIds(Collection<Integer> encrGrpIds) {
+        // Might be null even if final due to deserialization of previous version the object;
+        if (this.encrGrpIds == null) {
+            synchronized (this) {
+                if (this.encrGrpIds == null)
+                    this.encrGrpIds = new HashSet<>();
+            }
+        }
+
+        this.encrGrpIds.addAll(encrGrpIds);

Review comment:
       You guard initialization of encGrpIds with `this`, then it's assumed that code runs in concurrent env. But the map is an ordinary HashMap, and it's allowed to being modified (`addAll`) in separate thread without sync.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -993,8 +1008,19 @@ public void cancelLocalSnapshotTask(String name) {
                     grps.stream().collect(Collectors.toMap(CU::cacheId, v -> v));
 
                 for (List<SnapshotMetadata> nodeMetas : metas.values()) {
-                    for (SnapshotMetadata meta : nodeMetas)
+                    for (SnapshotMetadata meta : nodeMetas) {
+                        if (meta.masterKeyDigest() != null && !Arrays.equals(meta.masterKeyDigest(),
+                            kctx0.config().getEncryptionSpi().masterKeyDigest())) {

Review comment:
       `getEncryptionSpi().masterKeyDigest()` performs `KeystoreEncryptionSpi#makeDigest` operation on every iteration for this loop. Let's make a var for it before this loop.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java
##########
@@ -58,6 +60,9 @@
     @GridToStringInclude
     private final List<Integer> grpIds;
 
+    /** Encrypted group ids. */
+    private Set<Integer> encrGrpIds;

Review comment:
       Should it be marked with `GridToStringInclude`? The same question for the `masterKeyDigest`
   




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

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

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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -1517,6 +1525,12 @@ public void applyReencryptionStartRecord(ReencryptionStartRecord rec) {
                 "The previous change was not completed."));
         }
 
+        if (ctx.cache().context().snapshotMgr().isSnapshotCreating()

Review comment:
       What if snapshot create/restore started after this check and before the next line `masterKeyChangeRequest = req;`? Do we need a lock (instead of volatile var) that syncs changing of master key and snapshot process?
   
   I see it like this: 
   1. SnapshotManager tries to acquire the lock on master key, then wait for lock if other process own it.
   2. MasterKeyChangeProcess tries to acquire lock, and if failed then return error to user (without wait for lock).
   
   
   WDYT? Is the sync of those processes are already guaranteed with different approach?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java
##########
@@ -170,6 +178,54 @@ public boolean sameSnapshot(SnapshotMetadata compare) {
             Objects.equals(baselineNodes(), compare.baselineNodes());
     }
 
+    /**
+     * @param grpId Cache id or cache group id.
+     * @return {@code True} if cache group is encrypted. {@code False} otherwise.
+     */
+    public boolean isCacheGroupEncrypted(int grpId) {
+        Set<Integer> encrGrpIds = this.encrGrpIds;

Review comment:
       But the reference `encrGrpIds` is not synced: reference isn't volatile, method doesn't acquire a lock. So this reference isn't observable for other threads within this method, so this var doesn't make sense. 
   
   Make var volatile, or use lock there.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java
##########
@@ -170,6 +178,54 @@ public boolean sameSnapshot(SnapshotMetadata compare) {
             Objects.equals(baselineNodes(), compare.baselineNodes());
     }
 
+    /**
+     * @param grpId Cache id or cache group id.
+     * @return {@code True} if cache group is encrypted. {@code False} otherwise.
+     */
+    public boolean isCacheGroupEncrypted(int grpId) {
+        Set<Integer> encrGrpIds = this.encrGrpIds;
+
+        return encrGrpIds != null && encrGrpIds.contains(grpId);
+    }
+
+    /**
+     * @param masterKeyDigest Master key digest for encrypted caches.
+     * @return this meta.
+     */
+    public SnapshotMetadata masterKeyDigest(@Nullable byte[] masterKeyDigest) {
+        this.masterKeyDigest = masterKeyDigest == null ? null : masterKeyDigest.clone();
+
+        return this;
+    }
+
+    /**
+     * @return Master key digest for encrypted caches.
+     */
+    public byte[] masterKeyDigest() {
+        byte[] masterKeyDigest = this.masterKeyDigest;
+
+        return masterKeyDigest == null ? null : masterKeyDigest.clone();
+    }
+
+    /**
+     * Stores ids of encrypted cache groups.
+     *
+     * @return this meta.
+     */
+    public SnapshotMetadata encrGrpIds(Collection<Integer> encrGrpIds) {
+        // Might be null even if final due to deserialization of previous version the object;
+        if (this.encrGrpIds == null) {
+            synchronized (this) {
+                if (this.encrGrpIds == null)
+                    this.encrGrpIds = new HashSet<>();
+            }
+        }
+
+        this.encrGrpIds.addAll(encrGrpIds);

Review comment:
       In previous comment you said that it's possible for `encrGrpIds` be `nulled after encrGrpIds != null`. So we need a guard for this case there too.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
##########
@@ -3778,43 +3780,50 @@ private void checkReadOnlyState(
 
         GridPlainClosure2<Collection<byte[]>, byte[], IgniteInternalFuture<Boolean>> startCacheClsr =
             (grpKeys, masterKeyDigest) -> {
-            List<DynamicCacheChangeRequest> srvReqs = null;

Review comment:
       Could you please revert tabulation of this block? It's pretty hard to understand what is exactly changed.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -597,6 +606,12 @@ public File snapshotTmpDir() {
                 "prior to snapshot operation start: " + leftNodes));
         }
 
+        if (!cctx.localNode().isClient() && cctx.kernalContext().encryption().isMasterKeyChangeInProgress()

Review comment:
       > we fail at start
   
   Yes, so, do we need an explicit check for client, if it always pass or raise NPE? Maybe replace it with `assert cctx != null`? 




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

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

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



[GitHub] [ignite] Vladsz83 commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -597,6 +606,12 @@ public File snapshotTmpDir() {
                 "prior to snapshot operation start: " + leftNodes));
         }
 
+        if (!cctx.localNode().isClient() && cctx.kernalContext().encryption().isMasterKeyChangeInProgress()

Review comment:
       If cctx is null for client, we fail at start: `if (cctx.kernalContext().clientNode()`. Also snapshots is disables for client nodes because clisents do not have data and have nothing to snapshot.




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

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

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



[GitHub] [ignite] Vladsz83 commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
##########
@@ -3778,43 +3780,50 @@ private void checkReadOnlyState(
 
         GridPlainClosure2<Collection<byte[]>, byte[], IgniteInternalFuture<Boolean>> startCacheClsr =
             (grpKeys, masterKeyDigest) -> {
-            List<DynamicCacheChangeRequest> srvReqs = null;
-            Map<String, DynamicCacheChangeRequest> clientReqs = null;
-
-            Iterator<byte[]> grpKeysIter = grpKeys.iterator();
-
-            for (StoredCacheData ccfg : storedCacheDataList) {
-                assert !ccfg.config().isEncryptionEnabled() || grpKeysIter.hasNext();
-
-                DynamicCacheChangeRequest req = prepareCacheChangeRequest(
-                    ccfg.config(),
-                    ccfg.config().getName(),
-                    null,
-                    resolveCacheType(ccfg.config()),
-                    ccfg.sql(),
-                    failIfExists,
-                    true,
-                    restartId,
-                    disabledAfterStart,
-                    ccfg.queryEntities(),
-                    ccfg.config().isEncryptionEnabled() ? grpKeysIter.next() : null,
-                    ccfg.config().isEncryptionEnabled() ? masterKeyDigest : null);
-
-                if (req != null) {
-                    if (req.clientStartOnly()) {
-                        if (clientReqs == null)
-                            clientReqs = U.newLinkedHashMap(storedCacheDataList.size());
-
-                        clientReqs.put(req.cacheName(), req);
-                    }
-                    else {
-                        if (srvReqs == null)
-                            srvReqs = new ArrayList<>(storedCacheDataList.size());
+                List<DynamicCacheChangeRequest> srvReqs = null;
+                Map<String, DynamicCacheChangeRequest> clientReqs = null;
+
+                Iterator<byte[]> grpKeysIter = grpKeys.iterator();
+
+                for (StoredCacheData ccfg : storedCacheDataList) {
+                    assert ccfg.grpKeyEncrypted() == null || ccfg.config().isEncryptionEnabled();
+
+                    // Reuse encription key if passed for this group. Take next generated otherwise.
+                    GroupKeyEncrypted encrKey = ccfg.config().isEncryptionEnabled() ? (ccfg.grpKeyEncrypted() != null ?
+                        ccfg.grpKeyEncrypted() : new GroupKeyEncrypted(0, grpKeysIter.next())) : null;
+
+                    DynamicCacheChangeRequest req = prepareCacheChangeRequest(
+                        ccfg.config(),
+                        ccfg.config().getName(),
+                        null,
+                        resolveCacheType(ccfg.config()),
+                        ccfg.sql(),
+                        failIfExists,
+                        true,
+                        restartId,
+                        disabledAfterStart,
+                        ccfg.queryEntities(),
+                        encrKey != null ? encrKey.key() : null,
+                        encrKey != null ? masterKeyDigest : null);
+
+                    if (encrKey != null)
+                        req.encryptionKeyId(encrKey.id());

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.

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

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



[GitHub] [ignite] Vladsz83 commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java
##########
@@ -170,6 +178,54 @@ public boolean sameSnapshot(SnapshotMetadata compare) {
             Objects.equals(baselineNodes(), compare.baselineNodes());
     }
 
+    /**
+     * @param grpId Cache id or cache group id.
+     * @return {@code True} if cache group is encrypted. {@code False} otherwise.
+     */
+    public boolean isCacheGroupEncrypted(int grpId) {
+        Set<Integer> encrGrpIds = this.encrGrpIds;

Review comment:
       This is common technique to avoid NPE to me, introducing local varaible. But in our case, as mentioned, once encrGrpIds is assigned, it won't set to null. We can check it once. But for general purpose I wrote it correct

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java
##########
@@ -170,6 +178,54 @@ public boolean sameSnapshot(SnapshotMetadata compare) {
             Objects.equals(baselineNodes(), compare.baselineNodes());
     }
 
+    /**
+     * @param grpId Cache id or cache group id.
+     * @return {@code True} if cache group is encrypted. {@code False} otherwise.
+     */
+    public boolean isCacheGroupEncrypted(int grpId) {
+        Set<Integer> encrGrpIds = this.encrGrpIds;

Review comment:
       This is common technique to avoid NPE to me, introducing local varaible. But in our case, as mentioned, once encrGrpIds is assigned, it won't set to null. We can check it once. But for general purpose I wrote it correctly




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

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

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



[GitHub] [ignite] Vladsz83 commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -993,8 +1008,19 @@ public void cancelLocalSnapshotTask(String name) {
                     grps.stream().collect(Collectors.toMap(CU::cacheId, v -> v));
 
                 for (List<SnapshotMetadata> nodeMetas : metas.values()) {
-                    for (SnapshotMetadata meta : nodeMetas)
+                    for (SnapshotMetadata meta : nodeMetas) {
+                        if (meta.masterKeyDigest() != null && !Arrays.equals(meta.masterKeyDigest(),
+                            kctx0.config().getEncryptionSpi().masterKeyDigest())) {

Review comment:
       Fixed

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -597,6 +606,12 @@ public File snapshotTmpDir() {
                 "prior to snapshot operation start: " + leftNodes));
         }
 
+        if (!cctx.localNode().isClient() && cctx.kernalContext().encryption().isMasterKeyChangeInProgress()

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.

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

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



[GitHub] [ignite] Vladsz83 commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -1065,24 +1080,46 @@ public void cancelLocalSnapshotTask(String name) {
 
         GridKernalContext kctx0 = cctx.kernalContext();
 
-        Collection<ClusterNode> bltNodes = F.view(cctx.discovery().serverNodes(AffinityTopologyVersion.NONE),
-            (node) -> CU.baselineNode(node, kctx0.state().clusterState()));
-
-        kctx0.task().setThreadContext(TC_SKIP_AUTH, true);
-        kctx0.task().setThreadContext(TC_SUBGRID, bltNodes);
-
         kctx0.task().execute(SnapshotMetadataCollectorTask.class, name).listen(f0 -> {
             if (f0.error() == null) {
                 Map<ClusterNode, List<SnapshotMetadata>> metas = f0.result();
 
                 Map<Integer, String> grpIds = grps == null ? Collections.emptyMap() :
                     grps.stream().collect(Collectors.toMap(CU::cacheId, v -> v));
 
+                byte[] currentMasterKeyDigest = kctx0.config().getEncryptionSpi().masterKeyDigest();
+
                 for (List<SnapshotMetadata> nodeMetas : metas.values()) {
-                    for (SnapshotMetadata meta : nodeMetas)
+                    for (SnapshotMetadata meta : nodeMetas) {
+                        if (meta.masterKeyDigest() == null)
+                            continue;
+
+                        if (currentMasterKeyDigest == null) {
+                            res.onDone(new SnapshotPartitionsVerifyTaskResult(metas, new IdleVerifyResultV2(
+                                Collections.singletonMap(cctx.localNode(), new IllegalArgumentException("Snapshot '" + meta.snapshotName() +
+                                    "' has encrypted caches while encryption is disabled. No keys exist to decrypt data to validate.")))));
+
+                            return;
+                        }
+
+                        if (!Arrays.equals(meta.masterKeyDigest(), currentMasterKeyDigest)) {
+                            res.onDone(new SnapshotPartitionsVerifyTaskResult(metas, new IdleVerifyResultV2(
+                                Collections.singletonMap(cctx.localNode(), new IllegalArgumentException("Snapshot '" + meta.snapshotName() +
+                                    "' has different signature of the master key. Unable to decrypt data to validate.")))));
+
+                            return;
+                        }
+
                         grpIds.keySet().removeAll(meta.partitions().keySet());
+                    }
                 }
 
+                Collection<ClusterNode> bltNodes = F.view(cctx.discovery().serverNodes(AffinityTopologyVersion.NONE),
+                    (node) -> CU.baselineNode(node, kctx0.state().clusterState()));
+
+                kctx0.task().setThreadContext(TC_SKIP_AUTH, true);

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.

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

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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -1065,24 +1080,46 @@ public void cancelLocalSnapshotTask(String name) {
 
         GridKernalContext kctx0 = cctx.kernalContext();
 
-        Collection<ClusterNode> bltNodes = F.view(cctx.discovery().serverNodes(AffinityTopologyVersion.NONE),
-            (node) -> CU.baselineNode(node, kctx0.state().clusterState()));
-
-        kctx0.task().setThreadContext(TC_SKIP_AUTH, true);
-        kctx0.task().setThreadContext(TC_SUBGRID, bltNodes);
-
         kctx0.task().execute(SnapshotMetadataCollectorTask.class, name).listen(f0 -> {
             if (f0.error() == null) {
                 Map<ClusterNode, List<SnapshotMetadata>> metas = f0.result();
 
                 Map<Integer, String> grpIds = grps == null ? Collections.emptyMap() :
                     grps.stream().collect(Collectors.toMap(CU::cacheId, v -> v));
 
+                byte[] currentMasterKeyDigest = kctx0.config().getEncryptionSpi().masterKeyDigest();
+
                 for (List<SnapshotMetadata> nodeMetas : metas.values()) {
-                    for (SnapshotMetadata meta : nodeMetas)
+                    for (SnapshotMetadata meta : nodeMetas) {
+                        if (meta.masterKeyDigest() == null)
+                            continue;
+
+                        if (currentMasterKeyDigest == null) {
+                            res.onDone(new SnapshotPartitionsVerifyTaskResult(metas, new IdleVerifyResultV2(
+                                Collections.singletonMap(cctx.localNode(), new IllegalArgumentException("Snapshot '" + meta.snapshotName() +
+                                    "' has encrypted caches while encryption is disabled. No keys exist to decrypt data to validate.")))));
+
+                            return;
+                        }
+
+                        if (!Arrays.equals(meta.masterKeyDigest(), currentMasterKeyDigest)) {
+                            res.onDone(new SnapshotPartitionsVerifyTaskResult(metas, new IdleVerifyResultV2(
+                                Collections.singletonMap(cctx.localNode(), new IllegalArgumentException("Snapshot '" + meta.snapshotName() +
+                                    "' has different signature of the master key. Unable to decrypt data to validate.")))));
+
+                            return;
+                        }
+
                         grpIds.keySet().removeAll(meta.partitions().keySet());
+                    }
                 }
 
+                Collection<ClusterNode> bltNodes = F.view(cctx.discovery().serverNodes(AffinityTopologyVersion.NONE),
+                    (node) -> CU.baselineNode(node, kctx0.state().clusterState()));
+
+                kctx0.task().setThreadContext(TC_SKIP_AUTH, true);

Review comment:
       It duplicates lines below (L1132). What is a purpose?




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

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

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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandler.java
##########
@@ -249,4 +261,66 @@ public SnapshotPartitionsVerifyHandler(GridCacheSharedContext<?, ?> cctx) {
 
         throw new IgniteCheckedException(buf.toString());
     }
+
+    /**
+     * Provides encryption keys stored within snapshot.
+     */
+    private static class SnapshotEncrKeyProvider implements EncryptionCacheKeyProvider {
+        /** Kernal context */
+        private GridKernalContext ctx;

Review comment:
       Here and below, make field final.




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

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

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



[GitHub] [ignite] Vladsz83 commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
##########
@@ -3778,43 +3780,50 @@ private void checkReadOnlyState(
 
         GridPlainClosure2<Collection<byte[]>, byte[], IgniteInternalFuture<Boolean>> startCacheClsr =
             (grpKeys, masterKeyDigest) -> {
-            List<DynamicCacheChangeRequest> srvReqs = null;

Review comment:
       Why don't we follow the checkstyle? This is the code format. What's wrong?




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

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

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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/StoredCacheData.java
##########
@@ -118,6 +122,20 @@ public StoredCacheData sql(boolean sql) {
         return this;
     }
 
+    /**
+     * @return Chipered encryption key for this cache or cache group. {@code Null} if not encrypted.

Review comment:
       Here and below, replace "Chipered" with "Ciphered"

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java
##########
@@ -170,6 +178,54 @@ public boolean sameSnapshot(SnapshotMetadata compare) {
             Objects.equals(baselineNodes(), compare.baselineNodes());
     }
 
+    /**
+     * @param grpId Cache id or cache group id.
+     * @return {@code True} if cache group is encrypted. {@code False} otherwise.
+     */
+    public boolean isCacheGroupEncrypted(int grpId) {
+        Set<Integer> encrGrpIds = this.encrGrpIds;

Review comment:
       What is a purpose of this local var?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -597,6 +606,12 @@ public File snapshotTmpDir() {
                 "prior to snapshot operation start: " + leftNodes));
         }
 
+        if (!cctx.localNode().isClient() && cctx.kernalContext().encryption().isMasterKeyChangeInProgress()

Review comment:
       In the construction `A && B || C` the C operation will be invoked despite of whether the A is true of false. So it may lead to NPE, as cctx is `null` for client nodes.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -993,8 +1008,19 @@ public void cancelLocalSnapshotTask(String name) {
                     grps.stream().collect(Collectors.toMap(CU::cacheId, v -> v));
 
                 for (List<SnapshotMetadata> nodeMetas : metas.values()) {
-                    for (SnapshotMetadata meta : nodeMetas)
+                    for (SnapshotMetadata meta : nodeMetas) {
+                        if (meta.masterKeyDigest() != null && !Arrays.equals(meta.masterKeyDigest(),
+                            kctx0.config().getEncryptionSpi().masterKeyDigest())) {

Review comment:
       It's possible to get NPE for `kctx0.config().getEncryptionSpi().masterKeyDigest()` for config with disabled encryption.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java
##########
@@ -170,6 +178,54 @@ public boolean sameSnapshot(SnapshotMetadata compare) {
             Objects.equals(baselineNodes(), compare.baselineNodes());
     }
 
+    /**
+     * @param grpId Cache id or cache group id.
+     * @return {@code True} if cache group is encrypted. {@code False} otherwise.
+     */
+    public boolean isCacheGroupEncrypted(int grpId) {
+        Set<Integer> encrGrpIds = this.encrGrpIds;
+
+        return encrGrpIds != null && encrGrpIds.contains(grpId);
+    }
+
+    /**
+     * @param masterKeyDigest Master key digest for encrypted caches.
+     * @return this meta.
+     */
+    public SnapshotMetadata masterKeyDigest(@Nullable byte[] masterKeyDigest) {
+        this.masterKeyDigest = masterKeyDigest == null ? null : masterKeyDigest.clone();
+
+        return this;
+    }
+
+    /**
+     * @return Master key digest for encrypted caches.
+     */
+    public byte[] masterKeyDigest() {
+        byte[] masterKeyDigest = this.masterKeyDigest;
+
+        return masterKeyDigest == null ? null : masterKeyDigest.clone();
+    }
+
+    /**
+     * Stores ids of encrypted cache groups.
+     *
+     * @return this meta.
+     */
+    public SnapshotMetadata encrGrpIds(Collection<Integer> encrGrpIds) {
+        // Might be null even if final due to deserialization of previous version the object;
+        if (this.encrGrpIds == null) {
+            synchronized (this) {
+                if (this.encrGrpIds == null)
+                    this.encrGrpIds = new HashSet<>();
+            }
+        }
+
+        this.encrGrpIds.addAll(encrGrpIds);

Review comment:
       You guard initialization of encGrpIds with `this`, then it's assumed that code runs in concurrent env. But the map is an ordinary HashMap, and it's allowed to being modified (`addAll`) in separate thread without sync.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -993,8 +1008,19 @@ public void cancelLocalSnapshotTask(String name) {
                     grps.stream().collect(Collectors.toMap(CU::cacheId, v -> v));
 
                 for (List<SnapshotMetadata> nodeMetas : metas.values()) {
-                    for (SnapshotMetadata meta : nodeMetas)
+                    for (SnapshotMetadata meta : nodeMetas) {
+                        if (meta.masterKeyDigest() != null && !Arrays.equals(meta.masterKeyDigest(),
+                            kctx0.config().getEncryptionSpi().masterKeyDigest())) {

Review comment:
       `getEncryptionSpi().masterKeyDigest()` performs `KeystoreEncryptionSpi#makeDigest` operation on every iteration for this loop. Let's make a var for it before this loop.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java
##########
@@ -58,6 +60,9 @@
     @GridToStringInclude
     private final List<Integer> grpIds;
 
+    /** Encrypted group ids. */
+    private Set<Integer> encrGrpIds;

Review comment:
       Should it be marked with `GridToStringInclude`? The same question for the `masterKeyDigest`
   




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

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

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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
##########
@@ -3778,43 +3780,50 @@ private void checkReadOnlyState(
 
         GridPlainClosure2<Collection<byte[]>, byte[], IgniteInternalFuture<Boolean>> startCacheClsr =
             (grpKeys, masterKeyDigest) -> {
-            List<DynamicCacheChangeRequest> srvReqs = null;
-            Map<String, DynamicCacheChangeRequest> clientReqs = null;
-
-            Iterator<byte[]> grpKeysIter = grpKeys.iterator();
-
-            for (StoredCacheData ccfg : storedCacheDataList) {
-                assert !ccfg.config().isEncryptionEnabled() || grpKeysIter.hasNext();
-
-                DynamicCacheChangeRequest req = prepareCacheChangeRequest(
-                    ccfg.config(),
-                    ccfg.config().getName(),
-                    null,
-                    resolveCacheType(ccfg.config()),
-                    ccfg.sql(),
-                    failIfExists,
-                    true,
-                    restartId,
-                    disabledAfterStart,
-                    ccfg.queryEntities(),
-                    ccfg.config().isEncryptionEnabled() ? grpKeysIter.next() : null,
-                    ccfg.config().isEncryptionEnabled() ? masterKeyDigest : null);
-
-                if (req != null) {
-                    if (req.clientStartOnly()) {
-                        if (clientReqs == null)
-                            clientReqs = U.newLinkedHashMap(storedCacheDataList.size());
-
-                        clientReqs.put(req.cacheName(), req);
-                    }
-                    else {
-                        if (srvReqs == null)
-                            srvReqs = new ArrayList<>(storedCacheDataList.size());
+                List<DynamicCacheChangeRequest> srvReqs = null;
+                Map<String, DynamicCacheChangeRequest> clientReqs = null;
+
+                Iterator<byte[]> grpKeysIter = grpKeys.iterator();
+
+                for (StoredCacheData ccfg : storedCacheDataList) {
+                    assert ccfg.grpKeyEncrypted() == null || ccfg.config().isEncryptionEnabled();
+
+                    // Reuse encription key if passed for this group. Take next generated otherwise.
+                    GroupKeyEncrypted encrKey = ccfg.config().isEncryptionEnabled() ? (ccfg.grpKeyEncrypted() != null ?
+                        ccfg.grpKeyEncrypted() : new GroupKeyEncrypted(0, grpKeysIter.next())) : null;
+
+                    DynamicCacheChangeRequest req = prepareCacheChangeRequest(
+                        ccfg.config(),
+                        ccfg.config().getName(),
+                        null,
+                        resolveCacheType(ccfg.config()),
+                        ccfg.sql(),
+                        failIfExists,
+                        true,
+                        restartId,
+                        disabledAfterStart,
+                        ccfg.queryEntities(),
+                        encrKey != null ? encrKey.key() : null,
+                        encrKey != null ? masterKeyDigest : null);
+
+                    if (encrKey != null)
+                        req.encryptionKeyId(encrKey.id());

Review comment:
       Let's move this setting inside the `prepareCacheChangeRequest` method. Now you have 2 places to fill `DynamicCacheChangeRequest` with info about encrypted key - key and id are set separately. 




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

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

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



[GitHub] [ignite] Vladsz83 commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
##########
@@ -3778,43 +3780,50 @@ private void checkReadOnlyState(
 
         GridPlainClosure2<Collection<byte[]>, byte[], IgniteInternalFuture<Boolean>> startCacheClsr =
             (grpKeys, masterKeyDigest) -> {
-            List<DynamicCacheChangeRequest> srvReqs = null;
-            Map<String, DynamicCacheChangeRequest> clientReqs = null;
-
-            Iterator<byte[]> grpKeysIter = grpKeys.iterator();
-
-            for (StoredCacheData ccfg : storedCacheDataList) {
-                assert !ccfg.config().isEncryptionEnabled() || grpKeysIter.hasNext();
-
-                DynamicCacheChangeRequest req = prepareCacheChangeRequest(
-                    ccfg.config(),
-                    ccfg.config().getName(),
-                    null,
-                    resolveCacheType(ccfg.config()),
-                    ccfg.sql(),
-                    failIfExists,
-                    true,
-                    restartId,
-                    disabledAfterStart,
-                    ccfg.queryEntities(),
-                    ccfg.config().isEncryptionEnabled() ? grpKeysIter.next() : null,
-                    ccfg.config().isEncryptionEnabled() ? masterKeyDigest : null);
-
-                if (req != null) {
-                    if (req.clientStartOnly()) {
-                        if (clientReqs == null)
-                            clientReqs = U.newLinkedHashMap(storedCacheDataList.size());
-
-                        clientReqs.put(req.cacheName(), req);
-                    }
-                    else {
-                        if (srvReqs == null)
-                            srvReqs = new ArrayList<>(storedCacheDataList.size());
+                List<DynamicCacheChangeRequest> srvReqs = null;
+                Map<String, DynamicCacheChangeRequest> clientReqs = null;
+
+                Iterator<byte[]> grpKeysIter = grpKeys.iterator();
+
+                for (StoredCacheData ccfg : storedCacheDataList) {
+                    assert ccfg.grpKeyEncrypted() == null || ccfg.config().isEncryptionEnabled();
+
+                    // Reuse encription key if passed for this group. Take next generated otherwise.
+                    GroupKeyEncrypted encrKey = ccfg.config().isEncryptionEnabled() ? (ccfg.grpKeyEncrypted() != null ?
+                        ccfg.grpKeyEncrypted() : new GroupKeyEncrypted(0, grpKeysIter.next())) : null;
+
+                    DynamicCacheChangeRequest req = prepareCacheChangeRequest(
+                        ccfg.config(),
+                        ccfg.config().getName(),
+                        null,
+                        resolveCacheType(ccfg.config()),
+                        ccfg.sql(),
+                        failIfExists,
+                        true,
+                        restartId,
+                        disabledAfterStart,
+                        ccfg.queryEntities(),
+                        encrKey != null ? encrKey.key() : null,
+                        encrKey != null ? masterKeyDigest : null);
+
+                    if (encrKey != null)
+                        req.encryptionKeyId(encrKey.id());

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.

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

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



[GitHub] [ignite] Vladsz83 commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/GridEncryptionManager.java
##########
@@ -1517,6 +1525,12 @@ public void applyReencryptionStartRecord(ReencryptionStartRecord rec) {
                 "The previous change was not completed."));
         }
 
+        if (ctx.cache().context().snapshotMgr().isSnapshotCreating()

Review comment:
       Master key change and snapshots are distributed processes based on discovery messages which are guaranteed be sequential. Once one process is active, another process will see this state. There is no pure concurrency. Master key change isn't allowed during snapshot. Snapshot is not allowed during master key change and re-encryption. You can check ` testSnapshotFailsDuringCacheKeyChange()`, ` testSnapshotFailsDuringMasterKeyChange()`, ` testMasterKeyChangeDuringSnapshot()`, ` testMasterKeyChangeDuringRestore()`, ` testReencryptionDuringRestore()`, ` testReencryptionDuringSnapshot()`




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

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

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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -597,6 +606,12 @@ public File snapshotTmpDir() {
                 "prior to snapshot operation start: " + leftNodes));
         }
 
+        if (!cctx.localNode().isClient() && cctx.kernalContext().encryption().isMasterKeyChangeInProgress()

Review comment:
       Yes, you're right. `GridCacheContext` is null for client node in `GridQueryProcessor` only.
   
   So, what about to fix boolean logic then - `A && (B || C)`?




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

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

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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -993,8 +1008,19 @@ public void cancelLocalSnapshotTask(String name) {
                     grps.stream().collect(Collectors.toMap(CU::cacheId, v -> v));
 
                 for (List<SnapshotMetadata> nodeMetas : metas.values()) {
-                    for (SnapshotMetadata meta : nodeMetas)
+                    for (SnapshotMetadata meta : nodeMetas) {
+                        if (meta.masterKeyDigest() != null && !Arrays.equals(meta.masterKeyDigest(),
+                            kctx0.config().getEncryptionSpi().masterKeyDigest())) {

Review comment:
       > Is it worth?
   
   But masterKeyDigest() is method of _public_ interface EncryptionSpi that can be overloaded with Ignite user, you never knows what is going under the hood. It can goes by to external storage over unreliable network. Let's prevent this.




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

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

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



[GitHub] [ignite] Vladsz83 commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandler.java
##########
@@ -249,4 +261,66 @@ public SnapshotPartitionsVerifyHandler(GridCacheSharedContext<?, ?> cctx) {
 
         throw new IgniteCheckedException(buf.toString());
     }
+
+    /**
+     * Provides encryption keys stored within snapshot.
+     */
+    private static class SnapshotEncrKeyProvider implements EncryptionCacheKeyProvider {
+        /** Kernal context */
+        private GridKernalContext ctx;

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.

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

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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java
##########
@@ -170,6 +178,54 @@ public boolean sameSnapshot(SnapshotMetadata compare) {
             Objects.equals(baselineNodes(), compare.baselineNodes());
     }
 
+    /**
+     * @param grpId Cache id or cache group id.
+     * @return {@code True} if cache group is encrypted. {@code False} otherwise.
+     */
+    public boolean isCacheGroupEncrypted(int grpId) {
+        Set<Integer> encrGrpIds = this.encrGrpIds;

Review comment:
       You've initialized `encrGrpIds` with the DCL pattern (that requires the object to be declared with `volatile` keyword, and you missed it). It means that `encrGrpIds` is assigned from concurrent context, but your local var just copy thread local var (_that is not observable from other threads_ and may can't see that `encrGrpIds` is initialized) to local var.




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

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

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



[GitHub] [ignite] Vladsz83 commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java
##########
@@ -170,6 +178,54 @@ public boolean sameSnapshot(SnapshotMetadata compare) {
             Objects.equals(baselineNodes(), compare.baselineNodes());
     }
 
+    /**
+     * @param grpId Cache id or cache group id.
+     * @return {@code True} if cache group is encrypted. {@code False} otherwise.
+     */
+    public boolean isCacheGroupEncrypted(int grpId) {
+        Set<Integer> encrGrpIds = this.encrGrpIds;

Review comment:
       Exactly. This brings thread-safety. The test shows it. Makes sense?




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

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

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



[GitHub] [ignite] Vladsz83 commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -597,6 +606,12 @@ public File snapshotTmpDir() {
                 "prior to snapshot operation start: " + leftNodes));
         }
 
+        if (!cctx.localNode().isClient() && cctx.kernalContext().encryption().isMasterKeyChangeInProgress()

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.

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

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



[GitHub] [ignite] Vladsz83 commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -597,6 +606,12 @@ public File snapshotTmpDir() {
                 "prior to snapshot operation start: " + leftNodes));
         }
 
+        if (!cctx.localNode().isClient() && cctx.kernalContext().encryption().isMasterKeyChangeInProgress()

Review comment:
       I don't think so. We do not affect snapshot-on-client behavior in this ticket. It has been working well till now. Shapshots-on-client hasn't raised NPE before the ticket. I believe, because this DistributedProcess doesn't work on client regardless encrypted or not.




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

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

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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/StoredCacheData.java
##########
@@ -118,6 +122,20 @@ public StoredCacheData sql(boolean sql) {
         return this;
     }
 
+    /**
+     * @return Chipered encryption key for this cache or cache group. {@code Null} if not encrypted.

Review comment:
       Here and below, replace "Chipered" with "Ciphered"

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java
##########
@@ -170,6 +178,54 @@ public boolean sameSnapshot(SnapshotMetadata compare) {
             Objects.equals(baselineNodes(), compare.baselineNodes());
     }
 
+    /**
+     * @param grpId Cache id or cache group id.
+     * @return {@code True} if cache group is encrypted. {@code False} otherwise.
+     */
+    public boolean isCacheGroupEncrypted(int grpId) {
+        Set<Integer> encrGrpIds = this.encrGrpIds;

Review comment:
       What is a purpose of this local var?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -597,6 +606,12 @@ public File snapshotTmpDir() {
                 "prior to snapshot operation start: " + leftNodes));
         }
 
+        if (!cctx.localNode().isClient() && cctx.kernalContext().encryption().isMasterKeyChangeInProgress()

Review comment:
       In the construction `A && B || C` the C operation will be invoked despite of whether the A is true of false. So it may lead to NPE, as cctx is `null` for client nodes.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -993,8 +1008,19 @@ public void cancelLocalSnapshotTask(String name) {
                     grps.stream().collect(Collectors.toMap(CU::cacheId, v -> v));
 
                 for (List<SnapshotMetadata> nodeMetas : metas.values()) {
-                    for (SnapshotMetadata meta : nodeMetas)
+                    for (SnapshotMetadata meta : nodeMetas) {
+                        if (meta.masterKeyDigest() != null && !Arrays.equals(meta.masterKeyDigest(),
+                            kctx0.config().getEncryptionSpi().masterKeyDigest())) {

Review comment:
       It's possible to get NPE for `kctx0.config().getEncryptionSpi().masterKeyDigest()` for config with disabled encryption.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java
##########
@@ -170,6 +178,54 @@ public boolean sameSnapshot(SnapshotMetadata compare) {
             Objects.equals(baselineNodes(), compare.baselineNodes());
     }
 
+    /**
+     * @param grpId Cache id or cache group id.
+     * @return {@code True} if cache group is encrypted. {@code False} otherwise.
+     */
+    public boolean isCacheGroupEncrypted(int grpId) {
+        Set<Integer> encrGrpIds = this.encrGrpIds;
+
+        return encrGrpIds != null && encrGrpIds.contains(grpId);
+    }
+
+    /**
+     * @param masterKeyDigest Master key digest for encrypted caches.
+     * @return this meta.
+     */
+    public SnapshotMetadata masterKeyDigest(@Nullable byte[] masterKeyDigest) {
+        this.masterKeyDigest = masterKeyDigest == null ? null : masterKeyDigest.clone();
+
+        return this;
+    }
+
+    /**
+     * @return Master key digest for encrypted caches.
+     */
+    public byte[] masterKeyDigest() {
+        byte[] masterKeyDigest = this.masterKeyDigest;
+
+        return masterKeyDigest == null ? null : masterKeyDigest.clone();
+    }
+
+    /**
+     * Stores ids of encrypted cache groups.
+     *
+     * @return this meta.
+     */
+    public SnapshotMetadata encrGrpIds(Collection<Integer> encrGrpIds) {
+        // Might be null even if final due to deserialization of previous version the object;
+        if (this.encrGrpIds == null) {
+            synchronized (this) {
+                if (this.encrGrpIds == null)
+                    this.encrGrpIds = new HashSet<>();
+            }
+        }
+
+        this.encrGrpIds.addAll(encrGrpIds);

Review comment:
       You guard initialization of encGrpIds with `this`, then it's assumed that code runs in concurrent env. But the map is an ordinary HashMap, and it's allowed to being modified (`addAll`) in separate thread without sync.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -993,8 +1008,19 @@ public void cancelLocalSnapshotTask(String name) {
                     grps.stream().collect(Collectors.toMap(CU::cacheId, v -> v));
 
                 for (List<SnapshotMetadata> nodeMetas : metas.values()) {
-                    for (SnapshotMetadata meta : nodeMetas)
+                    for (SnapshotMetadata meta : nodeMetas) {
+                        if (meta.masterKeyDigest() != null && !Arrays.equals(meta.masterKeyDigest(),
+                            kctx0.config().getEncryptionSpi().masterKeyDigest())) {

Review comment:
       `getEncryptionSpi().masterKeyDigest()` performs `KeystoreEncryptionSpi#makeDigest` operation on every iteration for this loop. Let's make a var for it before this loop.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java
##########
@@ -58,6 +60,9 @@
     @GridToStringInclude
     private final List<Integer> grpIds;
 
+    /** Encrypted group ids. */
+    private Set<Integer> encrGrpIds;

Review comment:
       Should it be marked with `GridToStringInclude`? The same question for the `masterKeyDigest`
   




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

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

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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -1065,24 +1080,46 @@ public void cancelLocalSnapshotTask(String name) {
 
         GridKernalContext kctx0 = cctx.kernalContext();
 
-        Collection<ClusterNode> bltNodes = F.view(cctx.discovery().serverNodes(AffinityTopologyVersion.NONE),
-            (node) -> CU.baselineNode(node, kctx0.state().clusterState()));
-
-        kctx0.task().setThreadContext(TC_SKIP_AUTH, true);
-        kctx0.task().setThreadContext(TC_SUBGRID, bltNodes);
-
         kctx0.task().execute(SnapshotMetadataCollectorTask.class, name).listen(f0 -> {
             if (f0.error() == null) {
                 Map<ClusterNode, List<SnapshotMetadata>> metas = f0.result();
 
                 Map<Integer, String> grpIds = grps == null ? Collections.emptyMap() :
                     grps.stream().collect(Collectors.toMap(CU::cacheId, v -> v));
 
+                byte[] currentMasterKeyDigest = kctx0.config().getEncryptionSpi().masterKeyDigest();
+
                 for (List<SnapshotMetadata> nodeMetas : metas.values()) {
-                    for (SnapshotMetadata meta : nodeMetas)
+                    for (SnapshotMetadata meta : nodeMetas) {
+                        if (meta.masterKeyDigest() == null)
+                            continue;

Review comment:
       Should we remove the meta partitions from the `grpIds` collection (see 1113 line)? Now your changes affect only encrypted cached, what about non-encrypted?




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

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

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



[GitHub] [ignite] Vladsz83 commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -597,6 +606,12 @@ public File snapshotTmpDir() {
                 "prior to snapshot operation start: " + leftNodes));
         }
 
+        if (!cctx.localNode().isClient() && cctx.kernalContext().encryption().isMasterKeyChangeInProgress()

Review comment:
       I don't think so. We do not affect snapshot-on-client behavior in this ticket. It has been working well till now. Shapshots-on-client hasn't raised NPE before the ticket. I believe, because this DistributedProcess doesn't work on client regardless encrypted or not. There is `cctx.kernalContext().clientNode()` before. It hasn't been asserted earlier. I don't think we should care.




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

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

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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
##########
@@ -1929,17 +1929,27 @@ protected void assertEqualsCollections(Collection<?> exp, Collection<?> act) {
     }
 
     /**
-     *
+     * Cleans persistence directory including snapshots.
      */
     protected void cleanPersistenceDir() throws Exception {
+        cleanPersistenceDir(true);

Review comment:
       I don't like an idea that we change a signature of this wide used method. It is extremely rare case to not clean snapshots in tests. What are alternatives? For example, create snapshots in different directory for tests that don't clean them, or introduce a method for copy-pasting snapshots. WDYT?




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

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

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



[GitHub] [ignite] Vladsz83 commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/StoredCacheData.java
##########
@@ -118,6 +122,20 @@ public StoredCacheData sql(boolean sql) {
         return this;
     }
 
+    /**
+     * @return Chipered encryption key for this cache or cache group. {@code Null} if not encrypted.

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.

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

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



[GitHub] [ignite] Vladsz83 commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
##########
@@ -1929,17 +1929,27 @@ protected void assertEqualsCollections(Collection<?> exp, Collection<?> act) {
     }
 
     /**
-     *
+     * Cleans persistence directory including snapshots.
      */
     protected void cleanPersistenceDir() throws Exception {
+        cleanPersistenceDir(true);

Review comment:
       But we didn't change the signature. We added new method. You can use previous method as usual.




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

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

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



[GitHub] [ignite] Vladsz83 commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
##########
@@ -1929,17 +1929,27 @@ protected void assertEqualsCollections(Collection<?> exp, Collection<?> act) {
     }
 
     /**
-     *
+     * Cleans persistence directory including snapshots.
      */
     protected void cleanPersistenceDir() throws Exception {
+        cleanPersistenceDir(true);

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.

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

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



[GitHub] [ignite] Vladsz83 commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java
##########
@@ -170,6 +178,54 @@ public boolean sameSnapshot(SnapshotMetadata compare) {
             Objects.equals(baselineNodes(), compare.baselineNodes());
     }
 
+    /**
+     * @param grpId Cache id or cache group id.
+     * @return {@code True} if cache group is encrypted. {@code False} otherwise.
+     */
+    public boolean isCacheGroupEncrypted(int grpId) {
+        Set<Integer> encrGrpIds = this.encrGrpIds;
+
+        return encrGrpIds != null && encrGrpIds.contains(grpId);
+    }
+
+    /**
+     * @param masterKeyDigest Master key digest for encrypted caches.
+     * @return this meta.
+     */
+    public SnapshotMetadata masterKeyDigest(@Nullable byte[] masterKeyDigest) {
+        this.masterKeyDigest = masterKeyDigest == null ? null : masterKeyDigest.clone();
+
+        return this;
+    }
+
+    /**
+     * @return Master key digest for encrypted caches.
+     */
+    public byte[] masterKeyDigest() {
+        byte[] masterKeyDigest = this.masterKeyDigest;
+
+        return masterKeyDigest == null ? null : masterKeyDigest.clone();
+    }
+
+    /**
+     * Stores ids of encrypted cache groups.
+     *
+     * @return this meta.
+     */
+    public SnapshotMetadata encrGrpIds(Collection<Integer> encrGrpIds) {
+        // Might be null even if final due to deserialization of previous version the object;
+        if (this.encrGrpIds == null) {
+            synchronized (this) {
+                if (this.encrGrpIds == null)
+                    this.encrGrpIds = new HashSet<>();
+            }
+        }
+
+        this.encrGrpIds.addAll(encrGrpIds);

Review comment:
        no encrypted groups ids reside in SnapshotMeta any more. inactual




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

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

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



[GitHub] [ignite] Vladsz83 commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -993,8 +1008,19 @@ public void cancelLocalSnapshotTask(String name) {
                     grps.stream().collect(Collectors.toMap(CU::cacheId, v -> v));
 
                 for (List<SnapshotMetadata> nodeMetas : metas.values()) {
-                    for (SnapshotMetadata meta : nodeMetas)
+                    for (SnapshotMetadata meta : nodeMetas) {
+                        if (meta.masterKeyDigest() != null && !Arrays.equals(meta.masterKeyDigest(),
+                            kctx0.config().getEncryptionSpi().masterKeyDigest())) {

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.

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

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



[GitHub] [ignite] Vladsz83 commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java
##########
@@ -58,6 +60,9 @@
     @GridToStringInclude
     private final List<Integer> grpIds;
 
+    /** Encrypted group ids. */
+    private Set<Integer> encrGrpIds;

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.

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

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



[GitHub] [ignite] Vladsz83 commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java
##########
@@ -170,6 +178,54 @@ public boolean sameSnapshot(SnapshotMetadata compare) {
             Objects.equals(baselineNodes(), compare.baselineNodes());
     }
 
+    /**
+     * @param grpId Cache id or cache group id.
+     * @return {@code True} if cache group is encrypted. {@code False} otherwise.
+     */
+    public boolean isCacheGroupEncrypted(int grpId) {
+        Set<Integer> encrGrpIds = this.encrGrpIds;

Review comment:
       to avoid NPE if class member has been nulled after `encrGrpIds != null`




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

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

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



[GitHub] [ignite] Vladsz83 commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/StoredCacheData.java
##########
@@ -118,6 +122,20 @@ public StoredCacheData sql(boolean sql) {
         return this;
     }
 
+    /**
+     * @return Chipered encryption key for this cache or cache group. {@code Null} if not encrypted.

Review comment:
       Fixed

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java
##########
@@ -170,6 +178,54 @@ public boolean sameSnapshot(SnapshotMetadata compare) {
             Objects.equals(baselineNodes(), compare.baselineNodes());
     }
 
+    /**
+     * @param grpId Cache id or cache group id.
+     * @return {@code True} if cache group is encrypted. {@code False} otherwise.
+     */
+    public boolean isCacheGroupEncrypted(int grpId) {
+        Set<Integer> encrGrpIds = this.encrGrpIds;
+
+        return encrGrpIds != null && encrGrpIds.contains(grpId);
+    }
+
+    /**
+     * @param masterKeyDigest Master key digest for encrypted caches.
+     * @return this meta.
+     */
+    public SnapshotMetadata masterKeyDigest(@Nullable byte[] masterKeyDigest) {
+        this.masterKeyDigest = masterKeyDigest == null ? null : masterKeyDigest.clone();
+
+        return this;
+    }
+
+    /**
+     * @return Master key digest for encrypted caches.
+     */
+    public byte[] masterKeyDigest() {
+        byte[] masterKeyDigest = this.masterKeyDigest;
+
+        return masterKeyDigest == null ? null : masterKeyDigest.clone();
+    }
+
+    /**
+     * Stores ids of encrypted cache groups.
+     *
+     * @return this meta.
+     */
+    public SnapshotMetadata encrGrpIds(Collection<Integer> encrGrpIds) {
+        // Might be null even if final due to deserialization of previous version the object;
+        if (this.encrGrpIds == null) {
+            synchronized (this) {
+                if (this.encrGrpIds == null)
+                    this.encrGrpIds = new HashSet<>();
+            }
+        }
+
+        this.encrGrpIds.addAll(encrGrpIds);

Review comment:
       Fixed

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java
##########
@@ -170,6 +178,54 @@ public boolean sameSnapshot(SnapshotMetadata compare) {
             Objects.equals(baselineNodes(), compare.baselineNodes());
     }
 
+    /**
+     * @param grpId Cache id or cache group id.
+     * @return {@code True} if cache group is encrypted. {@code False} otherwise.
+     */
+    public boolean isCacheGroupEncrypted(int grpId) {
+        Set<Integer> encrGrpIds = this.encrGrpIds;

Review comment:
       to avoid NPE if class member has been nulled after `encrGrpIds != null`

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java
##########
@@ -58,6 +60,9 @@
     @GridToStringInclude
     private final List<Integer> grpIds;
 
+    /** Encrypted group ids. */
+    private Set<Integer> encrGrpIds;

Review comment:
       Fixed

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -993,8 +1008,19 @@ public void cancelLocalSnapshotTask(String name) {
                     grps.stream().collect(Collectors.toMap(CU::cacheId, v -> v));
 
                 for (List<SnapshotMetadata> nodeMetas : metas.values()) {
-                    for (SnapshotMetadata meta : nodeMetas)
+                    for (SnapshotMetadata meta : nodeMetas) {
+                        if (meta.masterKeyDigest() != null && !Arrays.equals(meta.masterKeyDigest(),
+                            kctx0.config().getEncryptionSpi().masterKeyDigest())) {

Review comment:
       "performs KeystoreEncryptionSpi#makeDigest operation on every" But verifying snapshot is slow operation which involves many disk reading, page reading one by one from the partition files. Is it worth?
   
   

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -993,8 +1008,19 @@ public void cancelLocalSnapshotTask(String name) {
                     grps.stream().collect(Collectors.toMap(CU::cacheId, v -> v));
 
                 for (List<SnapshotMetadata> nodeMetas : metas.values()) {
-                    for (SnapshotMetadata meta : nodeMetas)
+                    for (SnapshotMetadata meta : nodeMetas) {
+                        if (meta.masterKeyDigest() != null && !Arrays.equals(meta.masterKeyDigest(),
+                            kctx0.config().getEncryptionSpi().masterKeyDigest())) {

Review comment:
       "It's possible to get NPE for kctx0.config().getEncryptionSpi().masterKeyDigest() for config with disabled encryption." Impossible. Toy can find `if (cfg.getEncryptionSpi() == null)
                   cfg.setEncryptionSpi(new NoopEncryptionSpi());` in IgniteEx.initializeDefaultSpi() and in `EncryptedSnapshotTest.testValidatingSnapshotFailsWithNoEncryption`

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -597,6 +606,12 @@ public File snapshotTmpDir() {
                 "prior to snapshot operation start: " + leftNodes));
         }
 
+        if (!cctx.localNode().isClient() && cctx.kernalContext().encryption().isMasterKeyChangeInProgress()

Review comment:
       If cctx is null for client, we fail at start: `if (cctx.kernalContext().clientNode()`. Also snapshots is disables for client nodes because clisents do not have data and have nothing to snapshot.




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

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

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



[GitHub] [ignite] NSAmelchev merged pull request #9313: IGNITE-14999v3 : Support dynamic restoration of encrypted snapshots.

Posted by GitBox <gi...@apache.org>.
NSAmelchev merged pull request #9313:
URL: https://github.com/apache/ignite/pull/9313


   


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

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

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



[GitHub] [ignite] Vladsz83 commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -993,8 +1008,19 @@ public void cancelLocalSnapshotTask(String name) {
                     grps.stream().collect(Collectors.toMap(CU::cacheId, v -> v));
 
                 for (List<SnapshotMetadata> nodeMetas : metas.values()) {
-                    for (SnapshotMetadata meta : nodeMetas)
+                    for (SnapshotMetadata meta : nodeMetas) {
+                        if (meta.masterKeyDigest() != null && !Arrays.equals(meta.masterKeyDigest(),
+                            kctx0.config().getEncryptionSpi().masterKeyDigest())) {

Review comment:
       "performs KeystoreEncryptionSpi#makeDigest operation on every" But verifying snapshot is slow operation which involves many disk reading, page reading one by one from the partition files. Is it worth?
   
   




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

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

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



[GitHub] [ignite] Vladsz83 commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -993,8 +1008,19 @@ public void cancelLocalSnapshotTask(String name) {
                     grps.stream().collect(Collectors.toMap(CU::cacheId, v -> v));
 
                 for (List<SnapshotMetadata> nodeMetas : metas.values()) {
-                    for (SnapshotMetadata meta : nodeMetas)
+                    for (SnapshotMetadata meta : nodeMetas) {
+                        if (meta.masterKeyDigest() != null && !Arrays.equals(meta.masterKeyDigest(),
+                            kctx0.config().getEncryptionSpi().masterKeyDigest())) {

Review comment:
       "It's possible to get NPE for kctx0.config().getEncryptionSpi().masterKeyDigest() for config with disabled encryption." Impossible. Toy can find `if (cfg.getEncryptionSpi() == null)
                   cfg.setEncryptionSpi(new NoopEncryptionSpi());` in IgniteEx.initializeDefaultSpi() and in `EncryptedSnapshotTest.testValidatingSnapshotFailsWithNoEncryption`




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

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

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



[GitHub] [ignite] Vladsz83 commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java
##########
@@ -170,6 +178,54 @@ public boolean sameSnapshot(SnapshotMetadata compare) {
             Objects.equals(baselineNodes(), compare.baselineNodes());
     }
 
+    /**
+     * @param grpId Cache id or cache group id.
+     * @return {@code True} if cache group is encrypted. {@code False} otherwise.
+     */
+    public boolean isCacheGroupEncrypted(int grpId) {
+        Set<Integer> encrGrpIds = this.encrGrpIds;
+
+        return encrGrpIds != null && encrGrpIds.contains(grpId);
+    }
+
+    /**
+     * @param masterKeyDigest Master key digest for encrypted caches.
+     * @return this meta.
+     */
+    public SnapshotMetadata masterKeyDigest(@Nullable byte[] masterKeyDigest) {
+        this.masterKeyDigest = masterKeyDigest == null ? null : masterKeyDigest.clone();
+
+        return this;
+    }
+
+    /**
+     * @return Master key digest for encrypted caches.
+     */
+    public byte[] masterKeyDigest() {
+        byte[] masterKeyDigest = this.masterKeyDigest;
+
+        return masterKeyDigest == null ? null : masterKeyDigest.clone();
+    }
+
+    /**
+     * Stores ids of encrypted cache groups.
+     *
+     * @return this meta.
+     */
+    public SnapshotMetadata encrGrpIds(Collection<Integer> encrGrpIds) {
+        // Might be null even if final due to deserialization of previous version the object;
+        if (this.encrGrpIds == null) {
+            synchronized (this) {
+                if (this.encrGrpIds == null)
+                    this.encrGrpIds = new HashSet<>();
+            }
+        }
+
+        this.encrGrpIds.addAll(encrGrpIds);

Review comment:
       Nope. Once `encrGrpIds` gets assigned, it never takes null. User cannot null it.




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

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

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



[GitHub] [ignite] Vladsz83 commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -1065,24 +1080,46 @@ public void cancelLocalSnapshotTask(String name) {
 
         GridKernalContext kctx0 = cctx.kernalContext();
 
-        Collection<ClusterNode> bltNodes = F.view(cctx.discovery().serverNodes(AffinityTopologyVersion.NONE),
-            (node) -> CU.baselineNode(node, kctx0.state().clusterState()));
-
-        kctx0.task().setThreadContext(TC_SKIP_AUTH, true);
-        kctx0.task().setThreadContext(TC_SUBGRID, bltNodes);
-
         kctx0.task().execute(SnapshotMetadataCollectorTask.class, name).listen(f0 -> {
             if (f0.error() == null) {
                 Map<ClusterNode, List<SnapshotMetadata>> metas = f0.result();
 
                 Map<Integer, String> grpIds = grps == null ? Collections.emptyMap() :
                     grps.stream().collect(Collectors.toMap(CU::cacheId, v -> v));
 
+                byte[] currentMasterKeyDigest = kctx0.config().getEncryptionSpi().masterKeyDigest();
+
                 for (List<SnapshotMetadata> nodeMetas : metas.values()) {
-                    for (SnapshotMetadata meta : nodeMetas)
+                    for (SnapshotMetadata meta : nodeMetas) {
+                        if (meta.masterKeyDigest() == null)
+                            continue;

Review comment:
       What is "meta partitions"? it is snapshot metadata. We check encryption, master keys here. However, another fixes applied.




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

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

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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java
##########
@@ -170,6 +178,54 @@ public boolean sameSnapshot(SnapshotMetadata compare) {
             Objects.equals(baselineNodes(), compare.baselineNodes());
     }
 
+    /**
+     * @param grpId Cache id or cache group id.
+     * @return {@code True} if cache group is encrypted. {@code False} otherwise.
+     */
+    public boolean isCacheGroupEncrypted(int grpId) {
+        Set<Integer> encrGrpIds = this.encrGrpIds;

Review comment:
       You've initialized `encrGrpIds` with the DCL pattern (that requires the object to be declared with `volatile` keyword, and you missed it). It means that `encrGrpIds` is assigned from concurrent context, but you just copy thread local var (_that is not observable from other threads_ and may can't see that `encrGrpIds` is initialized) to local var.




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

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

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



[GitHub] [ignite] Vladsz83 commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java
##########
@@ -170,6 +178,54 @@ public boolean sameSnapshot(SnapshotMetadata compare) {
             Objects.equals(baselineNodes(), compare.baselineNodes());
     }
 
+    /**
+     * @param grpId Cache id or cache group id.
+     * @return {@code True} if cache group is encrypted. {@code False} otherwise.
+     */
+    public boolean isCacheGroupEncrypted(int grpId) {
+        Set<Integer> encrGrpIds = this.encrGrpIds;

Review comment:
       No encrypted groups ids reside in SnapshotMeta any more. Inactual.




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

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

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



[GitHub] [ignite] Vladsz83 commented on a change in pull request #9313: IGNITE-14999v3

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java
##########
@@ -170,6 +178,54 @@ public boolean sameSnapshot(SnapshotMetadata compare) {
             Objects.equals(baselineNodes(), compare.baselineNodes());
     }
 
+    /**
+     * @param grpId Cache id or cache group id.
+     * @return {@code True} if cache group is encrypted. {@code False} otherwise.
+     */
+    public boolean isCacheGroupEncrypted(int grpId) {
+        Set<Integer> encrGrpIds = this.encrGrpIds;
+
+        return encrGrpIds != null && encrGrpIds.contains(grpId);
+    }
+
+    /**
+     * @param masterKeyDigest Master key digest for encrypted caches.
+     * @return this meta.
+     */
+    public SnapshotMetadata masterKeyDigest(@Nullable byte[] masterKeyDigest) {
+        this.masterKeyDigest = masterKeyDigest == null ? null : masterKeyDigest.clone();
+
+        return this;
+    }
+
+    /**
+     * @return Master key digest for encrypted caches.
+     */
+    public byte[] masterKeyDigest() {
+        byte[] masterKeyDigest = this.masterKeyDigest;
+
+        return masterKeyDigest == null ? null : masterKeyDigest.clone();
+    }
+
+    /**
+     * Stores ids of encrypted cache groups.
+     *
+     * @return this meta.
+     */
+    public SnapshotMetadata encrGrpIds(Collection<Integer> encrGrpIds) {
+        // Might be null even if final due to deserialization of previous version the object;
+        if (this.encrGrpIds == null) {
+            synchronized (this) {
+                if (this.encrGrpIds == null)
+                    this.encrGrpIds = new HashSet<>();
+            }
+        }
+
+        this.encrGrpIds.addAll(encrGrpIds);

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.

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

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