[jira] [Commented] (PARQUET-1222) Specify a well-defined sorting order for float and double types

2022-09-30 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-1222:
---

[~emkornfield], I think we do not need to handle NaN values with a boolean to 
fix this issue. NaN is kind of similar than null values so we may even count 
them instead of having a boolean but this question is not tightly related to 
this topic.
What do you think about elevating the current suggestion in the thrift file to 
specification level for writing/reading FP min/max values?
{quote}Because the sorting order is not specified properly for floating point 
values (relations vs. total ordering) the following compatibility rules should 
be applied when reading statistics:
* If the min is a NaN, it should be ignored.
* If the max is a NaN, it should be ignored.
* If the min is +0, the row group may contain -0 values as well.
* If the max is -0, the row group may contain +0 values as well.
* When looking for NaN values, min and max should be ignored.{quote}
For writing we shall skip NaN values and use -0 for min and +0 for max any time 
when a 0 is to be taken into account.

With this solution we cannot do anything clever in case of searching for a NaN 
but it can be fixed separately. And we also need to double-check whether we 
really ignore the min/max stats in case of searching for a NaN.

I think it is a good idea to discuss such topics on the mailing list. However, 
we should also time-box the discussion and go forward with a proposed solution 
if there are no interests on the mailing list. (Personally, I do not follow the 
dev list anymore.)


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



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


