[GitHub] [parquet-mr] shangxinli commented on pull request #808: Parquet-1396: Cryptodata Interface for Schema Activation of Parquet E…

2020-10-22 Thread GitBox


shangxinli commented on pull request #808:
URL: https://github.com/apache/parquet-mr/pull/808#issuecomment-714828022


   I just pushed more comments and squashed into one. This is ready for review 
now. 



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




[GitHub] [parquet-mr] shangxinli commented on a change in pull request #808: Parquet-1396: Cryptodata Interface for Schema Activation of Parquet E…

2020-10-22 Thread GitBox


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



##
File path: 
parquet-hadoop/src/test/java/org/apache/parquet/crypto/propertiesfactory/SchemaControlEncryptionTest.java
##
@@ -0,0 +1,248 @@
+/*
+ * 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.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> crytoMetadata = 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, 
"AES_GCM_CTR_V1");
+runTest(conf);
+  }
+
+  @Test
+  public void testEncryptionGcmCtr() throws Exception {
+Configuration conf = new Configuration();
+conf.set(SchemaCryptoPropertiesFactory.CONF_ENCRYPTION_ALGORITHM, 
"AES_GCM_V1");
+runTest(conf);
+  }
+
+  @Test
+  public void testEncryptionWithFooter() throws Exception {
+Configuration conf = new Configuration();
+conf.setBoolean(SchemaCryptoPropertiesFactory.CONF_ENCRYPTION_FOOTER, 
true);
+runTest(conf);
+  }
+
+  private void runTest(Configuration conf ) throws Exception {
+conf.set(EncryptionPropertiesFactory.CRYPTO_FACTORY_CLASS_PROPERTY_NAME,
+  
"org.apache.parquet.crypto.propertiesfactory.SchemaCryptoPropertiesFactory");

Review comment:
   changed to 

[GitHub] [parquet-mr] shangxinli commented on a change in pull request #808: Parquet-1396: Cryptodata Interface for Schema Activation of Parquet E…

2020-10-22 Thread GitBox


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



##
File path: 
parquet-hadoop/src/test/java/org/apache/parquet/crypto/propertiesfactory/SchemaControlEncryptionTest.java
##
@@ -0,0 +1,248 @@
+/*
+ * 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.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> crytoMetadata = 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, 
"AES_GCM_CTR_V1");
+runTest(conf);
+  }
+
+  @Test
+  public void testEncryptionGcmCtr() throws Exception {
+Configuration conf = new Configuration();
+conf.set(SchemaCryptoPropertiesFactory.CONF_ENCRYPTION_ALGORITHM, 
"AES_GCM_V1");

Review comment:
   fixed

##
File path: 
parquet-hadoop/src/test/java/org/apache/parquet/crypto/propertiesfactory/SchemaControlEncryptionTest.java
##
@@ -0,0 +1,248 @@
+/*
+ * 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 

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

2020-10-22 Thread Xinli Shang (Jira)


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

Xinli Shang commented on PARQUET-1927:
--

[~gszadovszky], the problem is when rowCount is 0(line 966 
https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java#L966),
 readNextFilteredRowGroup() will just call  advanceToNextBlock() and then 
recurse itself to next row group. In that case, the returned count of 
[PageReadStore.getRowCount()|https://github.com/apache/parquet-mr/blob/master/parquet-column/src/main/java/org/apache/parquet/column/page/PageReadStore.java#L44]
 will be the filtered count of the next row group. Iceberg doesn't have the 
knowledge to know these row counts are from which row group. It has to assume 
it is from the previous group. The result is it is wrongly counted and Iceberg 
iterator will just return true in hasNext() even all the records are read. 

 

The fix could be just to add a count for a skipped count including the skipped 
count as a whole row group. 

 

> 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
>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] [Resolved] (PARQUET-1932) Bump Fastutil to 8.4.2

2020-10-22 Thread Fokko Driesprong (Jira)


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

Fokko Driesprong resolved PARQUET-1932.
---
Resolution: Fixed

> Bump Fastutil to 8.4.2
> --
>
> Key: PARQUET-1932
> URL: https://issues.apache.org/jira/browse/PARQUET-1932
> Project: Parquet
>  Issue Type: Improvement
>Reporter: Fokko Driesprong
>Assignee: Fokko Driesprong
>Priority: Major
> Fix For: 1.12.0
>
>




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


[jira] [Updated] (PARQUET-1932) Bump Fastutil to 8.4.2

2020-10-22 Thread Fokko Driesprong (Jira)


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

Fokko Driesprong updated PARQUET-1932:
--
Affects Version/s: 1.11.0

> Bump Fastutil to 8.4.2
> --
>
> Key: PARQUET-1932
> URL: https://issues.apache.org/jira/browse/PARQUET-1932
> Project: Parquet
>  Issue Type: Improvement
>Affects Versions: 1.11.0
>Reporter: Fokko Driesprong
>Assignee: Fokko Driesprong
>Priority: Major
> Fix For: 1.12.0
>
>




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


[jira] [Commented] (PARQUET-1932) Bump Fastutil to 8.4.2

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


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

ASF GitHub Bot commented on PARQUET-1932:
-

Fokko merged pull request #836:
URL: https://github.com/apache/parquet-mr/pull/836


   



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


> Bump Fastutil to 8.4.2
> --
>
> Key: PARQUET-1932
> URL: https://issues.apache.org/jira/browse/PARQUET-1932
> Project: Parquet
>  Issue Type: Improvement
>Reporter: Fokko Driesprong
>Assignee: Fokko Driesprong
>Priority: Major
> Fix For: 1.12.0
>
>




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


[GitHub] [parquet-mr] Fokko merged pull request #836: [PARQUET-1932] Bump Fastutil to 8.4.2

2020-10-22 Thread GitBox


Fokko merged pull request #836:
URL: https://github.com/apache/parquet-mr/pull/836


   



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




[GitHub] [parquet-mr] dossett commented on pull request #818: Remove brew install since thrift 0.12 isn't available

2020-10-22 Thread GitBox


dossett commented on pull request #818:
URL: https://github.com/apache/parquet-mr/pull/818#issuecomment-714624897


   That works too!



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] [Resolved] (PARQUET-1929) Bump Snappy to 1.1.8

2020-10-22 Thread Fokko Driesprong (Jira)


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

Fokko Driesprong resolved PARQUET-1929.
---
Fix Version/s: 1.12.0
   Resolution: Fixed

> Bump Snappy to 1.1.8
> 
>
> Key: PARQUET-1929
> URL: https://issues.apache.org/jira/browse/PARQUET-1929
> Project: Parquet
>  Issue Type: Improvement
>Reporter: Fokko Driesprong
>Assignee: Fokko Driesprong
>Priority: Major
> Fix For: 1.12.0
>
>




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


[jira] [Commented] (PARQUET-1929) Bump Snappy to 1.1.8

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


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

ASF GitHub Bot commented on PARQUET-1929:
-

Fokko merged pull request #833:
URL: https://github.com/apache/parquet-mr/pull/833


   



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


> Bump Snappy to 1.1.8
> 
>
> Key: PARQUET-1929
> URL: https://issues.apache.org/jira/browse/PARQUET-1929
> Project: Parquet
>  Issue Type: Improvement
>Reporter: Fokko Driesprong
>Assignee: Fokko Driesprong
>Priority: Major
> Fix For: 1.12.0
>
>




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


[GitHub] [parquet-mr] Fokko merged pull request #833: [PARQUET-1929] Bump Snappy to 1.1.8

2020-10-22 Thread GitBox


Fokko merged pull request #833:
URL: https://github.com/apache/parquet-mr/pull/833


   



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] [Updated] (PARQUET-1929) Bump Snappy to 1.1.8

2020-10-22 Thread Fokko Driesprong (Jira)


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

Fokko Driesprong updated PARQUET-1929:
--
Affects Version/s: 1.11.0

> Bump Snappy to 1.1.8
> 
>
> Key: PARQUET-1929
> URL: https://issues.apache.org/jira/browse/PARQUET-1929
> Project: Parquet
>  Issue Type: Improvement
>Affects Versions: 1.11.0
>Reporter: Fokko Driesprong
>Assignee: Fokko Driesprong
>Priority: Major
> Fix For: 1.12.0
>
>




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


[jira] [Resolved] (PARQUET-1931) Bump Junit 4.13.1

2020-10-22 Thread Fokko Driesprong (Jira)


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

Fokko Driesprong resolved PARQUET-1931.
---
Fix Version/s: 1.12.0
   Resolution: Fixed

> Bump Junit 4.13.1
> -
>
> Key: PARQUET-1931
> URL: https://issues.apache.org/jira/browse/PARQUET-1931
> Project: Parquet
>  Issue Type: Improvement
>Reporter: Fokko Driesprong
>Assignee: Fokko Driesprong
>Priority: Major
> Fix For: 1.12.0
>
>




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


[jira] [Commented] (PARQUET-1931) Bump Junit 4.13.1

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


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

ASF GitHub Bot commented on PARQUET-1931:
-

Fokko merged pull request #835:
URL: https://github.com/apache/parquet-mr/pull/835


   



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


> Bump Junit 4.13.1
> -
>
> Key: PARQUET-1931
> URL: https://issues.apache.org/jira/browse/PARQUET-1931
> Project: Parquet
>  Issue Type: Improvement
>Reporter: Fokko Driesprong
>Assignee: Fokko Driesprong
>Priority: Major
>




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


[jira] [Updated] (PARQUET-1931) Bump Junit 4.13.1

2020-10-22 Thread Fokko Driesprong (Jira)


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

Fokko Driesprong updated PARQUET-1931:
--
Affects Version/s: 1.11.0

> Bump Junit 4.13.1
> -
>
> Key: PARQUET-1931
> URL: https://issues.apache.org/jira/browse/PARQUET-1931
> Project: Parquet
>  Issue Type: Improvement
>Affects Versions: 1.11.0
>Reporter: Fokko Driesprong
>Assignee: Fokko Driesprong
>Priority: Major
> Fix For: 1.12.0
>
>




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


[GitHub] [parquet-mr] Fokko merged pull request #835: [PARQUET-1931] Bump Junit to 4.13.1

2020-10-22 Thread GitBox


Fokko merged pull request #835:
URL: https://github.com/apache/parquet-mr/pull/835


   



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




[GitHub] [parquet-mr] Fokko commented on pull request #818: Remove brew install since thrift 0.12 isn't available

2020-10-22 Thread GitBox


Fokko commented on pull request #818:
URL: https://github.com/apache/parquet-mr/pull/818#issuecomment-714619809


   Thanks @dossett for letting us know. Master is now at Apache Thrift 0.13.0 
(https://github.com/apache/parquet-mr/pull/834) which is available at Brew: 
`brew install thrift@0.13`



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




[GitHub] [parquet-mr] Fokko closed pull request #818: Remove brew install since thrift 0.12 isn't available

2020-10-22 Thread GitBox


Fokko closed pull request #818:
URL: https://github.com/apache/parquet-mr/pull/818


   



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] [Updated] (PARQUET-1930) Bump Apache Thrift to 0.13.0

2020-10-22 Thread Fokko Driesprong (Jira)


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

Fokko Driesprong updated PARQUET-1930:
--
Summary: Bump Apache Thrift to 0.13.0  (was: Bump Apache Thrift to 0.13)

> Bump Apache Thrift to 0.13.0
> 
>
> Key: PARQUET-1930
> URL: https://issues.apache.org/jira/browse/PARQUET-1930
> Project: Parquet
>  Issue Type: Improvement
>Affects Versions: 1.11.0
>Reporter: Fokko Driesprong
>Assignee: Fokko Driesprong
>Priority: Major
> Fix For: 1.12.0
>
>




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


[jira] [Assigned] (PARQUET-1910) Parquet-cli is broken after TransCompressionCommand was added

2020-10-22 Thread Fokko Driesprong (Jira)


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

Fokko Driesprong reassigned PARQUET-1910:
-

Assignee: Grisha Weintraub

> Parquet-cli is broken after TransCompressionCommand was added
> -
>
> Key: PARQUET-1910
> URL: https://issues.apache.org/jira/browse/PARQUET-1910
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-cli
>Affects Versions: 1.12.0
>Reporter: Grisha Weintraub
>Assignee: Grisha Weintraub
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.12.0
>
>
> +Scenario+
> Run parquet-cli
> {code:java}
> java -cp 'target/*:target/dependency/*' org.apache.parquet.cli.Main
> {code}
>  
> +Expected result+ 
> parquet-cli usage is presented
> {code:java}
> Usage: parquet [options] [command] [command options]
> {code}
>  
> +Actual result+
> the following error is presented
> {code:java}
> Exception in thread "main" com.beust.jcommander.ParameterException: Only one 
> @Parameter with no names attribute is allowed, 
> found:com.beust.jcommander.JCommander$MainParameter@6442b0a6 and 
> com.beust.jcommander.Parameterized@f5bfa59d
> at com.beust.jcommander.JCommander.addDescription(JCommander.java:606)
> at 
> com.beust.jcommander.JCommander.createDescriptions(JCommander.java:587)
> at com.beust.jcommander.JCommander.addCommand(JCommander.java:1533)
> at com.beust.jcommander.JCommander.addCommand(JCommander.java:1512)
> at org.apache.parquet.cli.Main.(Main.java:95)
> at org.apache.parquet.cli.Main.main(Main.java:181)
> {code}
>  
> +Notes+
> The error is related to the recent changes in PARQUET-1872.
> The fix is trivial - see PR.



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


[GitHub] [parquet-mr] Fokko merged pull request #834: [PARQUET-1930] Bump Apache Thrift to 0.13

2020-10-22 Thread GitBox


Fokko merged pull request #834:
URL: https://github.com/apache/parquet-mr/pull/834


   



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] [Updated] (PARQUET-1930) Bump Apache Thrift to 0.13

2020-10-22 Thread Fokko Driesprong (Jira)


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

Fokko Driesprong updated PARQUET-1930:
--
Affects Version/s: 1.11.0

> Bump Apache Thrift to 0.13
> --
>
> Key: PARQUET-1930
> URL: https://issues.apache.org/jira/browse/PARQUET-1930
> Project: Parquet
>  Issue Type: Improvement
>Affects Versions: 1.11.0
>Reporter: Fokko Driesprong
>Assignee: Fokko Driesprong
>Priority: Major
>




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


[jira] [Resolved] (PARQUET-1930) Bump Apache Thrift to 0.13

2020-10-22 Thread Fokko Driesprong (Jira)


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

Fokko Driesprong resolved PARQUET-1930.
---
Resolution: Fixed

> Bump Apache Thrift to 0.13
> --
>
> Key: PARQUET-1930
> URL: https://issues.apache.org/jira/browse/PARQUET-1930
> Project: Parquet
>  Issue Type: Improvement
>Affects Versions: 1.11.0
>Reporter: Fokko Driesprong
>Assignee: Fokko Driesprong
>Priority: Major
> Fix For: 1.12.0
>
>




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


[jira] [Commented] (PARQUET-1930) Bump Apache Thrift to 0.13

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


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

ASF GitHub Bot commented on PARQUET-1930:
-

Fokko merged pull request #834:
URL: https://github.com/apache/parquet-mr/pull/834


   



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


> Bump Apache Thrift to 0.13
> --
>
> Key: PARQUET-1930
> URL: https://issues.apache.org/jira/browse/PARQUET-1930
> Project: Parquet
>  Issue Type: Improvement
>Affects Versions: 1.11.0
>Reporter: Fokko Driesprong
>Assignee: Fokko Driesprong
>Priority: Major
>




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


[jira] [Updated] (PARQUET-1930) Bump Apache Thrift to 0.13

2020-10-22 Thread Fokko Driesprong (Jira)


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

Fokko Driesprong updated PARQUET-1930:
--
Fix Version/s: 1.12.0

> Bump Apache Thrift to 0.13
> --
>
> Key: PARQUET-1930
> URL: https://issues.apache.org/jira/browse/PARQUET-1930
> Project: Parquet
>  Issue Type: Improvement
>Affects Versions: 1.11.0
>Reporter: Fokko Driesprong
>Assignee: Fokko Driesprong
>Priority: Major
> Fix For: 1.12.0
>
>




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


[GitHub] [parquet-mr] shangxinli commented on a change in pull request #808: Parquet-1396: Cryptodata Interface for Schema Activation of Parquet E…

2020-10-22 Thread GitBox


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



##
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, new Path(file.toString()), 
writeContext);

Review comment:
   Good point. I like the idea of adding "Path getPath()" better. Since 
this feature is going to be in a major release. I think
   adding a new method to the interface could be fine. 





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




Re: Current status of Data Page V2?

2020-10-22 Thread Micah Kornfield
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 status in reference implementations
> > >
> > > Thanks,
> > > Micah
> > >
> > >
> > > On Tue, Oct 13, 2020 at 1:51 AM Gabor Szadovszky 
> > 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.
> > >> In this table many features are not related to the pages so I don't
> > think
> > >> the "Expected release" meant the v1/v2 pages. I guess there was an
> > earlier
> > >> plan to release parquet-mr 2.0 with the v2 pages but then v2 pages
> were
> > >> released in a 1.x release while 2.0 is not planned yet. (I was not in
> > the
> > >> community that time so I'm only guessing.)
> > >>
> > >> Also worth to mention that it seems to be not related to the
> > >> parquet-format
> > >> releases which means that based on the spec the implementations
> were/are
> > >> not limited by this table.
> > >>
> > >>
> > >> On Mon, Oct 12, 2020 at 6:53 PM Ryan Blue 
> > >> wrote:
> > >>
> > >> > I remembered that there used to be a table. Looks like it was
> removed:
> > >> >
> > >> >
> > >>
> >
> https://github.com/apache/parquet-mr/commit/aed9097640c7adffe1151b32e86b5efc3702c657?short_path=b335630#diff-04c6e90faac2675aa89e2176d2eec7d8
> > >> >
> > >> > The table used to list delta as a 2.0 feature.
> > >> >
> > >> > On Mon, Oct 12, 2020 at 1:38 AM Gabor Szadovszky 
> > >> wrote:
> > >> >
> > >> > > That answer I wrote to the other thread was based on the current
> > >> code. So
> > >> > > that is how parquet-mr is working now. It does not mean though how
> > >> shall
> > >> > it
> > >> > > work or how it works in other implementations. Unfortunately, the
> > spec
> > >> > does
> > >> > > not say anything about v1 and v2 in the context of encodings.
> > >> > > Meanwhile, enabling the "new" encodings in v1 may generate
> > >> compatibility
> > >> > > issues with other implementations. (I am not sure how would the
> > >> existing
> > >> > > releases of parquet-mr behave if they have to 

[jira] [Commented] (PARQUET-1925) Introduce Velocity Template Engine to Parquet Generator

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


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

ASF GitHub Bot commented on PARQUET-1925:
-

belugabehr commented on pull request #828:
URL: https://github.com/apache/parquet-mr/pull/828#issuecomment-714581308


   OK.  I added a unit test.  I don't love the test, but it's something (and 
better than nothing).  I'm also not sure how good of an example it will be for 
future tests in this 'generator' project.
   
   I ran into a host of issues trying to test this.  It's probably best to, in 
the future, change `Version` to implement an interface and then use normal 
getters instead of static fields.  Since this class is compiled without any 
interface, there is nothing to import into the test at compile time in order to 
cast it to something I can then easily test against.



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


> Introduce Velocity Template Engine to Parquet Generator
> ---
>
> Key: PARQUET-1925
> URL: https://issues.apache.org/jira/browse/PARQUET-1925
> Project: Parquet
>  Issue Type: New Feature
>Reporter: David Mollitor
>Assignee: David Mollitor
>Priority: Major
>
> Much easier than the current setup of manually outputting the strings.



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


[GitHub] [parquet-mr] belugabehr commented on pull request #828: PARQUET-1925: Introduce Velocity Template Engine to Parquet Generator

2020-10-22 Thread GitBox


belugabehr commented on pull request #828:
URL: https://github.com/apache/parquet-mr/pull/828#issuecomment-714581308


   OK.  I added a unit test.  I don't love the test, but it's something (and 
better than nothing).  I'm also not sure how good of an example it will be for 
future tests in this 'generator' project.
   
   I ran into a host of issues trying to test this.  It's probably best to, in 
the future, change `Version` to implement an interface and then use normal 
getters instead of static fields.  Since this class is compiled without any 
interface, there is nothing to import into the test at compile time in order to 
cast it to something I can then easily test against.



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-1917) [parquet-proto] default values are stored in oneOf fields that aren't set

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


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

ASF GitHub Bot commented on PARQUET-1917:
-

dossett commented on pull request #820:
URL: https://github.com/apache/parquet-mr/pull/820#issuecomment-714553061


   Thank you both!



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-proto] default values are stored in oneOf fields that aren't set
> -
>
> Key: PARQUET-1917
> URL: https://issues.apache.org/jira/browse/PARQUET-1917
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-protobuf
>Affects Versions: 1.12.0
>Reporter: Aaron Blake Niskode-Dossett
>Assignee: Aaron Blake Niskode-Dossett
>Priority: Major
>
> SCHEMA
> 
> {noformat}
> message Person {
>   int32 foo = 1;
>   oneof optional_bar {
>     int32 bar_int = 200;
>     int32 bar_int2 = 201;
>     string bar_string = 300;
>   }
> }{noformat}
>  
> CODE
> 
> I set values for foo and bar_string
>  
> {noformat}
> for (int i = 0; i < 3; i += 1) {
>                 com.etsy.grpcparquet.Person message = Person.newBuilder()
>                         .setFoo(i)
>                         .setBarString("hello world")
>                         .build();
>                 message.writeDelimitedTo(out);
>             }{noformat}
> And then I write the protobuf file out to parquet.
>  
> RESULT
> ---
> {noformat}
> $ parquet-tools show example.parquet                                          
>                                                                               
> +---+---++--+
> |   foo |   bar_int |   bar_int2 | bar_string   |
> |---+---++--|
> |     0 |         0 |          0 | hello world  |
> |     1 |         0 |          0 | hello world  |
> |     2 |         0 |          0 | hello world  |
> +---+---++--+{noformat}
>  
> bar_int and bar_int2 should be EMPTY for all three rows since only bar_string 
> is set in the oneof.  0 is the default value for int, but it should not be 
> stored.



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


[jira] [Resolved] (PARQUET-1914) Allow ProtoParquetReader To Support InputFile

2020-10-22 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky resolved PARQUET-1914.
---
Resolution: Fixed

> Allow ProtoParquetReader To Support InputFile
> -
>
> Key: PARQUET-1914
> URL: https://issues.apache.org/jira/browse/PARQUET-1914
> Project: Parquet
>  Issue Type: Improvement
>Reporter: David Mollitor
>Assignee: David Mollitor
>Priority: Major
>




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


[GitHub] [parquet-mr] gszadovszky merged pull request #817: PARQUET-1914: Allow ProtoParquetReader To Support InputFile

2020-10-22 Thread GitBox


gszadovszky merged pull request #817:
URL: https://github.com/apache/parquet-mr/pull/817


   



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-1914) Allow ProtoParquetReader To Support InputFile

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


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

ASF GitHub Bot commented on PARQUET-1914:
-

gszadovszky merged pull request #817:
URL: https://github.com/apache/parquet-mr/pull/817


   



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


> Allow ProtoParquetReader To Support InputFile
> -
>
> Key: PARQUET-1914
> URL: https://issues.apache.org/jira/browse/PARQUET-1914
> Project: Parquet
>  Issue Type: Improvement
>Reporter: David Mollitor
>Assignee: David Mollitor
>Priority: Major
>




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


[GitHub] [parquet-mr] dossett commented on pull request #820: PARQUET-1917: Don't write values for oneOf fields that aren't set

2020-10-22 Thread GitBox


dossett commented on pull request #820:
URL: https://github.com/apache/parquet-mr/pull/820#issuecomment-714553061


   Thank you both!



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] [Resolved] (PARQUET-1917) [parquet-proto] default values are stored in oneOf fields that aren't set

2020-10-22 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky resolved PARQUET-1917.
---
Resolution: Fixed

> [parquet-proto] default values are stored in oneOf fields that aren't set
> -
>
> Key: PARQUET-1917
> URL: https://issues.apache.org/jira/browse/PARQUET-1917
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-protobuf
>Affects Versions: 1.12.0
>Reporter: Aaron Blake Niskode-Dossett
>Assignee: Aaron Blake Niskode-Dossett
>Priority: Major
>
> SCHEMA
> 
> {noformat}
> message Person {
>   int32 foo = 1;
>   oneof optional_bar {
>     int32 bar_int = 200;
>     int32 bar_int2 = 201;
>     string bar_string = 300;
>   }
> }{noformat}
>  
> CODE
> 
> I set values for foo and bar_string
>  
> {noformat}
> for (int i = 0; i < 3; i += 1) {
>                 com.etsy.grpcparquet.Person message = Person.newBuilder()
>                         .setFoo(i)
>                         .setBarString("hello world")
>                         .build();
>                 message.writeDelimitedTo(out);
>             }{noformat}
> And then I write the protobuf file out to parquet.
>  
> RESULT
> ---
> {noformat}
> $ parquet-tools show example.parquet                                          
>                                                                               
> +---+---++--+
> |   foo |   bar_int |   bar_int2 | bar_string   |
> |---+---++--|
> |     0 |         0 |          0 | hello world  |
> |     1 |         0 |          0 | hello world  |
> |     2 |         0 |          0 | hello world  |
> +---+---++--+{noformat}
>  
> bar_int and bar_int2 should be EMPTY for all three rows since only bar_string 
> is set in the oneof.  0 is the default value for int, but it should not be 
> stored.



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


[jira] [Assigned] (PARQUET-1917) [parquet-proto] default values are stored in oneOf fields that aren't set

2020-10-22 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky reassigned PARQUET-1917:
-

Assignee: Aaron Blake Niskode-Dossett

> [parquet-proto] default values are stored in oneOf fields that aren't set
> -
>
> Key: PARQUET-1917
> URL: https://issues.apache.org/jira/browse/PARQUET-1917
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-protobuf
>Affects Versions: 1.12.0
>Reporter: Aaron Blake Niskode-Dossett
>Assignee: Aaron Blake Niskode-Dossett
>Priority: Major
>
> SCHEMA
> 
> {noformat}
> message Person {
>   int32 foo = 1;
>   oneof optional_bar {
>     int32 bar_int = 200;
>     int32 bar_int2 = 201;
>     string bar_string = 300;
>   }
> }{noformat}
>  
> CODE
> 
> I set values for foo and bar_string
>  
> {noformat}
> for (int i = 0; i < 3; i += 1) {
>                 com.etsy.grpcparquet.Person message = Person.newBuilder()
>                         .setFoo(i)
>                         .setBarString("hello world")
>                         .build();
>                 message.writeDelimitedTo(out);
>             }{noformat}
> And then I write the protobuf file out to parquet.
>  
> RESULT
> ---
> {noformat}
> $ parquet-tools show example.parquet                                          
>                                                                               
> +---+---++--+
> |   foo |   bar_int |   bar_int2 | bar_string   |
> |---+---++--|
> |     0 |         0 |          0 | hello world  |
> |     1 |         0 |          0 | hello world  |
> |     2 |         0 |          0 | hello world  |
> +---+---++--+{noformat}
>  
> bar_int and bar_int2 should be EMPTY for all three rows since only bar_string 
> is set in the oneof.  0 is the default value for int, but it should not be 
> stored.



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


[jira] [Commented] (PARQUET-1917) [parquet-proto] default values are stored in oneOf fields that aren't set

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


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

ASF GitHub Bot commented on PARQUET-1917:
-

gszadovszky merged pull request #820:
URL: https://github.com/apache/parquet-mr/pull/820


   



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-proto] default values are stored in oneOf fields that aren't set
> -
>
> Key: PARQUET-1917
> URL: https://issues.apache.org/jira/browse/PARQUET-1917
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-protobuf
>Affects Versions: 1.12.0
>Reporter: Aaron Blake Niskode-Dossett
>Priority: Major
>
> SCHEMA
> 
> {noformat}
> message Person {
>   int32 foo = 1;
>   oneof optional_bar {
>     int32 bar_int = 200;
>     int32 bar_int2 = 201;
>     string bar_string = 300;
>   }
> }{noformat}
>  
> CODE
> 
> I set values for foo and bar_string
>  
> {noformat}
> for (int i = 0; i < 3; i += 1) {
>                 com.etsy.grpcparquet.Person message = Person.newBuilder()
>                         .setFoo(i)
>                         .setBarString("hello world")
>                         .build();
>                 message.writeDelimitedTo(out);
>             }{noformat}
> And then I write the protobuf file out to parquet.
>  
> RESULT
> ---
> {noformat}
> $ parquet-tools show example.parquet                                          
>                                                                               
> +---+---++--+
> |   foo |   bar_int |   bar_int2 | bar_string   |
> |---+---++--|
> |     0 |         0 |          0 | hello world  |
> |     1 |         0 |          0 | hello world  |
> |     2 |         0 |          0 | hello world  |
> +---+---++--+{noformat}
>  
> bar_int and bar_int2 should be EMPTY for all three rows since only bar_string 
> is set in the oneof.  0 is the default value for int, but it should not be 
> stored.



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


[GitHub] [parquet-mr] gszadovszky merged pull request #820: PARQUET-1917: Don't write values for oneOf fields that aren't set

2020-10-22 Thread GitBox


gszadovszky merged pull request #820:
URL: https://github.com/apache/parquet-mr/pull/820


   



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




[GitHub] [parquet-mr] belugabehr commented on pull request #820: PARQUET-1917: Don't write values for oneOf fields that aren't set

2020-10-22 Thread GitBox


belugabehr commented on pull request #820:
URL: https://github.com/apache/parquet-mr/pull/820#issuecomment-714550284


   I looked at the unit test again, it's fine as-is.  +1
   
   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.

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




[jira] [Commented] (PARQUET-1917) [parquet-proto] default values are stored in oneOf fields that aren't set

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


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

ASF GitHub Bot commented on PARQUET-1917:
-

belugabehr commented on pull request #820:
URL: https://github.com/apache/parquet-mr/pull/820#issuecomment-714550284


   I looked at the unit test again, it's fine as-is.  +1
   
   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.

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


> [parquet-proto] default values are stored in oneOf fields that aren't set
> -
>
> Key: PARQUET-1917
> URL: https://issues.apache.org/jira/browse/PARQUET-1917
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-protobuf
>Affects Versions: 1.12.0
>Reporter: Aaron Blake Niskode-Dossett
>Priority: Major
>
> SCHEMA
> 
> {noformat}
> message Person {
>   int32 foo = 1;
>   oneof optional_bar {
>     int32 bar_int = 200;
>     int32 bar_int2 = 201;
>     string bar_string = 300;
>   }
> }{noformat}
>  
> CODE
> 
> I set values for foo and bar_string
>  
> {noformat}
> for (int i = 0; i < 3; i += 1) {
>                 com.etsy.grpcparquet.Person message = Person.newBuilder()
>                         .setFoo(i)
>                         .setBarString("hello world")
>                         .build();
>                 message.writeDelimitedTo(out);
>             }{noformat}
> And then I write the protobuf file out to parquet.
>  
> RESULT
> ---
> {noformat}
> $ parquet-tools show example.parquet                                          
>                                                                               
> +---+---++--+
> |   foo |   bar_int |   bar_int2 | bar_string   |
> |---+---++--|
> |     0 |         0 |          0 | hello world  |
> |     1 |         0 |          0 | hello world  |
> |     2 |         0 |          0 | hello world  |
> +---+---++--+{noformat}
>  
> bar_int and bar_int2 should be EMPTY for all three rows since only bar_string 
> is set in the oneof.  0 is the default value for int, but it should not be 
> stored.



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


[jira] [Commented] (PARQUET-1917) [parquet-proto] default values are stored in oneOf fields that aren't set

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


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

ASF GitHub Bot commented on PARQUET-1917:
-

gszadovszky commented on pull request #820:
URL: https://github.com/apache/parquet-mr/pull/820#issuecomment-714544732


   @dossett, it looks good to me. Let me wait for @belugabehr's approval then 
I'll approve and push 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.

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


> [parquet-proto] default values are stored in oneOf fields that aren't set
> -
>
> Key: PARQUET-1917
> URL: https://issues.apache.org/jira/browse/PARQUET-1917
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-protobuf
>Affects Versions: 1.12.0
>Reporter: Aaron Blake Niskode-Dossett
>Priority: Major
>
> SCHEMA
> 
> {noformat}
> message Person {
>   int32 foo = 1;
>   oneof optional_bar {
>     int32 bar_int = 200;
>     int32 bar_int2 = 201;
>     string bar_string = 300;
>   }
> }{noformat}
>  
> CODE
> 
> I set values for foo and bar_string
>  
> {noformat}
> for (int i = 0; i < 3; i += 1) {
>                 com.etsy.grpcparquet.Person message = Person.newBuilder()
>                         .setFoo(i)
>                         .setBarString("hello world")
>                         .build();
>                 message.writeDelimitedTo(out);
>             }{noformat}
> And then I write the protobuf file out to parquet.
>  
> RESULT
> ---
> {noformat}
> $ parquet-tools show example.parquet                                          
>                                                                               
> +---+---++--+
> |   foo |   bar_int |   bar_int2 | bar_string   |
> |---+---++--|
> |     0 |         0 |          0 | hello world  |
> |     1 |         0 |          0 | hello world  |
> |     2 |         0 |          0 | hello world  |
> +---+---++--+{noformat}
>  
> bar_int and bar_int2 should be EMPTY for all three rows since only bar_string 
> is set in the oneof.  0 is the default value for int, but it should not be 
> stored.



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


[GitHub] [parquet-mr] gszadovszky commented on pull request #820: PARQUET-1917: Don't write values for oneOf fields that aren't set

2020-10-22 Thread GitBox


gszadovszky commented on pull request #820:
URL: https://github.com/apache/parquet-mr/pull/820#issuecomment-714544732


   @dossett, it looks good to me. Let me wait for @belugabehr's approval then 
I'll approve and push 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.

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




[jira] [Commented] (PARQUET-1918) Avoid Copy of Bytes in Protobuf BinaryWriter

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


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

ASF GitHub Bot commented on PARQUET-1918:
-

gszadovszky commented on pull request #822:
URL: https://github.com/apache/parquet-mr/pull/822#issuecomment-714533411


   @belugabehr, what about blocking the jira with the thrift ticket so it is 
clear why we cannot step forward?
   Also, after fixing THRIFT-5288 we have to fix `Binary` as well and maybe 
other points in the code.



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


> Avoid Copy of Bytes in Protobuf BinaryWriter
> 
>
> Key: PARQUET-1918
> URL: https://issues.apache.org/jira/browse/PARQUET-1918
> Project: Parquet
>  Issue Type: Improvement
>Reporter: David Mollitor
>Assignee: David Mollitor
>Priority: Minor
>
> {code:java|title=ProtoWriteSupport.java}
>   class BinaryWriter extends FieldWriter {
> @Override
> final void writeRawValue(Object value) {
>   ByteString byteString = (ByteString) value;
>   Binary binary = Binary.fromConstantByteArray(byteString.toByteArray());
>   recordConsumer.addBinary(binary);
> }
>   }
> {code}
> {{toByteArray()}} creates a copy of the buffer.  There is already support 
> with Parquet and Protobuf to pass instead a ByteBuffer which avoids the copy.



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


[GitHub] [parquet-mr] gszadovszky commented on pull request #822: PARQUET-1918: Avoid Copy of Bytes in Protobuf BinaryWriter

2020-10-22 Thread GitBox


gszadovszky commented on pull request #822:
URL: https://github.com/apache/parquet-mr/pull/822#issuecomment-714533411


   @belugabehr, what about blocking the jira with the thrift ticket so it is 
clear why we cannot step forward?
   Also, after fixing THRIFT-5288 we have to fix `Binary` as well and maybe 
other points in the code.



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-1922) Deprecate IOExceptionUtils

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


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

ASF GitHub Bot commented on PARQUET-1922:
-

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



##
File path: 
parquet-column/src/main/java/org/apache/parquet/column/values/plain/PlainValuesWriter.java
##
@@ -127,7 +127,6 @@ public void reset() {
   @Override
   public void close() {
 arrayOut.close();
-out.close();

Review comment:
   I am fine fixing this in another ticket. What I think is a key in this 
change is to put the related comments (`out.close` is not required because...) 
in the code as well as in this review.





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


> Deprecate IOExceptionUtils
> --
>
> Key: PARQUET-1922
> URL: https://issues.apache.org/jira/browse/PARQUET-1922
> Project: Parquet
>  Issue Type: Improvement
>Reporter: David Mollitor
>Assignee: David Mollitor
>Priority: Major
>




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


[GitHub] [parquet-mr] gszadovszky commented on a change in pull request #825: PARQUET-1922: Deprecate IOExceptionUtils

2020-10-22 Thread GitBox


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



##
File path: 
parquet-column/src/main/java/org/apache/parquet/column/values/plain/PlainValuesWriter.java
##
@@ -127,7 +127,6 @@ public void reset() {
   @Override
   public void close() {
 arrayOut.close();
-out.close();

Review comment:
   I am fine fixing this in another ticket. What I think is a key in this 
change is to put the related comments (`out.close` is not required because...) 
in the code as well as in this review.





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




[GitHub] [parquet-mr] gszadovszky merged pull request #807: PARQUET-1893: H2SeekableInputStream readFully() doesn't respect start and len

2020-10-22 Thread GitBox


gszadovszky merged pull request #807:
URL: https://github.com/apache/parquet-mr/pull/807


   



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] [Resolved] (PARQUET-1893) H2SeekableInputStream readFully() doesn't respect start and len

2020-10-22 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky resolved PARQUET-1893.
---
Resolution: Fixed

> H2SeekableInputStream readFully() doesn't respect start and len 
> 
>
> Key: PARQUET-1893
> URL: https://issues.apache.org/jira/browse/PARQUET-1893
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Reporter: Xinli Shang
>Assignee: Xinli Shang
>Priority: Major
>
> The  readFully() throws away the parameters 'start' and 'len' as shown below. 
> public void readFully(byte[] bytes, int start, int len) throws IOException {
> stream.readFully(bytes);
>   }
> It should be corrected as below. 
> public void readFully(byte[] bytes, int start, int len) throws IOException {
> stream.readFully(bytes, start, len);
>   }
> H1SeekableInputStream() has been fixed. 



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


[jira] [Commented] (PARQUET-1893) H2SeekableInputStream readFully() doesn't respect start and len

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


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

ASF GitHub Bot commented on PARQUET-1893:
-

gszadovszky merged pull request #807:
URL: https://github.com/apache/parquet-mr/pull/807


   



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


> H2SeekableInputStream readFully() doesn't respect start and len 
> 
>
> Key: PARQUET-1893
> URL: https://issues.apache.org/jira/browse/PARQUET-1893
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Reporter: Xinli Shang
>Assignee: Xinli Shang
>Priority: Major
>
> The  readFully() throws away the parameters 'start' and 'len' as shown below. 
> public void readFully(byte[] bytes, int start, int len) throws IOException {
> stream.readFully(bytes);
>   }
> It should be corrected as below. 
> public void readFully(byte[] bytes, int start, int len) throws IOException {
> stream.readFully(bytes, start, len);
>   }
> H1SeekableInputStream() has been fixed. 



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


[jira] [Commented] (PARQUET-1917) [parquet-proto] default values are stored in oneOf fields that aren't set

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


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

ASF GitHub Bot commented on PARQUET-1917:
-

dossett commented on pull request #820:
URL: https://github.com/apache/parquet-mr/pull/820#issuecomment-714526665


   @gszadovszky Tagging you on this PR per discussion in the dev list.  If you 
approve the change I will also clean up the new tests a bit per David's 
comments.  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.

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


> [parquet-proto] default values are stored in oneOf fields that aren't set
> -
>
> Key: PARQUET-1917
> URL: https://issues.apache.org/jira/browse/PARQUET-1917
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-protobuf
>Affects Versions: 1.12.0
>Reporter: Aaron Blake Niskode-Dossett
>Priority: Major
>
> SCHEMA
> 
> {noformat}
> message Person {
>   int32 foo = 1;
>   oneof optional_bar {
>     int32 bar_int = 200;
>     int32 bar_int2 = 201;
>     string bar_string = 300;
>   }
> }{noformat}
>  
> CODE
> 
> I set values for foo and bar_string
>  
> {noformat}
> for (int i = 0; i < 3; i += 1) {
>                 com.etsy.grpcparquet.Person message = Person.newBuilder()
>                         .setFoo(i)
>                         .setBarString("hello world")
>                         .build();
>                 message.writeDelimitedTo(out);
>             }{noformat}
> And then I write the protobuf file out to parquet.
>  
> RESULT
> ---
> {noformat}
> $ parquet-tools show example.parquet                                          
>                                                                               
> +---+---++--+
> |   foo |   bar_int |   bar_int2 | bar_string   |
> |---+---++--|
> |     0 |         0 |          0 | hello world  |
> |     1 |         0 |          0 | hello world  |
> |     2 |         0 |          0 | hello world  |
> +---+---++--+{noformat}
>  
> bar_int and bar_int2 should be EMPTY for all three rows since only bar_string 
> is set in the oneof.  0 is the default value for int, but it should not be 
> stored.



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


[GitHub] [parquet-mr] dossett commented on pull request #820: PARQUET-1917 Don't write values for oneOf fields that aren't set

2020-10-22 Thread GitBox


dossett commented on pull request #820:
URL: https://github.com/apache/parquet-mr/pull/820#issuecomment-714526665


   @gszadovszky Tagging you on this PR per discussion in the dev list.  If you 
approve the change I will also clean up the new tests a bit per David's 
comments.  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.

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




[jira] [Commented] (PARQUET-1925) Introduce Velocity Template Engine to Parquet Generator

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


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

ASF GitHub Bot commented on PARQUET-1925:
-

belugabehr commented on pull request #828:
URL: https://github.com/apache/parquet-mr/pull/828#issuecomment-714525290


   > I like this very much. Would it be possible to have a test to check the 
output of the template?
   
   Great! I started working on some of the more complex templates, but I didn't 
want to invest too much time before getting some buy-in on this effort.
   
   I was manually generating and compiling the output for testing.  I'll see if 
I can write a unit test.



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


> Introduce Velocity Template Engine to Parquet Generator
> ---
>
> Key: PARQUET-1925
> URL: https://issues.apache.org/jira/browse/PARQUET-1925
> Project: Parquet
>  Issue Type: New Feature
>Reporter: David Mollitor
>Assignee: David Mollitor
>Priority: Major
>
> Much easier than the current setup of manually outputting the strings.



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


[jira] [Commented] (PARQUET-1903) Improve Parquet Protobuf Usability

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


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

ASF GitHub Bot commented on PARQUET-1903:
-

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



##
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetReader.java
##
@@ -136,6 +136,7 @@ public T read() throws IOException {
 return reader == null ? null : read();
   }
 } catch (InterruptedException e) {
+  Thread.currentThread().interrupt();

Review comment:
   I have no idea why some of the methods of `InternalParquetRecordReader` 
throw `InterruptedException`. I find no reason for that as there are no 
concurrency/thread handling is implemented there and not designed for 
extension. 
   What do you think about leaving as is for now and handle this in a separate 
issue? (Or if you have some answers please light me up :smiley:)

##
File path: 
parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoParquetReader.java
##
@@ -33,8 +33,14 @@
 public class ProtoParquetReader extends 
ParquetReader {
 
   @SuppressWarnings("unchecked")
-  public static  Builder builder(Path file) {
-return ParquetReader.builder(new ProtoReadSupport(), file);
+  public static  Builder builder(Path file) {

Review comment:
   I am not sure if it won't break backward compatibility.

##
File path: 
parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoParquetWriter.java
##
@@ -82,42 +84,70 @@ public ProtoParquetWriter(Path file, Class protoMessage) thro
 this(file, protoMessage, CompressionCodecName.UNCOMPRESSED,
 DEFAULT_BLOCK_SIZE, DEFAULT_PAGE_SIZE);
   }
-  
-  public static  Builder builder(Path file) {
-   return new Builder(file);
-   }
-
-   public static  Builder builder(OutputFile file) {
-   return new Builder(file);
-   }
-   
-   private static  WriteSupport 
writeSupport(Class protoMessage) {
-   return new ProtoWriteSupport(protoMessage);
-   }
- 
-   public static class Builder extends ParquetWriter.Builder> {
- 
-   Class protoMessage = null;
-
-   private Builder(Path file) {
-   super(file);
-   }
-
-   private Builder(OutputFile file) {
-   super(file);
-   }
-
-   protected Builder self() {
-   return this;
-   }
-   
-   public Builder withMessage(Class 
protoMessage){
-   this.protoMessage = protoMessage;
-   return this;
-   }
-
-   protected WriteSupport getWriteSupport(Configuration conf) {
-   return (WriteSupport) 
ProtoParquetWriter.writeSupport(protoMessage);
-   }
-   }
+
+  public static  Builder builder(Path file) {

Review comment:
   Might break backward compatibility.

##
File path: 
parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoReadSupport.java
##
@@ -74,24 +79,72 @@ public ReadContext init(InitContext context) {
 
   @Override
   public RecordMaterializer prepareForRead(Configuration configuration, 
Map keyValueMetaData, MessageType fileSchema, ReadContext 
readContext) {
-String headerProtoClass = keyValueMetaData.get(PB_CLASS);
-String configuredProtoClass = configuration.get(PB_CLASS);
-
-if (configuredProtoClass != null) {
-  LOG.debug("Replacing class " + headerProtoClass + " by " + 
configuredProtoClass);
-  headerProtoClass = configuredProtoClass;
-}
-
-if (headerProtoClass == null) {
-  throw new RuntimeException("I Need parameter " + PB_CLASS + " with 
Protocol Buffer class");
+final Optional headerProtoClass = 
Optional.ofNullable(keyValueMetaData.get(PB_CLASS));
+final Optional headerProtoType = 
Optional.ofNullable(keyValueMetaData.get(PB_TYPE));
+
+final Optional configuredProtoClass = 
Optional.ofNullable(configuration.get(PB_CLASS));
+final Optional configuredProtoType = 
Optional.ofNullable(configuration.get(PB_TYPE));
+
+final String candidateClass;
+
+/* 
+ * Load the class type, with backwards-compatibility for class-only 
support (no registry).
+ * Configured values override any values found in the meta section of the 
file.
+ */
+
+if (configuredProtoType.isPresent()) {
+  // No schema registry implemented yet
+  LOG.debug("Configured proto type: {}", configuredProtoType.get());
+  candidateClass = parseProtoType(configuredProtoType.get()).getValue();
+} else if (configuredProtoClass.isPresent()) {
+  LOG.debug("Configured proto class: {}", configuredProtoClass.get());
+  candidateClass = 

[GitHub] [parquet-mr] gszadovszky commented on a change in pull request #813: PARQUET-1903: Improve Parquet Protobuf Usability

2020-10-22 Thread GitBox


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



##
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetReader.java
##
@@ -136,6 +136,7 @@ public T read() throws IOException {
 return reader == null ? null : read();
   }
 } catch (InterruptedException e) {
+  Thread.currentThread().interrupt();

Review comment:
   I have no idea why some of the methods of `InternalParquetRecordReader` 
throw `InterruptedException`. I find no reason for that as there are no 
concurrency/thread handling is implemented there and not designed for 
extension. 
   What do you think about leaving as is for now and handle this in a separate 
issue? (Or if you have some answers please light me up :smiley:)

##
File path: 
parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoParquetReader.java
##
@@ -33,8 +33,14 @@
 public class ProtoParquetReader extends 
ParquetReader {
 
   @SuppressWarnings("unchecked")
-  public static  Builder builder(Path file) {
-return ParquetReader.builder(new ProtoReadSupport(), file);
+  public static  Builder builder(Path file) {

Review comment:
   I am not sure if it won't break backward compatibility.

##
File path: 
parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoParquetWriter.java
##
@@ -82,42 +84,70 @@ public ProtoParquetWriter(Path file, Class protoMessage) thro
 this(file, protoMessage, CompressionCodecName.UNCOMPRESSED,
 DEFAULT_BLOCK_SIZE, DEFAULT_PAGE_SIZE);
   }
-  
-  public static  Builder builder(Path file) {
-   return new Builder(file);
-   }
-
-   public static  Builder builder(OutputFile file) {
-   return new Builder(file);
-   }
-   
-   private static  WriteSupport 
writeSupport(Class protoMessage) {
-   return new ProtoWriteSupport(protoMessage);
-   }
- 
-   public static class Builder extends ParquetWriter.Builder> {
- 
-   Class protoMessage = null;
-
-   private Builder(Path file) {
-   super(file);
-   }
-
-   private Builder(OutputFile file) {
-   super(file);
-   }
-
-   protected Builder self() {
-   return this;
-   }
-   
-   public Builder withMessage(Class 
protoMessage){
-   this.protoMessage = protoMessage;
-   return this;
-   }
-
-   protected WriteSupport getWriteSupport(Configuration conf) {
-   return (WriteSupport) 
ProtoParquetWriter.writeSupport(protoMessage);
-   }
-   }
+
+  public static  Builder builder(Path file) {

Review comment:
   Might break backward compatibility.

##
File path: 
parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoReadSupport.java
##
@@ -74,24 +79,72 @@ public ReadContext init(InitContext context) {
 
   @Override
   public RecordMaterializer prepareForRead(Configuration configuration, 
Map keyValueMetaData, MessageType fileSchema, ReadContext 
readContext) {
-String headerProtoClass = keyValueMetaData.get(PB_CLASS);
-String configuredProtoClass = configuration.get(PB_CLASS);
-
-if (configuredProtoClass != null) {
-  LOG.debug("Replacing class " + headerProtoClass + " by " + 
configuredProtoClass);
-  headerProtoClass = configuredProtoClass;
-}
-
-if (headerProtoClass == null) {
-  throw new RuntimeException("I Need parameter " + PB_CLASS + " with 
Protocol Buffer class");
+final Optional headerProtoClass = 
Optional.ofNullable(keyValueMetaData.get(PB_CLASS));
+final Optional headerProtoType = 
Optional.ofNullable(keyValueMetaData.get(PB_TYPE));
+
+final Optional configuredProtoClass = 
Optional.ofNullable(configuration.get(PB_CLASS));
+final Optional configuredProtoType = 
Optional.ofNullable(configuration.get(PB_TYPE));
+
+final String candidateClass;
+
+/* 
+ * Load the class type, with backwards-compatibility for class-only 
support (no registry).
+ * Configured values override any values found in the meta section of the 
file.
+ */
+
+if (configuredProtoType.isPresent()) {
+  // No schema registry implemented yet
+  LOG.debug("Configured proto type: {}", configuredProtoType.get());
+  candidateClass = parseProtoType(configuredProtoType.get()).getValue();
+} else if (configuredProtoClass.isPresent()) {
+  LOG.debug("Configured proto class: {}", configuredProtoClass.get());
+  candidateClass = configuredProtoClass.get();
+} else if (headerProtoType.isPresent()) {
+  // No schema registry implemented yet
+  LOG.debug("Parquet meta proto type: {}", headerProtoType.get());
+  candidateClass = 

[GitHub] [parquet-mr] belugabehr commented on pull request #828: PARQUET-1925: Introduce Velocity Template Engine to Parquet Generator

2020-10-22 Thread GitBox


belugabehr commented on pull request #828:
URL: https://github.com/apache/parquet-mr/pull/828#issuecomment-714525290


   > I like this very much. Would it be possible to have a test to check the 
output of the template?
   
   Great! I started working on some of the more complex templates, but I didn't 
want to invest too much time before getting some buy-in on this effort.
   
   I was manually generating and compiling the output for testing.  I'll see if 
I can write a unit test.



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-1918) Avoid Copy of Bytes in Protobuf BinaryWriter

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


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

ASF GitHub Bot commented on PARQUET-1918:
-

belugabehr edited a comment on pull request #822:
URL: https://github.com/apache/parquet-mr/pull/822#issuecomment-714521089


   @gszadovszky Ya. I did discover that this is a bit more tricky than I had 
anticipated.  My expectation was that ByteBuffers were handled the same way as 
there are defined in the JDK specs (most methods have the side-affect of moving 
Buffer positions around). This particular work is blocked as I try to improve 
ByteBuffer handling in Thrift first: THRIFT-5288



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


> Avoid Copy of Bytes in Protobuf BinaryWriter
> 
>
> Key: PARQUET-1918
> URL: https://issues.apache.org/jira/browse/PARQUET-1918
> Project: Parquet
>  Issue Type: Improvement
>Reporter: David Mollitor
>Assignee: David Mollitor
>Priority: Minor
>
> {code:java|title=ProtoWriteSupport.java}
>   class BinaryWriter extends FieldWriter {
> @Override
> final void writeRawValue(Object value) {
>   ByteString byteString = (ByteString) value;
>   Binary binary = Binary.fromConstantByteArray(byteString.toByteArray());
>   recordConsumer.addBinary(binary);
> }
>   }
> {code}
> {{toByteArray()}} creates a copy of the buffer.  There is already support 
> with Parquet and Protobuf to pass instead a ByteBuffer which avoids the copy.



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


[GitHub] [parquet-mr] belugabehr edited a comment on pull request #822: PARQUET-1918: Avoid Copy of Bytes in Protobuf BinaryWriter

2020-10-22 Thread GitBox


belugabehr edited a comment on pull request #822:
URL: https://github.com/apache/parquet-mr/pull/822#issuecomment-714521089


   @gszadovszky Ya. I did discover that this is a bit more tricky than I had 
anticipated.  My expectation was that ByteBuffers were handled the same way as 
there are defined in the JDK specs (most methods have the side-affect of moving 
Buffer positions around). This particular work is blocked as I try to improve 
ByteBuffer handling in Thrift first: THRIFT-5288



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-1918) Avoid Copy of Bytes in Protobuf BinaryWriter

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


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

ASF GitHub Bot commented on PARQUET-1918:
-

belugabehr commented on pull request #822:
URL: https://github.com/apache/parquet-mr/pull/822#issuecomment-714521089


   @gszadovszky Ya. I did discover that this is a bit more tricky than I had 
anticipated.  My expectation was this ByteBuffers were handled the same way as 
there are defined in the JDK specs (most methods have the side-affect of moving 
Buffer positions around). This particular work is blocked as I try to improve 
ByteBuffer handling in Thrift first: THRIFT-5288



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


> Avoid Copy of Bytes in Protobuf BinaryWriter
> 
>
> Key: PARQUET-1918
> URL: https://issues.apache.org/jira/browse/PARQUET-1918
> Project: Parquet
>  Issue Type: Improvement
>Reporter: David Mollitor
>Assignee: David Mollitor
>Priority: Minor
>
> {code:java|title=ProtoWriteSupport.java}
>   class BinaryWriter extends FieldWriter {
> @Override
> final void writeRawValue(Object value) {
>   ByteString byteString = (ByteString) value;
>   Binary binary = Binary.fromConstantByteArray(byteString.toByteArray());
>   recordConsumer.addBinary(binary);
> }
>   }
> {code}
> {{toByteArray()}} creates a copy of the buffer.  There is already support 
> with Parquet and Protobuf to pass instead a ByteBuffer which avoids the copy.



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


[GitHub] [parquet-mr] belugabehr commented on pull request #822: PARQUET-1918: Avoid Copy of Bytes in Protobuf BinaryWriter

2020-10-22 Thread GitBox


belugabehr commented on pull request #822:
URL: https://github.com/apache/parquet-mr/pull/822#issuecomment-714521089


   @gszadovszky Ya. I did discover that this is a bit more tricky than I had 
anticipated.  My expectation was this ByteBuffers were handled the same way as 
there are defined in the JDK specs (most methods have the side-affect of moving 
Buffer positions around). This particular work is blocked as I try to improve 
ByteBuffer handling in Thrift first: THRIFT-5288



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-1922) Deprecate IOExceptionUtils

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


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

ASF GitHub Bot commented on PARQUET-1922:
-

belugabehr commented on a change in pull request #825:
URL: https://github.com/apache/parquet-mr/pull/825#discussion_r510189309



##
File path: 
parquet-column/src/main/java/org/apache/parquet/column/values/plain/PlainValuesWriter.java
##
@@ -127,7 +127,6 @@ public void reset() {
   @Override
   public void close() {
 arrayOut.close();
-out.close();

Review comment:
   @gszadovszky Thanks for the review!
   
   I think this line of work should be pursued in a different ticket.  This 
change makes this PR's changes possible, but also puts it in line with other 
implementations:
   
   
https://github.com/apache/parquet-mr/blob/dc61e510126aaa1a95a46fe39bf1529f394147e9/parquet-column/src/main/java/org/apache/parquet/column/values/plain/FixedLenByteArrayPlainValuesWriter.java#L88-L90
   
   So if we want to change how things are closed (nice, but technically not 
required) it can be addressed holistically. 
   
   
   





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


> Deprecate IOExceptionUtils
> --
>
> Key: PARQUET-1922
> URL: https://issues.apache.org/jira/browse/PARQUET-1922
> Project: Parquet
>  Issue Type: Improvement
>Reporter: David Mollitor
>Assignee: David Mollitor
>Priority: Major
>




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


[GitHub] [parquet-mr] belugabehr commented on a change in pull request #825: PARQUET-1922: Deprecate IOExceptionUtils

2020-10-22 Thread GitBox


belugabehr commented on a change in pull request #825:
URL: https://github.com/apache/parquet-mr/pull/825#discussion_r510189309



##
File path: 
parquet-column/src/main/java/org/apache/parquet/column/values/plain/PlainValuesWriter.java
##
@@ -127,7 +127,6 @@ public void reset() {
   @Override
   public void close() {
 arrayOut.close();
-out.close();

Review comment:
   @gszadovszky Thanks for the review!
   
   I think this line of work should be pursued in a different ticket.  This 
change makes this PR's changes possible, but also puts it in line with other 
implementations:
   
   
https://github.com/apache/parquet-mr/blob/dc61e510126aaa1a95a46fe39bf1529f394147e9/parquet-column/src/main/java/org/apache/parquet/column/values/plain/FixedLenByteArrayPlainValuesWriter.java#L88-L90
   
   So if we want to change how things are closed (nice, but technically not 
required) it can be addressed holistically. 
   
   
   





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-1914) Allow ProtoParquetReader To Support InputFile

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


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

ASF GitHub Bot commented on PARQUET-1914:
-

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



##
File path: 
parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoParquetReader.java
##
@@ -56,7 +61,25 @@ public ProtoParquetReader(Path file) throws IOException {
*/
   @Deprecated
   @SuppressWarnings("unchecked")
-  public ProtoParquetReader(Path file, UnboundRecordFilter recordFilter) 
throws IOException {
+  public ProtoParquetReader(Path file, UnboundRecordFilter recordFilter)
+  throws IOException {

Review comment:
   Untouched code should not be formatted. (BTW, the max line width is 120 
characters anyway.)





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


> Allow ProtoParquetReader To Support InputFile
> -
>
> Key: PARQUET-1914
> URL: https://issues.apache.org/jira/browse/PARQUET-1914
> Project: Parquet
>  Issue Type: Improvement
>Reporter: David Mollitor
>Assignee: David Mollitor
>Priority: Major
>




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


[GitHub] [parquet-mr] gszadovszky commented on a change in pull request #817: PARQUET-1914: Allow ProtoParquetReader To Support InputFile

2020-10-22 Thread GitBox


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



##
File path: 
parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoParquetReader.java
##
@@ -56,7 +61,25 @@ public ProtoParquetReader(Path file) throws IOException {
*/
   @Deprecated
   @SuppressWarnings("unchecked")
-  public ProtoParquetReader(Path file, UnboundRecordFilter recordFilter) 
throws IOException {
+  public ProtoParquetReader(Path file, UnboundRecordFilter recordFilter)
+  throws IOException {

Review comment:
   Untouched code should not be formatted. (BTW, the max line width is 120 
characters anyway.)





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-1922) Deprecate IOExceptionUtils

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


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

ASF GitHub Bot commented on PARQUET-1922:
-

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



##
File path: 
parquet-column/src/main/java/org/apache/parquet/column/values/plain/PlainValuesWriter.java
##
@@ -127,7 +127,6 @@ public void reset() {
   @Override
   public void close() {
 arrayOut.close();
-out.close();

Review comment:
   I would either close only `arrayOut` but with comments about what you 
have written here or close only `out` and catching the `IOException` that would 
never occur and comment in the catch block similarly. 





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


> Deprecate IOExceptionUtils
> --
>
> Key: PARQUET-1922
> URL: https://issues.apache.org/jira/browse/PARQUET-1922
> Project: Parquet
>  Issue Type: Improvement
>Reporter: David Mollitor
>Assignee: David Mollitor
>Priority: Major
>




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


[GitHub] [parquet-mr] gszadovszky commented on a change in pull request #825: PARQUET-1922: Deprecate IOExceptionUtils

2020-10-22 Thread GitBox


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



##
File path: 
parquet-column/src/main/java/org/apache/parquet/column/values/plain/PlainValuesWriter.java
##
@@ -127,7 +127,6 @@ public void reset() {
   @Override
   public void close() {
 arrayOut.close();
-out.close();

Review comment:
   I would either close only `arrayOut` but with comments about what you 
have written here or close only `out` and catching the `IOException` that would 
never occur and comment in the catch block similarly. 





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-22 Thread ASF GitHub Bot (Jira)


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

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_r510113275



##
File path: 
parquet-cli/src/main/java/org/apache/parquet/cli/commands/ColumnMaskingCommand.java
##
@@ -0,0 +1,104 @@
+/*
+ * 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.cli.commands;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.cli.BaseCommand;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.ParquetFileWriter;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.util.ColumnMasker;
+import org.apache.parquet.hadoop.util.ColumnMasker.MaskMode;
+import 
org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import 
org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileWriter;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.schema.MessageType;
+import org.slf4j.Logger;
+
+import java.io.IOException;
+import java.util.List;
+
+import static 
org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER;
+
+@Parameters(commandDescription="Replace columns with masked values and write 
to a new Parquet file")
+public class ColumnMaskingCommand extends BaseCommand {
+
+  private ColumnMasker masker;
+
+  public ColumnMaskingCommand(Logger console) {
+super(console);
+masker = new ColumnMasker();
+  }
+
+  @Parameter(description = "")
+  String mode;
+
+  @Parameter(description = "")
+  String input;
+
+  @Parameter(description = "")
+  String output;
+
+  @Parameter(description = "columns to be repalced with masked value")

Review comment:
   ```suggestion
 @Parameter(description = "columns to be replaced with masked value")
   ```

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

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

2020-10-22 Thread GitBox


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



##
File path: 
parquet-cli/src/main/java/org/apache/parquet/cli/commands/ColumnMaskingCommand.java
##
@@ -0,0 +1,104 @@
+/*
+ * 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.cli.commands;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.cli.BaseCommand;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.ParquetFileWriter;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.util.ColumnMasker;
+import org.apache.parquet.hadoop.util.ColumnMasker.MaskMode;
+import 
org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader;
+import 
org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileWriter;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.schema.MessageType;
+import org.slf4j.Logger;
+
+import java.io.IOException;
+import java.util.List;
+
+import static 
org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER;
+
+@Parameters(commandDescription="Replace columns with masked values and write 
to a new Parquet file")
+public class ColumnMaskingCommand extends BaseCommand {
+
+  private ColumnMasker masker;
+
+  public ColumnMaskingCommand(Logger console) {
+super(console);
+masker = new ColumnMasker();
+  }
+
+  @Parameter(description = "")
+  String mode;
+
+  @Parameter(description = "")
+  String input;
+
+  @Parameter(description = "")
+  String output;
+
+  @Parameter(description = "columns to be repalced with masked value")

Review comment:
   ```suggestion
 @Parameter(description = "columns to be replaced with masked value")
   ```

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

[jira] [Resolved] (PARQUET-1528) Add JSON support to `parquet-tools head`

2020-10-22 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky resolved PARQUET-1528.
---
Resolution: Fixed

> Add JSON support to `parquet-tools head`
> 
>
> Key: PARQUET-1528
> URL: https://issues.apache.org/jira/browse/PARQUET-1528
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Reporter: David Nies
>Assignee: Raphaël Afanyan
>Priority: Major
>
> {{parquet-tools cat}} supports JSON output via the {{-j}} flag. It would be 
> really handy if {{parquet-tools head}} would support this too.



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


[jira] [Assigned] (PARQUET-1528) Add JSON support to `parquet-tools head`

2020-10-22 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky reassigned PARQUET-1528:
-

Assignee: Raphaël Afanyan

> Add JSON support to `parquet-tools head`
> 
>
> Key: PARQUET-1528
> URL: https://issues.apache.org/jira/browse/PARQUET-1528
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Reporter: David Nies
>Assignee: Raphaël Afanyan
>Priority: Major
>
> {{parquet-tools cat}} supports JSON output via the {{-j}} flag. It would be 
> really handy if {{parquet-tools head}} would support this too.



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


[jira] [Commented] (PARQUET-1528) Add JSON support to `parquet-tools head`

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


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

ASF GitHub Bot commented on PARQUET-1528:
-

raph-af commented on pull request #829:
URL: https://github.com/apache/parquet-mr/pull/829#issuecomment-714358867


   @gszadovszky ty for having a look ! I have an account, my username is 
"Raphael Af"



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


> Add JSON support to `parquet-tools head`
> 
>
> Key: PARQUET-1528
> URL: https://issues.apache.org/jira/browse/PARQUET-1528
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Reporter: David Nies
>Priority: Major
>
> {{parquet-tools cat}} supports JSON output via the {{-j}} flag. It would be 
> really handy if {{parquet-tools head}} would support this too.



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


[GitHub] [parquet-mr] raph-af commented on pull request #829: PARQUET-1528: Add JSON support to `parquet-tools head`

2020-10-22 Thread GitBox


raph-af commented on pull request #829:
URL: https://github.com/apache/parquet-mr/pull/829#issuecomment-714358867


   @gszadovszky ty for having a look ! I have an account, my username is 
"Raphael Af"



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




Re: Current status of Data Page V2?

2020-10-22 Thread Gabor Szadovszky
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 status in reference implementations
> >
> > Thanks,
> > Micah
> >
> >
> > On Tue, Oct 13, 2020 at 1:51 AM Gabor Szadovszky 
> 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.
> >> In this table many features are not related to the pages so I don't
> think
> >> the "Expected release" meant the v1/v2 pages. I guess there was an
> earlier
> >> plan to release parquet-mr 2.0 with the v2 pages but then v2 pages were
> >> released in a 1.x release while 2.0 is not planned yet. (I was not in
> the
> >> community that time so I'm only guessing.)
> >>
> >> Also worth to mention that it seems to be not related to the
> >> parquet-format
> >> releases which means that based on the spec the implementations were/are
> >> not limited by this table.
> >>
> >>
> >> On Mon, Oct 12, 2020 at 6:53 PM Ryan Blue 
> >> wrote:
> >>
> >> > I remembered that there used to be a table. Looks like it was removed:
> >> >
> >> >
> >>
> https://github.com/apache/parquet-mr/commit/aed9097640c7adffe1151b32e86b5efc3702c657?short_path=b335630#diff-04c6e90faac2675aa89e2176d2eec7d8
> >> >
> >> > The table used to list delta as a 2.0 feature.
> >> >
> >> > On Mon, Oct 12, 2020 at 1:38 AM Gabor Szadovszky 
> >> wrote:
> >> >
> >> > > That answer I wrote to the other thread was based on the current
> >> code. So
> >> > > that is how parquet-mr is working now. It does not mean though how
> >> shall
> >> > it
> >> > > work or how it works in other implementations. Unfortunately, the
> spec
> >> > does
> >> > > not say anything about v1 and v2 in the context of encodings.
> >> > > Meanwhile, enabling the "new" encodings in v1 may generate
> >> compatibility
> >> > > issues with other implementations. (I am not sure how would the
> >> existing
> >> > > releases of parquet-mr behave if they have to read v1 pages with
> these
> >> > > encodings but I believe it would work fine.)
> >> > >
> >> > > I think, it would be a good idea to keep the existing default
> >> behavior as
> >> > > is but introduce some new flags where the user may set/suggest
> >> encodings
> >> > > for the different columns. This way the user can hold the risk of
> >> being
> >> > > potentially incompatible with other implementations (for the time
> >> being)
> >> > > and also can fine tune the encodings for the data. This way we can
> >> also
> >> > > introduce some new encodings that are better in some cases (e.g.
> lossy
> >> > > compression for floating point numbers).
> >> > >
> >> > > What do you guys thing?
> >> > > (I would be happy to help anyone would like to contribute in this
> >> topic.)
> >> > >
> >> > > Cheers,
> >> > > Gabor
> >> > >
> >> > > On Sat, Oct 10, 2020 at 2:18 AM Jacques Nadeau 
> >> > wrote:
> >> > >
> >> > > > Gabor seems to agree that delta is V2 only.
> >> > > >
> >> > > > To summarize, no delta encodings are used for V1 pages. They are
> >> > > available
> >> > > > > for V2 only.
> >> > > >
> >> > > >
> >> > > > https://www.mail-archive.com/dev@parquet.apache.org/msg11826.html
> >> > > >
> >> > > >
> >> > > >
> >> > > > On Fri, Oct 9, 2020 at 5:06 PM Jacques Nadeau  >
> >> > > wrote:
> >> > > >
> >> > > > > Good point. I had mentally categorized this as V2, not based on
> >> the
> >> > > docs?
> >> > > > >
> >> > > > > I don't think most tools write this but I can't see anywhere
> that
> >> it
> >> > > says
> >> > > > > it is limited to v2 readers/writers. I'm not sure how many tools
> >> > > > vectorize
> >> > > > > read it versus 

[jira] [Commented] (PARQUET-1528) Add JSON support to `parquet-tools head`

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


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

ASF GitHub Bot commented on PARQUET-1528:
-

gszadovszky merged pull request #829:
URL: https://github.com/apache/parquet-mr/pull/829


   



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


> Add JSON support to `parquet-tools head`
> 
>
> Key: PARQUET-1528
> URL: https://issues.apache.org/jira/browse/PARQUET-1528
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Reporter: David Nies
>Priority: Major
>
> {{parquet-tools cat}} supports JSON output via the {{-j}} flag. It would be 
> really handy if {{parquet-tools head}} would support this too.



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


[jira] [Commented] (PARQUET-1528) Add JSON support to `parquet-tools head`

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


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

ASF GitHub Bot commented on PARQUET-1528:
-

gszadovszky commented on pull request #829:
URL: https://github.com/apache/parquet-mr/pull/829#issuecomment-714322870


   @raph-af, do you have a jira account so I can assign this one to you?



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


> Add JSON support to `parquet-tools head`
> 
>
> Key: PARQUET-1528
> URL: https://issues.apache.org/jira/browse/PARQUET-1528
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Reporter: David Nies
>Priority: Major
>
> {{parquet-tools cat}} supports JSON output via the {{-j}} flag. It would be 
> really handy if {{parquet-tools head}} would support this too.



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


[GitHub] [parquet-mr] gszadovszky merged pull request #829: PARQUET-1528: Add JSON support to `parquet-tools head`

2020-10-22 Thread GitBox


gszadovszky merged pull request #829:
URL: https://github.com/apache/parquet-mr/pull/829


   



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




[GitHub] [parquet-mr] gszadovszky commented on pull request #829: PARQUET-1528: Add JSON support to `parquet-tools head`

2020-10-22 Thread GitBox


gszadovszky commented on pull request #829:
URL: https://github.com/apache/parquet-mr/pull/829#issuecomment-714322870


   @raph-af, do you have a jira account so I can assign this one to you?



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-1930) Bump Apache Thrift to 0.13

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


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

ASF GitHub Bot commented on PARQUET-1930:
-

gszadovszky commented on a change in pull request #162:
URL: https://github.com/apache/parquet-format/pull/162#discussion_r509964168



##
File path: parquet-format.iml
##
@@ -0,0 +1,24 @@
+

Review comment:
   Please, do not check in IDE files.





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


> Bump Apache Thrift to 0.13
> --
>
> Key: PARQUET-1930
> URL: https://issues.apache.org/jira/browse/PARQUET-1930
> Project: Parquet
>  Issue Type: Improvement
>Reporter: Fokko Driesprong
>Assignee: Fokko Driesprong
>Priority: Major
>




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


[GitHub] [parquet-format] gszadovszky commented on a change in pull request #162: [PARQUET-1930] Bump Apache Thrift to 0.13

2020-10-22 Thread GitBox


gszadovszky commented on a change in pull request #162:
URL: https://github.com/apache/parquet-format/pull/162#discussion_r509964168



##
File path: parquet-format.iml
##
@@ -0,0 +1,24 @@
+

Review comment:
   Please, do not check in IDE files.





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




Re: Create a parquet-protobuf JIRA component

2020-10-22 Thread Gabor Szadovszky
I'm afraid that is the main issue that we have no active committers who
have enough experience in protobuf. Meanwhile it is great that there are
more contributors who work on it so they can review each other's work.
So I would suggest cross reviewing your PRs first and if you got an
experienced review ping me and I'll push it. (I'll also review the PRs
before pushing but it is a great help if someone else who has more
experience in the area has had their comments before.)

Cheers,
Gabor

On Wed, Oct 21, 2020 at 8:34 PM Aaron Niskode-Dossett
 wrote:

> Gabor -- is there an active parquet committer who works in the protobuf
> module? There are several open PRs (mostly from David, one from me, perhaps
> others) that would constitute nice improvements to that module.
>
> Thanks, Aaron
>
> On Wed, Oct 21, 2020 at 7:39 AM Aaron Niskode-Dossett <
> aniskodedoss...@etsy.com> wrote:
>
> > Wonderful, thank you!  My company hopes to use proto+parquet a lot and I
> > look forward to contributing!
> >
> > On Wed, Oct 21, 2020 at 2:54 AM Gabor Szadovszky 
> wrote:
> >
> >> Sorry, I've missed this thread. Just created the component. Feel free to
> >> use it.
> >>
> >> On Tue, Oct 20, 2020 at 4:27 PM Aaron Niskode-Dossett
> >>  wrote:
> >>
> >> > Hi, just bumping this request for a parquet-protobuf JIRA component
> >> again.
> >> >
> >> > On Fri, Oct 2, 2020 at 9:03 AM David  wrote:
> >> >
> >> > > Hello Gang,
> >> > >
> >> > > I too would like to see this too.
> >> > >
> >> > > Aaron and I have put up a few PRs re: protobuf integration
> >> > >
> >> > > Is anyone able for review and potential push?
> >> > >
> >> > > Thanks.
> >> > >
> >> > > On Tue, Sep 29, 2020 at 12:01 PM Aaron Niskode-Dossett
> >> > >  wrote:
> >> > >
> >> > > > Hello Parquet project members,
> >> > > >
> >> > > > Could a parquet-protobuf component be added to the project JIRA?
> >> > There a
> >> > > > few open JIRA tickets that would be nice to categorize.  If the
> >> > component
> >> > > > is created, i would be happy to categorize the tickets.
> >> > > >
> >> > > > Thank you, Aaron
> >> > > >
> >> > > > --
> >> > > > Aaron Niskode-Dossett, Data Engineering -- Etsy
> >> > > >
> >> > >
> >> >
> >> >
> >> > --
> >> > Aaron Niskode-Dossett, Data Engineering -- Etsy
> >> >
> >>
> >
> >
> > --
> > Aaron Niskode-Dossett, Data Engineering -- Etsy
> >
>
>
> --
> Aaron Niskode-Dossett, Data Engineering -- Etsy
>


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

2020-10-22 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-1927:
---

Rechecked the code again and found that 
[PageReadStore.getRowCount()|https://github.com/apache/parquet-mr/blob/master/parquet-column/src/main/java/org/apache/parquet/column/page/PageReadStore.java#L44]
 should return the required value meaning the number of rows after column-index 
filtering. See the 
[implementation|https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ColumnChunkPageReadStore.java#L260]
 for details. So you do not need to calculate anything and extend the current 
API just use this value as total row count in the row group.

> 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
>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] [Created] (PARQUET-1934) Dictionary page is not decrypted in predicate pushdown path

2020-10-22 Thread Gidon Gershinsky (Jira)
Gidon Gershinsky created PARQUET-1934:
-

 Summary: Dictionary page is not decrypted in predicate pushdown 
path
 Key: PARQUET-1934
 URL: https://issues.apache.org/jira/browse/PARQUET-1934
 Project: Parquet
  Issue Type: Bug
  Components: parquet-mr
Affects Versions: 1.12.0
Reporter: Gidon Gershinsky
Assignee: Gidon Gershinsky


Predicate pushdown, based on dictionary pages, uses a page parsing code that 
doesn't support decryption yet. Will add a few lines to decrypt the dictionary 
page header and page (for encrypted columns).



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