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

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


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

ASF GitHub Bot commented on PARQUET-1711:
-

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





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



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

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


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

ASF GitHub Bot commented on PARQUET-1711:
-

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.





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



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

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


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

ASF GitHub Bot commented on PARQUET-1711:
-

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.





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



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

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


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

ASF GitHub Bot commented on PARQUET-1711:
-

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.





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



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

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


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

ASF GitHub Bot commented on PARQUET-1711:
-

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.





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

[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



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

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


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

ASF GitHub Bot commented on PARQUET-1711:
-

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.





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



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

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


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

ASF GitHub Bot commented on PARQUET-1711:
-

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

   > Hi @jinyius and @matthieun, Thank both of you for the contribution and we 
really appreciate your patience with us. Now we have two PRs for the same 
issue, we better merge them into one. Given this PR is earlier, would it be a 
good idea to incorporate #995 into this PR for what is missing? @matthieun can 
add @jinyius as a co-author in that case.
   > 
   > Does it make sense to both of you?
   
   i don't think merging will help here.  both approaches do similar things in 
terms of traversing and expanding out the schema on recursive fields.  the 
differ on the state used during the traversal, and they differ on how to deal 
with the remaining recursive data (this one silently ignores, but the mine 
stores as serialized bytes).
   
   i don't care about authorship.  i want this to get fixed, and fixed properly.
   
   




> [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 #988: PARQUET-1711: Break circular dependencies in proto definitions

2022-10-09 Thread GitBox


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

   > Hi @jinyius and @matthieun, Thank both of you for the contribution and we 
really appreciate your patience with us. Now we have two PRs for the same 
issue, we better merge them into one. Given this PR is earlier, would it be a 
good idea to incorporate #995 into this PR for what is missing? @matthieun can 
add @jinyius as a co-author in that case.
   > 
   > Does it make sense to both of you?
   
   i don't think merging will help here.  both approaches do similar things in 
terms of traversing and expanding out the schema on recursive fields.  the 
differ on the state used during the traversal, and they differ on how to deal 
with the remaining recursive data (this one silently ignores, but the mine 
stores as serialized bytes).
   
   i don't care about authorship.  i want this to get fixed, and fixed properly.
   
   


-- 
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-2202) Redundant String allocation on the hot path in CapacityByteArrayOutputStream.setByte

2022-10-09 Thread Andrei Pangin (Jira)


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

Andrei Pangin updated PARQUET-2202:
---
Description: 
Profiling of a Spark application revealed a performance issue in production:

{{CapacityByteArrayOutputStream.setByte}} consumed 2.2% of total CPU time and 
made up 4.6% of total allocations. However, in normal case, this method should 
allocate nothing at all.

Here is an excerpt from async-profiler report.

CPU profile:

!profile-cpu.png|width=560!

Allocation profile:

!profile-alloc.png|width=560!

The reason is a {{checkArgument()}} call with an unconditionally constructed 
dynamic String:

[https://github.com/apache/parquet-mr/blob/62b774cd0f0c60cfbe540bbfa60bee15929af5d4/parquet-common/src/main/java/org/apache/parquet/bytes/CapacityByteArrayOutputStream.java#L303]

The suggested fix is to move String construction under the condition:
{code:java}
if (index >= bytesUsed) {
  throw new IllegalArgumentException("Index: " + index +
  " is >= the current size of: " + bytesUsed);
}{code}

  was:
Profiling of a Spark application revealed a performance issue in production:

{{CapacityByteArrayOutputStream.setByte}} consumed 2.2% of total CPU time and 
made up 4.6% of total allocations. However, in normal case, this method should 
allocate nothing at all.

Here is an excerpt from async-profiler report.

CPU profile:

!profile-cpu.png|width=600!

Allocation profile:

!profile-alloc.png|width=600!

The reason is a {{checkArgument()}} call with an unconditionally constructed 
dynamic String:

[https://github.com/apache/parquet-mr/blob/62b774cd0f0c60cfbe540bbfa60bee15929af5d4/parquet-common/src/main/java/org/apache/parquet/bytes/CapacityByteArrayOutputStream.java#L303]

The suggested fix is to move String construction under the condition:
{code:java}
if (index >= bytesUsed) {
throw new IllegalArgumentException("Index: " + index + " is >= the current 
size of: " + bytesUsed);
}{code}


> Redundant String allocation on the hot path in 
> CapacityByteArrayOutputStream.setByte
> 
>
> Key: PARQUET-2202
> URL: https://issues.apache.org/jira/browse/PARQUET-2202
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.3
>Reporter: Andrei Pangin
>Priority: Major
>  Labels: performance
> Attachments: profile-alloc.png, profile-cpu.png
>
>
> Profiling of a Spark application revealed a performance issue in production:
> {{CapacityByteArrayOutputStream.setByte}} consumed 2.2% of total CPU time and 
> made up 4.6% of total allocations. However, in normal case, this method 
> should allocate nothing at all.
> Here is an excerpt from async-profiler report.
> CPU profile:
> !profile-cpu.png|width=560!
> Allocation profile:
> !profile-alloc.png|width=560!
> The reason is a {{checkArgument()}} call with an unconditionally constructed 
> dynamic String:
> [https://github.com/apache/parquet-mr/blob/62b774cd0f0c60cfbe540bbfa60bee15929af5d4/parquet-common/src/main/java/org/apache/parquet/bytes/CapacityByteArrayOutputStream.java#L303]
> The suggested fix is to move String construction under the condition:
> {code:java}
> if (index >= bytesUsed) {
>   throw new IllegalArgumentException("Index: " + index +
>   " is >= the current size of: " + bytesUsed);
> }{code}



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


[jira] [Updated] (PARQUET-2202) Redundant String allocation on the hot path in CapacityByteArrayOutputStream.setByte

2022-10-09 Thread Andrei Pangin (Jira)


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

Andrei Pangin updated PARQUET-2202:
---
Description: 
Profiling of a Spark application revealed a performance issue in production:

{{CapacityByteArrayOutputStream.setByte}} consumed 2.2% of total CPU time and 
made up 4.6% of total allocations. However, in normal case, this method should 
allocate nothing at all.

Here is an excerpt from async-profiler report.

CPU profile:

!profile-cpu.png|width=600!

Allocation profile:

!profile-alloc.png|width=600!

The reason is a {{checkArgument()}} call with an unconditionally constructed 
dynamic String:

[https://github.com/apache/parquet-mr/blob/62b774cd0f0c60cfbe540bbfa60bee15929af5d4/parquet-common/src/main/java/org/apache/parquet/bytes/CapacityByteArrayOutputStream.java#L303]

The suggested fix is to move String construction under the condition:
{code:java}
if (index >= bytesUsed) {
throw new IllegalArgumentException("Index: " + index + " is >= the current 
size of: " + bytesUsed);
}{code}

  was:
Profiling of a Spark application revealed a performance issue in production:

{{CapacityByteArrayOutputStream.setByte}} consumed 2.2% of total CPU time and 
made up 4.6% of total allocations. However, in normal case, this method should 
allocate nothing at all.

Here is an excerpt from async-profiler report.

CPU profile:

!profile-cpu.png!

Allocation profile:

!profile-alloc.png!

The reason is a {{checkArgument()}} call with an unconditionally constructed 
dynamic String:

[https://github.com/apache/parquet-mr/blob/62b774cd0f0c60cfbe540bbfa60bee15929af5d4/parquet-common/src/main/java/org/apache/parquet/bytes/CapacityByteArrayOutputStream.java#L303]

The suggested fix is to move String construction under the condition:
{code:java}
if (index >= bytesUsed) {
throw new IllegalArgumentException("Index: " + index + " is >= the current 
size of: " + bytesUsed);
}{code}


> Redundant String allocation on the hot path in 
> CapacityByteArrayOutputStream.setByte
> 
>
> Key: PARQUET-2202
> URL: https://issues.apache.org/jira/browse/PARQUET-2202
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.3
>Reporter: Andrei Pangin
>Priority: Major
>  Labels: performance
> Attachments: profile-alloc.png, profile-cpu.png
>
>
> Profiling of a Spark application revealed a performance issue in production:
> {{CapacityByteArrayOutputStream.setByte}} consumed 2.2% of total CPU time and 
> made up 4.6% of total allocations. However, in normal case, this method 
> should allocate nothing at all.
> Here is an excerpt from async-profiler report.
> CPU profile:
> !profile-cpu.png|width=600!
> Allocation profile:
> !profile-alloc.png|width=600!
> The reason is a {{checkArgument()}} call with an unconditionally constructed 
> dynamic String:
> [https://github.com/apache/parquet-mr/blob/62b774cd0f0c60cfbe540bbfa60bee15929af5d4/parquet-common/src/main/java/org/apache/parquet/bytes/CapacityByteArrayOutputStream.java#L303]
> The suggested fix is to move String construction under the condition:
> {code:java}
> if (index >= bytesUsed) {
> throw new IllegalArgumentException("Index: " + index + " is >= the 
> current size of: " + bytesUsed);
> }{code}



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


[jira] [Created] (PARQUET-2202) Redundant String allocation on the hot path in CapacityByteArrayOutputStream.setByte

2022-10-09 Thread Andrei Pangin (Jira)
Andrei Pangin created PARQUET-2202:
--

 Summary: Redundant String allocation on the hot path in 
CapacityByteArrayOutputStream.setByte
 Key: PARQUET-2202
 URL: https://issues.apache.org/jira/browse/PARQUET-2202
 Project: Parquet
  Issue Type: Bug
  Components: parquet-mr
Affects Versions: 1.12.3
Reporter: Andrei Pangin
 Attachments: profile-alloc.png, profile-cpu.png

Profiling of a Spark application revealed a performance issue in production:

{{CapacityByteArrayOutputStream.setByte}} consumed 2.2% of total CPU time and 
made up 4.6% of total allocations. However, in normal case, this method should 
allocate nothing at all.

Here is an excerpt from async-profiler report.

CPU profile:

!profile-cpu.png!

Allocation profile:

!profile-alloc.png!

The reason is a {{checkArgument()}} call with an unconditionally constructed 
dynamic String:

[https://github.com/apache/parquet-mr/blob/62b774cd0f0c60cfbe540bbfa60bee15929af5d4/parquet-common/src/main/java/org/apache/parquet/bytes/CapacityByteArrayOutputStream.java#L303]

The suggested fix is to move String construction under the condition:
{code:java}
if (index >= bytesUsed) {
throw new IllegalArgumentException("Index: " + index + " is >= the current 
size of: " + bytesUsed);
}{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-09 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1711:
-

shangxinli commented on PR #988:
URL: https://github.com/apache/parquet-mr/pull/988#issuecomment-1272621608

   Hi @jinyius and @matthieun, Thank both of you for the contribution and we 
really appreciate your patience with us. Now we have two PRs for the same 
issue, we better merge them into one. Given this PR is earlier, would it be a 
good idea to incorporate https://github.com/apache/parquet-mr/pull/995 into 
this PR for what is missing? @matthieun can add @jinyius as a co-author in that 
case. 
   
   Does it make sense to both of you? 




> [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] shangxinli commented on pull request #988: PARQUET-1711: Break circular dependencies in proto definitions

2022-10-09 Thread GitBox


shangxinli commented on PR #988:
URL: https://github.com/apache/parquet-mr/pull/988#issuecomment-1272621608

   Hi @jinyius and @matthieun, Thank both of you for the contribution and we 
really appreciate your patience with us. Now we have two PRs for the same 
issue, we better merge them into one. Given this PR is earlier, would it be a 
good idea to incorporate https://github.com/apache/parquet-mr/pull/995 into 
this PR for what is missing? @matthieun can add @jinyius as a co-author in that 
case. 
   
   Does it make sense to both of you? 


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

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] shangxinli commented on pull request #1003: Bump protobuf-java from 3.17.3 to 3.19.6 in /parquet-protobuf

2022-10-09 Thread GitBox


shangxinli commented on PR #1003:
URL: https://github.com/apache/parquet-mr/pull/1003#issuecomment-1272620318

   Not sure what does the 'compatibility' unknown mean?


-- 
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-2156) Column bloom filter: Show bloom filters in tools

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


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

ASF GitHub Bot commented on PARQUET-2156:
-

shangxinli commented on PR #974:
URL: https://github.com/apache/parquet-mr/pull/974#issuecomment-1272610018

   @panbingkun Do you still need this PR open? 




> Column bloom filter: Show bloom filters in tools
> 
>
> Key: PARQUET-2156
> URL: https://issues.apache.org/jira/browse/PARQUET-2156
> Project: Parquet
>  Issue Type: Improvement
>Reporter: BingKun Pan
>Priority: Minor
>
> command result as follow:
> parquet-tools bloom-filter BloomFilter.snappy.parquet
> row-group 0:
> bloom filter for column id:
> NONE
> bloom filter for column uuid:
> Hash strategy: block
> Algorithm: block
> Compression: uncompressed
> Bitset size: 1048576



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


[GitHub] [parquet-mr] shangxinli commented on pull request #974: PARQUET-2156: Column bloom filter: Show bloom filters in tools

2022-10-09 Thread GitBox


shangxinli commented on PR #974:
URL: https://github.com/apache/parquet-mr/pull/974#issuecomment-1272610018

   @panbingkun Do you still need this PR open? 


-- 
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-2142) parquet-cli without hadoop throws java.lang.NoSuchMethodError on any parquet file access command

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


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

ASF GitHub Bot commented on PARQUET-2142:
-

shangxinli merged PR #990:
URL: https://github.com/apache/parquet-mr/pull/990




> parquet-cli without hadoop throws java.lang.NoSuchMethodError on any parquet 
> file access command
> 
>
> Key: PARQUET-2142
> URL: https://issues.apache.org/jira/browse/PARQUET-2142
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-cli
>Affects Versions: 1.13.0
> Environment: Amazon Linux 2, Java 8
>Reporter: Timothy Miller
>Assignee: Kengo Seki
>Priority: Blocker
>
> I can't do even basic things with parquet-cli from 1.13.0-SNAPSHOT.
> Steps to reproduce:
> {noformat}
> git clone  
> cd parquet-mr
> mvn clean install -DskipTests
> cd parquet-cli
> mvn clean install -DskipTests
> mvn dependency:copy-dependencies
> java -cp 'target/*:target/dependency/*' org.apache.parquet.cli.Main cat 
> {noformat}
> Results:
> {noformat}
> Exception in thread "main" java.lang.NoSuchMethodError: 
> org.apache.parquet.avro.AvroSchemaConverter.convert(Lorg/apache/parquet/schema/MessageType;)Lorg/apache/avro/Schema;
>     at org.apache.parquet.cli.util.Schemas.fromParquet(Schemas.java:89)
>     at org.apache.parquet.cli.BaseCommand.getAvroSchema(BaseCommand.java:405)
>     at org.apache.parquet.cli.commands.CatCommand.run(CatCommand.java:66)
>     at org.apache.parquet.cli.Main.run(Main.java:157)
>     at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76)
>     at org.apache.parquet.cli.Main.main(Main.java:187){noformat}



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


[GitHub] [parquet-mr] shangxinli merged pull request #990: PARQUET-2142: Update the parquet-cli document to avoid NoSuchMethodError

2022-10-09 Thread GitBox


shangxinli merged PR #990:
URL: https://github.com/apache/parquet-mr/pull/990


-- 
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-2195) Add scan command to parquet-cli

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


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

ASF GitHub Bot commented on PARQUET-2195:
-

shangxinli commented on code in PR #998:
URL: https://github.com/apache/parquet-mr/pull/998#discussion_r990827325


##
parquet-cli/src/main/java/org/apache/parquet/cli/commands/ScanCommand.java:
##
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.parquet.cli.commands;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.io.Closeables;
+import org.apache.avro.Schema;
+import org.apache.parquet.cli.BaseCommand;
+import org.apache.parquet.cli.util.Expressions;
+import org.slf4j.Logger;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+
+@Parameters(commandDescription = "Scan all records from a file")
+public class ScanCommand extends BaseCommand {
+
+  @Parameter(description = "")
+  List sourceFiles;
+
+  @Parameter(
+names = {"-c", "--column", "--columns"},
+description = "List of columns")
+  List columns;
+
+  public ScanCommand(Logger console) {
+super(console);
+  }
+
+  @Override
+  public int run() throws IOException {
+Preconditions.checkArgument(
+  sourceFiles != null && !sourceFiles.isEmpty(),
+  "Missing file name");
+Preconditions.checkArgument(sourceFiles.size() == 1,
+  "Only one file can be given");
+
+final String source = sourceFiles.get(0);
+Schema schema = getAvroSchema(source);
+Schema projection = Expressions.filterSchema(schema, columns);
+
+long startTime = System.currentTimeMillis();
+Iterable reader = openDataFile(source, projection);
+boolean threw = true;
+long count = 0;
+try {
+  for (Object record : reader) {
+count += 1;

Review Comment:
   If your goal is only to get count, why not get it from the metadata? Iterate 
each record is an expensive operation. 





> Add scan command to parquet-cli
> ---
>
> Key: PARQUET-2195
> URL: https://issues.apache.org/jira/browse/PARQUET-2195
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-cli
>Reporter: Gang Wu
>Priority: Major
>
> parquet-cli has *cat* and *head* commands to print the records but it does 
> not have the capability to *scan* (w/o printing) all records to check if the 
> file is corrupted.



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


[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #998: PARQUET-2195: Add scan command to parquet-cli

2022-10-09 Thread GitBox


shangxinli commented on code in PR #998:
URL: https://github.com/apache/parquet-mr/pull/998#discussion_r990827325


##
parquet-cli/src/main/java/org/apache/parquet/cli/commands/ScanCommand.java:
##
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.parquet.cli.commands;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.io.Closeables;
+import org.apache.avro.Schema;
+import org.apache.parquet.cli.BaseCommand;
+import org.apache.parquet.cli.util.Expressions;
+import org.slf4j.Logger;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+
+@Parameters(commandDescription = "Scan all records from a file")
+public class ScanCommand extends BaseCommand {
+
+  @Parameter(description = "")
+  List sourceFiles;
+
+  @Parameter(
+names = {"-c", "--column", "--columns"},
+description = "List of columns")
+  List columns;
+
+  public ScanCommand(Logger console) {
+super(console);
+  }
+
+  @Override
+  public int run() throws IOException {
+Preconditions.checkArgument(
+  sourceFiles != null && !sourceFiles.isEmpty(),
+  "Missing file name");
+Preconditions.checkArgument(sourceFiles.size() == 1,
+  "Only one file can be given");
+
+final String source = sourceFiles.get(0);
+Schema schema = getAvroSchema(source);
+Schema projection = Expressions.filterSchema(schema, columns);
+
+long startTime = System.currentTimeMillis();
+Iterable reader = openDataFile(source, projection);
+boolean threw = true;
+long count = 0;
+try {
+  for (Object record : reader) {
+count += 1;

Review Comment:
   If your goal is only to get count, why not get it from the metadata? Iterate 
each record is an expensive operation. 



-- 
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-2195) Add scan command to parquet-cli

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


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

ASF GitHub Bot commented on PARQUET-2195:
-

shangxinli commented on code in PR #998:
URL: https://github.com/apache/parquet-mr/pull/998#discussion_r990827069


##
parquet-cli/src/main/java/org/apache/parquet/cli/commands/ScanCommand.java:
##
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.parquet.cli.commands;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.io.Closeables;
+import org.apache.avro.Schema;
+import org.apache.parquet.cli.BaseCommand;
+import org.apache.parquet.cli.util.Expressions;
+import org.slf4j.Logger;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+
+@Parameters(commandDescription = "Scan all records from a file")
+public class ScanCommand extends BaseCommand {
+
+  @Parameter(description = "")
+  List sourceFiles;
+
+  @Parameter(
+names = {"-c", "--column", "--columns"},
+description = "List of columns")
+  List columns;
+
+  public ScanCommand(Logger console) {
+super(console);
+  }
+
+  @Override
+  public int run() throws IOException {
+Preconditions.checkArgument(
+  sourceFiles != null && !sourceFiles.isEmpty(),
+  "Missing file name");
+Preconditions.checkArgument(sourceFiles.size() == 1,
+  "Only one file can be given");
+
+final String source = sourceFiles.get(0);
+Schema schema = getAvroSchema(source);
+Schema projection = Expressions.filterSchema(schema, columns);

Review Comment:
   What do we do if the file that doesn't have Avro Schema, 



##
parquet-cli/src/main/java/org/apache/parquet/cli/commands/ScanCommand.java:
##
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.parquet.cli.commands;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.io.Closeables;
+import org.apache.avro.Schema;
+import org.apache.parquet.cli.BaseCommand;
+import org.apache.parquet.cli.util.Expressions;
+import org.slf4j.Logger;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+
+@Parameters(commandDescription = "Scan all records from a file")
+public class ScanCommand extends BaseCommand {
+
+  @Parameter(description = "")
+  List sourceFiles;
+
+  @Parameter(
+names = {"-c", "--column", "--columns"},
+description = "List of columns")
+  List columns;
+
+  public ScanCommand(Logger console) {
+super(console);
+  }
+
+  @Override
+  public int run() throws IOException {
+Preconditions.checkArgument(
+  sourceFiles != null && !sourceFiles.isEmpty(),
+  "Missing file name");
+Preconditions.checkArgument(sourceFiles.size() == 1,
+  "Only one file can be given");
+
+final String source = sourceFiles.get(0);
+Schema schema = getAvroSchema(source);
+Schema projection = Expressions.filterSchema(schema, columns);

Review Comment:
   What do we do if the file that doesn't have Avro Schema?





> Add scan command to parquet-cli
> 

[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #998: PARQUET-2195: Add scan command to parquet-cli

2022-10-09 Thread GitBox


shangxinli commented on code in PR #998:
URL: https://github.com/apache/parquet-mr/pull/998#discussion_r990827069


##
parquet-cli/src/main/java/org/apache/parquet/cli/commands/ScanCommand.java:
##
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.parquet.cli.commands;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.io.Closeables;
+import org.apache.avro.Schema;
+import org.apache.parquet.cli.BaseCommand;
+import org.apache.parquet.cli.util.Expressions;
+import org.slf4j.Logger;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+
+@Parameters(commandDescription = "Scan all records from a file")
+public class ScanCommand extends BaseCommand {
+
+  @Parameter(description = "")
+  List sourceFiles;
+
+  @Parameter(
+names = {"-c", "--column", "--columns"},
+description = "List of columns")
+  List columns;
+
+  public ScanCommand(Logger console) {
+super(console);
+  }
+
+  @Override
+  public int run() throws IOException {
+Preconditions.checkArgument(
+  sourceFiles != null && !sourceFiles.isEmpty(),
+  "Missing file name");
+Preconditions.checkArgument(sourceFiles.size() == 1,
+  "Only one file can be given");
+
+final String source = sourceFiles.get(0);
+Schema schema = getAvroSchema(source);
+Schema projection = Expressions.filterSchema(schema, columns);

Review Comment:
   What do we do if the file that doesn't have Avro Schema, 



##
parquet-cli/src/main/java/org/apache/parquet/cli/commands/ScanCommand.java:
##
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.parquet.cli.commands;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.io.Closeables;
+import org.apache.avro.Schema;
+import org.apache.parquet.cli.BaseCommand;
+import org.apache.parquet.cli.util.Expressions;
+import org.slf4j.Logger;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+
+@Parameters(commandDescription = "Scan all records from a file")
+public class ScanCommand extends BaseCommand {
+
+  @Parameter(description = "")
+  List sourceFiles;
+
+  @Parameter(
+names = {"-c", "--column", "--columns"},
+description = "List of columns")
+  List columns;
+
+  public ScanCommand(Logger console) {
+super(console);
+  }
+
+  @Override
+  public int run() throws IOException {
+Preconditions.checkArgument(
+  sourceFiles != null && !sourceFiles.isEmpty(),
+  "Missing file name");
+Preconditions.checkArgument(sourceFiles.size() == 1,
+  "Only one file can be given");
+
+final String source = sourceFiles.get(0);
+Schema schema = getAvroSchema(source);
+Schema projection = Expressions.filterSchema(schema, columns);

Review Comment:
   What do we do if the file that doesn't have Avro Schema?



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

[jira] [Commented] (PARQUET-2195) Add scan command to parquet-cli

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


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

ASF GitHub Bot commented on PARQUET-2195:
-

shangxinli commented on code in PR #998:
URL: https://github.com/apache/parquet-mr/pull/998#discussion_r990826841


##
parquet-cli/src/main/java/org/apache/parquet/cli/commands/ScanCommand.java:
##
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.parquet.cli.commands;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.io.Closeables;
+import org.apache.avro.Schema;
+import org.apache.parquet.cli.BaseCommand;
+import org.apache.parquet.cli.util.Expressions;
+import org.slf4j.Logger;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+
+@Parameters(commandDescription = "Scan all records from a file")
+public class ScanCommand extends BaseCommand {
+
+  @Parameter(description = "")
+  List sourceFiles;
+
+  @Parameter(
+names = {"-c", "--column", "--columns"},
+description = "List of columns")
+  List columns;
+
+  public ScanCommand(Logger console) {
+super(console);
+  }
+
+  @Override
+  public int run() throws IOException {
+Preconditions.checkArgument(
+  sourceFiles != null && !sourceFiles.isEmpty(),
+  "Missing file name");
+Preconditions.checkArgument(sourceFiles.size() == 1,
+  "Only one file can be given");

Review Comment:
   Why not define it as String instead of List? 





> Add scan command to parquet-cli
> ---
>
> Key: PARQUET-2195
> URL: https://issues.apache.org/jira/browse/PARQUET-2195
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-cli
>Reporter: Gang Wu
>Priority: Major
>
> parquet-cli has *cat* and *head* commands to print the records but it does 
> not have the capability to *scan* (w/o printing) all records to check if the 
> file is corrupted.



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


[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #998: PARQUET-2195: Add scan command to parquet-cli

2022-10-09 Thread GitBox


shangxinli commented on code in PR #998:
URL: https://github.com/apache/parquet-mr/pull/998#discussion_r990826841


##
parquet-cli/src/main/java/org/apache/parquet/cli/commands/ScanCommand.java:
##
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.parquet.cli.commands;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.io.Closeables;
+import org.apache.avro.Schema;
+import org.apache.parquet.cli.BaseCommand;
+import org.apache.parquet.cli.util.Expressions;
+import org.slf4j.Logger;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+
+@Parameters(commandDescription = "Scan all records from a file")
+public class ScanCommand extends BaseCommand {
+
+  @Parameter(description = "")
+  List sourceFiles;
+
+  @Parameter(
+names = {"-c", "--column", "--columns"},
+description = "List of columns")
+  List columns;
+
+  public ScanCommand(Logger console) {
+super(console);
+  }
+
+  @Override
+  public int run() throws IOException {
+Preconditions.checkArgument(
+  sourceFiles != null && !sourceFiles.isEmpty(),
+  "Missing file name");
+Preconditions.checkArgument(sourceFiles.size() == 1,
+  "Only one file can be given");

Review Comment:
   Why not define it as String instead of List? 



-- 
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] shangxinli merged pull request #962: Performance optimization to ByteBitPackingValuesReader

2022-10-09 Thread GitBox


shangxinli merged PR #962:
URL: https://github.com/apache/parquet-mr/pull/962


-- 
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-2176) Parquet writers should allow for configurable index/statistics truncation

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


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

ASF GitHub Bot commented on PARQUET-2176:
-

shangxinli merged PR #989:
URL: https://github.com/apache/parquet-mr/pull/989




> Parquet writers should allow for configurable index/statistics truncation
> -
>
> Key: PARQUET-2176
> URL: https://issues.apache.org/jira/browse/PARQUET-2176
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Affects Versions: 1.12.3
>Reporter: patchwork01
>Priority: Major
>
> ParquetWriter does not expose any way to set the properties for column index 
> or statistics truncation.
> With ParquetOutputFormat those can be set with 
> parquet.columnindex.truncate.length and parquet.statistics.truncate.length. 
> These are not applied for ParquetWriter.
> These properties are documented here: 
> [https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/README.md]



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


[GitHub] [parquet-mr] shangxinli merged pull request #989: PARQUET-2176: Column index/statistics truncation in ParquetWriter

2022-10-09 Thread GitBox


shangxinli merged PR #989:
URL: https://github.com/apache/parquet-mr/pull/989


-- 
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-2196) Support LZ4_RAW codec

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


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

ASF GitHub Bot commented on PARQUET-2196:
-

shangxinli commented on code in PR #1000:
URL: https://github.com/apache/parquet-mr/pull/1000#discussion_r990824756


##
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/codec/TestInteropReadLz4RawCodec.java:
##
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.parquet.hadoop.codec;
+
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.example.GroupReadSupport;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestInteropReadLz4RawCodec {
+
+  // The link includes a reference to a specific commit. To take a newer 
version - update this link.
+  private static final String PARQUET_TESTING_REPO = 
"https://github.com/apache/parquet-testing/raw/19fcd4d/data/;;
+  private static String PARQUET_TESTING_PATH = "target/parquet-testing/data";
+  private static String SIMPLE_FILE = "lz4_raw_compressed.parquet";
+  private static String LARGER_FILE = "lz4_raw_compressed_larger.parquet";
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(TestInteropReadLz4RawCodec.class);
+  private OkHttpClient httpClient = new OkHttpClient();
+
+  @Test
+  public void testInteropReadLz4RawParquetFiles() throws IOException {
+Path rootPath = new Path(PARQUET_TESTING_PATH);
+LOG.info(" testInteropReadLz4RawParquetFiles {} ", 
rootPath.toString());
+
+// Test simple parquet file with lz4 raw compressed
+Path simpleFile = downloadInteropFiles(rootPath, SIMPLE_FILE, httpClient);
+readParquetFile(simpleFile, 4);
+
+// Test larger parquet file with lz4 raw compressed
+Path largerFile = downloadInteropFiles(rootPath, LARGER_FILE, httpClient);
+readParquetFile(largerFile, 1);
+  }
+
+  private Path downloadInteropFiles(Path rootPath, String fileName, 
OkHttpClient httpClient) throws IOException {
+LOG.info("Download interop files if needed");
+Configuration conf = new Configuration();
+FileSystem fs = rootPath.getFileSystem(conf);
+LOG.info(rootPath + " exists?: " + fs.exists(rootPath));
+if (!fs.exists(rootPath)) {
+  LOG.info("Create folder for interop files: " + rootPath);
+  if (!fs.mkdirs(rootPath)) {
+throw new IOException("Cannot create path " + rootPath);
+  }
+}
+
+Path file = new Path(rootPath, fileName);
+if (!fs.exists(file)) {
+  String downloadUrl = PARQUET_TESTING_REPO + fileName;
+  LOG.info("Download interop file: " + downloadUrl);
+  Request request = new Request.Builder().url(downloadUrl).build();

Review Comment:
   That would be the last option. Too much downloading will make the build 
unreliable. 





> Support LZ4_RAW codec
> -
>
> Key: PARQUET-2196
> URL: https://issues.apache.org/jira/browse/PARQUET-2196
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Reporter: Gang Wu
>Priority: Major
>
> There is a long history about the LZ4 interoperability of parquet files 
> between parquet-mr and parquet-cpp (which is now in the Apache Arrow). 
> Attached links are the evidence. In short, a new LZ4_RAW codec type has been 
> introduced since parquet format v2.9.0. However, only parquet-cpp supports 
> LZ4_RAW. The parquet-mr library still uses the old Hadoop-provided LZ4 codec 
> and cannot read parquet files with LZ4_RAW.



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


[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #1000: PARQUET-2196: Support LZ4_RAW codec

2022-10-09 Thread GitBox


shangxinli commented on code in PR #1000:
URL: https://github.com/apache/parquet-mr/pull/1000#discussion_r990824756


##
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/codec/TestInteropReadLz4RawCodec.java:
##
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.parquet.hadoop.codec;
+
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.Response;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.example.GroupReadSupport;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestInteropReadLz4RawCodec {
+
+  // The link includes a reference to a specific commit. To take a newer 
version - update this link.
+  private static final String PARQUET_TESTING_REPO = 
"https://github.com/apache/parquet-testing/raw/19fcd4d/data/;;
+  private static String PARQUET_TESTING_PATH = "target/parquet-testing/data";
+  private static String SIMPLE_FILE = "lz4_raw_compressed.parquet";
+  private static String LARGER_FILE = "lz4_raw_compressed_larger.parquet";
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(TestInteropReadLz4RawCodec.class);
+  private OkHttpClient httpClient = new OkHttpClient();
+
+  @Test
+  public void testInteropReadLz4RawParquetFiles() throws IOException {
+Path rootPath = new Path(PARQUET_TESTING_PATH);
+LOG.info(" testInteropReadLz4RawParquetFiles {} ", 
rootPath.toString());
+
+// Test simple parquet file with lz4 raw compressed
+Path simpleFile = downloadInteropFiles(rootPath, SIMPLE_FILE, httpClient);
+readParquetFile(simpleFile, 4);
+
+// Test larger parquet file with lz4 raw compressed
+Path largerFile = downloadInteropFiles(rootPath, LARGER_FILE, httpClient);
+readParquetFile(largerFile, 1);
+  }
+
+  private Path downloadInteropFiles(Path rootPath, String fileName, 
OkHttpClient httpClient) throws IOException {
+LOG.info("Download interop files if needed");
+Configuration conf = new Configuration();
+FileSystem fs = rootPath.getFileSystem(conf);
+LOG.info(rootPath + " exists?: " + fs.exists(rootPath));
+if (!fs.exists(rootPath)) {
+  LOG.info("Create folder for interop files: " + rootPath);
+  if (!fs.mkdirs(rootPath)) {
+throw new IOException("Cannot create path " + rootPath);
+  }
+}
+
+Path file = new Path(rootPath, fileName);
+if (!fs.exists(file)) {
+  String downloadUrl = PARQUET_TESTING_REPO + fileName;
+  LOG.info("Download interop file: " + downloadUrl);
+  Request request = new Request.Builder().url(downloadUrl).build();

Review Comment:
   That would be the last option. Too much downloading will make the build 
unreliable. 



-- 
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-2196) Support LZ4_RAW codec

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


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

ASF GitHub Bot commented on PARQUET-2196:
-

shangxinli commented on code in PR #1000:
URL: https://github.com/apache/parquet-mr/pull/1000#discussion_r990824600


##
parquet-hadoop/pom.xml:
##
@@ -102,6 +102,11 @@
   jar
   compile
 
+
+  io.airlift

Review Comment:
   Generally, we are strict to add more dependencies. Do we know if this 
dependency also brings in other dependencies too? What total libraries were 
introduced and what is their footprint of them? 





> Support LZ4_RAW codec
> -
>
> Key: PARQUET-2196
> URL: https://issues.apache.org/jira/browse/PARQUET-2196
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Reporter: Gang Wu
>Priority: Major
>
> There is a long history about the LZ4 interoperability of parquet files 
> between parquet-mr and parquet-cpp (which is now in the Apache Arrow). 
> Attached links are the evidence. In short, a new LZ4_RAW codec type has been 
> introduced since parquet format v2.9.0. However, only parquet-cpp supports 
> LZ4_RAW. The parquet-mr library still uses the old Hadoop-provided LZ4 codec 
> and cannot read parquet files with LZ4_RAW.



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


[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #1000: PARQUET-2196: Support LZ4_RAW codec

2022-10-09 Thread GitBox


shangxinli commented on code in PR #1000:
URL: https://github.com/apache/parquet-mr/pull/1000#discussion_r990824600


##
parquet-hadoop/pom.xml:
##
@@ -102,6 +102,11 @@
   jar
   compile
 
+
+  io.airlift

Review Comment:
   Generally, we are strict to add more dependencies. Do we know if this 
dependency also brings in other dependencies too? What total libraries were 
introduced and what is their footprint of them? 



-- 
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-2196) Support LZ4_RAW codec

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


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

ASF GitHub Bot commented on PARQUET-2196:
-

shangxinli commented on code in PR #1000:
URL: https://github.com/apache/parquet-mr/pull/1000#discussion_r990824259


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/Lz4RawCodec.java:
##
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.parquet.hadoop.codec;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.compress.*;

Review Comment:
   Please don't use wildcard





> Support LZ4_RAW codec
> -
>
> Key: PARQUET-2196
> URL: https://issues.apache.org/jira/browse/PARQUET-2196
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Reporter: Gang Wu
>Priority: Major
>
> There is a long history about the LZ4 interoperability of parquet files 
> between parquet-mr and parquet-cpp (which is now in the Apache Arrow). 
> Attached links are the evidence. In short, a new LZ4_RAW codec type has been 
> introduced since parquet format v2.9.0. However, only parquet-cpp supports 
> LZ4_RAW. The parquet-mr library still uses the old Hadoop-provided LZ4 codec 
> and cannot read parquet files with LZ4_RAW.



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


[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #1000: PARQUET-2196: Support LZ4_RAW codec

2022-10-09 Thread GitBox


shangxinli commented on code in PR #1000:
URL: https://github.com/apache/parquet-mr/pull/1000#discussion_r990824259


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/Lz4RawCodec.java:
##
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.parquet.hadoop.codec;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.compress.*;

Review Comment:
   Please don't use wildcard



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