You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@parquet.apache.org by GitBox <gi...@apache.org> on 2023/01/04 22:23:08 UTC

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

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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] shangxinli commented on pull request #1016: PARQUET-2223: Parquet Data Masking Enhancement for Column Encryption

Posted by "shangxinli (via GitHub)" <gi...@apache.org>.
shangxinli commented on PR #1016:
URL: https://github.com/apache/parquet-mr/pull/1016#issuecomment-1518702874

   @zhangjiashen The current change is incomplete. You only port the change of utilizes to hide the columns in schema, but you need to actually hide it in the readFooter(). And before that, you need to mark those columns as hidden when access denied is thrown from KMS. 


-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1016: PARQUET-2223: Parquet Data Masking Enhancement for Column Encryption

Posted by GitBox <gi...@apache.org>.
wgtmac commented on code in PR #1016:
URL: https://github.com/apache/parquet-mr/pull/1016#discussion_r1062349914


##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/DataMaskingUtil.java:
##########
@@ -0,0 +1,95 @@
+/*
+ *  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.util;
+
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.Type;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+public class DataMaskingUtil
+{

Review Comment:
   The left curly brace should be moved to the end of the above line.



##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/example/GroupReadSupport.java:
##########
@@ -35,8 +37,21 @@ public org.apache.parquet.hadoop.api.ReadSupport.ReadContext init(
       Configuration configuration, Map<String, String> keyValueMetaData,
       MessageType fileSchema) {
     String partialSchemaString = configuration.get(ReadSupport.PARQUET_READ_SCHEMA);
-    MessageType requestedProjection = getSchemaForRead(fileSchema, partialSchemaString);
-    return new ReadContext(requestedProjection);
+    String removeColumns = configuration.get(DataMaskingUtil.DATA_MASKING_COLUMNS);

Review Comment:
   It seems that only the example `ReadSupport` has enabled skipping masked columns. Does it mean that other ReadSupport implementations are required to apply the same approach? If yes, does the `AvroReadSupport` class require the change, too?



##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/DataMaskingUtil.java:
##########
@@ -0,0 +1,95 @@
+/*
+ *  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.util;
+
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.Type;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+public class DataMaskingUtil
+{
+  public final static String DATA_MASKING_COLUMNS = "parquet.data.masking.columns";
+  public final static String DELIMITER = ",";
+
+  public static MessageType removeColumnsInSchema(MessageType schema, String removeColumns)
+  {
+    if (removeColumns == null || removeColumns.isEmpty()) {
+      return schema;
+    }
+
+    Set<ColumnPath> paths = getColumnPaths(removeColumns);
+    List<String> currentPath = new ArrayList<>();
+    List<Type> prunedFields = removeColumnsInFields(schema.getFields(), currentPath, paths);
+    return new MessageType(schema.getName(), prunedFields);

Review Comment:
   What if all columns have been pruned and an empty schema is returned?



##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/DataMaskingUtil.java:
##########
@@ -0,0 +1,95 @@
+/*
+ *  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.util;
+
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.Type;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+public class DataMaskingUtil
+{
+  public final static String DATA_MASKING_COLUMNS = "parquet.data.masking.columns";

Review Comment:
   Should we update the [doc](https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/README.md#class-readsupport) to reflect the new config?



##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/DataMaskingUtil.java:
##########
@@ -0,0 +1,95 @@
+/*
+ *  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.util;
+
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.Type;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+public class DataMaskingUtil
+{
+  public final static String DATA_MASKING_COLUMNS = "parquet.data.masking.columns";
+  public final static String DELIMITER = ",";
+
+  public static MessageType removeColumnsInSchema(MessageType schema, String removeColumns)
+  {

Review Comment:
   ditto



##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/example/GroupReadSupport.java:
##########
@@ -35,8 +37,21 @@ public org.apache.parquet.hadoop.api.ReadSupport.ReadContext init(
       Configuration configuration, Map<String, String> keyValueMetaData,
       MessageType fileSchema) {
     String partialSchemaString = configuration.get(ReadSupport.PARQUET_READ_SCHEMA);
-    MessageType requestedProjection = getSchemaForRead(fileSchema, partialSchemaString);
-    return new ReadContext(requestedProjection);
+    String removeColumns = configuration.get(DataMaskingUtil.DATA_MASKING_COLUMNS);
+
+    if (removeColumns == null) {
+      MessageType requestedProjection = getSchemaForRead(fileSchema, partialSchemaString);
+      return new ReadContext(requestedProjection);
+    }
+
+    if (partialSchemaString == null) {
+      return new ReadContext(DataMaskingUtil.removeColumnsInSchema(fileSchema, removeColumns));
+    } else {
+      MessageType updatedSchema = DataMaskingUtil.removeColumnsInSchema(
+        MessageTypeParser.parseMessageType(partialSchemaString), removeColumns);
+      MessageType requestedProjection = getSchemaForRead(fileSchema, updatedSchema.toString());

Review Comment:
   Why not calling `MessageType requestedProjection = getSchemaForRead(fileSchema, updatedSchema);`



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] zhangjiashen commented on pull request #1016: PARQUET-2223: Parquet Data Masking Enhancement for Column Encryption

Posted by "zhangjiashen (via GitHub)" <gi...@apache.org>.
zhangjiashen commented on PR #1016:
URL: https://github.com/apache/parquet-mr/pull/1016#issuecomment-1590582561

   Created a new PR for data masking-> https://github.com/apache/parquet-mr/pull/1112


-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] zhangjiashen commented on pull request #1016: PARQUET-2223: Parquet Data Masking Enhancement for Column Encryption

Posted by "zhangjiashen (via GitHub)" <gi...@apache.org>.
zhangjiashen commented on PR #1016:
URL: https://github.com/apache/parquet-mr/pull/1016#issuecomment-1590582173

   > 
   
   Created a new PR for data masking-> https://github.com/apache/parquet-mr/pull/1112


-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] zhangjiashen closed pull request #1016: PARQUET-2223: Parquet Data Masking Enhancement for Column Encryption

Posted by "zhangjiashen (via GitHub)" <gi...@apache.org>.
zhangjiashen closed pull request #1016: PARQUET-2223: Parquet Data Masking Enhancement for Column Encryption
URL: https://github.com/apache/parquet-mr/pull/1016


-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] ggershinsky commented on pull request #1016: PARQUET-2223: Parquet Data Masking Enhancement for Column Encryption

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on PR #1016:
URL: https://github.com/apache/parquet-mr/pull/1016#issuecomment-1384054816

   I found the doc. Could you provide me with a "comment" access, so we'll discuss the goals and design there? 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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] shangxinli commented on pull request #1016: PARQUET-2223: Parquet Data Masking Enhancement for Column Encryption

Posted by GitBox <gi...@apache.org>.
shangxinli commented on PR #1016:
URL: https://github.com/apache/parquet-mr/pull/1016#issuecomment-1383006808

   @ggershinsky Do you have time to have a look? 


-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] ggershinsky commented on pull request #1016: PARQUET-2223: Parquet Data Masking Enhancement for Column Encryption

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on PR #1016:
URL: https://github.com/apache/parquet-mr/pull/1016#issuecomment-1383552737

   As far as I understand, _data masking_ replaces content of sensitive columns; it does not remove the columns (schema and content). The latter is done by _column pruning_ - when re-writing a file. All of that is not related to _column encryption_. So I'm not fully sure what is the goal of the mechanism in this PR. Maybe we can start with a googledoc that describes the problem, the goals and the solution design?


-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] zhangjiashen commented on pull request #1016: PARQUET-2223: Parquet Data Masking Enhancement for Column Encryption

Posted by GitBox <gi...@apache.org>.
zhangjiashen commented on PR #1016:
URL: https://github.com/apache/parquet-mr/pull/1016#issuecomment-1384639397

   > I found the doc. Could you provide me with a "comment" access, so we'll discuss the goals and design there? Thanks.
   
   @ggershinsky thanks for looking at this, I have added permission for you, feel free to add questions/comments!


-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] shangxinli commented on pull request #1016: PARQUET-2223: Parquet Data Masking Enhancement for Column Encryption

Posted by "shangxinli (via GitHub)" <gi...@apache.org>.
shangxinli commented on PR #1016:
URL: https://github.com/apache/parquet-mr/pull/1016#issuecomment-1559512037

   @zhangjiashen Do you have time to work on this? 


-- 
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-unsubscribe@parquet.apache.org

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