Re: Current status of Data Page V2?

2020-10-21 Thread Micah Kornfield
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 delegate to the legacy mr path (at least in java),
>> > > either.
>> > > > >
>> > > > > On Fri, Oct 9, 2020 at 2:25 PM Micah Kornfield <
>> > emkornfi...@gmail.com>
>> > > > > wrote:
>> > > > >
>> > > > >> The big win in v2 pages (if I remember correctly) is that the
>> > variable
>> > > > >>> length encoding is no longer interleaved. That would provide a
>> big
>> > > > >>> performance lift when pulling into arrow vectors (and variable
>> > length
>> > > > >>> decoding typically dominates total read processing time, 

[jira] [Commented] (PARQUET-1933) [Format] Clarify encodings and data page guidance.

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


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

ASF GitHub Bot commented on PARQUET-1933:
-

emkornfield opened a new pull request #163:
URL: https://github.com/apache/parquet-format/pull/163


   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [X ] My PR addresses the following [Parquet 
Jira](https://issues.apache.org/jira/browse/PARQUET-1933) issues and references 
them in the PR title. For example, "PARQUET-1234: My Parquet PR"
 - https://issues.apache.org/jira/browse/PARQUET-XXX
 - In case you are adding a dependency, check if the license complies with 
the [ASF 3rd Party License 
Policy](https://www.apache.org/legal/resolved.html#category-x).
   
   ### Commits
   
   - [X] My commits all reference Jira issues in their subject lines. In 
addition, my commits follow the guidelines from "[How to write a good git 
commit message](http://chris.beams.io/posts/git-commit/)":
 1. Subject is separated from body by a blank line
 1. Subject is limited to 50 characters (not including Jira issue reference)
 1. Subject does not end with a period
 1. Subject uses the imperative mood ("add", not "adding")
 1. Body wraps at 72 characters
 1. Body explains "what" and "why", not "how"
   
   



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


> [Format] Clarify encodings and data page guidance.
> --
>
> Key: PARQUET-1933
> URL: https://issues.apache.org/jira/browse/PARQUET-1933
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-format
>Reporter: Micah Kornfield
>Assignee: Micah Kornfield
>Priority: Major
>




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


[GitHub] [parquet-format] emkornfield opened a new pull request #163: PARQUET-1933: Clarify encodings relative to data page usage.

2020-10-21 Thread GitBox


emkornfield opened a new pull request #163:
URL: https://github.com/apache/parquet-format/pull/163


   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [X ] My PR addresses the following [Parquet 
Jira](https://issues.apache.org/jira/browse/PARQUET-1933) issues and references 
them in the PR title. For example, "PARQUET-1234: My Parquet PR"
 - https://issues.apache.org/jira/browse/PARQUET-XXX
 - In case you are adding a dependency, check if the license complies with 
the [ASF 3rd Party License 
Policy](https://www.apache.org/legal/resolved.html#category-x).
   
   ### Commits
   
   - [X] My commits all reference Jira issues in their subject lines. In 
addition, my commits follow the guidelines from "[How to write a good git 
commit message](http://chris.beams.io/posts/git-commit/)":
 1. Subject is separated from body by a blank line
 1. Subject is limited to 50 characters (not including Jira issue reference)
 1. Subject does not end with a period
 1. Subject uses the imperative mood ("add", not "adding")
 1. Body wraps at 72 characters
 1. Body explains "what" and "why", not "how"
   
   



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] [Created] (PARQUET-1933) [Format] Clarify encodings and data page guidance.

2020-10-21 Thread Micah Kornfield (Jira)
Micah Kornfield created PARQUET-1933:


 Summary: [Format] Clarify encodings and data page guidance.
 Key: PARQUET-1933
 URL: https://issues.apache.org/jira/browse/PARQUET-1933
 Project: Parquet
  Issue Type: Improvement
  Components: parquet-format
Reporter: Micah Kornfield
Assignee: Micah Kornfield






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


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

2020-10-21 Thread GitBox


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


   I did a push prematurely. That is not ready for review yet. I will rework on 
it soon and let you know when it is ready for 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] shangxinli commented on a change in pull request #808: Parquet-1396: Cryptodata Interface for Schema Activation of Parquet E…

2020-10-21 Thread GitBox


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



##
File path: 
parquet-hadoop/src/test/java/org/apache/parquet/crypto/propertiesfactory/SchemaCryptoPropertiesFactory.java
##
@@ -0,0 +1,135 @@
+/*
+ * 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.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.crypto.ColumnEncryptionProperties;
+import org.apache.parquet.crypto.DecryptionKeyRetrieverMock;
+import org.apache.parquet.crypto.DecryptionPropertiesFactory;
+import org.apache.parquet.crypto.EncryptionPropertiesFactory;
+import org.apache.parquet.crypto.FileDecryptionProperties;
+import org.apache.parquet.crypto.FileEncryptionProperties;
+import org.apache.parquet.crypto.ParquetCipher;
+import org.apache.parquet.crypto.ParquetCryptoRuntimeException;
+import org.apache.parquet.hadoop.api.WriteSupport.WriteContext;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.schema.MessageType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class SchemaCryptoPropertiesFactory implements 
EncryptionPropertiesFactory, DecryptionPropertiesFactory {
+
+  private static Logger log = 
LoggerFactory.getLogger(SchemaCryptoPropertiesFactory.class);
+
+  public static final String CONF_ENCRYPTION_ALGORITHM = 
"parquet.encryption.algorithm";
+  public static final String CONF_ENCRYPTION_FOOTER = "parquet.encrypt.footer";
+  private static final byte[] FOOTER_KEY = {0x01, 0x02, 0x03, 0x4, 0x05, 0x06, 
0x07, 0x08, 0x09, 0x0a,
+0x0b, 0x0c, 0x0d, 0x0e, 0x0f, 0x10};
+  private static final byte[] FOOTER_KEY_METADATA = 
"footkey".getBytes(StandardCharsets.UTF_8);
+  private static final byte[] COL_KEY = {0x02, 0x03, 0x4, 0x05, 0x06, 0x07, 
0x08, 0x09, 0x0a, 0x0b,
+0x0c, 0x0d, 0x0e, 0x0f, 0x10, 0x11};
+  private static final byte[] COL_KEY_METADATA = 
"col".getBytes(StandardCharsets.UTF_8);
+
+  @Override
+  public FileEncryptionProperties getFileEncryptionProperties(Configuration 
conf, Path tempFilePath,
+  WriteContext 
fileWriteContext) throws ParquetCryptoRuntimeException {
+MessageType schema = fileWriteContext.getSchema();
+List paths = schema.getPaths();
+if (paths == null || paths.isEmpty()) {
+  throw new ParquetCryptoRuntimeException("Null or empty fields is found");
+}
+
+Map columnPropertyMap = new 
HashMap<>();
+
+for (String[] path : paths) {
+  getColumnEncryptionProperties(path, columnPropertyMap, conf);
+}
+
+if (columnPropertyMap.size() == 0) {
+  log.debug("No column is encrypted. Returning null so that Parquet can 
skip. Empty properties will cause Parquet exception");
+  return null;
+}
+
+/**
+ * Why we still need footerKeyMetadata even withEncryptedFooter as false? 
According to the
+ * 'Plaintext Footer' section of
+ * https://github.com/apache/parquet-format/blob/encryption/Encryption.md, 
the plaintext footer
+ * is signed in order to prevent tampering with the FileMetaData contents. 
So footerKeyMetadata
+ * is always needed. This signature will be verified if parquet-mr code is 
with parquet-1178.
+ * Otherwise, it will be ignored.
+ */
+boolean shouldEncryptFooter = getEncryptFooter(conf);
+FileEncryptionProperties.Builder encryptionPropertiesBuilder =
+  FileEncryptionProperties.builder(FOOTER_KEY)
+.withFooterKeyMetadata(FOOTER_KEY_METADATA)
+.withAlgorithm(getParquetCipherOrDefault(conf))
+.withEncryptedColumns(columnPropertyMap);
+if (!shouldEncryptFooter) {
+  encryptionPropertiesBuilder = 
encryptionPropertiesBuilder.withPlaintextFooter();
+}
+FileEncryptionProperties encryptionProperties = 
encryptionPropertiesBuilder.build();
+log.info(
+  "FileEncryptionProperties is built with, algorithm:{}, 
footerEncrypted:{}",
+  encryptionProperties.getAlgorithm(),

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

2020-10-21 Thread Xinli Shang (Jira)


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

Xinli Shang commented on PARQUET-1396:
--

Most of the functionality of this Jira has been addressed by "PARQUET-1817: 
Crypto Properties Factory". Hence change the name of this Jira to 'Example of 
Using EncryptionPropertiesFactory/DecryptionPropertiesFactory. 

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



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


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

2020-10-21 Thread Xinli Shang (Jira)


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

Xinli Shang updated PARQUET-1396:
-
Summary: Example of using EncryptionPropertiesFactory and 
DecryptionPropertiesFactory  (was: Cryptodata Interface for Schema Activation 
of Parquet Encryption)

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



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


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

2020-10-21 Thread GitBox


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



##
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<>();

Review comment:
   good catch





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-21 Thread GitBox


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



##
File path: parquet-column/src/main/java/org/apache/parquet/schema/Type.java
##
@@ -362,5 +362,4 @@ void checkContains(Type subType) {
* @return the converted tree
*/
abstract  T convert(List path, TypeConverter converter);
-
-}
+ }

Review comment:
   true





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-1929) Bump Snappy to 1.1.8

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


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

ASF GitHub Bot commented on PARQUET-1929:
-

maropu commented on pull request #833:
URL: https://github.com/apache/parquet-mr/pull/833#issuecomment-714013870


   Thanks~, @Fokko 



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
>




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


[GitHub] [parquet-mr] maropu commented on pull request #833: [PARQUET-1929] Bump Snappy to 1.1.8

2020-10-21 Thread GitBox


maropu commented on pull request #833:
URL: https://github.com/apache/parquet-mr/pull/833#issuecomment-714013870


   Thanks~, @Fokko 



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-1932) Bump Fastutil to 8.4.2

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


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

ASF GitHub Bot commented on PARQUET-1932:
-

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


   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [ ] My PR addresses the following [Parquet 
Jira](https://issues.apache.org/jira/browse/PARQUET/) issues and references 
them in the PR title. For example, "PARQUET-1234: My Parquet PR"
 - https://issues.apache.org/jira/browse/PARQUET-XXX
 - In case you are adding a dependency, check if the license complies with 
the [ASF 3rd Party License 
Policy](https://www.apache.org/legal/resolved.html#category-x).
   
   ### Tests
   
   - [ ] My PR adds the following unit tests __OR__ does not need testing for 
this extremely good reason:
   
   ### Commits
   
   - [ ] My commits all reference Jira issues in their subject lines. In 
addition, my commits follow the guidelines from "[How to write a good git 
commit message](http://chris.beams.io/posts/git-commit/)":
 1. Subject is separated from body by a blank line
 1. Subject is limited to 50 characters (not including Jira issue reference)
 1. Subject does not end with a period
 1. Subject uses the imperative mood ("add", not "adding")
 1. Body wraps at 72 characters
 1. Body explains "what" and "why", not "how"
   
   ### Documentation
   
   - [ ] In case of new functionality, my PR adds documentation that describes 
how to use it.
 - All the public functions and the classes in the PR contain Javadoc that 
explain what it does
   



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 opened a new pull request #836: [PARQUET-1932] Bump Fastutil to 8.4.2

2020-10-21 Thread GitBox


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


   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [ ] My PR addresses the following [Parquet 
Jira](https://issues.apache.org/jira/browse/PARQUET/) issues and references 
them in the PR title. For example, "PARQUET-1234: My Parquet PR"
 - https://issues.apache.org/jira/browse/PARQUET-XXX
 - In case you are adding a dependency, check if the license complies with 
the [ASF 3rd Party License 
Policy](https://www.apache.org/legal/resolved.html#category-x).
   
   ### Tests
   
   - [ ] My PR adds the following unit tests __OR__ does not need testing for 
this extremely good reason:
   
   ### Commits
   
   - [ ] My commits all reference Jira issues in their subject lines. In 
addition, my commits follow the guidelines from "[How to write a good git 
commit message](http://chris.beams.io/posts/git-commit/)":
 1. Subject is separated from body by a blank line
 1. Subject is limited to 50 characters (not including Jira issue reference)
 1. Subject does not end with a period
 1. Subject uses the imperative mood ("add", not "adding")
 1. Body wraps at 72 characters
 1. Body explains "what" and "why", not "how"
   
   ### Documentation
   
   - [ ] In case of new functionality, my PR adds documentation that describes 
how to use it.
 - All the public functions and the classes in the PR contain Javadoc that 
explain what it does
   



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

2020-10-21 Thread Fokko Driesprong (Jira)
Fokko Driesprong created PARQUET-1932:
-

 Summary: 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
 Fix For: 1.12.0






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


[jira] [Commented] (PARQUET-1900) Run mvn clean in CI

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


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

ASF GitHub Bot commented on PARQUET-1900:
-

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


   



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


> Run mvn clean in CI
> ---
>
> Key: PARQUET-1900
> URL: https://issues.apache.org/jira/browse/PARQUET-1900
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Reporter: Qinghui Xu
>Assignee: Qinghui Xu
>Priority: Major
>
> Currently parquet-mr CI does not run `mvn clean`, modules without changes are 
> not recompiled each time.



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


[GitHub] [parquet-mr] Fokko closed pull request #812: PARQUET-1900: Add mvn clean to CI

2020-10-21 Thread GitBox


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


   



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 #812: PARQUET-1900: Add mvn clean to CI

2020-10-21 Thread GitBox


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


   Thanks @qinghui-xu for following up. Let me know if you find out where the 
problem lies.



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-1910) Parquet-cli is broken after TransCompressionCommand was added

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


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

ASF GitHub Bot commented on PARQUET-1910:
-

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


   



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-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
>Priority: Major
>  Labels: pull-request-available
>
> +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)


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

2020-10-21 Thread Fokko Driesprong (Jira)


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

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

> 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
>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 #814: PARQUET-1910: Fix broken parquet-cli

2020-10-21 Thread GitBox


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


   



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-1924) Do not Instantiate a New LongHashFunction

2020-10-21 Thread Fokko Driesprong (Jira)


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

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

> Do not Instantiate a New LongHashFunction 
> --
>
> Key: PARQUET-1924
> URL: https://issues.apache.org/jira/browse/PARQUET-1924
> Project: Parquet
>  Issue Type: Improvement
>Reporter: David Mollitor
>Assignee: David Mollitor
>Priority: Minor
> Fix For: 1.12.0
>
>
> {code:java|title=XxHash.java}
> /**
>  * The implementation of HashFunction interface. The XxHash uses XXH64 
> version xxHash
>  * with a seed of 0.
>  */
> public class XxHash implements HashFunction {
>   @Override
>   public long hashBytes(byte[] input) {
> return LongHashFunction.xx(0).hashBytes(input);
>   }
>   @Override
>   public long hashByteBuffer(ByteBuffer input) {
> return LongHashFunction.xx(0).hashBytes(input);
>   }
> {code}
> Since the seed is always zero, the {{static}} implementation provided by the 
> library can be used here.



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


[jira] [Commented] (PARQUET-1924) Do not Instantiate a New LongHashFunction

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


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

ASF GitHub Bot commented on PARQUET-1924:
-

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


   



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


> Do not Instantiate a New LongHashFunction 
> --
>
> Key: PARQUET-1924
> URL: https://issues.apache.org/jira/browse/PARQUET-1924
> Project: Parquet
>  Issue Type: Improvement
>Reporter: David Mollitor
>Assignee: David Mollitor
>Priority: Minor
>
> {code:java|title=XxHash.java}
> /**
>  * The implementation of HashFunction interface. The XxHash uses XXH64 
> version xxHash
>  * with a seed of 0.
>  */
> public class XxHash implements HashFunction {
>   @Override
>   public long hashBytes(byte[] input) {
> return LongHashFunction.xx(0).hashBytes(input);
>   }
>   @Override
>   public long hashByteBuffer(ByteBuffer input) {
> return LongHashFunction.xx(0).hashBytes(input);
>   }
> {code}
> Since the seed is always zero, the {{static}} implementation provided by the 
> library can be used here.



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


[GitHub] [parquet-mr] Fokko merged pull request #827: PARQUET-1924: Do not Instantiate a New LongHashFunction

2020-10-21 Thread GitBox


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


   



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-1931) Bump Junit 4.13.1

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


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

ASF GitHub Bot commented on PARQUET-1931:
-

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


   Looks like there is a vulnerability in Junit:
   https://github.com/Fokko/parquet-mr/pull/44
   
   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [ ] My PR addresses the following [Parquet 
Jira](https://issues.apache.org/jira/browse/PARQUET/) issues and references 
them in the PR title. For example, "PARQUET-1234: My Parquet PR"
 - https://issues.apache.org/jira/browse/PARQUET-1931
 - In case you are adding a dependency, check if the license complies with 
the [ASF 3rd Party License 
Policy](https://www.apache.org/legal/resolved.html#category-x).
   
   ### Tests
   
   - [ ] My PR adds the following unit tests __OR__ does not need testing for 
this extremely good reason:
   
   ### Commits
   
   - [ ] My commits all reference Jira issues in their subject lines. In 
addition, my commits follow the guidelines from "[How to write a good git 
commit message](http://chris.beams.io/posts/git-commit/)":
 1. Subject is separated from body by a blank line
 1. Subject is limited to 50 characters (not including Jira issue reference)
 1. Subject does not end with a period
 1. Subject uses the imperative mood ("add", not "adding")
 1. Body wraps at 72 characters
 1. Body explains "what" and "why", not "how"
   
   ### Documentation
   
   - [ ] In case of new functionality, my PR adds documentation that describes 
how to use it.
 - All the public functions and the classes in the PR contain Javadoc that 
explain what it does
   



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] [Created] (PARQUET-1931) Bump Junit 4.13.1

2020-10-21 Thread Fokko Driesprong (Jira)
Fokko Driesprong created PARQUET-1931:
-

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






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


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

2020-10-21 Thread GitBox


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


   Looks like there is a vulnerability in Junit:
   https://github.com/Fokko/parquet-mr/pull/44
   
   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [ ] My PR addresses the following [Parquet 
Jira](https://issues.apache.org/jira/browse/PARQUET/) issues and references 
them in the PR title. For example, "PARQUET-1234: My Parquet PR"
 - https://issues.apache.org/jira/browse/PARQUET-1931
 - In case you are adding a dependency, check if the license complies with 
the [ASF 3rd Party License 
Policy](https://www.apache.org/legal/resolved.html#category-x).
   
   ### Tests
   
   - [ ] My PR adds the following unit tests __OR__ does not need testing for 
this extremely good reason:
   
   ### Commits
   
   - [ ] My commits all reference Jira issues in their subject lines. In 
addition, my commits follow the guidelines from "[How to write a good git 
commit message](http://chris.beams.io/posts/git-commit/)":
 1. Subject is separated from body by a blank line
 1. Subject is limited to 50 characters (not including Jira issue reference)
 1. Subject does not end with a period
 1. Subject uses the imperative mood ("add", not "adding")
 1. Body wraps at 72 characters
 1. Body explains "what" and "why", not "how"
   
   ### Documentation
   
   - [ ] In case of new functionality, my PR adds documentation that describes 
how to use it.
 - All the public functions and the classes in the PR contain Javadoc that 
explain what it does
   



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


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

ASF GitHub Bot commented on PARQUET-1930:
-

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


   Make sure you have checked _all_ steps below.
   
   Changelog: https://github.com/apache/thrift/blob/master/CHANGES.md#0130
   
   Thrift 0.13.0 makes the first steps to be compatible with Java 9+, and it 
isn't available anymore on Brew: https://github.com/apache/parquet-mr/pull/818 
Which is just a minor thing of course, but might be an indicator that we should 
upgrade to a newer version ;)
   
   ### Jira
   
   - [ ] My PR addresses the following [Parquet 
Jira](https://issues.apache.org/jira/browse/PARQUET/) issues and references 
them in the PR title. For example, "PARQUET-1234: My Parquet PR"
 - https://issues.apache.org/jira/browse/PARQUET-XXX
 - In case you are adding a dependency, check if the license complies with 
the [ASF 3rd Party License 
Policy](https://www.apache.org/legal/resolved.html#category-x).
   
   ### Tests
   
   - [ ] My PR adds the following unit tests __OR__ does not need testing for 
this extremely good reason:
   
   ### Commits
   
   - [ ] My commits all reference Jira issues in their subject lines. In 
addition, my commits follow the guidelines from "[How to write a good git 
commit message](http://chris.beams.io/posts/git-commit/)":
 1. Subject is separated from body by a blank line
 1. Subject is limited to 50 characters (not including Jira issue reference)
 1. Subject does not end with a period
 1. Subject uses the imperative mood ("add", not "adding")
 1. Body wraps at 72 characters
 1. Body explains "what" and "why", not "how"
   
   ### Documentation
   
   - [ ] In case of new functionality, my PR adds documentation that describes 
how to use it.
 - All the public functions and the classes in the PR contain Javadoc that 
explain what it does
   



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-mr] Fokko opened a new pull request #834: [PARQUET-1930] Bump Apache Thrift to 0.13

2020-10-21 Thread GitBox


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


   Make sure you have checked _all_ steps below.
   
   Changelog: https://github.com/apache/thrift/blob/master/CHANGES.md#0130
   
   Thrift 0.13.0 makes the first steps to be compatible with Java 9+, and it 
isn't available anymore on Brew: https://github.com/apache/parquet-mr/pull/818 
Which is just a minor thing of course, but might be an indicator that we should 
upgrade to a newer version ;)
   
   ### Jira
   
   - [ ] My PR addresses the following [Parquet 
Jira](https://issues.apache.org/jira/browse/PARQUET/) issues and references 
them in the PR title. For example, "PARQUET-1234: My Parquet PR"
 - https://issues.apache.org/jira/browse/PARQUET-XXX
 - In case you are adding a dependency, check if the license complies with 
the [ASF 3rd Party License 
Policy](https://www.apache.org/legal/resolved.html#category-x).
   
   ### Tests
   
   - [ ] My PR adds the following unit tests __OR__ does not need testing for 
this extremely good reason:
   
   ### Commits
   
   - [ ] My commits all reference Jira issues in their subject lines. In 
addition, my commits follow the guidelines from "[How to write a good git 
commit message](http://chris.beams.io/posts/git-commit/)":
 1. Subject is separated from body by a blank line
 1. Subject is limited to 50 characters (not including Jira issue reference)
 1. Subject does not end with a period
 1. Subject uses the imperative mood ("add", not "adding")
 1. Body wraps at 72 characters
 1. Body explains "what" and "why", not "how"
   
   ### Documentation
   
   - [ ] In case of new functionality, my PR adds documentation that describes 
how to use it.
 - All the public functions and the classes in the PR contain Javadoc that 
explain what it does
   



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


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

ASF GitHub Bot commented on PARQUET-1930:
-

Fokko opened a new pull request #162:
URL: https://github.com/apache/parquet-format/pull/162


   Make sure you have checked _all_ steps below.
   
   Changelog: https://github.com/apache/thrift/blob/master/CHANGES.md#0130
   
   ### Jira
   
   - [ ] My PR addresses the following [Parquet 
Jira](https://issues.apache.org/jira/browse/PARQUET/) issues and references 
them in the PR title. For example, "PARQUET-1234: My Parquet PR"
 - https://issues.apache.org/jira/browse/PARQUET-1930
 - In case you are adding a dependency, check if the license complies with 
the [ASF 3rd Party License 
Policy](https://www.apache.org/legal/resolved.html#category-x).
   
   ### Commits
   
   - [ ] My commits all reference Jira issues in their subject lines. In 
addition, my commits follow the guidelines from "[How to write a good git 
commit message](http://chris.beams.io/posts/git-commit/)":
 1. Subject is separated from body by a blank line
 1. Subject is limited to 50 characters (not including Jira issue reference)
 1. Subject does not end with a period
 1. Subject uses the imperative mood ("add", not "adding")
 1. Body wraps at 72 characters
 1. Body explains "what" and "why", not "how"
   
   ### Documentation
   
   - [ ] In case of new functionality, my PR adds documentation that describes 
how to use it.
 - All the public functions and the classes in the PR contain Javadoc that 
explain what it does
   



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] Fokko opened a new pull request #162: [PARQUET-1930] Bump Apache Thrift to 0.13

2020-10-21 Thread GitBox


Fokko opened a new pull request #162:
URL: https://github.com/apache/parquet-format/pull/162


   Make sure you have checked _all_ steps below.
   
   Changelog: https://github.com/apache/thrift/blob/master/CHANGES.md#0130
   
   ### Jira
   
   - [ ] My PR addresses the following [Parquet 
Jira](https://issues.apache.org/jira/browse/PARQUET/) issues and references 
them in the PR title. For example, "PARQUET-1234: My Parquet PR"
 - https://issues.apache.org/jira/browse/PARQUET-1930
 - In case you are adding a dependency, check if the license complies with 
the [ASF 3rd Party License 
Policy](https://www.apache.org/legal/resolved.html#category-x).
   
   ### Commits
   
   - [ ] My commits all reference Jira issues in their subject lines. In 
addition, my commits follow the guidelines from "[How to write a good git 
commit message](http://chris.beams.io/posts/git-commit/)":
 1. Subject is separated from body by a blank line
 1. Subject is limited to 50 characters (not including Jira issue reference)
 1. Subject does not end with a period
 1. Subject uses the imperative mood ("add", not "adding")
 1. Body wraps at 72 characters
 1. Body explains "what" and "why", not "how"
   
   ### Documentation
   
   - [ ] In case of new functionality, my PR adds documentation that describes 
how to use it.
 - All the public functions and the classes in the PR contain Javadoc that 
explain what it does
   



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

2020-10-21 Thread Fokko Driesprong (Jira)
Fokko Driesprong created PARQUET-1930:
-

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






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


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

2020-10-21 Thread GitBox


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


   For performance improvements;
   
   the released snappy-java bundles the latest Snappy v1.1.8 binaries with 
small performance improvements.
   
   - snappy-java release note: 
https://github.com/xerial/snappy-java/releases/tag/1.1.8
   - snappy release note: https://github.com/google/snappy/releases/tag/1.1.8
   
   And to keep in line with https://github.com/apache/spark/pull/30120
   
   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [ ] My PR addresses the following [Parquet 
Jira](https://issues.apache.org/jira/browse/PARQUET/) issues and references 
them in the PR title. For example, "PARQUET-1234: My Parquet PR"
 - https://issues.apache.org/jira/browse/PARQUET-XXX
 - In case you are adding a dependency, check if the license complies with 
the [ASF 3rd Party License 
Policy](https://www.apache.org/legal/resolved.html#category-x).
   
   ### Tests
   
   - [ ] My PR adds the following unit tests __OR__ does not need testing for 
this extremely good reason:
   
   ### Commits
   
   - [ ] My commits all reference Jira issues in their subject lines. In 
addition, my commits follow the guidelines from "[How to write a good git 
commit message](http://chris.beams.io/posts/git-commit/)":
 1. Subject is separated from body by a blank line
 1. Subject is limited to 50 characters (not including Jira issue reference)
 1. Subject does not end with a period
 1. Subject uses the imperative mood ("add", not "adding")
 1. Body wraps at 72 characters
 1. Body explains "what" and "why", not "how"
   
   ### Documentation
   
   - [ ] In case of new functionality, my PR adds documentation that describes 
how to use it.
 - All the public functions and the classes in the PR contain Javadoc that 
explain what it does
   



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-1929) Bump Snappy to 1.1.8

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


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

ASF GitHub Bot commented on PARQUET-1929:
-

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


   For performance improvements;
   
   the released snappy-java bundles the latest Snappy v1.1.8 binaries with 
small performance improvements.
   
   - snappy-java release note: 
https://github.com/xerial/snappy-java/releases/tag/1.1.8
   - snappy release note: https://github.com/google/snappy/releases/tag/1.1.8
   
   And to keep in line with https://github.com/apache/spark/pull/30120
   
   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [ ] My PR addresses the following [Parquet 
Jira](https://issues.apache.org/jira/browse/PARQUET/) issues and references 
them in the PR title. For example, "PARQUET-1234: My Parquet PR"
 - https://issues.apache.org/jira/browse/PARQUET-XXX
 - In case you are adding a dependency, check if the license complies with 
the [ASF 3rd Party License 
Policy](https://www.apache.org/legal/resolved.html#category-x).
   
   ### Tests
   
   - [ ] My PR adds the following unit tests __OR__ does not need testing for 
this extremely good reason:
   
   ### Commits
   
   - [ ] My commits all reference Jira issues in their subject lines. In 
addition, my commits follow the guidelines from "[How to write a good git 
commit message](http://chris.beams.io/posts/git-commit/)":
 1. Subject is separated from body by a blank line
 1. Subject is limited to 50 characters (not including Jira issue reference)
 1. Subject does not end with a period
 1. Subject uses the imperative mood ("add", not "adding")
 1. Body wraps at 72 characters
 1. Body explains "what" and "why", not "how"
   
   ### Documentation
   
   - [ ] In case of new functionality, my PR adds documentation that describes 
how to use it.
 - All the public functions and the classes in the PR contain Javadoc that 
explain what it does
   



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
>




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


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

2020-10-21 Thread Fokko Driesprong (Jira)
Fokko Driesprong created PARQUET-1929:
-

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






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


Re: Create a parquet-protobuf JIRA component

2020-10-21 Thread Aaron Niskode-Dossett
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] [Updated] (PARQUET-1917) [parquet-proto] default values are stored in oneOf fields that aren't set

2020-10-21 Thread Aaron Blake Niskode-Dossett (Jira)


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

Aaron Blake Niskode-Dossett updated PARQUET-1917:
-
Component/s: parquet-protobuf

> [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-1927) ColumnIndex should provide number of records skipped

2020-10-21 Thread Xinli Shang (Jira)


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

Xinli Shang commented on PARQUET-1927:
--

The workaround I can think of is to apply ColumnIndex to row groups, something 
like (columnIndex, rowGroup) => recordCount, before calling 
readNextFilteredRowGroup() in Iceberg. If recordCount is 0, we skip calling 
readNextFilteredRowGroup() for that row group. By doing this way, it is ensured 
that readNextFilteredRowGroup() will never advance to the next row group 
without Iceberg's knowledge. But this workaround has several issues. 1) It is 
not a trivial implementation because we need to implement all types of filters 
against columnIndex, which pretty much duplicate the implementation in Parquet. 
2) The two implementations(in Parquet and in Iceberg) have to be consistent. If 
one has issues, it will cause Iceberg to be in an unknown state. 3) It requires 
other adoption like Hive, Spark to reimplement their own too.  

This is not regression because ColumnIndex is a new feature in 1.11.x. But I 
think releasing 1.11.2 would be better because it helps the adoption of 1.11.x  
as the ColumnIndex feature is one of the major features in 1.11.x. 

 

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


Re: Create a parquet-protobuf JIRA component

2020-10-21 Thread Aaron Niskode-Dossett
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


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

2020-10-21 Thread GitBox


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



##
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:
   eg upcasting to `HadoopOutputFile`. Or even better, adding `Path 
getPath()` method to the `OutputFile` - this should be general enough.





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] ggershinsky commented on a change in pull request #808: Parquet-1396: Cryptodata Interface for Schema Activation of Parquet E…

2020-10-21 Thread GitBox


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



##
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:
   This assumes that `file.toString()` returns the full file path. However, 
the `file` is a `public abstract interface org.apache.parquet.io.OutputFile`, 
which doesn't have such method, so `toString()` is up to the implementation; no 
guarantees it will return the path. Also, `new Path(string full_path)` is not 
aware of the right filesystem (?) Maybe can be handled with an upcast to a 
known implementing class - preferably one that already has a `Path getPath()` 
method. 
   But of course, this won't be very general.

##
File path: parquet-column/src/main/java/org/apache/parquet/schema/Type.java
##
@@ -362,5 +362,4 @@ void checkContains(Type subType) {
* @return the converted tree
*/
abstract  T convert(List path, TypeConverter converter);
-
-}
+ }

Review comment:
   probably no need in changing this file





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] ggershinsky commented on pull request #808: Parquet-1396: Cryptodata Interface for Schema Activation of Parquet E…

2020-10-21 Thread GitBox


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


   > I have some comments otherwise I am fine with this change.
   > 
   > Meanwhile, the title and maybe the jira description do not fit this 
change. I think, they should be reformulated.
   
   +1



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 a change in pull request #808: Parquet-1396: Cryptodata Interface for Schema Activation of Parquet E…

2020-10-21 Thread GitBox


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



##
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:
   Since the target class is already on 

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

2020-10-21 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-1927:
---

I think, it is fine extending the current API if it is required by one of our 
clients. Also, Iceberg might be the first one who already uses 1.11. This 
extension might be useful for others (e.g. Hive, Spark).

Releasing 1.11.2 depends on the issues we would like to fix in it. If they are 
regressions introduced in 1.11 and they are sever we clearly would like to 
release the fix in a maintenance release. So the question is if this issue is 
severe enough and doesn't have a proper workaround.

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


Re: Create a parquet-protobuf JIRA component

2020-10-21 Thread Gabor Szadovszky
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
>