Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
48 commits
Select commit Hold shift + click to select a range
5456ff2
HUDI-2370
liujinhui1994 Sep 7, 2021
926c941
exclude other parquet
liujinhui1994 Sep 8, 2021
ea401bc
Merge branch 'master' into HUDI-2370
liujinhui1994 Sep 15, 2021
6697169
update spark.sql.parquet.enableVectorizedReader=true
liujinhui1994 Sep 16, 2021
4fb2b5a
Merge branch 'master' into HUDI-2370
liujinhui1994 Sep 16, 2021
dd1bc3d
TEST ci
liujinhui1994 Oct 13, 2021
8328d5b
Merge branch 'master' into HUDI-2370
liujinhui1994 Oct 13, 2021
b9cbda1
test ci
liujinhui1994 Oct 18, 2021
b4d1f3c
Merge branch 'master' into HUDI-2370
liujinhui1994 Oct 21, 2021
6357b3e
Merge remote-tracking branch 'origin/HUDI-2370' into HUDI-2370
liujinhui1994 Oct 21, 2021
30f5c98
HUDI-2370
liujinhui1994 Oct 26, 2021
9637ae9
Merge remote-tracking branch 'origin/HUDI-2370' into HUDI-2370
liujinhui1994 Oct 26, 2021
5e0672f
HUDI-2370
liujinhui1994 Oct 26, 2021
9207f2c
HUDI-2370
liujinhui1994 Oct 26, 2021
917d5ea
Merge branch 'master' into HUDI-2370
liujinhui1994 Oct 27, 2021
115d606
HUDI-2370
liujinhui1994 Oct 27, 2021
cb2d4c6
HUDI-2370
liujinhui1994 Oct 27, 2021
f6d9204
HUDI-2370
liujinhui1994 Oct 27, 2021
c665b68
HUDI-2370
liujinhui1994 Oct 27, 2021
1e344f7
HUDI-2370
liujinhui1994 Oct 29, 2021
3352417
HUDI-2370
liujinhui1994 Nov 1, 2021
03c5308
HUDI-2370
liujinhui1994 Nov 2, 2021
c17ca61
HUDI-2370
liujinhui1994 Nov 2, 2021
db2bd88
Merge branch 'master' into HUDI-2370
liujinhui1994 Nov 2, 2021
61156c4
HUDI-2370
liujinhui1994 Nov 2, 2021
a3677e6
HUDI-2370
liujinhui1994 Nov 3, 2021
624c64a
HUDI-2370
liujinhui1994 Nov 3, 2021
429f7a6
test class exception
liujinhui1994 Nov 3, 2021
ce26ca8
test class exception
liujinhui1994 Nov 3, 2021
8bd01e8
test class exception
liujinhui1994 Nov 3, 2021
7c7ffa1
test class exception
liujinhui1994 Nov 4, 2021
491b4ea
test class exception
liujinhui1994 Nov 4, 2021
4548198
test class exception
liujinhui1994 Nov 4, 2021
c646ab9
test class exception
liujinhui1994 Nov 4, 2021
f8efb1f
test class exception
liujinhui1994 Nov 5, 2021
661c2d4
test class exception
liujinhui1994 Nov 5, 2021
303a3f3
Merge branch 'master' into HUDI-2370
liujinhui1994 Dec 31, 2021
4ca8852
HUDI-2370
liujinhui1994 Jan 25, 2022
f85aeac
HUDI-2370
liujinhui1994 Jan 25, 2022
8babbfa
HUDI-2370
liujinhui1994 Jan 25, 2022
a4688a9
HUDI-2370
liujinhui1994 Jan 25, 2022
1981bf7
HUDI-2370
liujinhui1994 Jan 25, 2022
c9e4ff7
Merge branch 'master' into HUDI-2370
liujinhui1994 Jan 25, 2022
26c72bc
HUDI-2370
liujinhui1994 Feb 7, 2022
ff998e2
Merge branch 'master' into HUDI-2370
liujinhui1994 Feb 8, 2022
cce3365
HUDI-2370
liujinhui1994 Feb 9, 2022
f74353d
HUDI-2370
liujinhui1994 Feb 9, 2022
c36dc87
HUDI-2370
liujinhui1994 Feb 9, 2022
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 6 additions & 0 deletions hudi-spark-datasource/hudi-spark3/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -161,6 +161,12 @@
<optional>true</optional>
</dependency>

<dependency>
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-hadoop</artifactId>
<version>1.12.0</version>
</dependency>

<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,106 @@
/*
* 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.spark3.crypto;

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).
* This function is only recommended for use in spark3.2+.
* The encryption function is experimental, because InMemoryKMS is not reliable.
*/
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.keys")
.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,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.spark3.crypto;

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;

/**
* This function is only recommended for use in spark3.2+.
*/
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.keys", "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.keys", "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));
}
}