[ 
https://issues.apache.org/jira/browse/PARQUET-1229?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17113247#comment-17113247
 ] 

ASF GitHub Bot commented on PARQUET-1229:
-----------------------------------------

ggershinsky commented on a change in pull request #776:
URL: https://github.com/apache/parquet-mr/pull/776#discussion_r428710799



##########
File path: 
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestEncryption.java
##########
@@ -0,0 +1,215 @@
+/* 
+ * 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.hadoop;
+
+import static org.junit.Assert.assertEquals;
+import static org.apache.parquet.hadoop.TestUtils.enforceEmptyDir;
+import static 
org.apache.parquet.hadoop.metadata.CompressionCodecName.UNCOMPRESSED;
+import static org.apache.parquet.schema.MessageTypeParser.parseMessageType;
+
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Random;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.junit.Rule;
+import org.junit.Test;
+
+import org.apache.parquet.crypto.ColumnEncryptionProperties;
+import org.apache.parquet.crypto.FileDecryptionProperties;
+import org.apache.parquet.crypto.FileEncryptionProperties;
+import org.apache.parquet.crypto.ParquetCipher;
+import org.apache.parquet.crypto.StringKeyIdRetriever;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.example.data.simple.SimpleGroupFactory;
+
+
+import org.apache.parquet.hadoop.example.GroupReadSupport;
+import org.apache.parquet.hadoop.example.GroupWriteSupport;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.MessageType;
+import org.junit.rules.TemporaryFolder;
+
+public class TestEncryption {
+
+  @Test
+  public void test() throws Exception {
+    Configuration conf = new Configuration();
+    Path root = new Path("target/tests/TestEncryption/");
+    enforceEmptyDir(conf, root);
+
+    Random random = new Random();
+    int numberOfEncryptionModes = 5;
+    FileEncryptionProperties[] encryptionPropertiesList = new 
FileEncryptionProperties[numberOfEncryptionModes];
+    FileDecryptionProperties[] decryptionPropertiesList = new 
FileDecryptionProperties[numberOfEncryptionModes];
+
+    // #0 Unencrypted - make sure null encryption properties don't break 
regular Parquet
+    encryptionPropertiesList[0] = null;
+    decryptionPropertiesList[0] = null;
+
+    // #1 Basic encryption setup
+    byte[] encryptionKey = new byte[16];
+    random.nextBytes(encryptionKey);
+    FileEncryptionProperties encryptionProperties = 
FileEncryptionProperties.builder(encryptionKey).build();
+    FileDecryptionProperties decryptionProperties = 
FileDecryptionProperties.builder().withFooterKey(encryptionKey).build();
+    encryptionPropertiesList[1] = encryptionProperties;
+    decryptionPropertiesList[1] = decryptionProperties;
+
+    // #2 Default algorithm, non-uniform encryption, key metadata, key 
retriever, AAD prefix
+    byte[] footerKey = new byte[16];
+    random.nextBytes(footerKey);
+    byte[] columnKey0 = new byte[16];
+    random.nextBytes(columnKey0);
+    byte[] columnKey1 = new byte[16];
+    random.nextBytes(columnKey1);
+    ColumnEncryptionProperties columnProperties0 = 
ColumnEncryptionProperties.builder("binary_field")
+        .withKey(columnKey0)
+        .withKeyID("ck0")
+        .build();
+    ColumnEncryptionProperties columnProperties1 = 
ColumnEncryptionProperties.builder("int32_field")
+        .withKey(columnKey1)
+        .withKeyID("ck1")
+        .build();
+    HashMap<ColumnPath, ColumnEncryptionProperties> columnPropertiesMap = new 
HashMap<ColumnPath, ColumnEncryptionProperties>();
+    columnPropertiesMap.put(columnProperties0.getPath(), columnProperties0);
+    columnPropertiesMap.put(columnProperties1.getPath(), columnProperties1);
+    byte[] AADPrefix = root.getName().getBytes(StandardCharsets.UTF_8);
+    encryptionProperties = FileEncryptionProperties.builder(footerKey)
+        .withFooterKeyID("fk")
+        .withAADPrefix(AADPrefix)
+        .withEncryptedColumns(columnPropertiesMap)
+        .build();
+    StringKeyIdRetriever keyRetriever = new StringKeyIdRetriever();
+    keyRetriever.putKey("fk", footerKey);
+    keyRetriever.putKey("ck0", columnKey0);
+    keyRetriever.putKey("ck1", columnKey1);
+    decryptionProperties = FileDecryptionProperties.builder()
+        .withKeyRetriever(keyRetriever)
+        .build();
+    encryptionPropertiesList[2] = encryptionProperties;
+    decryptionPropertiesList[2] = decryptionProperties;
+
+    // #3 GCM_CTR algorithm, non-uniform encryption, key metadata, key 
retriever, AAD
+    columnProperties0 = ColumnEncryptionProperties.builder("binary_field")
+        .withKey(columnKey0)
+        .withKeyID("ck0")
+        .build();
+    columnProperties1 = ColumnEncryptionProperties.builder("int32_field")
+        .withKey(columnKey1)
+        .withKeyID("ck1")
+        .build();
+    columnPropertiesMap = new HashMap<ColumnPath, 
ColumnEncryptionProperties>();
+    columnPropertiesMap.put(columnProperties0.getPath(), columnProperties0);
+    columnPropertiesMap.put(columnProperties1.getPath(), columnProperties1);
+    encryptionProperties = FileEncryptionProperties.builder(footerKey)
+        .withAlgorithm(ParquetCipher.AES_GCM_CTR_V1)
+        .withFooterKeyID("fk")
+        .withAADPrefix(AADPrefix)
+        .withEncryptedColumns(columnPropertiesMap)
+        .build();
+    encryptionPropertiesList[3] = encryptionProperties;
+    decryptionPropertiesList[3] = decryptionProperties; // Same decryption 
properties
+
+    // #4  Plaintext footer, default algorithm, key metadata, key retriever, 
AAD
+    columnProperties0 = ColumnEncryptionProperties.builder("binary_field")
+        .withKey(columnKey0)
+        .withKeyID("ck0")
+        .build();
+    columnProperties1 = ColumnEncryptionProperties.builder("int32_field")
+        .withKey(columnKey1)
+        .withKeyID("ck1")
+        .build();
+    columnPropertiesMap = new HashMap<ColumnPath, 
ColumnEncryptionProperties>();
+    columnPropertiesMap.put(columnProperties0.getPath(), columnProperties0);
+    columnPropertiesMap.put(columnProperties1.getPath(), columnProperties1);
+    encryptionProperties = FileEncryptionProperties.builder(footerKey)
+        .withFooterKeyID("fk")
+        .withPlaintextFooter()
+        .withAADPrefix(AADPrefix)
+        .withEncryptedColumns(columnPropertiesMap)
+        .build();
+    encryptionPropertiesList[4] = encryptionProperties;
+    decryptionPropertiesList[4] = decryptionProperties; // Same decryption 
properties
+
+
+    MessageType schema = parseMessageType(
+        "message test { "
+            + "required binary binary_field; "
+            + "required int32 int32_field; "
+            + "required int64 int64_field; "
+            + "required boolean boolean_field; "
+            + "required float float_field; "
+            + "required double double_field; "
+            + "required fixed_len_byte_array(3) flba_field; "
+            + "required int96 int96_field; "
+            + "} ");
+    GroupWriteSupport.setSchema(schema, conf);
+    SimpleGroupFactory f = new SimpleGroupFactory(schema);
+
+    for (int encryptionMode = 0; encryptionMode < numberOfEncryptionModes; 
encryptionMode++) {
+      System.out.println("MODE: "+encryptionMode);
+      
+      Path file = new Path(root, "m_" + encryptionMode + ".parquet.encrypted");
+      ParquetWriter<Group> writer = new ParquetWriter<Group>(
+          file,
+          new GroupWriteSupport(),
+          UNCOMPRESSED, 1024, 1024, 512, true, false, 
ParquetWriter.DEFAULT_WRITER_VERSION, conf, 
+          encryptionPropertiesList[encryptionMode]);
+      for (int i = 0; i < 1000; i++) {

Review comment:
       we're working on a better unitest for encryption, in #782 . it should be 
possible to drop this one.




----------------------------------------------------------------
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.

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


> parquet-mr code changes for encryption support
> ----------------------------------------------
>
>                 Key: PARQUET-1229
>                 URL: https://issues.apache.org/jira/browse/PARQUET-1229
>             Project: Parquet
>          Issue Type: Sub-task
>          Components: parquet-mr
>            Reporter: Gidon Gershinsky
>            Assignee: Gidon Gershinsky
>            Priority: Major
>              Labels: pull-request-available
>
> Addition of encryption/decryption support to the existing Parquet classes and 
> APIs



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to