You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by se...@apache.org on 2022/06/30 13:00:19 UTC

[ignite] branch master updated: IGNITE-17251 Fix parsing of encrypted records by IgniteWalConverter - Fixes #10118.

This is an automated email from the ASF dual-hosted git repository.

sergeychugunov pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/master by this push:
     new 21ba39ae8d8 IGNITE-17251 Fix parsing of encrypted records by IgniteWalConverter - Fixes #10118.
21ba39ae8d8 is described below

commit 21ba39ae8d82e079f6e43982976068800bbea2e5
Author: Aleksandr Polovtsev <al...@gmail.com>
AuthorDate: Thu Jun 30 15:59:48 2022 +0300

    IGNITE-17251 Fix parsing of encrypted records by IgniteWalConverter - Fixes #10118.
    
    Signed-off-by: Sergey Chugunov <se...@gmail.com>
---
 .../pagemem/wal/record/EncryptedRecord.java        |  13 ++-
 .../wal/serializer/DecryptionResult.java           |  88 +++++++++++++++
 .../wal/serializer/RecordDataV1Serializer.java     |  91 ++++++----------
 .../ignite/development/utils/DataEntryWrapper.java |   9 +-
 .../development/utils/DevUtilsTestSuite.java       |   1 +
 .../utils/IgniteEncryptedWalConverterTest.java     | 120 +++++++++++++++++++++
 6 files changed, 258 insertions(+), 64 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/EncryptedRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/EncryptedRecord.java
index 234292b18e1..02b99d31974 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/EncryptedRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/EncryptedRecord.java
@@ -1,6 +1,6 @@
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
+ * contributor license agreements. See the NOTICE file distributed with
  * this work for additional information regarding copyright ownership.
  * The ASF licenses this file to You under the Apache License, Version 2.0
  * (the "License"); you may not use this file except in compliance with
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record;
 
+import org.apache.ignite.internal.util.typedef.internal.S;
+
 /**
  * Encrypted record from WAL.
  * That types of record returned from a {@code RecordDataSerializer} on offline WAL iteration.
@@ -25,12 +27,12 @@ public class EncryptedRecord extends WALRecord implements WalRecordCacheGroupAwa
     /**
      * Group id.
      */
-    private int grpId;
+    private final int grpId;
 
     /**
      * Type of plain record.
      */
-    private RecordType plainRecType;
+    private final RecordType plainRecType;
 
     /**
      * @param grpId Group id
@@ -57,4 +59,9 @@ public class EncryptedRecord extends WALRecord implements WalRecordCacheGroupAwa
     public RecordType plainRecordType() {
         return plainRecType;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(EncryptedRecord.class, this, "super", super.toString());
+    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/DecryptionResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/DecryptionResult.java
new file mode 100644
index 00000000000..6577ea5740c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/DecryptionResult.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 DecryptionResult {
+    /** Decrypted WAL record or {@code null} if it couldn't be decrypted. */
+    @Nullable
+    private final ByteBufferBackedDataInput decryptedData;
+
+    /**
+     * Type of the encrypted WAL record or {@code null} if this instance contains an encrypted data entry
+     * and not a WAL record.
+     */
+    @Nullable
+    private final WALRecord.RecordType recordType;
+
+    /** Cache group id. */
+    private final int grpId;
+
+    /**
+     * @param decryptedData Decrypted WAL record or {@code null} if it couldn't be decrypted.
+     * @param recordType Type of the encrypted WAL record or {@code null}
+     *                   if this instance contains an encrypted data entry and not a WAL record.
+     * @param grpId Cache group id.
+     */
+    DecryptionResult(
+        @Nullable ByteBuffer decryptedData,
+        @Nullable WALRecord.RecordType recordType,
+        int grpId
+    ) {
+        this.decryptedData = decryptedData == null ? null : new ByteBufferBackedDataInputImpl().buffer(decryptedData);
+        this.recordType = recordType;
+        this.grpId = grpId;
+    }
+
+    /**
+     * Returns the decrypted WAL record or {@code null} if it couldn't be decrypted.
+     */
+    @Nullable ByteBufferBackedDataInput decryptedData() {
+        return decryptedData;
+    }
+
+    /**
+     * Returns the type of the encrypted WAL record or {@code null} if this instance contains an encrypted data entry
+     * and not a WAL record.
+     */
+    @Nullable WALRecord.RecordType recordType() {
+        return recordType;
+    }
+
+    /**
+     * Returns cache group id that this record belongs to.
+     */
+    int grpId() {
+        return grpId;
+    }
+
+    /**
+     * Returns {@code true} if this instance contains decrypted data.
+     */
+    boolean isDecryptedSuccessfully() {
+        return decryptedData != null;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java
index 81ddabb8eca..ac4c29188fa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java
@@ -109,13 +109,11 @@ import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.CacheVersionIO;
 import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferBackedDataInput;
-import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferBackedDataInputImpl;
 import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer;
 import org.apache.ignite.internal.processors.cache.persistence.wal.record.HeaderRecord;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
 import org.apache.ignite.internal.util.typedef.T2;
-import org.apache.ignite.internal.util.typedef.T3;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteProductVersion;
@@ -210,22 +208,15 @@ public class RecordDataV1Serializer implements RecordDataSerializer {
     @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);
+            DecryptionResult decryptionResult = readEncryptedData(in, true, type == ENCRYPTED_RECORD_V2);
 
-                //This happen on offline WAL iteration(we don't have encryption keys available).
-                return new EncryptedRecord(knownData.get1(), knownData.get2());
-            }
-
-            T3<ByteBufferBackedDataInput, Integer, RecordType> clData =
-                readEncryptedData(in, true, type == ENCRYPTED_RECORD_V2);
+            if (decryptionResult.isDecryptedSuccessfully()) {
+                ByteBufferBackedDataInput data = decryptionResult.decryptedData();
 
-            //This happen during startup. On first WAL iteration we restore only metastore.
-            //So, no encryption keys available. See GridCacheDatabaseSharedManager#readMetastore
-            if (clData.get1() == null)
-                return new EncryptedRecord(clData.get2(), clData.get3());
-
-            return readPlainRecord(clData.get3(), clData.get1(), true, clData.get1().buffer().capacity());
+                return readPlainRecord(decryptionResult.recordType(), data, true, data.buffer().capacity());
+            }
+            else
+                return new EncryptedRecord(decryptionResult.grpId(), decryptionResult.recordType());
         }
 
         return readPlainRecord(type, in, false, size);
