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

2022-10-09 Thread GitBox


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


##
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:
   no, afaict: 
https://www.javadoc.io/doc/org.apache.parquet/parquet-column/latest/org/apache/parquet/io/api/RecordConsumer.html



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

2022-10-09 Thread GitBox


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


##
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:
   i think the existing non-recursive proto tests exercise the existing and 
newly added (the skipping behavior) code paths.



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

2022-10-09 Thread GitBox


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


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

Review Comment:
   this is parquet schema, not proto.  protos should/would have a .proto suffix.



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

2022-10-09 Thread GitBox


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


##
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:
   this is intended.  for a real-time, production pipeline i'm working on, 
losing data as it passes through or killing the job b/c of an uncaught 
exception is problematic as it could lead to data loss and down time.  this 
way, there's some way to know what the problematic data was and fix it properly 
asap.



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

2022-10-09 Thread GitBox


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


##
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:
   wdym by "tpe"?
   
   if this isn't blocking, i'd rather avoid the busy-work to undo and redo in a 
different branch.



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

2022-10-09 Thread GitBox


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


##
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:
   i'm not sure encapsulation helps with readability or protection in this 
case.  they are really tracking different things, and should be understood by 
readers of the traversal code to know how each piece of state is used.



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

2022-09-30 Thread GitBox


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


##
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:
   ?
   
   the `seen` map does encode the seen fields along with their depth as a 
single datastructure.  `depth` being a separate arg is important b/c it's the 
current depth in the traversal, and is used to update the seen data structure.



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

2022-09-30 Thread GitBox


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


##
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:
   ?
   
   the seen map does encode the depth as a single datastructure.  the depth 
being a separate arg is important b/c it's the current depth in the traversal, 
and is used to update the seen data structure.



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

2022-09-30 Thread GitBox


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


##
parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoSchemaConverter.java:
##
@@ -124,35 +164,61 @@ private  Builder>, GroupBuilder> addR
 .named("list");
   }
 
-  private  GroupBuilder> addRepeatedMessage(FieldDescriptor 
descriptor, GroupBuilder builder) {
-GroupBuilder>>> result =
-  builder
+  private  GroupBuilder> addRepeatedMessage(FieldDescriptor 
descriptor, GroupBuilder builder, ImmutableSetMultimap 
seen, int depth) {
+GroupBuilder>>> result = builder
 .group(Type.Repetition.OPTIONAL).as(listType())
 .group(Type.Repetition.REPEATED)
 .group(Type.Repetition.OPTIONAL);
 
-convertFields(result, descriptor.getMessageType().getFields());
+convertFields(result, descriptor.getMessageType().getFields(), seen, 
depth);
 
 return result.named("element").named("list");
   }
 
-  private  GroupBuilder> addMessageField(FieldDescriptor 
descriptor, final GroupBuilder builder) {
+  private  Builder>, GroupBuilder> 
addMessageField(FieldDescriptor descriptor, final GroupBuilder builder, 
ImmutableSetMultimap seen, int depth) {
+// Prevent recursion by terminating with optional proto bytes.
+depth += 1;
+String typeName = getInnerTypeName(descriptor);
+LOG.trace("addMessageField: " + descriptor.getFullName() + " type: " + 
typeName + " depth: " + depth);
+if (typeName != null) {
+  if (seen.get(typeName).size() > maxRecursion) {
+return builder.primitive(BINARY, 
Type.Repetition.OPTIONAL).as((LogicalTypeAnnotation) null);
+  }
+}
+
 if (descriptor.isMapField() && parquetSpecsCompliant) {
   // the old schema style did not include the MAP wrapper around map groups
-  return addMapField(descriptor, builder);
+  return addMapField(descriptor, builder, seen, depth);
 }
+
+seen = ImmutableSetMultimap.builder().putAll(seen).put(typeName, depth).build();

Review Comment:
   it's actually not as costly as you think.  guava's immutable structures are 
written to simply remove method access what not needed, and takes tries its 
best to avoid memory reallocations when using copyOf or builder patterns 
[[1](https://github.com/google/guava/wiki/ImmutableCollectionsExplained)][[2](https://github.com/google/guava/blob/master/guava/src/com/google/common/collect/ImmutableSetMultimap.java#L365)][[3](https://github.com/google/guava/blob/master/guava/src/com/google/common/collect/ImmutableSetMultimap.java#L306)][[4](https://github.com/google/guava/blob/master/guava/src/com/google/common/collect/ImmutableSetMultimap.java#L291)]
 
[generally](https://stackoverflow.com/questions/1284727/mutable-or-immutable-class).
  it's pretty 
[efficient](https://github.com/DimitrisAndreou/memory-measurer/blob/master/ElementCostInDataStructures.txt).
  because of depth first traversal, we do want to "go back" and let the 
previous state of counts start again as the basis for other bra
 nch traversals.  this is exactly the benefit as it helps in avoiding defensive 
copying of mutable data structures or clearing of fields trying to use a single 
instance when traversing and going back up the stack.



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

2022-09-30 Thread GitBox


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


##
parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoSchemaConverter.java:
##
@@ -124,35 +164,61 @@ private  Builder>, GroupBuilder> addR
 .named("list");
   }
 
-  private  GroupBuilder> addRepeatedMessage(FieldDescriptor 
descriptor, GroupBuilder builder) {
-GroupBuilder>>> result =
-  builder
+  private  GroupBuilder> addRepeatedMessage(FieldDescriptor 
descriptor, GroupBuilder builder, ImmutableSetMultimap 
seen, int depth) {
+GroupBuilder>>> result = builder
 .group(Type.Repetition.OPTIONAL).as(listType())
 .group(Type.Repetition.REPEATED)
 .group(Type.Repetition.OPTIONAL);
 
-convertFields(result, descriptor.getMessageType().getFields());
+convertFields(result, descriptor.getMessageType().getFields(), seen, 
depth);
 
 return result.named("element").named("list");
   }
 
-  private  GroupBuilder> addMessageField(FieldDescriptor 
descriptor, final GroupBuilder builder) {
+  private  Builder>, GroupBuilder> 
addMessageField(FieldDescriptor descriptor, final GroupBuilder builder, 
ImmutableSetMultimap seen, int depth) {
+// Prevent recursion by terminating with optional proto bytes.
+depth += 1;
+String typeName = getInnerTypeName(descriptor);
+LOG.trace("addMessageField: " + descriptor.getFullName() + " type: " + 
typeName + " depth: " + depth);

Review Comment:
   you're correct.  however, proto schema conversion shouldn't happen 
repeatedly in the greater flow of a processing job (ideally, just once), so 
this overhead isn't too bad.
   
   i'll move to the parameterized/formatted logging calls in the files i touch 
here.  i would suggest the rest of the codebase do the same to avoid this 
penalty as well, but it's beyond the scope of this pr.



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