Re: Vectored IO in Parquet ( https://issues.apache.org/jira/browse/PARQUET-2171)

2022-10-08 Thread Xinli shang
Thanks, Mukund! As spoken at the conference, this is a great feature! Look
forward to reviewing the changes!

On Tue, Sep 27, 2022 at 9:29 AM Mukund Madhav Thakur
 wrote:

> Hi Team,
> We in hadoop project recently added a new feature in Hadoop Vectored IO
> which will be released in the upcoming 3.3.5 hadoop release.
> This is a high performance scatter/gather extension of PositionedReadable
> API optimized for reading columnar data in cloud storage.
> https://issues.apache.org/jira/browse/HADOOP-18103.
> We observed really good performance improvements in hive tpch and tpcds
> benchmark for orc data stored in S3.
>
> We are now looking at Parquet integration as well.
> https://issues.apache.org/jira/browse/PARQUET-2171
> I have a draft patch which works locally through sparks file reader.
> https://github.com/apache/parquet-mr/pull/999
>
> We know Parquet likes to support builds against the older versions of
> hadoop, we are working on a solution to offer the API through a
> shim library.
> As I have never contributed to the Parquet codebase and it is totally new
> for me, I would really appreciate some help in implementing, testing and
> releasing this feature in the best possible way.
>
> I will be talking about all these in the upcoming Apache Conference NA next
> week Tuesday, October 04, 4:10 PM CDT. It would be really great to meet
> anyone who would be interested in getting involved in this.
>
>
>
> Thanks,
> Mukund
>


-- 
Xinli Shang


[GitHub] [parquet-mr] danielcweeks commented on a diff in pull request #999: [DRAFT] PR to show Vectored IO integration, compilation fails now.

2022-10-08 Thread GitBox


danielcweeks commented on code in PR #999:
URL: https://github.com/apache/parquet-mr/pull/999#discussion_r990665831


##
parquet-common/src/main/java/org/apache/parquet/io/SeekableInputStream.java:
##
@@ -23,6 +23,10 @@
 import java.io.IOException;
 import java.io.InputStream;
 import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.function.IntFunction;
+
+import org.apache.hadoop.fs.FileRange;

Review Comment:
   I feel like this might be an issue.  We probably don't want to introduce a 
Hadoop dependency here because it breaks the separation from Hadoop in the IO 
path.



-- 
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] emkornfield commented on a diff in pull request #995: PARQUET-1711: support recursive proto schemas by limiting recursion depth

2022-10-08 Thread GitBox


emkornfield commented on code in PR #995:
URL: https://github.com/apache/parquet-mr/pull/995#discussion_r990726331


