Re: Modular encryption to support arrays and nested arrays

2022-10-31 Thread Gidon Gershinsky
Parquet columnar encryption supports these types. Currently, it requires an
explicit full path for each column to be encrypted.
Your sample will work with
*spark.sparkContext.hadoopConfiguration.set("parquet.encryption.column.keys",
"k2:rider.list.element.foo,rider.list.element.bar")*

Having said that, there are a couple of things that can be improved (thank
you for running these checks!)

- the exception text is not informative enough, doesn't help much in
correcting the parameters. I've opened a Jira for this (and for updating
the parameter documentation).
The goal is to make the exception print something like:
*Caused by: org.apache.parquet.crypto.ParquetCryptoRuntimeException:
Encrypted column [rider] not in file schema column list: [foo] ,
[rider.list.element.foo] , [rider.list.element.bar] , [ts] , [uuid]*

- Configuring a key for all children of a nested schema node (eg "
*k2:rider.*"*). This had been discussed in the past, but not followed up..
Is this something you'd be interested to build? Alternatively, I can do it,
but this will take me a while to get to.


Cheers, Gidon


On Sat, Oct 29, 2022 at 12:45 AM nicolas paris 
wrote:

> Hello,
>
> apparently, modular encryption does not yet support **arrays** types.
>
> ```scala
> spark.sparkContext.hadoopConfiguration.set("parquet.crypto.factory.class",
> "org.apache.parquet.crypto.keytools.PropertiesDrivenCryptoFactory")
> spark.sparkContext.hadoopConfiguration.set("parquet.encryption.kms.client.class"
> , "org.apache.parquet.crypto.keytools.mocks.InMemoryKMS")
> spark.sparkContext.hadoopConfiguration.set("parquet.encryption.key.list",
> "k1:AAECAwQFBgcICQoLDA0ODw==, k2:AAECAAECAAECAAECAAECAA==")
> spark.sparkContext.hadoopConfiguration.set("parquet.encryption.plaintext.footer",
> "true")
> spark.sparkContext.hadoopConfiguration.set("parquet.encryption.footer.key",
> "k1")
> spark.sparkContext.hadoopConfiguration.set("parquet.encryption.column.keys",
> "k2:rider")
>
> val df = spark.sql("select 1 as foo, array(named_struct('foo',2, 'bar',3))
> as rider, 3 as ts, uuid() as uuid")
> df.write.format("parquet").mode("overwrite").save("/tmp/enc")
>
> Caused by: org.apache.parquet.crypto.ParquetCryptoRuntimeException:
> Encrypted column [rider] not in file schema
>
> ```
>
> also, the doted columnpath would not support to encrypt within nested
> structure mixed with arrays. For example, there is no way I am aware of to
> target "all foo in rider".
>
> ```
> root
>  |-- foo: integer (nullable = false)
>  |-- rider: array (nullable = false)
>  ||-- element: struct (containsNull = false)
>  |||-- foo: integer (nullable = false)
>  |||-- bar: integer (nullable = false)
>  |-- ts: integer (nullable = false)
>  |-- uuid: string (nullable = false)
> ```
>
> so far, those two issues makes arrays of confidential information
> impossible to encrypt, or am I missing something ?
>
> Thanks,
>


[jira] [Created] (PARQUET-2208) Add details to nested column encryption config doc and exception text

2022-10-31 Thread Gidon Gershinsky (Jira)
Gidon Gershinsky created PARQUET-2208:
-

 Summary: Add details to nested column encryption config doc and 
exception text
 Key: PARQUET-2208
 URL: https://issues.apache.org/jira/browse/PARQUET-2208
 Project: Parquet
  Issue Type: Improvement
  Components: parquet-mr
Affects Versions: 1.12.3
Reporter: Gidon Gershinsky


Parquet columnar encryption requires an explicit full path for each column to 
be encrypted. If a partial path is configured, the thrown exception is not 
informative enough, doesn't help much in correcting the parameters.
The goal is to make the exception print something like:
_Caused by: org.apache.parquet.crypto.ParquetCryptoRuntimeException: Encrypted 
column [rider] not in file schema column list: [foo] , [rider.list.element.foo] 
, [rider.list.element.bar] , [ts] , [uuid]_
 



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


[GitHub] [parquet-mr] ggershinsky commented on pull request #995: PARQUET-1711: support recursive proto schemas by limiting recursion depth

2022-10-31 Thread GitBox


ggershinsky commented on PR #995:
URL: https://github.com/apache/parquet-mr/pull/995#issuecomment-1297193514

   yep, just the squash/merge button.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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



[jira] [Commented] (PARQUET-1711) [parquet-protobuf] stack overflow when work with well known json type

2022-10-31 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1711:
-

ggershinsky commented on PR #995:
URL: https://github.com/apache/parquet-mr/pull/995#issuecomment-1297193514

   yep, just the squash/merge button.




> [parquet-protobuf] stack overflow when work with well known json type
> -
>
> Key: PARQUET-1711
> URL: https://issues.apache.org/jira/browse/PARQUET-1711
> Project: Parquet
>  Issue Type: Bug
>Affects Versions: 1.10.1
>Reporter: Lawrence He
>Priority: Major
>
> Writing following protobuf message as parquet file is not possible: 
> {code:java}
> syntax = "proto3";
> import "google/protobuf/struct.proto";
> package test;
> option java_outer_classname = "CustomMessage";
> message TestMessage {
> map data = 1;
> } {code}
> Protobuf introduced "well known json type" such like 
> [ListValue|https://developers.google.com/protocol-buffers/docs/reference/google.protobuf#listvalue]
>  to work around json schema conversion. 
> However writing above messages traps parquet writer into an infinite loop due 
> to the "general type" support in protobuf. Current implementation will keep 
> referencing 6 possible types defined in protobuf (null, bool, number, string, 
> struct, list) and entering infinite loop when referencing "struct".
> {code:java}
> java.lang.StackOverflowErrorjava.lang.StackOverflowError at 
> java.base/java.util.Arrays$ArrayItr.(Arrays.java:4418) at 
> java.base/java.util.Arrays$ArrayList.iterator(Arrays.java:4410) at 
> java.base/java.util.Collections$UnmodifiableCollection$1.(Collections.java:1044)
>  at 
> java.base/java.util.Collections$UnmodifiableCollection.iterator(Collections.java:1043)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.convertFields(ProtoSchemaConverter.java:64)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.addField(ProtoSchemaConverter.java:96)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.convertFields(ProtoSchemaConverter.java:66)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.addField(ProtoSchemaConverter.java:96)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.convertFields(ProtoSchemaConverter.java:66)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.addField(ProtoSchemaConverter.java:96)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.convertFields(ProtoSchemaConverter.java:66)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.addField(ProtoSchemaConverter.java:96)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.convertFields(ProtoSchemaConverter.java:66)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.addField(ProtoSchemaConverter.java:96)
>  {code}



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


[GitHub] [parquet-mr] jinyius commented on pull request #995: PARQUET-1711: support recursive proto schemas by limiting recursion depth

2022-10-31 Thread GitBox


jinyius commented on PR #995:
URL: https://github.com/apache/parquet-mr/pull/995#issuecomment-1297295385

   @ggershinsky 
   
   i'd love to just hit the button.  i don't see it.  the workflow for travis 
ci had a failure due to a transient connection issue, and so it wasn't giving 
me the option to merge.  the ui messaging also states that "Only those with 
[write 
access](https://docs.github.com/articles/what-are-the-different-access-permissions)
 to this repository can merge pull requests." 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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



[jira] [Updated] (PARQUET-2209) Optimize skip for the case that number of values to skip equals page size

2022-10-31 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot updated PARQUET-2209:

Labels: pull-request-available  (was: )

> Optimize skip for the case that number of values to skip equals page size
> -
>
> Key: PARQUET-2209
> URL: https://issues.apache.org/jira/browse/PARQUET-2209
> Project: Parquet
>  Issue Type: Improvement
>Reporter: fatemah
>Priority: Minor
>  Labels: pull-request-available
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> Optimize skip for the case that the number of values to skip equals page 
> size. Right now, we end up reading to the end of the page and throwing away 
> the rep/defs and values that we have read, which is unnecessary.



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


[jira] [Commented] (PARQUET-1711) [parquet-protobuf] stack overflow when work with well known json type

2022-10-31 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1711:
-

jinyius commented on PR #995:
URL: https://github.com/apache/parquet-mr/pull/995#issuecomment-1297295385

   @ggershinsky 
   
   i'd love to just hit the button.  i don't see it.  the workflow for travis 
ci had a failure due to a transient connection issue, and so it wasn't giving 
me the option to merge.  the ui messaging also states that "Only those with 
[write 
access](https://docs.github.com/articles/what-are-the-different-access-permissions)
 to this repository can merge pull requests." 




> [parquet-protobuf] stack overflow when work with well known json type
> -
>
> Key: PARQUET-1711
> URL: https://issues.apache.org/jira/browse/PARQUET-1711
> Project: Parquet
>  Issue Type: Bug
>Affects Versions: 1.10.1
>Reporter: Lawrence He
>Priority: Major
>
> Writing following protobuf message as parquet file is not possible: 
> {code:java}
> syntax = "proto3";
> import "google/protobuf/struct.proto";
> package test;
> option java_outer_classname = "CustomMessage";
> message TestMessage {
> map data = 1;
> } {code}
> Protobuf introduced "well known json type" such like 
> [ListValue|https://developers.google.com/protocol-buffers/docs/reference/google.protobuf#listvalue]
>  to work around json schema conversion. 
> However writing above messages traps parquet writer into an infinite loop due 
> to the "general type" support in protobuf. Current implementation will keep 
> referencing 6 possible types defined in protobuf (null, bool, number, string, 
> struct, list) and entering infinite loop when referencing "struct".
> {code:java}
> java.lang.StackOverflowErrorjava.lang.StackOverflowError at 
> java.base/java.util.Arrays$ArrayItr.(Arrays.java:4418) at 
> java.base/java.util.Arrays$ArrayList.iterator(Arrays.java:4410) at 
> java.base/java.util.Collections$UnmodifiableCollection$1.(Collections.java:1044)
>  at 
> java.base/java.util.Collections$UnmodifiableCollection.iterator(Collections.java:1043)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.convertFields(ProtoSchemaConverter.java:64)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.addField(ProtoSchemaConverter.java:96)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.convertFields(ProtoSchemaConverter.java:66)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.addField(ProtoSchemaConverter.java:96)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.convertFields(ProtoSchemaConverter.java:66)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.addField(ProtoSchemaConverter.java:96)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.convertFields(ProtoSchemaConverter.java:66)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.addField(ProtoSchemaConverter.java:96)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.convertFields(ProtoSchemaConverter.java:66)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.addField(ProtoSchemaConverter.java:96)
>  {code}



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


[jira] [Created] (PARQUET-2209) Optimize skip for the case that number of values to skip equals page size

2022-10-31 Thread fatemah (Jira)
fatemah created PARQUET-2209:


 Summary: Optimize skip for the case that number of values to skip 
equals page size
 Key: PARQUET-2209
 URL: https://issues.apache.org/jira/browse/PARQUET-2209
 Project: Parquet
  Issue Type: Improvement
Reporter: fatemah


Optimize skip for the case that the number of values to skip equals page size. 
Right now, we end up reading to the end of the page and throwing away the 
rep/defs and values that we have read, which is unnecessary.



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


[jira] [Commented] (PARQUET-1711) [parquet-protobuf] stack overflow when work with well known json type

2022-10-31 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1711:
-

jinyius closed pull request #995: PARQUET-1711: support recursive proto schemas 
by limiting recursion depth
URL: https://github.com/apache/parquet-mr/pull/995




> [parquet-protobuf] stack overflow when work with well known json type
> -
>
> Key: PARQUET-1711
> URL: https://issues.apache.org/jira/browse/PARQUET-1711
> Project: Parquet
>  Issue Type: Bug
>Affects Versions: 1.10.1
>Reporter: Lawrence He
>Priority: Major
>
> Writing following protobuf message as parquet file is not possible: 
> {code:java}
> syntax = "proto3";
> import "google/protobuf/struct.proto";
> package test;
> option java_outer_classname = "CustomMessage";
> message TestMessage {
> map data = 1;
> } {code}
> Protobuf introduced "well known json type" such like 
> [ListValue|https://developers.google.com/protocol-buffers/docs/reference/google.protobuf#listvalue]
>  to work around json schema conversion. 
> However writing above messages traps parquet writer into an infinite loop due 
> to the "general type" support in protobuf. Current implementation will keep 
> referencing 6 possible types defined in protobuf (null, bool, number, string, 
> struct, list) and entering infinite loop when referencing "struct".
> {code:java}
> java.lang.StackOverflowErrorjava.lang.StackOverflowError at 
> java.base/java.util.Arrays$ArrayItr.(Arrays.java:4418) at 
> java.base/java.util.Arrays$ArrayList.iterator(Arrays.java:4410) at 
> java.base/java.util.Collections$UnmodifiableCollection$1.(Collections.java:1044)
>  at 
> java.base/java.util.Collections$UnmodifiableCollection.iterator(Collections.java:1043)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.convertFields(ProtoSchemaConverter.java:64)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.addField(ProtoSchemaConverter.java:96)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.convertFields(ProtoSchemaConverter.java:66)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.addField(ProtoSchemaConverter.java:96)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.convertFields(ProtoSchemaConverter.java:66)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.addField(ProtoSchemaConverter.java:96)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.convertFields(ProtoSchemaConverter.java:66)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.addField(ProtoSchemaConverter.java:96)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.convertFields(ProtoSchemaConverter.java:66)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.addField(ProtoSchemaConverter.java:96)
>  {code}



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


[GitHub] [parquet-mr] jinyius closed pull request #995: PARQUET-1711: support recursive proto schemas by limiting recursion depth

2022-10-31 Thread GitBox


jinyius closed pull request #995: PARQUET-1711: support recursive proto schemas 
by limiting recursion depth
URL: https://github.com/apache/parquet-mr/pull/995


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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



[GitHub] [parquet-mr] jinyius commented on pull request #995: PARQUET-1711: support recursive proto schemas by limiting recursion depth

2022-10-31 Thread GitBox


jinyius commented on PR #995:
URL: https://github.com/apache/parquet-mr/pull/995#issuecomment-1297292700

   > 
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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



[jira] [Commented] (PARQUET-1711) [parquet-protobuf] stack overflow when work with well known json type

2022-10-31 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1711:
-

jinyius commented on PR #995:
URL: https://github.com/apache/parquet-mr/pull/995#issuecomment-1297292700

   > 
   
   




> [parquet-protobuf] stack overflow when work with well known json type
> -
>
> Key: PARQUET-1711
> URL: https://issues.apache.org/jira/browse/PARQUET-1711
> Project: Parquet
>  Issue Type: Bug
>Affects Versions: 1.10.1
>Reporter: Lawrence He
>Priority: Major
>
> Writing following protobuf message as parquet file is not possible: 
> {code:java}
> syntax = "proto3";
> import "google/protobuf/struct.proto";
> package test;
> option java_outer_classname = "CustomMessage";
> message TestMessage {
> map data = 1;
> } {code}
> Protobuf introduced "well known json type" such like 
> [ListValue|https://developers.google.com/protocol-buffers/docs/reference/google.protobuf#listvalue]
>  to work around json schema conversion. 
> However writing above messages traps parquet writer into an infinite loop due 
> to the "general type" support in protobuf. Current implementation will keep 
> referencing 6 possible types defined in protobuf (null, bool, number, string, 
> struct, list) and entering infinite loop when referencing "struct".
> {code:java}
> java.lang.StackOverflowErrorjava.lang.StackOverflowError at 
> java.base/java.util.Arrays$ArrayItr.(Arrays.java:4418) at 
> java.base/java.util.Arrays$ArrayList.iterator(Arrays.java:4410) at 
> java.base/java.util.Collections$UnmodifiableCollection$1.(Collections.java:1044)
>  at 
> java.base/java.util.Collections$UnmodifiableCollection.iterator(Collections.java:1043)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.convertFields(ProtoSchemaConverter.java:64)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.addField(ProtoSchemaConverter.java:96)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.convertFields(ProtoSchemaConverter.java:66)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.addField(ProtoSchemaConverter.java:96)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.convertFields(ProtoSchemaConverter.java:66)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.addField(ProtoSchemaConverter.java:96)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.convertFields(ProtoSchemaConverter.java:66)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.addField(ProtoSchemaConverter.java:96)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.convertFields(ProtoSchemaConverter.java:66)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.addField(ProtoSchemaConverter.java:96)
>  {code}



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


[jira] [Commented] (PARQUET-1711) [parquet-protobuf] stack overflow when work with well known json type

2022-10-31 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1711:
-

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

   - This is an alternative approach to supporting recursion to apache#445 and 
apache#988.
   - This approach could address the other recursion related issues 
(PARQUET-129, PARQUET-554).
   - TODO: ReadSupport
   
   ### Jira
   
   - [x] 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-1711
 - 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
   
   - [x] My PR adds the following unit tests __OR__ does not need testing for 
this extremely good reason:
 - `ProtoSchemaConverterTest#test*Recursion`
 - `ProtoWriteSupportTest#test*Recursion`
   
   ### 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"
   
   ### Documentation
   
   - [x] 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
   




> [parquet-protobuf] stack overflow when work with well known json type
> -
>
> Key: PARQUET-1711
> URL: https://issues.apache.org/jira/browse/PARQUET-1711
> Project: Parquet
>  Issue Type: Bug
>Affects Versions: 1.10.1
>Reporter: Lawrence He
>Priority: Major
>
> Writing following protobuf message as parquet file is not possible: 
> {code:java}
> syntax = "proto3";
> import "google/protobuf/struct.proto";
> package test;
> option java_outer_classname = "CustomMessage";
> message TestMessage {
> map data = 1;
> } {code}
> Protobuf introduced "well known json type" such like 
> [ListValue|https://developers.google.com/protocol-buffers/docs/reference/google.protobuf#listvalue]
>  to work around json schema conversion. 
> However writing above messages traps parquet writer into an infinite loop due 
> to the "general type" support in protobuf. Current implementation will keep 
> referencing 6 possible types defined in protobuf (null, bool, number, string, 
> struct, list) and entering infinite loop when referencing "struct".
> {code:java}
> java.lang.StackOverflowErrorjava.lang.StackOverflowError at 
> java.base/java.util.Arrays$ArrayItr.(Arrays.java:4418) at 
> java.base/java.util.Arrays$ArrayList.iterator(Arrays.java:4410) at 
> java.base/java.util.Collections$UnmodifiableCollection$1.(Collections.java:1044)
>  at 
> java.base/java.util.Collections$UnmodifiableCollection.iterator(Collections.java:1043)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.convertFields(ProtoSchemaConverter.java:64)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.addField(ProtoSchemaConverter.java:96)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.convertFields(ProtoSchemaConverter.java:66)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.addField(ProtoSchemaConverter.java:96)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.convertFields(ProtoSchemaConverter.java:66)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.addField(ProtoSchemaConverter.java:96)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.convertFields(ProtoSchemaConverter.java:66)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.addField(ProtoSchemaConverter.java:96)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.convertFields(ProtoSchemaConverter.java:66)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.addField(ProtoSchemaConverter.java:96)
>  {code}



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


[jira] [Updated] (PARQUET-2209) [C++] Optimize skip for the case that number of values to skip equals page size

2022-10-31 Thread Antoine Pitrou (Jira)


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

Antoine Pitrou updated PARQUET-2209:

Component/s: parquet-cpp

> [C++] Optimize skip for the case that number of values to skip equals page 
> size
> ---
>
> Key: PARQUET-2209
> URL: https://issues.apache.org/jira/browse/PARQUET-2209
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-cpp
>Reporter: fatemah
>Priority: Minor
>  Labels: pull-request-available
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> Optimize skip for the case that the number of values to skip equals page 
> size. Right now, we end up reading to the end of the page and throwing away 
> the rep/defs and values that we have read, which is unnecessary.



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


[jira] [Updated] (PARQUET-2209) [C++] Optimize skip for the case that number of values to skip equals page size

2022-10-31 Thread Antoine Pitrou (Jira)


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

Antoine Pitrou updated PARQUET-2209:

Summary: [C++] Optimize skip for the case that number of values to skip 
equals page size  (was: Optimize skip for the case that number of values to 
skip equals page size)

> [C++] Optimize skip for the case that number of values to skip equals page 
> size
> ---
>
> Key: PARQUET-2209
> URL: https://issues.apache.org/jira/browse/PARQUET-2209
> Project: Parquet
>  Issue Type: Improvement
>Reporter: fatemah
>Priority: Minor
>  Labels: pull-request-available
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> Optimize skip for the case that the number of values to skip equals page 
> size. Right now, we end up reading to the end of the page and throwing away 
> the rep/defs and values that we have read, which is unnecessary.



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


Add FilteredPageReader to filter rows based on page statistics

2022-10-31 Thread Fatemah Panahi
-- Sending as an email in case Jira messages are filtered out. Please let
me know your thoughts on this. Thanks!

Jira ticket: https://issues.apache.org/jira/browse/PARQUET-2210

Currently, we do not use the statistics that is stored in the page headers
for pruning the rows that we read. Row group pruning is very coarse-grained
and in many cases does not prune the row group. I propose adding a
FilteredPageReader that would accept a filter and would not return the
pages that do not match the filter based on page statistics.

Initial set of filters can be: EQUALS, IS NULL, IS NOT NULL.

Also, the FilteredPageReader will keep track of what row ranges matched and
not matched. We could use this to skip reading rows that do not match from
the rest of the columns. Note that the SkipRecords API is being added to
the Parquet reader (https://issues.apache.org/jira/browse/PARQUET-2188)


[jira] [Created] (PARQUET-2210) Add FilteredPageReader to filter rows based on page statistics

2022-10-31 Thread fatemah (Jira)
fatemah created PARQUET-2210:


 Summary: Add FilteredPageReader to filter rows based on page 
statistics
 Key: PARQUET-2210
 URL: https://issues.apache.org/jira/browse/PARQUET-2210
 Project: Parquet
  Issue Type: New Feature
Reporter: fatemah


Currently, we do not use the statistics that is stored in the page headers for 
pruning the rows that we read. Row group pruning is very coarse-grained and in 
many cases does not prune the row group. I propose adding a FilteredPageReader 
that would accept a filter and would not return the pages that do not match the 
filter based on page statistics.

Initial set of filters can be: EQUALS, IS NULL, IS NOT NULL.

Also, the FilteredPageReader will keep track of what row ranges matched and not 
matched. We could use this to skip reading rows that do not match from the rest 
of the columns. Note that the SkipRecords API was recently added to the Parquet 
reader (https://issues.apache.org/jira/browse/PARQUET-2188)



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


Re: Add FilteredPageReader to filter rows based on page statistics

2022-10-31 Thread Micah Kornfield
Hi Fatemah,
I think there are likely two things to consider here:
1.  How will expressions be modeled?  There are already some examples of
using expressions in Arrow for pruning predicates [1].  Do you plan to
re-use them?
2.  Along these lines is the proposed approach taken because the API to
expose the raw data necessary and filter externally too unwieldy?

Thanks,
Micah


[1]
https://github.com/apache/arrow/blob/5e49174d69deb9d1cbbdf82bc8041b90098f560b/cpp/src/arrow/dataset/file_parquet.cc

On Mon, Oct 31, 2022 at 9:50 AM Fatemah Panahi 
wrote:

> -- Sending as an email in case Jira messages are filtered out. Please let
> me know your thoughts on this. Thanks!
>
> Jira ticket: https://issues.apache.org/jira/browse/PARQUET-2210
>
> Currently, we do not use the statistics that is stored in the page headers
> for pruning the rows that we read. Row group pruning is very coarse-grained
> and in many cases does not prune the row group. I propose adding a
> FilteredPageReader that would accept a filter and would not return the
> pages that do not match the filter based on page statistics.
>
> Initial set of filters can be: EQUALS, IS NULL, IS NOT NULL.
>
> Also, the FilteredPageReader will keep track of what row ranges matched and
> not matched. We could use this to skip reading rows that do not match from
> the rest of the columns. Note that the SkipRecords API is being added to
> the Parquet reader (https://issues.apache.org/jira/browse/PARQUET-2188)
>


JIRA issue tracker registration

2022-10-31 Thread Antoine Pitrou


Hello,

I don't know if everyone here is already aware, but the Apache Software
Foundation has decided that user registration on JIRA will very soon be
moderated in order to fight against issue/comment spam. Concretely, user
creation requests will soon have to be approved by a project's PMC
instead of going through the current automated process.

The announcement was done by the ASF on the announce@infra
mailing-list (and forwarded to the PMCs of the various Apache projects),
which unfortunately I couldn't find a public record of.

I don't know if the Parquet project will want to take any steps
following this announcement, but I would like to mention that the Arrow
project (which has ties to Parquet through the C++ implementation of
Parquet which is hosted inside the Arrow repository) has voted to
migrate its issues to Github:
https://lists.apache.org/thread/nkzbg0481k0dt0l2wq9b2k60kpg5hk62

Regards

Antoine.




[jira] [Resolved] (PARQUET-2188) Add SkipRecords API to RecordReader

2022-10-31 Thread Antoine Pitrou (Jira)


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

Antoine Pitrou resolved PARQUET-2188.
-
Fix Version/s: cpp-11.0.0
   Resolution: Fixed

Issue resolved by pull request 14142
[https://github.com/apache/arrow/pull/14142]

> Add SkipRecords API to RecordReader
> ---
>
> Key: PARQUET-2188
> URL: https://issues.apache.org/jira/browse/PARQUET-2188
> Project: Parquet
>  Issue Type: New Feature
>  Components: parquet-cpp
>Reporter: fatemah
>Priority: Major
>  Labels: pull-request-available
> Fix For: cpp-11.0.0
>
>  Time Spent: 16.5h
>  Remaining Estimate: 0h
>
> The RecordReader is missing an API to skip records. There is a Skip method in 
> the ColumnReader, but that skips based on the number of values/levels and not 
> records. For repeated fields, this SkipRecords API will detect the record 
> boundaries and correctly skip the right number of values for the requested 
> number of records.



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


[jira] [Assigned] (PARQUET-2188) Add SkipRecords API to RecordReader

2022-10-31 Thread Antoine Pitrou (Jira)


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

Antoine Pitrou reassigned PARQUET-2188:
---

Assignee: fatemah

> Add SkipRecords API to RecordReader
> ---
>
> Key: PARQUET-2188
> URL: https://issues.apache.org/jira/browse/PARQUET-2188
> Project: Parquet
>  Issue Type: New Feature
>  Components: parquet-cpp
>Reporter: fatemah
>Assignee: fatemah
>Priority: Major
>  Labels: pull-request-available
> Fix For: cpp-11.0.0
>
>  Time Spent: 16.5h
>  Remaining Estimate: 0h
>
> The RecordReader is missing an API to skip records. There is a Skip method in 
> the ColumnReader, but that skips based on the number of values/levels and not 
> records. For repeated fields, this SkipRecords API will detect the record 
> boundaries and correctly skip the right number of values for the requested 
> number of records.



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