-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-2370] Supports data encryption #3614
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 1 commit
5456ff2
926c941
ea401bc
6697169
4fb2b5a
dd1bc3d
8328d5b
b9cbda1
b4d1f3c
6357b3e
30f5c98
9637ae9
5e0672f
9207f2c
917d5ea
115d606
cb2d4c6
f6d9204
c665b68
1e344f7
3352417
03c5308
c17ca61
db2bd88
61156c4
a3677e6
624c64a
429f7a6
ce26ca8
8bd01e8
7c7ffa1
491b4ea
4548198
c646ab9
f8efb1f
661c2d4
303a3f3
4ca8852
f85aeac
8babbfa
a4688a9
1981bf7
c9e4ff7
26c72bc
ff998e2
cce3365
f74353d
c36dc87
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,104 @@ | ||
| /* | ||
| * 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.hudi.crypot.kms; | ||
|
|
||
| import org.apache.hadoop.conf.Configuration; | ||
| import org.apache.hudi.common.config.ConfigProperty; | ||
| import org.apache.parquet.crypto.KeyAccessDeniedException; | ||
| import org.apache.parquet.crypto.ParquetCryptoRuntimeException; | ||
| import org.apache.parquet.crypto.keytools.KeyToolkit; | ||
| import org.apache.parquet.crypto.keytools.KmsClient; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| import java.nio.charset.StandardCharsets; | ||
| import java.util.Base64; | ||
| import java.util.HashMap; | ||
| import java.util.Map; | ||
|
|
||
| /** | ||
| * This is a mock class, built for testing only. Don't use it as an example of KmsClient implementation. | ||
| * (org.apache.parquet.crypto.keytools.samples.VaultClient is the sample implementation). | ||
| */ | ||
| public class InMemoryKMS implements KmsClient { | ||
| private static final Logger LOG = LoggerFactory.getLogger(InMemoryKMS.class); | ||
|
|
||
| private static final ConfigProperty<String> KEY_LIST_PROPERTY_NAME = ConfigProperty | ||
| .key("hoodie.parquet.encryption.key.list") | ||
| .noDefaultValue() | ||
| .withDocumentation("hudi parquet encryption key list, example: keyName_1:value_1, keyName_2:value_2"); | ||
|
|
||
| private static Map<String,byte[]> masterKeyMap; | ||
|
|
||
| @Override | ||
| public synchronized void initialize(Configuration configuration, String kmsInstanceID, String kmsInstanceURL, String accessToken) { | ||
| // Parse master keys | ||
| String[] masterKeys = configuration.getTrimmedStrings(KEY_LIST_PROPERTY_NAME.key()); | ||
| if (null == masterKeys || masterKeys.length == 0) { | ||
| throw new ParquetCryptoRuntimeException("No encryption key list"); | ||
| } | ||
| masterKeyMap = parseKeyList(masterKeys); | ||
| } | ||
|
|
||
| private static Map<String, byte[]> parseKeyList(String[] masterKeys) { | ||
|
|
||
| Map<String,byte[]> keyMap = new HashMap<>(); | ||
|
|
||
| int nKeys = masterKeys.length; | ||
| for (int i = 0; i < nKeys; i++) { | ||
| String[] parts = masterKeys[i].split(":"); | ||
| String keyName = parts[0].trim(); | ||
| if (parts.length != 2) { | ||
| throw new IllegalArgumentException("Key '" + keyName + "' is not formatted correctly"); | ||
| } | ||
| String key = parts[1].trim(); | ||
| try { | ||
| byte[] keyBytes = Base64.getDecoder().decode(key); | ||
| keyMap.put(keyName, keyBytes); | ||
| } catch (IllegalArgumentException e) { | ||
| LOG.warn("Could not decode key '" + keyName + "'!"); | ||
| throw e; | ||
| } | ||
| } | ||
| return keyMap; | ||
| } | ||
|
|
||
| @Override | ||
| public synchronized String wrapKey(byte[] keyBytes, String masterKeyIdentifier) | ||
| throws KeyAccessDeniedException, UnsupportedOperationException { | ||
|
|
||
| byte[] masterKey = masterKeyMap.get(masterKeyIdentifier); | ||
| if (null == masterKey) { | ||
| throw new ParquetCryptoRuntimeException("Key not found: " + masterKeyIdentifier); | ||
| } | ||
| byte[] add = masterKeyIdentifier.getBytes(StandardCharsets.UTF_8); | ||
| return KeyToolkit.encryptKeyLocally(keyBytes, masterKey, add); | ||
| } | ||
|
|
||
| @Override | ||
| public synchronized byte[] unwrapKey(String wrappedKey, String masterKeyIdentifier) | ||
| throws KeyAccessDeniedException, UnsupportedOperationException { | ||
| byte[] masterKey = masterKeyMap.get(masterKeyIdentifier); | ||
| if (null == masterKey) { | ||
| throw new ParquetCryptoRuntimeException("Key not found: " + masterKeyIdentifier); | ||
| } | ||
| byte[] add = masterKeyIdentifier.getBytes(StandardCharsets.UTF_8); | ||
| return KeyToolkit.decryptKeyLocally(wrappedKey, masterKey, add); | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,102 @@ | ||
| /* | ||
| * 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.hudi.crypot; | ||
|
|
||
| import org.apache.hudi.crypot.kms.InMemoryKMS; | ||
| import org.apache.hudi.testutils.HoodieClientTestBase; | ||
| import org.apache.parquet.crypto.keytools.PropertiesDrivenCryptoFactory; | ||
| import org.apache.spark.api.java.JavaSparkContext; | ||
| import org.apache.spark.sql.Dataset; | ||
| import org.apache.spark.sql.Row; | ||
| import org.apache.spark.sql.SaveMode; | ||
| import org.apache.spark.sql.SparkSession; | ||
| import org.junit.jupiter.api.AfterEach; | ||
| import org.junit.jupiter.api.Assertions; | ||
| import org.junit.jupiter.api.BeforeEach; | ||
| import org.junit.jupiter.api.Test; | ||
|
|
||
| import java.io.IOException; | ||
| import java.util.HashMap; | ||
| import java.util.List; | ||
|
|
||
| import static org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings; | ||
|
|
||
| public class TestParquetEncryption extends HoodieClientTestBase { | ||
|
|
||
| private SparkSession spark; | ||
|
|
||
| private HashMap<String, String> commonOpts = new HashMap(); | ||
|
|
||
| @BeforeEach | ||
| public void setUp() throws Exception { | ||
| commonOpts.put("hoodie.insert.shuffle.parallelism", "4"); | ||
| commonOpts.put("hoodie.upsert.shuffle.parallelism", "4"); | ||
| commonOpts.put("hoodie.bulkinsert.shuffle.parallelism", "4"); | ||
| commonOpts.put("hoodie.delete.shuffle.parallelism", "4"); | ||
| commonOpts.put("hoodie.datasource.write.recordkey.field", "_row_key"); | ||
| commonOpts.put("hoodie.datasource.write.partitionpath.field", "partition"); | ||
| commonOpts.put("hoodie.datasource.write.precombine.field", "timestamp"); | ||
| commonOpts.put("hoodie.table.name", "hoodie_test"); | ||
|
|
||
| initPath(); | ||
| initSparkContexts(); | ||
| spark = sqlContext.sparkSession(); | ||
| initTestDataGenerator(); | ||
| initFileSystem(); | ||
| } | ||
|
|
||
| @AfterEach | ||
| public void tearDown() throws IOException { | ||
| cleanupSparkContexts(); | ||
| cleanupTestDataGenerator(); | ||
| cleanupFileSystem(); | ||
| } | ||
|
|
||
| @Test | ||
| public void testEncryption() { | ||
|
|
||
| JavaSparkContext jsc = new JavaSparkContext(spark.sparkContext()); | ||
| jsc.hadoopConfiguration().set("parquet.crypto.factory.class", PropertiesDrivenCryptoFactory.class.getName()); | ||
| jsc.hadoopConfiguration().set("parquet.encryption.kms.client.class", InMemoryKMS.class.getName()); | ||
| jsc.hadoopConfiguration().set("parquet.encryption.footer.key", "k1"); | ||
| jsc.hadoopConfiguration().set("parquet.encryption.column.keys", "k2:rider,_row_key"); | ||
| jsc.hadoopConfiguration().set("hoodie.parquet.encryption.key.list", "k1:AAECAwQFBgcICQoLDA0ODw==, k2:AAECAAECAAECAAECAAECAA=="); | ||
|
|
||
| List<String> records1 = recordsToStrings(dataGen.generateInserts("000", 100)); | ||
| Dataset<Row> inputDF1 = spark.read().json(jsc.parallelize(records1, 2)); | ||
|
|
||
| inputDF1.write().format("org.apache.hudi") | ||
| .options(commonOpts) | ||
| .mode(SaveMode.Overwrite) | ||
| .save(basePath); | ||
|
|
||
|
|
||
| //1. no footer key, no column key | ||
| jsc.hadoopConfiguration().clear(); | ||
| Assertions.assertThrows(Exception.class, () -> spark.read().format("org.apache.hudi").load(basePath).count()); | ||
| Assertions.assertThrows(Exception.class, () -> spark.read().format("org.apache.hudi").load(basePath).select("rider").show(1)); | ||
|
|
||
| //2 has footer key, has column key | ||
| jsc.hadoopConfiguration().set("parquet.crypto.factory.class", PropertiesDrivenCryptoFactory.class.getName()); | ||
| jsc.hadoopConfiguration().set("parquet.encryption.kms.client.class", InMemoryKMS.class.getName()); | ||
| jsc.hadoopConfiguration().set("hoodie.parquet.encryption.key.list", "k1:AAECAwQFBgcICQoLDA0ODw==, k2:AAECAAECAAECAAECAAECAA=="); | ||
| Assertions.assertEquals(100, spark.read().format("org.apache.hudi").load(basePath).count()); | ||
| Assertions.assertDoesNotThrow(() -> spark.read().format("org.apache.hudi").load(basePath).select("rider").show(1)); | ||
| } | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -91,7 +91,7 @@ | |
| <kafka.version>2.0.0</kafka.version> | ||
| <confluent.version>5.3.4</confluent.version> | ||
| <glassfish.version>2.17</glassfish.version> | ||
| <parquet.version>1.10.1</parquet.version> | ||
| <parquet.version>1.12.0</parquet.version> | ||
|
||
| <junit.jupiter.version>5.7.0-M1</junit.jupiter.version> | ||
| <junit.vintage.version>5.7.0-M1</junit.vintage.version> | ||
| <junit.platform.version>1.7.0-M1</junit.platform.version> | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.