##
parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoWriteSupport.java:
##
@@ -559,7 +564,14 @@ final void writeRawValue(Object value) {
   class BinaryWriter extends FieldWriter {
 @Override
 final void writeRawValue(Object value) {
-  ByteString byteString = (ByteString) value;
+  // Non-ByteString values can happen when recursions gets truncated.
+  ByteString byteString = value instanceof ByteString
+  ? (ByteString) value
+  // TODO: figure out a way to use MessageOrBuilder
+  : value instanceof Message
+  ? ((Message) value).toByteString()

Review Comment:
   does recordconsumer offer a stream API or something else to avoid the 
additional array/bytestring copies?



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


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

ASF GitHub Bot commented on PARQUET-1711:
-

emkornfield commented on code in PR #995:
URL: https://github.com/apache/parquet-mr/pull/995#discussion_r990726272


##
parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoWriteSupport.java:
##
@@ -559,7 +564,14 @@ final void writeRawValue(Object value) {
   class BinaryWriter extends FieldWriter {
 @Override
 final void writeRawValue(Object value) {
-  ByteString byteString = (ByteString) value;
+  // Non-ByteString values can happen when recursions gets truncated.
+  ByteString byteString = value instanceof ByteString
+  ? (ByteString) value
+  // TODO: figure out a way to use MessageOrBuilder
+  : value instanceof Message
+  ? ((Message) value).toByteString()
+  // Worst-case, just dump as plain java string.
+  : ByteString.copyFromUtf8(value.toString());

Review Comment:
   is this actually an intended state?  If not it is probably better to raise 
an exception then writing data that could possibly be hard to recover.





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


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

ASF GitHub Bot commented on PARQUET-1711:
-

emkornfield commented on code in PR #995:
URL: https://github.com/apache/parquet-mr/pull/995#discussion_r990726331


##
parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoWriteSupport.java:
##
@@ -559,7 +564,14 @@ final void writeRawValue(Object value) {
   class BinaryWriter extends FieldWriter {
 @Override
 final void writeRawValue(Object value) {
-  ByteString byteString = (ByteString) value;
+  // Non-ByteString values can happen when recursions gets truncated.
+  ByteString byteString = value instanceof ByteString
+  ? (ByteString) value
+  // TODO: figure out a way to use MessageOrBuilder
+  : value instanceof Message
+  ? ((Message) value).toByteString()

Review Comment:
   does recordconsumer offer a stream API or something else to avoid the 
additional array/bytestring copies?





> [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-1222) Specify a well-defined sorting order for float and double types

2022-10-08 Thread Micah Kornfield (Jira)


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

Micah Kornfield commented on PARQUET-1222:
--

Elevating the specification level seems fine.  I was under the impression the 
thrift file was the specification?  Where do we need to do the PR to elevate 
them?

> Specify a well-defined sorting order for float and double types
> ---
>
> Key: PARQUET-1222
> URL: https://issues.apache.org/jira/browse/PARQUET-1222
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-format
>Reporter: Zoltan Ivanfi
>Priority: Critical
>
> Currently parquet-format specifies the sort order for floating point numbers 
> as follows:
> {code:java}
>*   FLOAT - signed comparison of the represented value
>*   DOUBLE - signed comparison of the represented value
> {code}
> The problem is that the comparison of floating point numbers is only a 
> partial ordering with strange behaviour in specific corner cases. For 
> example, according to IEEE 754, -0 is neither less nor more than \+0 and 
> comparing NaN to anything always returns false. This ordering is not suitable 
> for statistics. Additionally, the Java implementation already uses a 
> different (total) ordering that handles these cases correctly but differently 
> than the C\+\+ implementations, which leads to interoperability problems.
> TypeDefinedOrder for doubles and floats should be deprecated and a new 
> TotalFloatingPointOrder should be introduced. The default for writing doubles 
> and floats would be the new TotalFloatingPointOrder. This ordering should be 
> effective and easy to implement in all programming languages.



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


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

ASF GitHub Bot commented on PARQUET-1711:
-

emkornfield commented on code in PR #995:
URL: https://github.com/apache/parquet-mr/pull/995#discussion_r990727282


##
parquet-protobuf/src/test/resources/BinaryTree.par:
##
@@ -0,0 +1,50 @@
+message Trees.BinaryTree {
+  optional group value = 1 {

Review Comment:
   or is par not proto?





> [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] emkornfield commented on a diff in pull request #995: PARQUET-1711: support recursive proto schemas by limiting recursion depth

2022-10-08 Thread GitBox


emkornfield commented on code in PR #995:
URL: https://github.com/apache/parquet-mr/pull/995#discussion_r990727282


##
parquet-protobuf/src/test/resources/BinaryTree.par:
##
@@ -0,0 +1,50 @@
+message Trees.BinaryTree {
+  optional group value = 1 {

Review Comment:
   or is par not proto?



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


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

ASF GitHub Bot commented on PARQUET-1711:
-

emkornfield commented on code in PR #995:
URL: https://github.com/apache/parquet-mr/pull/995#discussion_r990725805


##
parquet-protobuf/src/test/java/org/apache/parquet/proto/ProtoSchemaConverterTest.java:
##
@@ -82,264 +93,447 @@ public void testConvertAllDatatypes() throws Exception {
* Tests that all protocol buffer datatypes are converted to correct parquet 
datatypes.
*/
   @Test
-  public void testProto3ConvertAllDatatypes() throws Exception {
-String expectedSchema =
-  "message TestProto3.SchemaConverterAllDatatypes {\n" +
-"  optional double optionalDouble = 1;\n" +
-"  optional float optionalFloat = 2;\n" +
-"  optional int32 optionalInt32 = 3;\n" +
-"  optional int64 optionalInt64 = 4;\n" +
-"  optional int32 optionalUInt32 = 5;\n" +
-"  optional int64 optionalUInt64 = 6;\n" +
-"  optional int32 optionalSInt32 = 7;\n" +
-"  optional int64 optionalSInt64 = 8;\n" +
-"  optional int32 optionalFixed32 = 9;\n" +
-"  optional int64 optionalFixed64 = 10;\n" +
-"  optional int32 optionalSFixed32 = 11;\n" +
-"  optional int64 optionalSFixed64 = 12;\n" +
-"  optional boolean optionalBool = 13;\n" +
-"  optional binary optionalString (UTF8) = 14;\n" +
-"  optional binary optionalBytes = 15;\n" +
-"  optional group optionalMessage = 16 {\n" +
-"optional int32 someId = 3;\n" +
-"  }\n" +
-"  optional binary optionalEnum (ENUM) = 18;" +
-"  optional int32 someInt32 = 19;" +
-"  optional binary someString (UTF8) = 20;" +
-"  optional group optionalMap (MAP) = 21 {\n" +
-"repeated group key_value {\n" +
-"  required int64 key;\n" +
-"  optional group value {\n" +
-"optional int32 someId = 3;\n" +
-"  }\n" +
-"}\n" +
-"  }\n" +
-"}";
+  public void testProto3ConvertAllDatatypes() {
+String expectedSchema = JOINER.join(

Review Comment:
   is it possible to separate this tpe of code style cleanup from functional 
changes?





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

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

2022-10-08 Thread GitBox


emkornfield commented on code in PR #995:
URL: https://github.com/apache/parquet-mr/pull/995#discussion_r990725805


##
parquet-protobuf/src/test/java/org/apache/parquet/proto/ProtoSchemaConverterTest.java:
##
@@ -82,264 +93,447 @@ public void testConvertAllDatatypes() throws Exception {
* Tests that all protocol buffer datatypes are converted to correct parquet 
datatypes.
*/
   @Test
-  public void testProto3ConvertAllDatatypes() throws Exception {
-String expectedSchema =
-  "message TestProto3.SchemaConverterAllDatatypes {\n" +
-"  optional double optionalDouble = 1;\n" +
-"  optional float optionalFloat = 2;\n" +
-"  optional int32 optionalInt32 = 3;\n" +
-"  optional int64 optionalInt64 = 4;\n" +
-"  optional int32 optionalUInt32 = 5;\n" +
-"  optional int64 optionalUInt64 = 6;\n" +
-"  optional int32 optionalSInt32 = 7;\n" +
-"  optional int64 optionalSInt64 = 8;\n" +
-"  optional int32 optionalFixed32 = 9;\n" +
-"  optional int64 optionalFixed64 = 10;\n" +
-"  optional int32 optionalSFixed32 = 11;\n" +
-"  optional int64 optionalSFixed64 = 12;\n" +
-"  optional boolean optionalBool = 13;\n" +
-"  optional binary optionalString (UTF8) = 14;\n" +
-"  optional binary optionalBytes = 15;\n" +
-"  optional group optionalMessage = 16 {\n" +
-"optional int32 someId = 3;\n" +
-"  }\n" +
-"  optional binary optionalEnum (ENUM) = 18;" +
-"  optional int32 someInt32 = 19;" +
-"  optional binary someString (UTF8) = 20;" +
-"  optional group optionalMap (MAP) = 21 {\n" +
-"repeated group key_value {\n" +
-"  required int64 key;\n" +
-"  optional group value {\n" +
-"optional int32 someId = 3;\n" +
-"  }\n" +
-"}\n" +
-"  }\n" +
-"}";
+  public void testProto3ConvertAllDatatypes() {
+String expectedSchema = JOINER.join(

Review Comment:
   is it possible to separate this tpe of code style cleanup from functional 
changes?



-- 
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] emkornfield commented on a diff in pull request #995: PARQUET-1711: support recursive proto schemas by limiting recursion depth

2022-10-08 Thread GitBox


emkornfield commented on code in PR #995:
URL: https://github.com/apache/parquet-mr/pull/995#discussion_r990727030


##
parquet-protobuf/src/test/resources/BinaryTree.par:
##
@@ -0,0 +1,50 @@
+message Trees.BinaryTree {
+  optional group value = 1 {

Review Comment:
   Aren't groups 
[deprecated](https://developers.google.com/protocol-buffers/docs/proto#groups)?



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


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

ASF GitHub Bot commented on PARQUET-1711:
-

emkornfield commented on code in PR #995:
URL: https://github.com/apache/parquet-mr/pull/995#discussion_r990727030


##
parquet-protobuf/src/test/resources/BinaryTree.par:
##
@@ -0,0 +1,50 @@
+message Trees.BinaryTree {
+  optional group value = 1 {

Review Comment:
   Aren't groups 
[deprecated](https://developers.google.com/protocol-buffers/docs/proto#groups)?





> [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] emkornfield commented on a diff in pull request #995: PARQUET-1711: support recursive proto schemas by limiting recursion depth

2022-10-08 Thread GitBox


emkornfield commented on code in PR #995:
URL: https://github.com/apache/parquet-mr/pull/995#discussion_r990726272


##
parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoWriteSupport.java:
##
@@ -559,7 +564,14 @@ final void writeRawValue(Object value) {
   class BinaryWriter extends FieldWriter {
 @Override
 final void writeRawValue(Object value) {
-  ByteString byteString = (ByteString) value;
+  // Non-ByteString values can happen when recursions gets truncated.
+  ByteString byteString = value instanceof ByteString
+  ? (ByteString) value
+  // TODO: figure out a way to use MessageOrBuilder
+  : value instanceof Message
+  ? ((Message) value).toByteString()
+  // Worst-case, just dump as plain java string.
+  : ByteString.copyFromUtf8(value.toString());

Review Comment:
   is this actually an intended state?  If not it is probably better to raise 
an exception then writing data that could possibly be hard to recover.



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


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

ASF GitHub Bot commented on PARQUET-1711:
-

emkornfield commented on code in PR #995:
URL: https://github.com/apache/parquet-mr/pull/995#discussion_r990727498


##
parquet-protobuf/src/test/resources/Trees.proto:
##
@@ -0,0 +1,37 @@
+//
+// 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.
+//
+
+syntax = "proto3";
+
+package Trees;
+
+import "google/protobuf/any.proto";
+
+option java_package = "org.apache.parquet.proto.test";
+
+message BinaryTree {
+google.protobuf.Any value = 1;

Review Comment:
   it would be good to verify that something like:
   
   message WrappedTree {
  google.protobuf.Any non_recursive = 1;
  BinaryTree tree = 2;
   }
   
   Also gives expected results (non_recursive doesn't accidentally trigger any 
of the recursio logic). 
   }





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


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

ASF GitHub Bot commented on PARQUET-1711:
-

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

   Mostly looks reasonable, I'm not too familiar with parquet-mr @shangxinli 
can you recommend someone who might be able to give a better review?




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


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

ASF GitHub Bot commented on PARQUET-1711:
-

emkornfield commented on code in PR #995:
URL: https://github.com/apache/parquet-mr/pull/995#discussion_r990726095


##
parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoSchemaConverter.java:
##
@@ -99,9 +139,9 @@ private Type.Repetition getRepetition(FieldDescriptor 
descriptor) {
 }
   }
 
-  private  Builder>, GroupBuilder> 
addField(FieldDescriptor descriptor, final GroupBuilder builder) {
+  private  Builder>, GroupBuilder> 
addField(FieldDescriptor descriptor, final GroupBuilder builder, 
ImmutableSetMultimap seen, int depth) {

Review Comment:
   right, I was thinking of encapsulating this logic into its own class, so 
they can be recorded and updated together, to  1.  Reduce additional parameters 
that have to be passed through.
   2.  Encapsulate the logic behind more mnemonic method names (e.g. 
AddRecursiveStep())





> [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] emkornfield commented on a diff in pull request #995: PARQUET-1711: support recursive proto schemas by limiting recursion depth

2022-10-08 Thread GitBox


emkornfield commented on code in PR #995:
URL: https://github.com/apache/parquet-mr/pull/995#discussion_r990726095


##
parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoSchemaConverter.java:
##
@@ -99,9 +139,9 @@ private Type.Repetition getRepetition(FieldDescriptor 
descriptor) {
 }
   }
 
-  private  Builder>, GroupBuilder> 
addField(FieldDescriptor descriptor, final GroupBuilder builder) {
+  private  Builder>, GroupBuilder> 
addField(FieldDescriptor descriptor, final GroupBuilder builder, 
ImmutableSetMultimap seen, int depth) {

Review Comment:
   right, I was thinking of encapsulating this logic into its own class, so 
they can be recorded and updated together, to  1.  Reduce additional parameters 
that have to be passed through.
   2.  Encapsulate the logic behind more mnemonic method names (e.g. 
AddRecursiveStep())



-- 
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] emkornfield commented on a diff in pull request #995: PARQUET-1711: support recursive proto schemas by limiting recursion depth

2022-10-08 Thread GitBox


emkornfield commented on code in PR #995:
URL: https://github.com/apache/parquet-mr/pull/995#discussion_r990727498


##
parquet-protobuf/src/test/resources/Trees.proto:
##
@@ -0,0 +1,37 @@
+//
+// 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.
+//
+
+syntax = "proto3";
+
+package Trees;
+
+import "google/protobuf/any.proto";
+
+option java_package = "org.apache.parquet.proto.test";
+
+message BinaryTree {
+google.protobuf.Any value = 1;

Review Comment:
   it would be good to verify that something like:
   
   message WrappedTree {
  google.protobuf.Any non_recursive = 1;
  BinaryTree tree = 2;
   }
   
   Also gives expected results (non_recursive doesn't accidentally trigger any 
of the recursio logic). 
   }



-- 
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] emkornfield commented on pull request #995: PARQUET-1711: support recursive proto schemas by limiting recursion depth

2022-10-08 Thread GitBox


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

   Mostly looks reasonable, I'm not too familiar with parquet-mr @shangxinli 
can you recommend someone who might be able to give a better 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.

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

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