[jira] [Commented] (PARQUET-1927) ColumnIndex should provide number of records skipped

2020-10-26 Thread Xinli Shang (Jira)


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

Xinli Shang commented on PARQUET-1927:
--

ParquetFileReader.getFilteredRecordCount() cannot be used because Iceberg 
applied RowGroup stats filter and Dcitionary filter also.

I think what we can do is to make getRowRanges() public. Iceberg call 
getRowRanges() to calculate the filteredRecordCount for the RowGroup that is 
determined(by RowGroup stats and Dictionary filter) to be read.   

> ColumnIndex should provide number of records skipped 
> -
>
> Key: PARQUET-1927
> URL: https://issues.apache.org/jira/browse/PARQUET-1927
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Affects Versions: 1.11.0
>Reporter: Xinli Shang
>Assignee: Xinli Shang
>Priority: Major
> Fix For: 1.12.0
>
>
> When integrating Parquet ColumnIndex, I found we need to know from Parquet 
> that how many records that we skipped due to ColumnIndex filtering. When 
> rowCount is 0, readNextFilteredRowGroup() just advance to next without 
> telling the caller. See code here 
> [https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java#L969]
>  
> In Iceberg, it reads Parquet record with an iterator. The hasNext() has the 
> following code():
> valuesRead + skippedValues < totalValues
> See 
> ([https://github.com/apache/iceberg/pull/1566/commits/cd70cac279d3f14ba61f0143f9988d4cc9413651#diff-d80c15b3e5376265436aeab8b79d5a92fb629c6b81f58ad10a11b9b9d3bfcffcR115).]
>  
> So without knowing the skipped values, it is hard to determine hasNext() or 
> not. 
>  
> Currently, we can workaround by using a flag. When readNextFilteredRowGroup() 
> returns null, we consider it is done for the whole file. Then hasNext() just 
> retrun false. 
>  
>  
>  



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


[jira] [Commented] (PARQUET-1396) Example of using EncryptionPropertiesFactory and DecryptionPropertiesFactory

2020-10-26 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1396:
-

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



##
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java
##
@@ -279,6 +279,11 @@ public ParquetWriter(Path file, Configuration conf, 
WriteSupport writeSupport
 WriteSupport.WriteContext writeContext = writeSupport.init(conf);
 MessageType schema = writeContext.getSchema();
 
+// encryptionProperties could be built from the implementation of 
EncryptionPropertiesFactory when it is attached.
+if (encryptionProperties == null) {
+  encryptionProperties = 
ParquetOutputFormat.createEncryptionProperties(conf, file.getPath(), 
writeContext);

Review comment:
   per the previous comment, can be changed to `encryptionProperties = 
ParquetOutputFormat.createEncryptionProperties(conf, new Path(file.getPath()), 
writeContext);`





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)


[GitHub] [parquet-mr] ggershinsky commented on a change in pull request #808: PARQUET-1396: Example of using EncryptionPropertiesFactory and DecryptionPropertiesFactory

2020-10-26 Thread GitBox


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



##
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java
##
@@ -279,6 +279,11 @@ public ParquetWriter(Path file, Configuration conf, 
WriteSupport writeSupport
 WriteSupport.WriteContext writeContext = writeSupport.init(conf);
 MessageType schema = writeContext.getSchema();
 
+// encryptionProperties could be built from the implementation of 
EncryptionPropertiesFactory when it is attached.
+if (encryptionProperties == null) {
+  encryptionProperties = 
ParquetOutputFormat.createEncryptionProperties(conf, file.getPath(), 
writeContext);

Review comment:
   per the previous comment, can be changed to `encryptionProperties = 
ParquetOutputFormat.createEncryptionProperties(conf, new Path(file.getPath()), 
writeContext);`





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




[jira] [Commented] (PARQUET-1396) Example of using EncryptionPropertiesFactory and DecryptionPropertiesFactory

2020-10-26 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1396:
-

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



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

Review comment:
   apologies, this is a recent change done due to my suggestion; making 
`common` dependent on `hadoop` is indeed not a good idea. @shangxinli - I've 
checked the hadoop Path and FileSystem code, and it looks like having a Path 
class here is not a must. A regular String would do, eg `String getPath` in 
this interface (`OutputFile`). 
   Then the ParquetWriter can use the `new Path(String)` call.





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)


[GitHub] [parquet-mr] ggershinsky commented on a change in pull request #808: PARQUET-1396: Example of using EncryptionPropertiesFactory and DecryptionPropertiesFactory

2020-10-26 Thread GitBox


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



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

Review comment:
   apologies, this is a recent change done due to my suggestion; making 
`common` dependent on `hadoop` is indeed not a good idea. @shangxinli - I've 
checked the hadoop Path and FileSystem code, and it looks like having a Path 
class here is not a must. A regular String would do, eg `String getPath` in 
this interface (`OutputFile`). 
   Then the ParquetWriter can use the `new Path(String)` call.





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




[jira] [Commented] (PARQUET-1915) Add null command

2020-10-26 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1915:
-

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



##
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/ColumnMasker.java
##
@@ -0,0 +1,274 @@
+/*
+ * 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.bytes.BytesInput;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.ColumnReader;
+import org.apache.parquet.column.ColumnWriter;
+import org.apache.parquet.column.Encoding;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.column.ParquetProperties.WriterVersion;
+import org.apache.parquet.column.impl.ColumnReadStoreImpl;
+import org.apache.parquet.column.page.DataPage;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.page.PageWriteStore;
+import org.apache.parquet.column.page.PageWriter;
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import 
org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import 
org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileWriter;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.io.ParquetEncodingException;
+import org.apache.parquet.io.api.Converter;
+import org.apache.parquet.io.api.GroupConverter;
+import org.apache.parquet.io.api.PrimitiveConverter;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class ColumnMasker {
+  /**
+   *
+   * @param reader Reader of source file
+   * @param writer Writer of destination file
+   * @param meta Metadata of source file
+   * @param schema Schema of source file
+   * @param paths Column Paths need to be masked
+   * @param maskMode Mode to mask
+   * @throws IOException
+   */
+  public void processBlocks(TransParquetFileReader reader, 
TransParquetFileWriter writer, ParquetMetadata meta,
+MessageType schema, List paths, MaskMode 
maskMode) throws IOException {
+Set nullifyColumns = convertToColumnPaths(paths);
+int blockIndex = 0;
+PageReadStore store = reader.readNextRowGroup();
+
+while (store != null) {
+  writer.startBlock(store.getRowCount());
+  List columnsInOrder = 
meta.getBlocks().get(blockIndex).getColumns();
+  Map descriptorsMap = 
schema.getColumns().stream().collect(
+Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x));
+  ColumnReadStoreImpl crStore = new ColumnReadStoreImpl(store, new 
DummyGroupConverter(), schema,
+meta.getFileMetaData().getCreatedBy());
+
+  for (int i = 0; i < columnsInOrder.size(); i += 1) {
+ColumnChunkMetaData chunk = columnsInOrder.get(i);
+ColumnDescriptor descriptor = descriptorsMap.get(chunk.getPath());
+processChunk(descriptor, chunk, crStore, reader, writer, schema, 
nullifyColumns, maskMode);
+  }
+
+  writer.endBlock();
+  store = reader.readNextRowGroup();
+  blockIndex++;
+}
+  }
+
+  private void processChunk(ColumnDescriptor descriptor, ColumnChunkMetaData 
chunk, ColumnReadStoreImpl crStore,
+TransParquetFileReader reader, 
TransParquetFileWriter wr

[GitHub] [parquet-mr] gszadovszky commented on a change in pull request #819: PARQUET-1915: Add nullify column

2020-10-26 Thread GitBox


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



##
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/ColumnMasker.java
##
@@ -0,0 +1,274 @@
+/*
+ * 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.bytes.BytesInput;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.ColumnReader;
+import org.apache.parquet.column.ColumnWriter;
+import org.apache.parquet.column.Encoding;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.column.ParquetProperties.WriterVersion;
+import org.apache.parquet.column.impl.ColumnReadStoreImpl;
+import org.apache.parquet.column.page.DataPage;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.page.PageWriteStore;
+import org.apache.parquet.column.page.PageWriter;
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import 
org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import 
org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileWriter;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.io.ParquetEncodingException;
+import org.apache.parquet.io.api.Converter;
+import org.apache.parquet.io.api.GroupConverter;
+import org.apache.parquet.io.api.PrimitiveConverter;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class ColumnMasker {
+  /**
+   *
+   * @param reader Reader of source file
+   * @param writer Writer of destination file
+   * @param meta Metadata of source file
+   * @param schema Schema of source file
+   * @param paths Column Paths need to be masked
+   * @param maskMode Mode to mask
+   * @throws IOException
+   */
+  public void processBlocks(TransParquetFileReader reader, 
TransParquetFileWriter writer, ParquetMetadata meta,
+MessageType schema, List paths, MaskMode 
maskMode) throws IOException {
+Set nullifyColumns = convertToColumnPaths(paths);
+int blockIndex = 0;
+PageReadStore store = reader.readNextRowGroup();
+
+while (store != null) {
+  writer.startBlock(store.getRowCount());
+  List columnsInOrder = 
meta.getBlocks().get(blockIndex).getColumns();
+  Map descriptorsMap = 
schema.getColumns().stream().collect(
+Collectors.toMap(x -> ColumnPath.get(x.getPath()), x -> x));
+  ColumnReadStoreImpl crStore = new ColumnReadStoreImpl(store, new 
DummyGroupConverter(), schema,
+meta.getFileMetaData().getCreatedBy());
+
+  for (int i = 0; i < columnsInOrder.size(); i += 1) {
+ColumnChunkMetaData chunk = columnsInOrder.get(i);
+ColumnDescriptor descriptor = descriptorsMap.get(chunk.getPath());
+processChunk(descriptor, chunk, crStore, reader, writer, schema, 
nullifyColumns, maskMode);
+  }
+
+  writer.endBlock();
+  store = reader.readNextRowGroup();
+  blockIndex++;
+}
+  }
+
+  private void processChunk(ColumnDescriptor descriptor, ColumnChunkMetaData 
chunk, ColumnReadStoreImpl crStore,
+TransParquetFileReader reader, 
TransParquetFileWriter writer, MessageType schema,
+Set paths, MaskMode maskMode) throws 
IOException {
+reader.setStreamPosition(chunk.getStartingPos());
+
+if (paths.contains(chunk.getPath())) {
+  if (maskMode.equals(MaskMode.NULLIFY)) {
+

[jira] [Commented] (PARQUET-1927) ColumnIndex should provide number of records skipped

2020-10-26 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-1927:
---

[~sha...@uber.com], sorry for keep bothering with my ideas but it seems I still 
not get the concept.

As far as I understand iceberg keeps reading the rows until it reaches the 
total number of rows in the row group or the file (not sure which one). Both 
the numbers of (filtered) rows are available for the row group 
([PageReadStore.getRowCount()|https://github.com/apache/parquet-mr/blob/master/parquet-column/src/main/java/org/apache/parquet/column/page/PageReadStore.java#L44])
 and the whole file 
([ParquetFileReader.getFilteredRecordCount()|https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java#L829]).
 I am not sure why you try to align the number of rows already read with the 
number of filtered rows instead of using the proper number for the total. 
(Instead of {{valuesRead + skippedValues < totalValues}} you may use 
{{valuesRead < totalFilteredValues}}.)
Of course if you have to use the number of total (filtered) rows in the file 
you have to calculate the filtering for all row groups before starting to read 
any value but you have to do it anyway so I don't think it should be a problem.

Meanwhile, if you think the API change is required I am happy to review the 
related PR.

> ColumnIndex should provide number of records skipped 
> -
>
> Key: PARQUET-1927
> URL: https://issues.apache.org/jira/browse/PARQUET-1927
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Affects Versions: 1.11.0
>Reporter: Xinli Shang
>Assignee: Xinli Shang
>Priority: Major
> Fix For: 1.12.0
>
>
> When integrating Parquet ColumnIndex, I found we need to know from Parquet 
> that how many records that we skipped due to ColumnIndex filtering. When 
> rowCount is 0, readNextFilteredRowGroup() just advance to next without 
> telling the caller. See code here 
> [https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java#L969]
>  
> In Iceberg, it reads Parquet record with an iterator. The hasNext() has the 
> following code():
> valuesRead + skippedValues < totalValues
> See 
> ([https://github.com/apache/iceberg/pull/1566/commits/cd70cac279d3f14ba61f0143f9988d4cc9413651#diff-d80c15b3e5376265436aeab8b79d5a92fb629c6b81f58ad10a11b9b9d3bfcffcR115).]
>  
> So without knowing the skipped values, it is hard to determine hasNext() or 
> not. 
>  
> Currently, we can workaround by using a flag. When readNextFilteredRowGroup() 
> returns null, we consider it is done for the whole file. Then hasNext() just 
> retrun false. 
>  
>  
>  



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


[jira] [Commented] (PARQUET-1396) Example of using EncryptionPropertiesFactory and DecryptionPropertiesFactory

2020-10-26 Thread ASF GitHub Bot (Jira)


[ 
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> cryptoMetadata = new HashMap<>();
+  private Map 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_

[GitHub] [parquet-mr] gszadovszky commented on a change in pull request #808: PARQUET-1396: Example of using EncryptionPropertiesFactory and DecryptionPropertiesFactory

2020-10-26 Thread GitBox


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> cryptoMetadata = new HashMap<>();
+  private Map 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
##
@@ -6

Re: Current status of Data Page V2?

2020-10-26 Thread Gabor Szadovszky
Hi Micah,

V2 pages are not only about the new encodings but a couple of other things
the community thought would be better than V1.
One of these improvements was to break pages at row boundaries. This one is
outdated because during the development of column-indexes we had to
implement the same for V1 pages.
Another improvement is to store the repetition and definition levels
separately so they can be read without decompressing data. I don't know if
we still think it has significant benefits but we never implemented
anything that makes use of it.
The default encodings are also differences in V1 and V2 pages but they are
not properly specified. Also, the encodings are not tied to V1 or V2 by
design. It is only a matter of specification.

I don't think that anything is blocking V2 pages to be introduced in
production (I believe there are a couple of users who already use V2 pages
in production) but I think it isn't worth the effort. As far as I know
parquet-mr is the only one that supports V2 pages.

I agree it is complicated to introduce new encodings and keep backward
compatibility but this is how compression codecs work currently. Supporting
compression codecs is not only about supporting a specific parquet-format
release or having a specific parquet-mr release but you also have to have
codec implementation available. We might handle the encodings similarly
that there are default ones guaranteed would work and there are others that
might not be supported by older readers.
But this is only an example. We might want to introduce some version levels
or compatibility groups. What I want to say there is no reason to tie the
new codecs to V2 pages. We should handle the introduction of the new codecs
(and handling their compatibility) and the V2 pages separately.

It would be nice to hear other opinions on this topic.

Regards,
Gabor

On Thu, Oct 22, 2020 at 5:54 PM Micah Kornfield 
wrote:

> Hi Gabor,
>
> > It is still not clear to me if we want to recommend V2 for production use
> > at all
>
> Again, I'm missing context here, but what is blocking V2 for production
> use?  Is it specification finalization, implementation finalization?
> Something else?
>
> or simply introduce the new encodings for V1.
>
>
> This doesn't seem simple to me.  It will break V1 readers that don't
> support these encodings. It also sounds like this is taking the frame or
> reference of parquet-mr, and not Parquet as a specification.  My main goal
> in this thread is to try to capture information, so we can get better
> specification docs (and to potentially help drive consensus on V2).
>
> There was another recent e-mail thread [1] that also pointed out our gap in
> explaining how versioning works and what is expected to be included in
> compliant reader/writers (if there are docs like this I apologize if I
> missed them and would appreciate pointers).
>
>  I would suggest
> > discussing this topic on the parquet sync next Tuesday.
>
>
> I will try to make it but I think I might have a conflict.  In this case I
> think continuing the email thread would be useful, so we have a good
> historical snapshot of the discussion (a lot of context gets lost in sync
> notes in my experience).
>
> [1]
>
> https://mail-archives.apache.org/mod_mbox/parquet-dev/202010.mbox/%3CMWHPR03MB31842468BE51022392B7F2D1CE020%40MWHPR03MB3184.namprd03.prod.outlook.com%3E
>
>
> On Thursday, October 22, 2020, Gabor Szadovszky  wrote:
>
> > It is still not clear to me if we want to recommend V2 for production use
> > at all or simply introduce the new encodings for V1. I would suggest
> > discussing this topic on the parquet sync next Tuesday.
> >
> > On Thu, Oct 22, 2020 at 6:04 AM Micah Kornfield 
> > wrote:
> >
> > > I've created https://github.com/apache/parquet-format/pull/163 to try
> to
> > > document these (note I really don't have historical context here so
> > please
> > > review carefully).
> > >
> > > I would appreciate it if someone could point me to a reference on what
> > the
> > > current status of V2 is?  What is left unsettled? When can we start
> > > recommending it for production use?
> > >
> > > Thanks,
> > > Micah
> > >
> > > On Tue, Oct 13, 2020 at 9:23 AM Micah Kornfield  >
> > > wrote:
> > >
> > > > I am not sure 2.0 means the v2 pages here. I think there was/is a bit
> > of
> > > >> confusion between the v1/v2 pages and the parquet-mr releases. Maybe
> > the
> > > >> parquet-format releases are also part of it.
> > > >
> > > >
> > > > +1 to the confusion part.  The reason why I originally started this
> > > thread
> > > > is that none of this is entirely clear to me from existing
> > documentation.
> > > >
> > > > In particular it is confusing to me to say that the V2 Spec is not
> yet
> > > > finished when it looks like there have been multiple V2 Format
> > releases.
> > > >
> > > > It would be extremely useful to have documentation relating features
> > to:
> > > > 1.  The version of the spec they are part of
> > > > 2.  There current