[jira] [Commented] (PARQUET-2196) Support LZ4_RAW codec

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


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-cli/src/main/java/org/apache/parquet/cli/Util.java:
##
@@ -151,6 +151,8 @@ public static String shortCodec(CompressionCodecName codec) 
{
 return "B";
   case LZ4:
 return "4";
+  case LZ4_RAW:
+return "4";

Review Comment:
   In fact they are the same compression except that the LZ4_HADOOP adds extra 
8 bytes for uncompressed size and compressed size. Check here for reference: 
https://github.com/airlift/aircompressor/blob/master/src/main/java/io/airlift/compress/lz4/HadoopLz4OutputStream.java#L97



##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/Lz4RawCompressor.java:
##
@@ -0,0 +1,44 @@
+/* 
+ * 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 io.airlift.compress.lz4.Lz4Compressor;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class Lz4RawCompressor extends NonBlockedCompressor {
+
+  private Lz4Compressor compressor = new Lz4Compressor();

Review Comment:
   I don't think this is an issue. parquet-mr creates CompressionInputStream 
and CompressionOutputStream based on new allocated Compressor and Decompressor 
objects.  ZstandardCodec even returns null for the createCompressor and 
createDecompressor methods.



##
parquet-common/src/main/java/org/apache/parquet/hadoop/metadata/CompressionCodecName.java:
##
@@ -30,7 +30,8 @@ public enum CompressionCodecName {
   LZO("com.hadoop.compression.lzo.LzoCodec", CompressionCodec.LZO, ".lzo"),
   BROTLI("org.apache.hadoop.io.compress.BrotliCodec", CompressionCodec.BROTLI, 
".br"),
   LZ4("org.apache.hadoop.io.compress.Lz4Codec", CompressionCodec.LZ4, ".lz4"),
-  ZSTD("org.apache.parquet.hadoop.codec.ZstandardCodec", 
CompressionCodec.ZSTD, ".zstd");
+  ZSTD("org.apache.parquet.hadoop.codec.ZstandardCodec", 
CompressionCodec.ZSTD, ".zstd"),
+  LZ4_RAW("org.apache.parquet.hadoop.codec.Lz4RawCodec", 
CompressionCodec.LZ4_RAW, ".lz4");

Review Comment:
   I haven't found the suffix parity in the C++ implementation. But I can 
switch it to "lz4_raw" if that makes sense: 
https://github.com/apache/arrow/blob/master/cpp/src/arrow/util/compression.cc#L51



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

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

2022-09-30 Thread GitBox


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


##
parquet-common/src/main/java/org/apache/parquet/hadoop/metadata/CompressionCodecName.java:
##
@@ -30,7 +30,8 @@ public enum CompressionCodecName {
   LZO("com.hadoop.compression.lzo.LzoCodec", CompressionCodec.LZO, ".lzo"),
   BROTLI("org.apache.hadoop.io.compress.BrotliCodec", CompressionCodec.BROTLI, 
".br"),
   LZ4("org.apache.hadoop.io.compress.Lz4Codec", CompressionCodec.LZ4, ".lz4"),
-  ZSTD("org.apache.parquet.hadoop.codec.ZstandardCodec", 
CompressionCodec.ZSTD, ".zstd");
+  ZSTD("org.apache.parquet.hadoop.codec.ZstandardCodec", 
CompressionCodec.ZSTD, ".zstd"),
+  LZ4_RAW("org.apache.parquet.hadoop.codec.Lz4RawCodec", 
CompressionCodec.LZ4_RAW, ".lz4");

Review Comment:
   We don't register file extensions in C++.
   That said, ".lz4" probably can't be appropriate for the Hadoop LZ4 codec.



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


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

ASF GitHub Bot commented on PARQUET-2196:
-

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


##
parquet-common/src/main/java/org/apache/parquet/hadoop/metadata/CompressionCodecName.java:
##
@@ -30,7 +30,8 @@ public enum CompressionCodecName {
   LZO("com.hadoop.compression.lzo.LzoCodec", CompressionCodec.LZO, ".lzo"),
   BROTLI("org.apache.hadoop.io.compress.BrotliCodec", CompressionCodec.BROTLI, 
".br"),
   LZ4("org.apache.hadoop.io.compress.Lz4Codec", CompressionCodec.LZ4, ".lz4"),
-  ZSTD("org.apache.parquet.hadoop.codec.ZstandardCodec", 
CompressionCodec.ZSTD, ".zstd");
+  ZSTD("org.apache.parquet.hadoop.codec.ZstandardCodec", 
CompressionCodec.ZSTD, ".zstd"),
+  LZ4_RAW("org.apache.parquet.hadoop.codec.Lz4RawCodec", 
CompressionCodec.LZ4_RAW, ".lz4");

Review Comment:
   We don't register file extensions in C++.
   That said, ".lz4" probably can't be appropriate for the Hadoop LZ4 codec.





> 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] wgtmac commented on a diff in pull request #1000: PARQUET-2196: Support LZ4_RAW codec

2022-09-30 Thread GitBox


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


##
parquet-cli/src/main/java/org/apache/parquet/cli/Util.java:
##
@@ -151,6 +151,8 @@ public static String shortCodec(CompressionCodecName codec) 
{
 return "B";
   case LZ4:
 return "4";
+  case LZ4_RAW:
+return "4";

Review Comment:
   In fact they are the same compression except that the LZ4_HADOOP adds extra 
8 bytes for uncompressed size and compressed size. Check here for reference: 
https://github.com/airlift/aircompressor/blob/master/src/main/java/io/airlift/compress/lz4/HadoopLz4OutputStream.java#L97



##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/Lz4RawCompressor.java:
##
@@ -0,0 +1,44 @@
+/* 
+ * 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 io.airlift.compress.lz4.Lz4Compressor;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class Lz4RawCompressor extends NonBlockedCompressor {
+
+  private Lz4Compressor compressor = new Lz4Compressor();

Review Comment:
   I don't think this is an issue. parquet-mr creates CompressionInputStream 
and CompressionOutputStream based on new allocated Compressor and Decompressor 
objects.  ZstandardCodec even returns null for the createCompressor and 
createDecompressor methods.



##
parquet-common/src/main/java/org/apache/parquet/hadoop/metadata/CompressionCodecName.java:
##
@@ -30,7 +30,8 @@ public enum CompressionCodecName {
   LZO("com.hadoop.compression.lzo.LzoCodec", CompressionCodec.LZO, ".lzo"),
   BROTLI("org.apache.hadoop.io.compress.BrotliCodec", CompressionCodec.BROTLI, 
".br"),
   LZ4("org.apache.hadoop.io.compress.Lz4Codec", CompressionCodec.LZ4, ".lz4"),
-  ZSTD("org.apache.parquet.hadoop.codec.ZstandardCodec", 
CompressionCodec.ZSTD, ".zstd");
+  ZSTD("org.apache.parquet.hadoop.codec.ZstandardCodec", 
CompressionCodec.ZSTD, ".zstd"),
+  LZ4_RAW("org.apache.parquet.hadoop.codec.Lz4RawCodec", 
CompressionCodec.LZ4_RAW, ".lz4");

Review Comment:
   I haven't found the suffix parity in the C++ implementation. But I can 
switch it to "lz4_raw" if that makes sense: 
https://github.com/apache/arrow/blob/master/cpp/src/arrow/util/compression.cc#L51



##
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 = "targ

[jira] [Commented] (PARQUET-1222) Specify a well-defined sorting order for float and double types

2022-09-30 Thread Antoine Pitrou (Jira)


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

Antoine Pitrou commented on PARQUET-1222:
-

(side note: the ML is mostly a firehose of notifications nowadays, which 
doesn't make it easy to follow...)

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



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


[jira] [Commented] (PARQUET-1222) Specify a well-defined sorting order for float and double types

2022-09-30 Thread Antoine Pitrou (Jira)


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

Antoine Pitrou commented on PARQUET-1222:
-

I agree with [~gszadovszky] for elevating these rules at the specification 
level.

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



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


[GitHub] [parquet-mr] ggershinsky commented on a diff in pull request #959: PARQUET-2126: Make cached (de)compressors thread-safe

2022-09-30 Thread GitBox


ggershinsky commented on code in PR #959:
URL: https://github.com/apache/parquet-mr/pull/959#discussion_r984394753


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/CodecFactory.java:
##
@@ -244,16 +272,60 @@ protected CompressionCodec getCodec(CompressionCodecName 
codecName) {
 }
   }
 
+  /**
+   * Modified for https://issues.apache.org/jira/browse/PARQUET-2126
+   * This releases all cached instances of all compressors and decompressors 
created by all threads that share
+   * this CodeFactory instance.
+   * Note: A problem might occur if release() were called while some codec 
instances were still in use, but it
+   * would not make sense to call close() or release() on a shared 
CodecFactory while some threads are still
+   * actively using it. The usage pattern should be:

Review Comment:
   It would be also good to have a look at an example of such callers, eg in 
Apache Spark, to start analyzing the implications.



##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/CodecFactory.java:
##
@@ -244,16 +272,60 @@ protected CompressionCodec getCodec(CompressionCodecName 
codecName) {
 }
   }
 
+  /**
+   * Modified for https://issues.apache.org/jira/browse/PARQUET-2126
+   * This releases all cached instances of all compressors and decompressors 
created by all threads that share
+   * this CodeFactory instance.
+   * Note: A problem might occur if release() were called while some codec 
instances were still in use, but it
+   * would not make sense to call close() or release() on a shared 
CodecFactory while some threads are still
+   * actively using it. The usage pattern should be:

Review Comment:
   regarding the code parts that call this release() method. For those parts 
inside the parquet-mr codebase, can  some of them implement/enforce this 
pattern fully internally? I'd guess most/all of them eventually pass this 
responsibility to the "app" code above parquet-mr API; so probably this pattern 
documentation should be moved/copied/referenced in these APIs. 



-- 
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-2126) Thread safety bug in CodecFactory

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


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

ASF GitHub Bot commented on PARQUET-2126:
-

ggershinsky commented on code in PR #959:
URL: https://github.com/apache/parquet-mr/pull/959#discussion_r984394753


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/CodecFactory.java:
##
@@ -244,16 +272,60 @@ protected CompressionCodec getCodec(CompressionCodecName 
codecName) {
 }
   }
 
+  /**
+   * Modified for https://issues.apache.org/jira/browse/PARQUET-2126
+   * This releases all cached instances of all compressors and decompressors 
created by all threads that share
+   * this CodeFactory instance.
+   * Note: A problem might occur if release() were called while some codec 
instances were still in use, but it
+   * would not make sense to call close() or release() on a shared 
CodecFactory while some threads are still
+   * actively using it. The usage pattern should be:

Review Comment:
   It would be also good to have a look at an example of such callers, eg in 
Apache Spark, to start analyzing the implications.



##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/CodecFactory.java:
##
@@ -244,16 +272,60 @@ protected CompressionCodec getCodec(CompressionCodecName 
codecName) {
 }
   }
 
+  /**
+   * Modified for https://issues.apache.org/jira/browse/PARQUET-2126
+   * This releases all cached instances of all compressors and decompressors 
created by all threads that share
+   * this CodeFactory instance.
+   * Note: A problem might occur if release() were called while some codec 
instances were still in use, but it
+   * would not make sense to call close() or release() on a shared 
CodecFactory while some threads are still
+   * actively using it. The usage pattern should be:

Review Comment:
   regarding the code parts that call this release() method. For those parts 
inside the parquet-mr codebase, can  some of them implement/enforce this 
pattern fully internally? I'd guess most/all of them eventually pass this 
responsibility to the "app" code above parquet-mr API; so probably this pattern 
documentation should be moved/copied/referenced in these APIs. 





> Thread safety bug in CodecFactory
> -
>
> Key: PARQUET-2126
> URL: https://issues.apache.org/jira/browse/PARQUET-2126
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.2
>Reporter: James Turton
>Priority: Major
>
> The code for returning Compressor objects to the caller goes to some lengths 
> to achieve thread safety, including keeping Codec objects in an Apache 
> Commons pool that has thread-safe borrow semantics.  This is all undone by 
> the BytesCompressor and BytesDecompressor Maps in 
> org.apache.parquet.hadoop.CodecFactory which end up caching single compressor 
> and decompressor instances due to code in CodecFactory@getCompressor and 
> CodecFactory@getDecompressor.  When the caller runs multiple threads, those 
> threads end up sharing compressor and decompressor instances.
> For compressors based on Xerial Snappy this bug has no effect because that 
> library is itself thread safe.  But when BuiltInGzipCompressor from Hadoop is 
> selected for the CompressionCodecName.GZIP case, serious problems ensue.  
> That class is not thread safe and sharing one instance of it between threads 
> produces both silent data corruption and JVM crashes.
> To fix this situation, parquet-mr should stop caching single compressor and 
> decompressor instances.



--
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-09-30 Thread GitBox


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


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

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



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

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

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



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

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


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

ASF GitHub Bot commented on PARQUET-1711:
-

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


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

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





> [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-09-30 Thread GitBox


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


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

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



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

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

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



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

2022-09-30 Thread GitBox


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


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

Review Comment:
   ?
   
   the seen map does encode the depth as a single datastructure.  the depth 
being a separate arg is important b/c it's the current depth in the traversal, 
and is used to update the seen data structure.



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

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

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



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

2022-09-30 Thread GitBox


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


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

Review Comment:
   ?
   
   the `seen` map does encode the seen fields along with their depth as a 
single datastructure.  `depth` being a separate arg is important b/c it's the 
current depth in the traversal, and is used to update the seen data structure.



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

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

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



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

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


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

ASF GitHub Bot commented on PARQUET-1711:
-

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


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

Review Comment:
   ?
   
   the seen map does encode the depth as a single datastructure.  the depth 
being a separate arg is important b/c it's the current depth in the traversal, 
and is used to update the seen data structure.





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



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


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

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


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

ASF GitHub Bot commented on PARQUET-1711:
-

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


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

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





> [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.(C

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

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


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

ASF GitHub Bot commented on PARQUET-1711:
-

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


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

Review Comment:
   ?
   
   the `seen` map does encode the seen fields along with their depth as a 
single datastructure.  `depth` being a separate arg is important b/c it's the 
current depth in the traversal, and is used to update the seen data structure.





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



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


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

2022-09-30 Thread GitBox


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

   thanks for the review.  updated to handle the logging perf concern as well 
as fixing the javadoc errors.


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


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

ASF GitHub Bot commented on PARQUET-1711:
-

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

   thanks for the review.  updated to handle the logging perf concern as well 
as fixing the javadoc errors.




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