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

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

gszadovszky commented on a change in pull request #808:
URL: https://github.com/apache/parquet-mr/pull/808#discussion_r511805317



##########
File path: parquet-common/src/main/java/org/apache/parquet/io/OutputFile.java
##########
@@ -31,4 +33,5 @@
 
   long defaultBlockSize();
 
+  Path getPath();

Review comment:
       I don't know if I overlooked this one before or it is a new change. The 
module `parquet-column` should not depend on hadoop. That's why we have the 
separate module `parquet-hadoop`. We already have struggling issues that 
parquet-mr cannot be used without hadoop, let's not make it worse.

##########
File path: 
parquet-hadoop/src/test/java/org/apache/parquet/crypto/propertiesfactory/SchemaControlEncryptionTest.java
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.propertiesfactory;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.crypto.EncryptionPropertiesFactory;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.example.data.simple.SimpleGroup;
+import org.apache.parquet.format.EncryptionAlgorithm;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.ParquetWriter;
+import org.apache.parquet.hadoop.api.WriteSupport;
+import org.apache.parquet.hadoop.example.GroupReadSupport;
+import org.apache.parquet.hadoop.example.GroupWriteSupport;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.ThreadLocalRandom;
+
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64;
+import static org.apache.parquet.schema.Type.Repetition.OPTIONAL;
+import static org.apache.parquet.schema.Type.Repetition.REPEATED;
+import static org.apache.parquet.schema.Type.Repetition.REQUIRED;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+public class SchemaControlEncryptionTest {
+
+  private final static Log LOG = 
LogFactory.getLog(SchemaControlEncryptionTest.class);
+  private final static int numRecord = 1000;
+  private Random rnd = new Random(5);
+  
+  // In the test We use a map to tell WriteSupport which columns to be 
encrypted with what key. In real use cases, people
+  // can find whatever easy way to do so basing on how do they get these 
information, for example people can choose to 
+  // store in HMS, or other metastore. 
+  private Map<String, Map<String, Object>> cryptoMetadata = new HashMap<>();
+  private Map<String, Object[]> testData = new HashMap<>();
+
+  @Before
+  public void generateTestData() {
+    String[] names = new String[numRecord];
+    Long[] ages = new Long[numRecord];
+    String[] linkedInWebs = new String[numRecord];
+    String[] twitterWebs = new String[numRecord];
+    for (int i = 0; i < numRecord; i++) {
+      names[i] = getString();
+      ages[i] = getLong();
+      linkedInWebs[i] = getString();
+      twitterWebs[i] = getString();
+    }
+
+    testData.put("Name", names);
+    testData.put("Age", ages);
+    testData.put("LinkedIn", linkedInWebs);
+    testData.put("Twitter", twitterWebs);
+  }
+
+  @Test
+  public void testEncryptionDefault() throws Exception {
+    Configuration conf = new Configuration();
+    runTest(conf);
+  }
+
+  @Test
+  public void testEncryptionGcm() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(SchemaCryptoPropertiesFactory.CONF_ENCRYPTION_ALGORITHM, 
EncryptionAlgorithm._Fields.AES__GCM__CTR__V1.getFieldName());

Review comment:
       I think it is nicer to use the enum `ParquetCypher` instead of the 
parquet-format generated class.

##########
File path: parquet-common/pom.xml
##########
@@ -67,6 +67,20 @@
       <artifactId>audience-annotations</artifactId>
       <version>0.12.0</version>
     </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client</artifactId>
+      <version>${hadoop.version}</version>
+      <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+

Review comment:
       See at OutputFile.getPath.

##########
File path: 
parquet-hadoop/src/test/java/org/apache/parquet/crypto/propertiesfactory/SchemaControlEncryptionTest.java
##########
@@ -0,0 +1,250 @@
+/*
+ * 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.propertiesfactory;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.crypto.EncryptionPropertiesFactory;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.example.data.simple.SimpleGroup;
+import org.apache.parquet.format.EncryptionAlgorithm;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.ParquetWriter;
+import org.apache.parquet.hadoop.api.WriteSupport;
+import org.apache.parquet.hadoop.example.GroupReadSupport;
+import org.apache.parquet.hadoop.example.GroupWriteSupport;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.ThreadLocalRandom;
+
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64;
+import static org.apache.parquet.schema.Type.Repetition.OPTIONAL;
+import static org.apache.parquet.schema.Type.Repetition.REPEATED;
+import static org.apache.parquet.schema.Type.Repetition.REQUIRED;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+public class SchemaControlEncryptionTest {
+
+  private final static Log LOG = 
LogFactory.getLog(SchemaControlEncryptionTest.class);
+  private final static int numRecord = 1000;
+  private Random rnd = new Random(5);
+  
+  // In the test We use a map to tell WriteSupport which columns to be 
encrypted with what key. In real use cases, people
+  // can find whatever easy way to do so basing on how do they get these 
information, for example people can choose to 
+  // store in HMS, or other metastore. 
+  private Map<String, Map<String, Object>> cryptoMetadata = new HashMap<>();
+  private Map<String, Object[]> testData = new HashMap<>();
+
+  @Before
+  public void generateTestData() {
+    String[] names = new String[numRecord];
+    Long[] ages = new Long[numRecord];
+    String[] linkedInWebs = new String[numRecord];
+    String[] twitterWebs = new String[numRecord];
+    for (int i = 0; i < numRecord; i++) {
+      names[i] = getString();
+      ages[i] = getLong();
+      linkedInWebs[i] = getString();
+      twitterWebs[i] = getString();
+    }
+
+    testData.put("Name", names);
+    testData.put("Age", ages);
+    testData.put("LinkedIn", linkedInWebs);
+    testData.put("Twitter", twitterWebs);
+  }
+
+  @Test
+  public void testEncryptionDefault() throws Exception {
+    Configuration conf = new Configuration();
+    runTest(conf);
+  }
+
+  @Test
+  public void testEncryptionGcm() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(SchemaCryptoPropertiesFactory.CONF_ENCRYPTION_ALGORITHM, 
EncryptionAlgorithm._Fields.AES__GCM__CTR__V1.getFieldName());
+    runTest(conf);
+  }
+
+  @Test
+  public void testEncryptionGcmCtr() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(SchemaCryptoPropertiesFactory.CONF_ENCRYPTION_ALGORITHM, 
EncryptionAlgorithm._Fields.AES__GCM__V1.getFieldName());

Review comment:
       I think it is nicer to use the enum `ParquetCypher` instead of the 
parquet-format generated class.




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


> Example of using EncryptionPropertiesFactory and DecryptionPropertiesFactory
> ----------------------------------------------------------------------------
>
>                 Key: PARQUET-1396
>                 URL: https://issues.apache.org/jira/browse/PARQUET-1396
>             Project: Parquet
>          Issue Type: New Feature
>          Components: parquet-mr
>    Affects Versions: 1.10.0, 1.10.1
>            Reporter: Xinli Shang
>            Priority: Major
>              Labels: pull-request-available
>
> This JIRA is an extension to Parquet Modular Encryption Jira(PARQUET-1178) 
> that will provide the basic building blocks and APIs for the encryption 
> support. 
> This JIRA provides a crypto data interface for schema activation of Parquet 
> encryption and serves as a high-level layer on top of PARQUET-1178 to make 
> the adoption of Parquet-1178 easier, with pluggable key access module, 
> without a need to use the low-level encryption APIs. Also, this feature will 
> enable seamless integration with existing clients.
> No change to specifications (Parquet-format), no new Parquet APIs, and no 
> changes in existing Parquet APIs. All current applications, tests, etc, will 
> work.
> From developer perspective, they can just implement the interface into a 
> plugin which can be attached any Parquet application like Hive/Spark etc. 
> This decouples the complexity of dealing with KMS and schema from Parquet 
> applications. In large organization, they may have hundreds or even thousands 
> of Parquet applications and pipelines. The decoupling would make Parquet 
> encryption easier to be adopted.  
> From end user(for example data owner) perspective, if they think a column is 
> sensitive, they can just set that column’s schema as sensitive and then the 
> Parquet application just encrypt that column automatically. This makes end 
> user easy to manage the encryptions of their columns.  



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

Reply via email to