You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@parquet.apache.org by ga...@apache.org on 2020/04/22 09:11:06 UTC
[parquet-mr] 04/04: PARQUET-1817: Crypto Properties Factory (#769)
This is an automated email from the ASF dual-hosted git repository.
gabor pushed a commit to branch encryption
in repository https://gitbox.apache.org/repos/asf/parquet-mr.git
commit 42827b16e5a1a0f23106b0625d0df60a9e11b319
Author: shangxinli <31...@users.noreply.github.com>
AuthorDate: Mon Mar 30 01:57:48 2020 -0700
PARQUET-1817: Crypto Properties Factory (#769)
---
.../crypto/DecryptionPropertiesFactory.java | 86 ++++++++++++++++++++
.../crypto/EncryptionPropertiesFactory.java | 93 ++++++++++++++++++++++
.../crypto/ParquetCryptoRuntimeException.java | 42 ++++++++++
.../crypto/DecryptionPropertiesFactoryTest.java | 43 ++++++++++
.../crypto/EncryptionPropertiesFactoryTest.java | 45 +++++++++++
.../crypto/SampleDecryptionPropertiesFactory.java | 57 +++++++++++++
.../crypto/SampleEncryptionPropertiesFactory.java | 58 ++++++++++++++
7 files changed, 424 insertions(+)
diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/crypto/DecryptionPropertiesFactory.java b/parquet-hadoop/src/main/java/org/apache/parquet/crypto/DecryptionPropertiesFactory.java
new file mode 100644
index 0000000..4e1f5ac
--- /dev/null
+++ b/parquet-hadoop/src/main/java/org/apache/parquet/crypto/DecryptionPropertiesFactory.java
@@ -0,0 +1,86 @@
+/*
+ * 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.parquet.crypto;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.hadoop.BadConfigurationException;
+import org.apache.parquet.hadoop.util.ConfigurationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * DecryptionPropertiesFactory interface enables transparent activation of Parquet decryption.
+ *
+ * It's customized implementations produce decryption properties for each Parquet file, using the input information
+ * available in Parquet file readers: file path and Hadoop configuration properties that can pass custom parameters
+ * required by a crypto factory. A factory implementation can use or ignore any of these inputs.
+ *
+ * The example usage could be as below.
+ * 1. Write a class to implement DecryptionPropertiesFactory.
+ * 2. Set configuration of "parquet.crypto.factory.class" with the fully qualified name of this class.
+ * For example, we can set the configuration in SparkSession as below.
+ * SparkSession spark = SparkSession
+ * .config("parquet.crypto.factory.class",
+ * "xxx.xxx.DecryptionPropertiesClassLoaderImpl")
+ *
+ * The implementation of this interface will be instantiated by {@link #loadFactory(Configuration)}.
+ */
+public interface DecryptionPropertiesFactory {
+
+ Logger LOG = LoggerFactory.getLogger(DecryptionPropertiesFactory.class);
+ String CRYPTO_FACTORY_CLASS_PROPERTY_NAME = "parquet.crypto.factory.class";
+
+ /**
+ * Load DecryptionPropertiesFactory class specified by CRYPTO_FACTORY_CLASS_PROPERTY_NAME as the path in the configuration
+ *
+ * @param conf Configuration where user specifies the class path
+ * @return object with class DecryptionPropertiesFactory if user specified the class path and invoking of
+ * the class succeeds, null if user doesn't specify the class path
+ * @throws BadConfigurationException if the instantiation of the configured class fails
+ */
+ static DecryptionPropertiesFactory loadFactory(Configuration conf) {
+ final Class<?> decryptionPropertiesFactoryClass = ConfigurationUtil.getClassFromConfig(conf,
+ CRYPTO_FACTORY_CLASS_PROPERTY_NAME, DecryptionPropertiesFactory.class);
+
+ if (null == decryptionPropertiesFactoryClass) {
+ LOG.debug("DecryptionPropertiesFactory is not configured - name not found in hadoop config");
+ return null;
+ }
+
+ try {
+ return (DecryptionPropertiesFactory) decryptionPropertiesFactoryClass.newInstance();
+ } catch (InstantiationException | IllegalAccessException e) {
+ throw new BadConfigurationException("could not instantiate decryptionPropertiesFactoryClass class: "
+ + decryptionPropertiesFactoryClass, e);
+ }
+ }
+
+ /**
+ * Get FileDecryptionProperties object which is created by the implementation of this interface. Please see
+ * the unit test SampleDecryptionPropertiesFactory for example
+ *
+ * @param hadoopConfig Configuration that is used to pass the needed information, e.g. KMS uri
+ * @param filePath File path of the parquet file
+ * Can be used for AAD prefix verification, part of key metadata etc
+ * @return object with class of FileDecryptionProperties
+ * @throws ParquetCryptoRuntimeException if there is an exception while creating the object
+ */
+ FileDecryptionProperties getFileDecryptionProperties(Configuration hadoopConfig, Path filePath) throws ParquetCryptoRuntimeException;
+}
diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/crypto/EncryptionPropertiesFactory.java b/parquet-hadoop/src/main/java/org/apache/parquet/crypto/EncryptionPropertiesFactory.java
new file mode 100644
index 0000000..84cbadd
--- /dev/null
+++ b/parquet-hadoop/src/main/java/org/apache/parquet/crypto/EncryptionPropertiesFactory.java
@@ -0,0 +1,93 @@
+/*
+ * 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.parquet.crypto;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.hadoop.BadConfigurationException;
+import org.apache.parquet.hadoop.api.WriteSupport.WriteContext;
+import org.apache.parquet.hadoop.util.ConfigurationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * EncryptionPropertiesFactory interface enables transparent activation of Parquet encryption.
+ *
+ * It's customized implementations produce encryption properties for each Parquet file, using the input information
+ * available in Parquet file writers: file path, file extended schema - and also Hadoop configuration properties that
+ * can pass custom parameters required by a crypto factory. A factory implementation can use or ignore any of these
+ * inputs.
+ *
+ * The example usage could be as below.
+ * 1. Write a class to implement EncryptionPropertiesFactory.
+ * 2. Set configuration of "parquet.crypto.factory.class" with the fully qualified name of this class.
+ * For example, we can set the configuration in SparkSession as below.
+ * SparkSession spark = SparkSession
+ * .config("parquet.crypto.factory.class",
+ * "xxx.xxx.EncryptionPropertiesClassLoaderImpl")
+ *
+ * The implementation of this interface will be instantiated by {@link #loadFactory(Configuration)}.
+ */
+public interface EncryptionPropertiesFactory {
+
+ Logger LOG = LoggerFactory.getLogger(EncryptionPropertiesFactory.class);
+ String CRYPTO_FACTORY_CLASS_PROPERTY_NAME = "parquet.crypto.factory.class";
+
+ /**
+ * Load EncryptionPropertiesFactory class specified by CRYPTO_FACTORY_CLASS_PROPERTY_NAME as the path in the
+ * configuration
+ *
+ * @param conf Configuration where user specifies the class path
+ * @return object with class EncryptionPropertiesFactory if user specified the class path and invoking of
+ * the class succeeds, null if user doesn't specify the class path
+ * @throws BadConfigurationException if the instantiation of the configured class fails
+ */
+ static EncryptionPropertiesFactory loadFactory(Configuration conf) {
+ final Class<?> encryptionPropertiesFactoryClass = ConfigurationUtil.getClassFromConfig(conf,
+ CRYPTO_FACTORY_CLASS_PROPERTY_NAME, EncryptionPropertiesFactory.class);
+
+ if (null == encryptionPropertiesFactoryClass) {
+ LOG.debug("EncryptionPropertiesFactory is not configured - name not found in hadoop config");
+ return null;
+ }
+
+ try {
+ return (EncryptionPropertiesFactory) encryptionPropertiesFactoryClass.newInstance();
+ } catch (InstantiationException | IllegalAccessException e) {
+ throw new BadConfigurationException("could not instantiate encryptionPropertiesFactoryClass class: "
+ + encryptionPropertiesFactoryClass, e);
+ }
+ }
+
+ /**
+ * Get FileEncryptionProperties object which is created by the implementation of this interface. Please see
+ * the unit test SampleEncryptionPropertiesFactory for example
+ *
+ * @param fileHadoopConfig Configuration that is used to pass the needed information, e.g. KMS uri
+ * @param tempFilePath File path of the parquet file being written.
+ * Can be used for AAD prefix creation, key material management, etc.
+ * Implementations must not presume the path is permanent,
+ * as the file can be moved or renamed later
+ * @param fileWriteContext WriteContext to provide information like schema to build the FileEncryptionProperties
+ * @return object with class of FileEncryptionProperties
+ * @throws ParquetCryptoRuntimeException if there is an exception while creating the object
+ */
+ FileEncryptionProperties getFileEncryptionProperties(Configuration fileHadoopConfig, Path tempFilePath,
+ WriteContext fileWriteContext) throws ParquetCryptoRuntimeException;
+}
diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/crypto/ParquetCryptoRuntimeException.java b/parquet-hadoop/src/main/java/org/apache/parquet/crypto/ParquetCryptoRuntimeException.java
new file mode 100644
index 0000000..60255b5
--- /dev/null
+++ b/parquet-hadoop/src/main/java/org/apache/parquet/crypto/ParquetCryptoRuntimeException.java
@@ -0,0 +1,42 @@
+/*
+ * 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.parquet.crypto;
+
+import org.apache.parquet.ParquetRuntimeException;
+
+/**
+ * Thrown when an encryption or decryption operation problem is occurred
+ */
+public class ParquetCryptoRuntimeException extends ParquetRuntimeException {
+ private static final long serialVersionUID = 1L;
+
+ public ParquetCryptoRuntimeException() {}
+
+ public ParquetCryptoRuntimeException(String message, Throwable cause) {
+ super(message, cause);
+ }
+
+ public ParquetCryptoRuntimeException(String message) {
+ super(message);
+ }
+
+ public ParquetCryptoRuntimeException(Throwable cause) {
+ super(cause);
+ }
+}
diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/crypto/DecryptionPropertiesFactoryTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/crypto/DecryptionPropertiesFactoryTest.java
new file mode 100644
index 0000000..8c8b920
--- /dev/null
+++ b/parquet-hadoop/src/test/java/org/apache/parquet/crypto/DecryptionPropertiesFactoryTest.java
@@ -0,0 +1,43 @@
+/*
+ * 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.parquet.crypto;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Test;
+
+import static org.junit.Assert.assertArrayEquals;
+
+public class DecryptionPropertiesFactoryTest {
+ @Test
+ public void testLoadDecPropertiesFactory() {
+ Configuration conf = new Configuration();
+ conf.set(EncryptionPropertiesFactory.CRYPTO_FACTORY_CLASS_PROPERTY_NAME,
+ "org.apache.parquet.crypto.SampleDecryptionPropertiesFactory");
+
+ DecryptionPropertiesFactory decryptionPropertiesFactory = DecryptionPropertiesFactory.loadFactory(conf);
+ FileDecryptionProperties decryptionProperties = decryptionPropertiesFactory.getFileDecryptionProperties(conf, null);
+
+ assertArrayEquals(decryptionProperties.getFooterKey(), SampleDecryptionPropertiesFactory.FOOTER_KEY);
+ assertArrayEquals(decryptionProperties.getColumnKey(SampleDecryptionPropertiesFactory.COL1),
+ SampleDecryptionPropertiesFactory.COL1_KEY);
+ assertArrayEquals(decryptionProperties.getColumnKey(SampleDecryptionPropertiesFactory.COL2),
+ SampleDecryptionPropertiesFactory.COL2_KEY);
+ }
+}
diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/crypto/EncryptionPropertiesFactoryTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/crypto/EncryptionPropertiesFactoryTest.java
new file mode 100644
index 0000000..6fa0816
--- /dev/null
+++ b/parquet-hadoop/src/test/java/org/apache/parquet/crypto/EncryptionPropertiesFactoryTest.java
@@ -0,0 +1,45 @@
+/*
+ * 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.parquet.crypto;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Test;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+public class EncryptionPropertiesFactoryTest {
+ @Test
+ public void testLoadEncPropertiesFactory() {
+ Configuration conf = new Configuration();
+ conf.set(EncryptionPropertiesFactory.CRYPTO_FACTORY_CLASS_PROPERTY_NAME,
+ "org.apache.parquet.crypto.SampleEncryptionPropertiesFactory");
+
+ EncryptionPropertiesFactory encryptionPropertiesFactory = EncryptionPropertiesFactory.loadFactory(conf);
+ FileEncryptionProperties encryptionProperties = encryptionPropertiesFactory.getFileEncryptionProperties(conf,
+ null, null);
+
+ assertArrayEquals(encryptionProperties.getFooterKey(), SampleEncryptionPropertiesFactory.FOOTER_KEY);
+ assertEquals(encryptionProperties.getColumnProperties(SampleEncryptionPropertiesFactory.COL1),
+ SampleEncryptionPropertiesFactory.COL1_ENCR_PROPERTIES);
+ assertEquals(encryptionProperties.getColumnProperties(SampleEncryptionPropertiesFactory.COL2),
+ SampleEncryptionPropertiesFactory.COL2_ENCR_PROPERTIES);
+ }
+}
diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/crypto/SampleDecryptionPropertiesFactory.java b/parquet-hadoop/src/test/java/org/apache/parquet/crypto/SampleDecryptionPropertiesFactory.java
new file mode 100644
index 0000000..d495656
--- /dev/null
+++ b/parquet-hadoop/src/test/java/org/apache/parquet/crypto/SampleDecryptionPropertiesFactory.java
@@ -0,0 +1,57 @@
+/*
+ * 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.parquet.crypto;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class SampleDecryptionPropertiesFactory implements DecryptionPropertiesFactory {
+
+ public static final byte[] FOOTER_KEY = {0x01, 0x02, 0x03, 0x4, 0x05, 0x06, 0x07, 0x08, 0x09, 0x0a,
+ 0x0b, 0x0c, 0x0d, 0x0e, 0x0f, 0x10};
+ public static final ColumnPath COL1 = ColumnPath.fromDotString("col_1");
+ public static final byte[] COL1_KEY = {0x02, 0x03, 0x4, 0x05, 0x06, 0x07, 0x08, 0x09, 0x0a, 0x0b,
+ 0x0c, 0x0d, 0x0e, 0x0f, 0x10, 0x11};
+ public static final ColumnDecryptionProperties COL1_ENCR_PROPERTIES = ColumnDecryptionProperties.builder(COL1.toDotString())
+ .withKey(COL1_KEY).build();
+ public static final ColumnPath COL2 = ColumnPath.fromDotString("col_2");
+ public static final byte[] COL2_KEY = {0x03, 0x4, 0x05, 0x06, 0x07, 0x08, 0x09, 0x0a, 0x0b, 0x0c,
+ 0x0d, 0x0e, 0x0f, 0x10, 0x11, 0x12};
+ public static final ColumnDecryptionProperties COL2_ENCR_PROPERTIES = ColumnDecryptionProperties.builder(COL2.toDotString())
+ .withKey(COL2_KEY).build();
+
+ @Override
+ public FileDecryptionProperties getFileDecryptionProperties(Configuration hadoopConfig, Path filePath)
+ throws ParquetCryptoRuntimeException {
+
+ Map<ColumnPath, ColumnDecryptionProperties> columnDecPropertiesMap = new HashMap<>();
+
+ columnDecPropertiesMap.put(COL1, COL1_ENCR_PROPERTIES);
+ columnDecPropertiesMap.put(COL2, COL2_ENCR_PROPERTIES);
+
+ FileDecryptionProperties.Builder fileDecBuilder = FileDecryptionProperties.builder();
+
+ return fileDecBuilder.withFooterKey(FOOTER_KEY).withColumnKeys(columnDecPropertiesMap).build();
+ }
+}
diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/crypto/SampleEncryptionPropertiesFactory.java b/parquet-hadoop/src/test/java/org/apache/parquet/crypto/SampleEncryptionPropertiesFactory.java
new file mode 100644
index 0000000..539a24f
--- /dev/null
+++ b/parquet-hadoop/src/test/java/org/apache/parquet/crypto/SampleEncryptionPropertiesFactory.java
@@ -0,0 +1,58 @@
+/*
+ * 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.parquet.crypto;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.hadoop.api.WriteSupport;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class SampleEncryptionPropertiesFactory implements EncryptionPropertiesFactory {
+
+ public static final byte[] FOOTER_KEY = {0x01, 0x02, 0x03, 0x4, 0x05, 0x06, 0x07, 0x08, 0x09, 0x0a,
+ 0x0b, 0x0c, 0x0d, 0x0e, 0x0f, 0x10};
+ public static final ColumnPath COL1 = ColumnPath.fromDotString("col_1");
+ public static final byte[] COL1_KEY = {0x02, 0x03, 0x4, 0x05, 0x06, 0x07, 0x08, 0x09, 0x0a, 0x0b,
+ 0x0c, 0x0d, 0x0e, 0x0f, 0x10, 0x11};
+ public static final ColumnEncryptionProperties COL1_ENCR_PROPERTIES = ColumnEncryptionProperties.builder(
+ COL1.toDotString()).withKey(COL1_KEY).build();
+ public static final ColumnPath COL2 = ColumnPath.fromDotString("col_2");
+ public static final byte[] COL2_KEY = {0x03, 0x4, 0x05, 0x06, 0x07, 0x08, 0x09, 0x0a, 0x0b, 0x0c,
+ 0x0d, 0x0e, 0x0f, 0x10, 0x11, 0x12};
+ public static final ColumnEncryptionProperties COL2_ENCR_PROPERTIES = ColumnEncryptionProperties.builder(
+ COL2.toDotString()).withKey(COL2_KEY).build();
+
+ @Override
+ public FileEncryptionProperties getFileEncryptionProperties(Configuration fileHadoopConfig, Path tempFilePath,
+ WriteSupport.WriteContext fileWriteContext) throws ParquetCryptoRuntimeException {
+
+ Map<ColumnPath, ColumnEncryptionProperties> columnEncPropertiesMap = new HashMap<>();
+
+ columnEncPropertiesMap.put(COL1, COL1_ENCR_PROPERTIES);
+ columnEncPropertiesMap.put(COL2, COL2_ENCR_PROPERTIES);
+
+ FileEncryptionProperties.Builder fileEncBuilder = FileEncryptionProperties.builder(FOOTER_KEY);
+
+ return fileEncBuilder.withAlgorithm(ParquetCipher.AES_GCM_V1).withEncryptedColumns(columnEncPropertiesMap).build();
+ }
+}