[GitHub] [parquet-mr] wgtmac commented on pull request #1014: PARQUET-2075: Implement unified file rewriter

2023-01-04 Thread GitBox


wgtmac commented on PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#issuecomment-1371885483

   > If you can add more unit tests, particularly the combinations of prune, 
mask, trans-compression etc, it would be better.
   
   I have added some test cases in the `ParquetRewriterTest` to mix config of 
prune, nullify and trans-compression. 
   
   There are two outstanding issues as below:
   - Masking and encrypting same column is not supported yet.
   - Fix `createdBy` of rewritten file to keep both original info and new 
rewriter info.
   
   I am inclined to fix them in separate JIRAs because this patch is simply a 
refactoring patch to unify different rewriters without changing any behavior, 
and it is large enough.
   
   @ggershinsky @shangxinli Please review again when you have time. Thanks!


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

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

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



[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool

2023-01-04 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2075:
-

wgtmac commented on PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#issuecomment-1371885483

   > If you can add more unit tests, particularly the combinations of prune, 
mask, trans-compression etc, it would be better.
   
   I have added some test cases in the `ParquetRewriterTest` to mix config of 
prune, nullify and trans-compression. 
   
   There are two outstanding issues as below:
   - Masking and encrypting same column is not supported yet.
   - Fix `createdBy` of rewritten file to keep both original info and new 
rewriter info.
   
   I am inclined to fix them in separate JIRAs because this patch is simply a 
refactoring patch to unify different rewriters without changing any behavior, 
and it is large enough.
   
   @ggershinsky @shangxinli Please review again when you have time. Thanks!




> Unified Rewriter Tool  
> ---
>
> Key: PARQUET-2075
> URL: https://issues.apache.org/jira/browse/PARQUET-2075
> Project: Parquet
>  Issue Type: New Feature
>Reporter: Xinli Shang
>Assignee: Gang Wu
>Priority: Major
>
> During the discussion of PARQUET-2071, we came up with the idea of a 
> universal tool to translate the existing file to a different state while 
> skipping some level steps like encoding/decoding, to gain speed. For example, 
> only decompress pages and then compress directly. For PARQUET-2071, we only 
> decrypt and then encrypt directly. This will be useful for the existing data 
> to onboard Parquet features like column encryption, zstd etc. 
> We already have tools like trans-compression, column pruning etc. We will 
> consolidate all these tools with this universal tool. 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool

2023-01-04 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2075:
-

wgtmac commented on code in PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r1062185479


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java:
##
@@ -0,0 +1,144 @@
+/*
+ * 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.rewrite;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.crypto.FileEncryptionProperties;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+import java.util.List;
+import java.util.Map;
+
+public class RewriteOptions {
+
+  final Configuration conf;
+  final Path inputFile;
+  final Path outputFile;
+  final List pruneColumns;
+  final CompressionCodecName codecName;
+  final Map maskColumns;
+  final List encryptColumns;
+  final FileEncryptionProperties fileEncryptionProperties;
+
+  private RewriteOptions(Configuration conf,
+ Path inputFile,
+ Path outputFile,
+ List pruneColumns,
+ CompressionCodecName codecName,
+ Map maskColumns,
+ List encryptColumns,
+ FileEncryptionProperties fileEncryptionProperties) {
+this.conf = conf;
+this.inputFile = inputFile;
+this.outputFile = outputFile;
+this.pruneColumns = pruneColumns;
+this.codecName = codecName;
+this.maskColumns = maskColumns;
+this.encryptColumns = encryptColumns;
+this.fileEncryptionProperties = fileEncryptionProperties;
+  }
+
+  public Configuration getConf() {
+return conf;
+  }
+
+  public Path getInputFile() {
+return inputFile;
+  }
+
+  public Path getOutputFile() {
+return outputFile;
+  }
+
+  public List getPruneColumns() {
+return pruneColumns;
+  }
+
+  public CompressionCodecName getCodecName() {
+return codecName;
+  }
+
+  public Map getMaskColumns() {
+return maskColumns;
+  }
+
+  public List getEncryptColumns() {
+return encryptColumns;
+  }
+
+  public FileEncryptionProperties getFileEncryptionProperties() {
+return fileEncryptionProperties;
+  }
+
+  public static class Builder {
+private Configuration conf;
+private Path inputFile;
+private Path outputFile;
+private List pruneColumns;
+private CompressionCodecName codecName;
+private Map maskColumns;
+private List encryptColumns;
+private FileEncryptionProperties fileEncryptionProperties;
+
+public Builder(Configuration conf, Path inputFile, Path outputFile) {
+  this.conf = conf;
+  this.inputFile = inputFile;
+  this.outputFile = outputFile;
+}
+
+public Builder prune(List columns) {
+  this.pruneColumns = columns;
+  return this;
+}
+
+public Builder transform(CompressionCodecName codecName) {
+  this.codecName = codecName;
+  return this;
+}
+
+public Builder mask(Map maskColumns) {
+  this.maskColumns = maskColumns;
+  return this;
+}
+
+public Builder encrypt(List encryptColumns) {
+  this.encryptColumns = encryptColumns;
+  return this;
+}
+
+public Builder encryptionProperties(FileEncryptionProperties 
fileEncryptionProperties) {
+  this.fileEncryptionProperties = fileEncryptionProperties;
+  return this;
+}
+
+public RewriteOptions build() {
+  // TODO: validate any conflict setting

Review Comment:
   Fixed and removed the TODO



##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java:
##
@@ -0,0 +1,144 @@
+/*
+ * 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 

[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1014: PARQUET-2075: Implement unified file rewriter

2023-01-04 Thread GitBox


wgtmac commented on code in PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r1062185479


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java:
##
@@ -0,0 +1,144 @@
+/*
+ * 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.rewrite;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.crypto.FileEncryptionProperties;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+import java.util.List;
+import java.util.Map;
+
+public class RewriteOptions {
+
+  final Configuration conf;
+  final Path inputFile;
+  final Path outputFile;
+  final List pruneColumns;
+  final CompressionCodecName codecName;
+  final Map maskColumns;
+  final List encryptColumns;
+  final FileEncryptionProperties fileEncryptionProperties;
+
+  private RewriteOptions(Configuration conf,
+ Path inputFile,
+ Path outputFile,
+ List pruneColumns,
+ CompressionCodecName codecName,
+ Map maskColumns,
+ List encryptColumns,
+ FileEncryptionProperties fileEncryptionProperties) {
+this.conf = conf;
+this.inputFile = inputFile;
+this.outputFile = outputFile;
+this.pruneColumns = pruneColumns;
+this.codecName = codecName;
+this.maskColumns = maskColumns;
+this.encryptColumns = encryptColumns;
+this.fileEncryptionProperties = fileEncryptionProperties;
+  }
+
+  public Configuration getConf() {
+return conf;
+  }
+
+  public Path getInputFile() {
+return inputFile;
+  }
+
+  public Path getOutputFile() {
+return outputFile;
+  }
+
+  public List getPruneColumns() {
+return pruneColumns;
+  }
+
+  public CompressionCodecName getCodecName() {
+return codecName;
+  }
+
+  public Map getMaskColumns() {
+return maskColumns;
+  }
+
+  public List getEncryptColumns() {
+return encryptColumns;
+  }
+
+  public FileEncryptionProperties getFileEncryptionProperties() {
+return fileEncryptionProperties;
+  }
+
+  public static class Builder {
+private Configuration conf;
+private Path inputFile;
+private Path outputFile;
+private List pruneColumns;
+private CompressionCodecName codecName;
+private Map maskColumns;
+private List encryptColumns;
+private FileEncryptionProperties fileEncryptionProperties;
+
+public Builder(Configuration conf, Path inputFile, Path outputFile) {
+  this.conf = conf;
+  this.inputFile = inputFile;
+  this.outputFile = outputFile;
+}
+
+public Builder prune(List columns) {
+  this.pruneColumns = columns;
+  return this;
+}
+
+public Builder transform(CompressionCodecName codecName) {
+  this.codecName = codecName;
+  return this;
+}
+
+public Builder mask(Map maskColumns) {
+  this.maskColumns = maskColumns;
+  return this;
+}
+
+public Builder encrypt(List encryptColumns) {
+  this.encryptColumns = encryptColumns;
+  return this;
+}
+
+public Builder encryptionProperties(FileEncryptionProperties 
fileEncryptionProperties) {
+  this.fileEncryptionProperties = fileEncryptionProperties;
+  return this;
+}
+
+public RewriteOptions build() {
+  // TODO: validate any conflict setting

Review Comment:
   Fixed and removed the TODO



##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java:
##
@@ -0,0 +1,144 @@
+/*
+ * 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 

[jira] [Commented] (PARQUET-2223) Parquet Data Masking for Column Encryption

2023-01-04 Thread Jiashen Zhang (Jira)


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

Jiashen Zhang commented on PARQUET-2223:


PARQUET-2223: Parquet Data Masking Enhancement

https://docs.google.com/document/d/1K0juUXOg0wWXBlTSBiZYs4snCKeY5MsoVYxWEixL2qU/edit#heading=h.1kzvc0jyhlyb

> Parquet Data Masking for Column Encryption
> --
>
> Key: PARQUET-2223
> URL: https://issues.apache.org/jira/browse/PARQUET-2223
> Project: Parquet
>  Issue Type: Task
>Reporter: Jiashen Zhang
>Priority: Minor
>
> h1. Background
> h2. What is Data Masking?
> Data masking is the process of obfuscating sensitive data. Instead of 
> revealing PII data, masking allows us to return NULLs, hashes or redacted 
> data in its place. With data masking, users who are in the correct permission 
> groups can retrieve the original data and users without permissions will 
> receive masked data.
> h2. Why do we need it?
>  * Fined-Grained Access Control
> h2. Why do we want to enhance data masking?
>  
> Users might not have all permissions for all columns, existing code doesn’t 
> have support for us to skip columns that users don’t have permissions to 
> access. This enhancement will add this support so that users can decide to 
> skip some columns to avoid decryption error.
> h1. Design Requirements
>  # Users can skip some columns with a configuration
> h1. Proposed solution
> Key idea is to modify the request schema by removing skipped columns from the 
> schema.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Updated] (PARQUET-2223) Parquet Data Masking for Column Encryption

2023-01-04 Thread Jiashen Zhang (Jira)


 [ 
https://issues.apache.org/jira/browse/PARQUET-2223?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jiashen Zhang updated PARQUET-2223:
---
Description: 
h1. Background
h2. What is Data Masking?

Data masking is the process of obfuscating sensitive data. Instead of revealing 
PII data, masking allows us to return NULLs, hashes or redacted data in its 
place. With data masking, users who are in the correct permission groups can 
retrieve the original data and users without permissions will receive masked 
data.
h2. Why do we need it?
 * Fined-Grained Access Control

h2. Why do we want to enhance data masking?

 

Users might not have all permissions for all columns, existing code doesn’t 
have support for us to skip columns that users don’t have permissions to 
access. This enhancement will add this support so that users can decide to skip 
some columns to avoid decryption error.
h1. Design Requirements
 # Users can skip some columns with a configuration

h1. Proposed solution

Key idea is to modify the request schema by removing skipped columns from the 
schema.

> Parquet Data Masking for Column Encryption
> --
>
> Key: PARQUET-2223
> URL: https://issues.apache.org/jira/browse/PARQUET-2223
> Project: Parquet
>  Issue Type: Task
>Reporter: Jiashen Zhang
>Priority: Minor
>
> h1. Background
> h2. What is Data Masking?
> Data masking is the process of obfuscating sensitive data. Instead of 
> revealing PII data, masking allows us to return NULLs, hashes or redacted 
> data in its place. With data masking, users who are in the correct permission 
> groups can retrieve the original data and users without permissions will 
> receive masked data.
> h2. Why do we need it?
>  * Fined-Grained Access Control
> h2. Why do we want to enhance data masking?
>  
> Users might not have all permissions for all columns, existing code doesn’t 
> have support for us to skip columns that users don’t have permissions to 
> access. This enhancement will add this support so that users can decide to 
> skip some columns to avoid decryption error.
> h1. Design Requirements
>  # Users can skip some columns with a configuration
> h1. Proposed solution
> Key idea is to modify the request schema by removing skipped columns from the 
> schema.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [parquet-mr] zhangjiashen opened a new pull request, #1016: Parquet Data Masking for Column Encryption

2023-01-04 Thread GitBox


zhangjiashen opened a new pull request, #1016:
URL: https://github.com/apache/parquet-mr/pull/1016

   ### Jira
   
 - https://issues.apache.org/jira/browse/PARQUET-2223
   
   ### Tests
   
   - Unit Tests
   
   ### Commits
   
   - Add Data Masking functionalities to skip reading encrypted columns for 
Column Encryption
   
   ### Documentation
   
   - TODO


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

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

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



[jira] [Created] (PARQUET-2223) Parquet Data Masking for Column Encryption

2023-01-04 Thread Jiashen Zhang (Jira)
Jiashen Zhang created PARQUET-2223:
--

 Summary: Parquet Data Masking for Column Encryption
 Key: PARQUET-2223
 URL: https://issues.apache.org/jira/browse/PARQUET-2223
 Project: Parquet
  Issue Type: Task
Reporter: Jiashen Zhang






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


IMPORTANT: specification bugs around v2 data pages

2023-01-04 Thread Antoine Pitrou


Hello,

I would like to bring this list's attention to two alleged bugs in the
specification around v2 data pages:

- https://issues.apache.org/jira/browse/PARQUET-2221: Encoding spec
  incorrect for dictionary fallback

- https://issues.apache.org/jira/browse/PARQUET-: RLE encoding spec
  incorrect for v2 data pages

Regards

Antoine.




[jira] [Commented] (PARQUET-2222) [Format] RLE encoding spec incorrect for v2 data pages

2023-01-04 Thread Antoine Pitrou (Jira)


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

Antoine Pitrou commented on PARQUET-:
-

cc [~julienledem] [~pnarang] [~rdblue] [~alexlevenson]

> [Format] RLE encoding spec incorrect for v2 data pages
> --
>
> Key: PARQUET-
> URL: https://issues.apache.org/jira/browse/PARQUET-
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-format
>Reporter: Antoine Pitrou
>Priority: Critical
> Fix For: format-2.10.0
>
>
> The spec 
> (https://github.com/apache/parquet-format/blob/master/Encodings.md#run-length-encoding--bit-packing-hybrid-rle--3)
>  has this:
> {code}
> rle-bit-packed-hybrid:  
> length := length of the  in bytes stored as 4 bytes little 
> endian (unsigned int32)
> {code}
> But the length is actually prepended only in v1 data pages, not in v2 data 
> pages.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (PARQUET-2222) [Format] RLE encoding spec incorrect for v2 data pages

2023-01-04 Thread Antoine Pitrou (Jira)
Antoine Pitrou created PARQUET-:
---

 Summary: [Format] RLE encoding spec incorrect for v2 data pages
 Key: PARQUET-
 URL: https://issues.apache.org/jira/browse/PARQUET-
 Project: Parquet
  Issue Type: Bug
  Components: parquet-format
Reporter: Antoine Pitrou
 Fix For: format-2.10.0


The spec 
(https://github.com/apache/parquet-format/blob/master/Encodings.md#run-length-encoding--bit-packing-hybrid-rle--3)
 has this:
{code}
rle-bit-packed-hybrid:  
length := length of the  in bytes stored as 4 bytes little endian 
(unsigned int32)
{code}

But the length is actually prepended only in v1 data pages, not in v2 data 
pages.





--
This message was sent by Atlassian Jira
(v8.20.10#820010)