You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2022/06/28 16:52:17 UTC

[GitHub] [ignite] sashapolo opened a new pull request, #10118: IGNITE-17251 Fix parsing of encrypted records by IgniteWalConverter

sashapolo opened a new pull request, #10118:
URL: https://github.com/apache/ignite/pull/10118

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


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

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

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


[GitHub] [ignite] sashapolo commented on a diff in pull request #10118: IGNITE-17251 Fix parsing of encrypted records by IgniteWalConverter

Posted by GitBox <gi...@apache.org>.
sashapolo commented on code in PR #10118:
URL: https://github.com/apache/ignite/pull/10118#discussion_r910912783


##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java:
##########
@@ -287,57 +278,45 @@ private boolean needEncryption(int grpId) {
      * @throws IOException If failed.
      * @throws IgniteCheckedException If failed.
      */
-    private T3<ByteBufferBackedDataInput, Integer, RecordType> readEncryptedData(
+    private EncryptedData readEncryptedData(
         ByteBufferBackedDataInput in,
         boolean readType,
         boolean readKeyId
     ) throws IOException, IgniteCheckedException {
         int grpId = in.readInt();
         int encRecSz = in.readInt();
 
-        RecordType plainRecType = null;
-
-        if (readType)
-            plainRecType = RecordV1Serializer.readRecordType(in);
+        RecordType plainRecType = readType ? RecordV1Serializer.readRecordType(in) : null;
 
         int keyId = readKeyId ? in.readUnsignedByte() : GridEncryptionManager.INITIAL_KEY_ID;
 
-        byte[] encData = new byte[encRecSz];
+        // Encryption Manager can be null during offline WAL iteration
+        if (encMgr == null) {
+            int skipped = in.skipBytes(encRecSz);
 
-        in.readFully(encData);
+            assert skipped == encRecSz;
 
-        GroupKey grpKey = encMgr.groupKey(grpId, keyId);
+            return new EncryptedData(null, plainRecType, grpId);
+        }
 
-        if (grpKey == null)
-            return new T3<>(null, grpId, plainRecType);
+        GroupKey grpKey = encMgr.groupKey(grpId, keyId);
 
-        byte[] clData = encSpi.decrypt(encData, grpKey.key());
+        // Encryption key is not available when restoring the MetaStorage
+        if (grpKey == null) {
+            int skipped = in.skipBytes(encRecSz);

Review Comment:
   This block is three lines of code and used in only two places inside one method. I think it is better as it is



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

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

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


[GitHub] [ignite] sashapolo commented on a diff in pull request #10118: IGNITE-17251 Fix parsing of encrypted records by IgniteWalConverter

Posted by GitBox <gi...@apache.org>.
sashapolo commented on code in PR #10118:
URL: https://github.com/apache/ignite/pull/10118#discussion_r910912783


##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java:
##########
@@ -287,57 +278,45 @@ private boolean needEncryption(int grpId) {
      * @throws IOException If failed.
      * @throws IgniteCheckedException If failed.
      */
-    private T3<ByteBufferBackedDataInput, Integer, RecordType> readEncryptedData(
+    private EncryptedData readEncryptedData(
         ByteBufferBackedDataInput in,
         boolean readType,
         boolean readKeyId
     ) throws IOException, IgniteCheckedException {
         int grpId = in.readInt();
         int encRecSz = in.readInt();
 
-        RecordType plainRecType = null;
-
-        if (readType)
-            plainRecType = RecordV1Serializer.readRecordType(in);
+        RecordType plainRecType = readType ? RecordV1Serializer.readRecordType(in) : null;
 
         int keyId = readKeyId ? in.readUnsignedByte() : GridEncryptionManager.INITIAL_KEY_ID;
 
-        byte[] encData = new byte[encRecSz];
+        // Encryption Manager can be null during offline WAL iteration
+        if (encMgr == null) {
+            int skipped = in.skipBytes(encRecSz);
 
-        in.readFully(encData);
+            assert skipped == encRecSz;
 
-        GroupKey grpKey = encMgr.groupKey(grpId, keyId);
+            return new EncryptedData(null, plainRecType, grpId);
+        }
 
-        if (grpKey == null)
-            return new T3<>(null, grpId, plainRecType);
+        GroupKey grpKey = encMgr.groupKey(grpId, keyId);
 
-        byte[] clData = encSpi.decrypt(encData, grpKey.key());
+        // Encryption key is not available when restoring the MetaStorage
+        if (grpKey == null) {
+            int skipped = in.skipBytes(encRecSz);

Review Comment:
   This block is three lines of code and used in only two places. I think it is better as it is



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

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

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


[GitHub] [ignite] sashapolo commented on pull request #10118: IGNITE-17251 Fix parsing of encrypted records by IgniteWalConverter

Posted by GitBox <gi...@apache.org>.
sashapolo commented on PR #10118:
URL: https://github.com/apache/ignite/pull/10118#issuecomment-1171176679

   I've re-run the basic tests: https://ci.ignite.apache.org/buildConfiguration/IgniteTests24Java8_RunBasicTests/6657118?buildTab=overview


-- 
This is an automated message from the 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] rpuch commented on pull request #10118: IGNITE-17251 Fix parsing of encrypted records by IgniteWalConverter

Posted by GitBox <gi...@apache.org>.
rpuch commented on PR #10118:
URL: https://github.com/apache/ignite/pull/10118#issuecomment-1171124605

   The patch looks good to me, thanks!


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

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

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


[GitHub] [ignite] sashapolo commented on a diff in pull request #10118: IGNITE-17251 Fix parsing of encrypted records by IgniteWalConverter

Posted by GitBox <gi...@apache.org>.
sashapolo commented on code in PR #10118:
URL: https://github.com/apache/ignite/pull/10118#discussion_r910915392


##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/EncryptedData.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.persistence.wal.serializer;
+
+import java.nio.ByteBuffer;
+import org.apache.ignite.internal.pagemem.wal.record.WALRecord;
+import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferBackedDataInput;
+import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferBackedDataInputImpl;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Class that represents information about an encrypted WAL record or data entry.
+ */
+class EncryptedData {
+    /** Decrypted WAL record or {@code null} if it couldn't be decrypted. */
+    @Nullable
+    private final ByteBufferBackedDataInput data;

Review Comment:
   Renamed to `decryptedData`



-- 
This is an automated message from the 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] sashapolo commented on a diff in pull request #10118: IGNITE-17251 Fix parsing of encrypted records by IgniteWalConverter

Posted by GitBox <gi...@apache.org>.
sashapolo commented on code in PR #10118:
URL: https://github.com/apache/ignite/pull/10118#discussion_r910916065


##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java:
##########
@@ -210,22 +208,15 @@ public RecordDataV1Serializer(GridCacheSharedContext cctx) {
     @Override public WALRecord readRecord(RecordType type, ByteBufferBackedDataInput in, int size)
         throws IOException, IgniteCheckedException {
         if (type == ENCRYPTED_RECORD || type == ENCRYPTED_RECORD_V2) {
-            if (encSpi == null) {

Review Comment:
   > Is this because encMgr will always be null if encSpi is null?
   
   Yes, I'll add a check



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

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

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


[GitHub] [ignite] rpuch commented on a diff in pull request #10118: IGNITE-17251 Fix parsing of encrypted records by IgniteWalConverter

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


##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java:
##########
@@ -210,22 +208,15 @@ public RecordDataV1Serializer(GridCacheSharedContext cctx) {
     @Override public WALRecord readRecord(RecordType type, ByteBufferBackedDataInput in, int size)
         throws IOException, IgniteCheckedException {
         if (type == ENCRYPTED_RECORD || type == ENCRYPTED_RECORD_V2) {
-            if (encSpi == null) {

Review Comment:
   Why was this check dropped? `readEncryptedData()` accesses `encSpi`. Is this because `encMgr` will always be `null` if `encSpi` is null? If this is true, then it could still make sense to still add a check that `encSpi` is not null inside `readEncryptedData()`.



##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/EncryptedData.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.persistence.wal.serializer;
+
+import java.nio.ByteBuffer;
+import org.apache.ignite.internal.pagemem.wal.record.WALRecord;
+import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferBackedDataInput;
+import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferBackedDataInputImpl;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Class that represents information about an encrypted WAL record or data entry.
+ */
+class EncryptedData {

Review Comment:
   The name seems questionable. It makes you think it contains encrypted data, but it actually POSSIBLY contains DEcrypted data (that was originall encrypted), or it contains no data meaning it could not be decrypted.
   
   I suggest the following:
   
   1. Rename the class to `DecryptionResult`. 'result' seems better because a result might be a failure; `DecryptedData` would suggest that the decryption was successful and we can get plain text data (which is not true).
   2. Rename `isDecrypted()` to `isDecryptedSuccessfully()`



##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/EncryptedData.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.persistence.wal.serializer;
+
+import java.nio.ByteBuffer;
+import org.apache.ignite.internal.pagemem.wal.record.WALRecord;
+import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferBackedDataInput;
+import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferBackedDataInputImpl;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Class that represents information about an encrypted WAL record or data entry.
+ */
+class EncryptedData {
+    /** Decrypted WAL record or {@code null} if it couldn't be decrypted. */
+    @Nullable
+    private final ByteBufferBackedDataInput data;

Review Comment:
   How about renaming the field to `plainTextData` to highlight that it is the data without encryption?



##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java:
##########
@@ -210,22 +208,15 @@ public RecordDataV1Serializer(GridCacheSharedContext cctx) {
     @Override public WALRecord readRecord(RecordType type, ByteBufferBackedDataInput in, int size)
         throws IOException, IgniteCheckedException {
         if (type == ENCRYPTED_RECORD || type == ENCRYPTED_RECORD_V2) {
-            if (encSpi == null) {
-                T2<Integer, RecordType> knownData = skipEncryptedRecord(in, true);
+            EncryptedData encryptedData = readEncryptedData(in, true, type == ENCRYPTED_RECORD_V2);

Review Comment:
   It seems a bit weird that a public method of a class related to v1 actually does all the work for both v1 and v2. Probaly, this led to confusion and caused the bug you are fixiing here.
   
   How about the following:
   
   1. Leave the implementation of `readRecord()` here, but make it a template method
   2. Extract things which define the differences between v1 and v2 to protected methods and overriding them in `RecordDataV2Serializer`?
   
   For example, the template method could look like this:
   
   ```
       @Override public WALRecord readRecord(RecordType type, ByteBufferBackedDataInput in, int size)
           throws IOException, IgniteCheckedException {
           if (type == encryptedRecordType()) {
               EncryptedData encryptedData = readEncryptedData(in, true);
   
               if (encryptedData.isDecrypted()) {
                   ByteBufferBackedDataInput data = encryptedData.data();
   
                   return readPlainRecord(encryptedData.recordType(), data, true, data.buffer().capacity());
               }
               else
                   return new EncryptedRecord(encryptedData.grpId(), encryptedData.recordType());
           }
   
           return readPlainRecord(type, in, false, size);
       }
   ```
   
   and there would also be
   
   ```
   protected RecordType encryptedRecordType();
   ```
   
   returning `ENCRYPTED_RECORD` and `ENCRYPTED_RECORD_V2` in the corresponding implementations, and
   
   ```
       protected EncryptedData readEncryptedData(
           ByteBufferBackedDataInput in,
           boolean readType
       ) throws IOException, IgniteCheckedException {
   ```
   
   calling `readEncryptedData(ByteBufferBackedDataInput, boolean, boolean)` with the corresponding third argument specific for v1 or v2.
   
   WDYT?



##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java:
##########
@@ -287,57 +278,45 @@ private boolean needEncryption(int grpId) {
      * @throws IOException If failed.
      * @throws IgniteCheckedException If failed.
      */
-    private T3<ByteBufferBackedDataInput, Integer, RecordType> readEncryptedData(
+    private EncryptedData readEncryptedData(
         ByteBufferBackedDataInput in,
         boolean readType,
         boolean readKeyId
     ) throws IOException, IgniteCheckedException {
         int grpId = in.readInt();
         int encRecSz = in.readInt();
 
-        RecordType plainRecType = null;
-
-        if (readType)
-            plainRecType = RecordV1Serializer.readRecordType(in);
+        RecordType plainRecType = readType ? RecordV1Serializer.readRecordType(in) : null;
 
         int keyId = readKeyId ? in.readUnsignedByte() : GridEncryptionManager.INITIAL_KEY_ID;
 
-        byte[] encData = new byte[encRecSz];
+        // Encryption Manager can be null during offline WAL iteration
+        if (encMgr == null) {
+            int skipped = in.skipBytes(encRecSz);
 
-        in.readFully(encData);
+            assert skipped == encRecSz;
 
-        GroupKey grpKey = encMgr.groupKey(grpId, keyId);
+            return new EncryptedData(null, plainRecType, grpId);
+        }
 
-        if (grpKey == null)
-            return new T3<>(null, grpId, plainRecType);
+        GroupKey grpKey = encMgr.groupKey(grpId, keyId);
 
-        byte[] clData = encSpi.decrypt(encData, grpKey.key());
+        // Encryption key is not available when restoring the MetaStorage
+        if (grpKey == null) {
+            int skipped = in.skipBytes(encRecSz);

Review Comment:
   This block duplicates the block above. How about extracting it to a tiny method?



##########
modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteEncryptedWalConverterTest.java:
##########
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.development.utils;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.PrintStream;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.encryption.AbstractEncryptionTest;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.encryption.keystore.KeystoreEncryptionSpi;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static java.util.Collections.emptyList;
+import static org.apache.ignite.configuration.DataStorageConfiguration.DFLT_BINARY_METADATA_PATH;
+import static org.apache.ignite.configuration.DataStorageConfiguration.DFLT_MARSHALLER_PATH;
+import static org.apache.ignite.configuration.DataStorageConfiguration.DFLT_WAL_ARCHIVE_PATH;
+import static org.apache.ignite.configuration.DataStorageConfiguration.DFLT_WAL_PATH;
+import static org.apache.ignite.development.utils.IgniteWalConverter.convert;
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Class that contains tests on interaction between the {@link IgniteWalConverter} and encrypted WALs.
+ */
+public class IgniteEncryptedWalConverterTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        cleanPersistenceDir();
+
+        super.afterTest();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        KeystoreEncryptionSpi encSpi = new KeystoreEncryptionSpi();
+
+        encSpi.setKeyStorePath(AbstractEncryptionTest.KEYSTORE_PATH);
+        encSpi.setKeyStorePassword(AbstractEncryptionTest.KEYSTORE_PASSWORD.toCharArray());
+
+        return super.getConfiguration(igniteInstanceName)
+            .setEncryptionSpi(encSpi)
+            .setCacheConfiguration(defaultCacheConfiguration().setEncryptionEnabled(true))
+            .setDataStorageConfiguration(new DataStorageConfiguration()
+                .setDefaultDataRegionConfiguration(new DataRegionConfiguration()
+                    .setPersistenceEnabled(true))
+            );
+    }
+
+    /**
+     * Populates an encrypted cache and checks that its WAL contains encrypted records.
+     */
+    @Test
+    public void testIgniteWalConverter() throws Exception {
+        String nodeFolder = createWal();
+
+        ByteArrayOutputStream outByte = new ByteArrayOutputStream();
+
+        PrintStream out = new PrintStream(outByte);
+
+        IgniteWalConverterArguments arg = new IgniteWalConverterArguments(
+            U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_WAL_PATH, false),
+            U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_WAL_ARCHIVE_PATH, false),
+            DataStorageConfiguration.DFLT_PAGE_SIZE,
+            new File(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_BINARY_METADATA_PATH, false), nodeFolder),
+            U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_MARSHALLER_PATH, false),
+            false,
+            null,
+            null,
+            null,
+            null,
+            null,
+            false,
+            false,
+            emptyList()
+        );
+
+        convert(out, arg);

Review Comment:
   Should we avoid a static import here? It does not seem to bring any benefits, but it increases confusion



-- 
This is an automated message from the 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] asfgit closed pull request #10118: IGNITE-17251 Fix parsing of encrypted records by IgniteWalConverter

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #10118: IGNITE-17251 Fix parsing of encrypted records by IgniteWalConverter
URL: https://github.com/apache/ignite/pull/10118


-- 
This is an automated message from the 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] sashapolo commented on a diff in pull request #10118: IGNITE-17251 Fix parsing of encrypted records by IgniteWalConverter

Posted by GitBox <gi...@apache.org>.
sashapolo commented on code in PR #10118:
URL: https://github.com/apache/ignite/pull/10118#discussion_r910929286


##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java:
##########
@@ -210,22 +208,15 @@ public RecordDataV1Serializer(GridCacheSharedContext cctx) {
     @Override public WALRecord readRecord(RecordType type, ByteBufferBackedDataInput in, int size)
         throws IOException, IgniteCheckedException {
         if (type == ENCRYPTED_RECORD || type == ENCRYPTED_RECORD_V2) {
-            if (encSpi == null) {
-                T2<Integer, RecordType> knownData = skipEncryptedRecord(in, true);
+            EncryptedData encryptedData = readEncryptedData(in, true, type == ENCRYPTED_RECORD_V2);

Review Comment:
   I don't think I fully understand your suggestion: `ENCRYPTED_RECORD` and `ENCRYPTED_RECORD_V2` are not serialized solely by corresponding serializers, but rather by both of them.



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

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

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


[GitHub] [ignite] rpuch commented on a diff in pull request #10118: IGNITE-17251 Fix parsing of encrypted records by IgniteWalConverter

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


##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java:
##########
@@ -210,22 +208,15 @@ public RecordDataV1Serializer(GridCacheSharedContext cctx) {
     @Override public WALRecord readRecord(RecordType type, ByteBufferBackedDataInput in, int size)
         throws IOException, IgniteCheckedException {
         if (type == ENCRYPTED_RECORD || type == ENCRYPTED_RECORD_V2) {
-            if (encSpi == null) {
-                T2<Integer, RecordType> knownData = skipEncryptedRecord(in, true);
+            EncryptedData encryptedData = readEncryptedData(in, true, type == ENCRYPTED_RECORD_V2);

Review Comment:
   Nevermind, the hierarchy of record serializer versions is orthogonal to the hierarchy of message versions



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