@@ -287,7 +278,7 @@ public class RecordDataV1Serializer implements RecordDataSerializer {
      * @throws IOException If failed.
      * @throws IgniteCheckedException If failed.
      */
-    private T3<ByteBufferBackedDataInput, Integer, RecordType> readEncryptedData(
+    private DecryptionResult readEncryptedData(
         ByteBufferBackedDataInput in,
         boolean readType,
         boolean readKeyId
@@ -295,49 +286,37 @@ public class RecordDataV1Serializer implements RecordDataSerializer {
         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 || encSpi == null) {
+            int skipped = in.skipBytes(encRecSz);
 
-        in.readFully(encData);
+            assert skipped == encRecSz;
 
-        GroupKey grpKey = encMgr.groupKey(grpId, keyId);
+            return new DecryptionResult(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);
 
-        return new T3<>(new ByteBufferBackedDataInputImpl().buffer(ByteBuffer.wrap(clData)), grpId, plainRecType);
-    }
+            assert skipped == encRecSz;
 
-    /**
-     * Reads encrypted record without decryption.
-     * Should be used only for a offline WAL iteration.
-     *
-     * @param in Data stream.
-     * @param readType If {@code true} plain record type will be read from {@code in}.
-     * @return Group id and type of skipped record.
-     */
-    private T2<Integer, RecordType> skipEncryptedRecord(ByteBufferBackedDataInput in, boolean readType)
-        throws IOException, IgniteCheckedException {
-        int grpId = in.readInt();
-        int encRecSz = in.readInt();
-        RecordType plainRecType = null;
+            return new DecryptionResult(null, plainRecType, grpId);
+        }
 
-        if (readType)
-            plainRecType = RecordV1Serializer.readRecordType(in);
+        byte[] encData = new byte[encRecSz];
 
-        int skipped = in.skipBytes(encRecSz);
+        in.readFully(encData);
 
-        assert skipped == encRecSz;
+        byte[] clData = encSpi.decrypt(encData, grpKey.key());
 
-        return new T2<>(grpId, plainRecType);
+        return new DecryptionResult(ByteBuffer.wrap(clData), plainRecType, grpId);
     }
 
     /**
@@ -2069,19 +2048,13 @@ public class RecordDataV1Serializer implements RecordDataSerializer {
         RecordType dataRecordType = recType == ENCRYPTED_DATA_RECORD_V3 ? DATA_RECORD_V2 : DATA_RECORD;
 
         if (needDecryption) {
-            if (encSpi == null) {
-                skipEncryptedRecord(in, false);
-
-                return new EncryptedDataEntry();
-            }
-
-            T3<ByteBufferBackedDataInput, Integer, RecordType> clData = readEncryptedData(in, false,
-                recType == ENCRYPTED_DATA_RECORD_V2 || recType == ENCRYPTED_DATA_RECORD_V3);
+            boolean readKeyId = recType == ENCRYPTED_DATA_RECORD_V2 || recType == ENCRYPTED_DATA_RECORD_V3;
 
-            if (clData.get1() == null)
-                return null;
+            DecryptionResult decryptionResult = readEncryptedData(in, false, readKeyId);
 
-            return readPlainDataEntry(clData.get1(), dataRecordType);
+            return decryptionResult.isDecryptedSuccessfully()
+                ? readPlainDataEntry(decryptionResult.decryptedData(), dataRecordType)
+                : new EncryptedDataEntry();
         }
 
         return readPlainDataEntry(in, dataRecordType);
@@ -2306,7 +2279,7 @@ public class RecordDataV1Serializer implements RecordDataSerializer {
             /*part ID*/4 +
             /*expire Time*/8 +
             /*part cnt*/8 +
-            /*primary*/(entry instanceof MvccDataEntry ? 0 : 1);
+            /*flags*/(entry instanceof MvccDataEntry ? 0 : 1);
     }
 
     /**
diff --git a/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/DataEntryWrapper.java b/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/DataEntryWrapper.java
index fd660de8e9f..dfdc26ba40b 100644
--- a/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/DataEntryWrapper.java
+++ b/modules/dev-utils/src/main/java/org/apache/ignite/development/utils/DataEntryWrapper.java
@@ -25,10 +25,10 @@ import org.apache.ignite.internal.pagemem.wal.record.DataEntry;
 import org.apache.ignite.internal.pagemem.wal.record.UnwrapDataEntry;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectValueContext;
+import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordDataV1Serializer;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.SB;
-import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.development.utils.ProcessSensitiveData.HASH;
 import static org.apache.ignite.development.utils.ProcessSensitiveData.HIDE;
@@ -41,7 +41,7 @@ class DataEntryWrapper extends DataEntry {
     /**
      * Source DataEntry.
      */
-    @Nullable private final DataEntry source;
+    private final DataEntry source;
 
     /** Strategy for the processing of sensitive data. */
     private final ProcessSensitiveData sensitiveData;
@@ -85,6 +85,11 @@ class DataEntryWrapper extends DataEntry {
 
             valueStr = toString(unwrappedDataEntry.unwrappedValue(), this.source.value());
         }
+        else if (source instanceof RecordDataV1Serializer.EncryptedDataEntry) {
+            keyStr = "<encrypted>";
+
+            valueStr = "<encrypted>";
+        }
         else {
             keyStr = toString(null, this.source.key());
 
diff --git a/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/DevUtilsTestSuite.java b/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/DevUtilsTestSuite.java
index 919fb9b057c..5d9315f2879 100644
--- a/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/DevUtilsTestSuite.java
+++ b/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/DevUtilsTestSuite.java
@@ -26,6 +26,7 @@ import org.junit.runners.Suite;
 @RunWith(Suite.class)
 @Suite.SuiteClasses({
     IgniteWalConverterTest.class,
+    IgniteEncryptedWalConverterTest.class,
     IgniteWalConverterArgumentsTest.class,
     IgniteWalConverterSensitiveDataTest.class
 })
diff --git a/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteEncryptedWalConverterTest.java b/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteEncryptedWalConverterTest.java
new file mode 100644
index 00000000000..878a5db9ee1
--- /dev/null
+++ b/modules/dev-utils/src/test/java/org/apache/ignite/development/utils/IgniteEncryptedWalConverterTest.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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()
+        );
+
+        IgniteWalConverter.convert(out, arg);
+
+        String result = outByte.toString();
+
+        assertThat(result, containsString("EncryptedRecord"));
+    }
+
+    /**
+     * Populates a cache and returns the name of its node's folder.
+     */
+    private String createWal() throws Exception {
+        try (IgniteEx node = startGrid(0)) {
+            node.cluster().state(ClusterState.ACTIVE);
+
+            IgniteCache<Integer, Integer> cache = node.cache(DEFAULT_CACHE_NAME);
+
+            for (int i = 0; i < 10; i++)
+                cache.put(i, i);
+
+            return node.context().pdsFolderResolver().resolveFolders().folderName();
+        }
+    }
+}