[jira] [Resolved] (PARQUET-758) [Format] HALF precision FLOAT Logical type

2023-10-28 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky resolved PARQUET-758.
--
Resolution: Fixed

> [Format] HALF precision FLOAT Logical type
> --
>
> Key: PARQUET-758
> URL: https://issues.apache.org/jira/browse/PARQUET-758
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-format
>Reporter: Julien Le Dem
>Assignee: Anja Boskovic
>Priority: Minor
>




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


[jira] [Assigned] (PARQUET-758) [Format] HALF precision FLOAT Logical type

2023-10-28 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky reassigned PARQUET-758:


Assignee: Anja Boskovic

> [Format] HALF precision FLOAT Logical type
> --
>
> Key: PARQUET-758
> URL: https://issues.apache.org/jira/browse/PARQUET-758
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-format
>Reporter: Julien Le Dem
>Assignee: Anja Boskovic
>Priority: Minor
>




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


[jira] [Commented] (PARQUET-2340) appendRowGroup will loose pageIndex

2023-08-22 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-2340:
---

[~NathanKan], I don't think these methods are used anymore. {{parquet-cli}} has 
another concept to merge files and that supports column indexes AFAIK. 
[~wgtmac], could you confirm this? Maybe, we can close this jira?

> appendRowGroup will loose pageIndex
> ---
>
> Key: PARQUET-2340
> URL: https://issues.apache.org/jira/browse/PARQUET-2340
> Project: Parquet
>  Issue Type: Sub-task
>  Components: parquet-mr
>Reporter: GANHONGNAN
>Priority: Major
>
> Currently, 
> org.apache.parquet.hadoop.ParquetFileWriter#appendFile(org.apache.parquet.io.InputFile)
>  uses appendRowGroup method to concate parquet row group. However, 
> appendRowGroup method *looses* column index.
> {code:java}
> // code placeholder
>   public void appendRowGroup(SeekableInputStream from, BlockMetaData rowGroup,
>                              boolean dropColumns) throws IOException {
>   
>       // TODO: column/offset indexes are not copied
>       // (it would require seeking to the end of the file for each row groups)
>       currentColumnIndexes.add(null);
>       currentOffsetIndexes.add(null);
>   } {code}
>  
> [https://github.com/apache/parquet-mr/blob/f8465a274b42e0a96996c76f3be0b50cf85ecf15/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java#L1033C19-L1033C19]
>  
> Look forward to functionality that support append with page index.
>  



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


[jira] [Resolved] (PARQUET-2318) Implement a tool to list page headers

2023-06-30 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky resolved PARQUET-2318.
---
Resolution: Fixed

> Implement a tool to list page headers
> -
>
> Key: PARQUET-2318
> URL: https://issues.apache.org/jira/browse/PARQUET-2318
> Project: Parquet
>  Issue Type: New Feature
>  Components: parquet-cli
>    Reporter: Gabor Szadovszky
>Assignee: Gabor Szadovszky
>Priority: Major
>
> Needs a tool which lists the page headers in a Parquet file for debugging 
> purposes.



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


[jira] [Created] (PARQUET-2318) Implement a tool to list page headers

2023-06-27 Thread Gabor Szadovszky (Jira)
Gabor Szadovszky created PARQUET-2318:
-

 Summary: Implement a tool to list page headers
 Key: PARQUET-2318
 URL: https://issues.apache.org/jira/browse/PARQUET-2318
 Project: Parquet
  Issue Type: New Feature
  Components: parquet-cli
Reporter: Gabor Szadovszky
Assignee: Gabor Szadovszky


Needs a tool which lists the page headers in a Parquet file for debugging 
purposes.



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


[jira] [Commented] (PARQUET-2317) parquet-format and parquet-format-structures defines Util with inconsitent methods provided

2023-06-25 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-2317:
---

[~wgtmac], Let me summarize the history of this. parquet-format contains all 
the specification docs and the parquet.thrift itself which is a kind of source 
code and spec at the same time. This is good to have all of these separated 
from the implementations. Meanwhile, since the thrift file is there, it was 
natural to have Thrift code generation and the Util there as well. But it was 
not a good choice since we only had the java code there. In some new features 
we had to extend Util which is clearly related to parquet-mr. So, we decided to 
deprecate all of the java related stuff in parquet-format and moved them to 
parquet-format-structures under parquet-mr.
So, it would be good to not only have Util be removed but all the other java 
classes including the Thrift generated ones to be part of the jar.
The catch is we still need to have some mechanism that validates the thrift 
file so we won't add invalid changes. Also, the distribution should be changed 
because providing a jar file without java classes would not make sense. I 
think, we should release a tarball instead that contains all the specs and the 
thrift file as well. Of course, we would need to update the parquet-mr (and 
maybe other affected implementations) to download that tarball instead of the 
jar file.

> parquet-format and parquet-format-structures defines Util with inconsitent 
> methods provided
> ---
>
> Key: PARQUET-2317
> URL: https://issues.apache.org/jira/browse/PARQUET-2317
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-format
>Affects Versions: 1.12.0, 1.13.0
>Reporter: Joey Pereira
>Priority: Major
>
> I have been running into a bug due to {{parquet-format}} and 
> {{parquet-format-structures}} both defining the 
> {{org.apache.parquet.format.Util}} class but doing so inconsistently.
> Examples of this are several methods which include a {{BlockCipher}} 
> parameter that are defined from {{parquet-format-structures}} but not 
> {{{}parquet-format{}}}. While invoking code that happens to use these, such 
> as {{{}org.apache.parquet.hadoop.ParquetFileReader.readFooter{}}}, the code 
> will fail if the {{parquet-format}} happens to be loaded first on the 
> classpath.
> Here is an example stack trace for a Scala Spark application.
> {code:java}
> Caused by: java.lang.NoSuchMethodError: 
> 'org.apache.parquet.format.FileMetaData 
> org.apache.parquet.format.Util.readFileMetaData(java.io.InputStream, 
> org.apache.parquet.format.BlockCipher$Decryptor, byte[])'
> at 
> org.apache.parquet.format.converter.ParquetMetadataConverter$3.visit(ParquetMetadataConverter.java:1441)
>  ~[parquet_hadoop.jar:1.13.1]
> at 
> org.apache.parquet.format.converter.ParquetMetadataConverter$3.visit(ParquetMetadataConverter.java:1438)
>  ~[parquet_hadoop.jar:1.13.1]
> at 
> org.apache.parquet.format.converter.ParquetMetadataConverter$NoFilter.accept(ParquetMetadataConverter.java:1173)
>  ~[parquet_hadoop.jar:1.13.1]
> at 
> org.apache.parquet.format.converter.ParquetMetadataConverter.readParquetMetadata(ParquetMetadataConverter.java:1438)
>  ~[parquet_hadoop.jar:1.13.1]
> at 
> org.apache.parquet.hadoop.ParquetFileReader.readFooter(ParquetFileReader.java:591)
>  ~[parquet_hadoop.jar:1.13.1]
> at 
> org.apache.parquet.hadoop.ParquetFileReader.readFooter(ParquetFileReader.java:536)
>  ~[parquet_hadoop.jar:1.13.1]
> at 
> org.apache.parquet.hadoop.ParquetFileReader.readFooter(ParquetFileReader.java:530)
>  ~[parquet_hadoop.jar:1.13.1]
> at 
> org.apache.parquet.hadoop.ParquetFileReader.readFooter(ParquetFileReader.java:478)
>  ~[parquet_hadoop.jar:1.13.1]
> ... (my application code invoking the above)
> {code}
> Because of issues external to Parquet that I have yet to figure out (a 
> complex Spark and dependency setup), my classpaths are not deterministically 
> ordered and I am unable to pin the {{parquet-format-structures}} ahead hence 
> why I'm chiming in about this.
> Even if that weren't the case, this is a fairly prickly edge to run into as 
> both modules define overlapping classes. {{Util}} is not the only class that 
> appears to be defined by both, just what I have been focusing on due to this 
> bug.
> It appears these methods were introduced in at least 1.12: 
> [https://github.com/apache/parquet-mr/commit/65b95fb72be8f5a8a193a6f7bc4560fdcd742fc7#diff-852341c99dcae06c8fa2b764bcf3d9e6860e40442d0ab1cf5b935df80a9cacb7]



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


[jira] [Comment Edited] (PARQUET-2222) [Format] RLE encoding spec incorrect for v2 data pages

2023-06-09 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky edited comment on PARQUET- at 6/9/23 2:40 PM:
---

[~mwish], -This is specifically about BOOLEAN values (data pages), not rl/dl. 
(In parquet-mr we write rl/dl and dictionary indices using RLE for both v1 and 
v2 settings.)-
Sorry, misread your comment. So parquet-cpp does not write BOOLEAN data pages 
in any case using RLE?


was (Author: gszadovszky):
[~mwish], This is specifically about BOOLEAN values (data pages), not rl/dl. 
(In parquet-mr we write rl/dl and dictionary indices using RLE for both v1 and 
v2 settings.)

> [Format] RLE encoding spec incorrect for v2 data pages
> --
>
> Key: PARQUET-
> URL: https://issues.apache.org/jira/browse/PARQUET-
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-format
>Reporter: Antoine Pitrou
>Assignee: Gang Wu
>Priority: Critical
> Fix For: format-2.10.0
>
>
> The spec 
> (https://github.com/apache/parquet-format/blob/master/Encodings.md#run-length-encoding--bit-packing-hybrid-rle--3)
>  has this:
> {code}
> rle-bit-packed-hybrid:  
> length := length of the  in bytes stored as 4 bytes little 
> endian (unsigned int32)
> {code}
> But the length is actually prepended only in v1 data pages, not in v2 data 
> pages.



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


[jira] [Commented] (PARQUET-2222) [Format] RLE encoding spec incorrect for v2 data pages

2023-06-09 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-:
---

[~mwish], This is specifically about BOOLEAN values (data pages), not rl/dl. 
(In parquet-mr we write rl/dl and dictionary indices using RLE for both v1 and 
v2 settings.)

> [Format] RLE encoding spec incorrect for v2 data pages
> --
>
> Key: PARQUET-
> URL: https://issues.apache.org/jira/browse/PARQUET-
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-format
>Reporter: Antoine Pitrou
>Assignee: Gang Wu
>Priority: Critical
> Fix For: format-2.10.0
>
>
> The spec 
> (https://github.com/apache/parquet-format/blob/master/Encodings.md#run-length-encoding--bit-packing-hybrid-rle--3)
>  has this:
> {code}
> rle-bit-packed-hybrid:  
> length := length of the  in bytes stored as 4 bytes little 
> endian (unsigned int32)
> {code}
> But the length is actually prepended only in v1 data pages, not in v2 data 
> pages.



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


[jira] [Commented] (PARQUET-2222) [Format] RLE encoding spec incorrect for v2 data pages

2023-06-09 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-:
---

[~apitrou], [~wgtmac],

It seems my review was not deep enough. Sorry for that. So, parquet-mr does not 
use RLE encoding for boolean values in case of V1 but only bit packing: 
* 
[V1|https://github.com/apache/parquet-mr/blob/master/parquet-column/src/main/java/org/apache/parquet/column/values/factory/DefaultV1ValuesWriterFactory.java#L53]
 -> ... -> [Bit 
packing|https://github.com/apache/parquet-mr/blob/9d80330ae4948787ac0bf4e4b0d990917f106440/parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ByteBitPackingValuesWriter.java]
 (encoding written to page header: PLAIN)
* 
[V2|https://github.com/apache/parquet-mr/blob/master/parquet-column/src/main/java/org/apache/parquet/column/values/factory/DefaultV2ValuesWriterFactory.java#L57]
 -> ... -> 
[RLE|https://github.com/apache/parquet-mr/blob/9d80330ae4948787ac0bf4e4b0d990917f106440/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridValuesWriter.java]
 (encoding written to page header: RLE)

[~apitrou], could you please confirm that is the same for parquet cpp?

So the table we added in this PR about prepending the length is misleading. 
Also, the link in the PLAIN encoding for boolean is dead and misleading. It 
should point to BIT_PACKED. In the definition of BIT_PACKED it is also wrongly 
stated that it is valid only for RL/DL. I think, the deprecation is valid since 
the "BIT_PACKED" encoding should not be written to anywhere but the actual 
encoding is used under PLAIN for boolean.
Would you guys like to work on this? We probably want to add this to the 
current format release.

> [Format] RLE encoding spec incorrect for v2 data pages
> --
>
> Key: PARQUET-
> URL: https://issues.apache.org/jira/browse/PARQUET-
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-format
>Reporter: Antoine Pitrou
>Assignee: Gang Wu
>Priority: Critical
> Fix For: format-2.10.0
>
>
> The spec 
> (https://github.com/apache/parquet-format/blob/master/Encodings.md#run-length-encoding--bit-packing-hybrid-rle--3)
>  has this:
> {code}
> rle-bit-packed-hybrid:  
> length := length of the  in bytes stored as 4 bytes little 
> endian (unsigned int32)
> {code}
> But the length is actually prepended only in v1 data pages, not in v2 data 
> pages.



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


[jira] [Assigned] (PARQUET-2222) [Format] RLE encoding spec incorrect for v2 data pages

2023-06-09 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky reassigned PARQUET-:
-

Assignee: Gang Wu

> [Format] RLE encoding spec incorrect for v2 data pages
> --
>
> Key: PARQUET-
> URL: https://issues.apache.org/jira/browse/PARQUET-
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-format
>Reporter: Antoine Pitrou
>Assignee: Gang Wu
>Priority: Critical
> Fix For: format-2.10.0
>
>
> The spec 
> (https://github.com/apache/parquet-format/blob/master/Encodings.md#run-length-encoding--bit-packing-hybrid-rle--3)
>  has this:
> {code}
> rle-bit-packed-hybrid:  
> length := length of the  in bytes stored as 4 bytes little 
> endian (unsigned int32)
> {code}
> But the length is actually prepended only in v1 data pages, not in v2 data 
> pages.



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


[jira] [Commented] (PARQUET-758) [Format] HALF precision FLOAT Logical type

2023-06-06 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-758:
--

Thanks for your reply, [~anjakefala]!

I've mentioned {{bfloat16}} only because of the ease of converting back and 
forth to java/c++ {{float}} which we will probably need to be implemented for 
{{IEEE Float16}} as well. But I agree, we should not block the format release 
because of additional discussions about this additional topic.

> [Format] HALF precision FLOAT Logical type
> --
>
> Key: PARQUET-758
> URL: https://issues.apache.org/jira/browse/PARQUET-758
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-format
>Reporter: Julien Le Dem
>Priority: Minor
>




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


[jira] [Commented] (PARQUET-758) [Format] HALF precision FLOAT Logical type

2023-06-05 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-758:
--

Hey everyone, who is interested in the half-float type,

When I've reviewed the format change it was obvious to me to use the "2-byte 
IEEE little-endian format". Now, I've faced another approach to encode 2 byte 
FP numbers: 
[bfloat16|https://en.wikipedia.org/wiki/Bfloat16_floating-point_format]. Since 
neither java nor c++ support 2 byte FP numbers natively we probably need to 
convert the encoded numbers to {{float}}. For {{bfloat16}} it would be more 
performant to do so.
It might worth adding {{bfloat16}} to the format as well and add 
implementations for it in the same round. WDYT?

> [Format] HALF precision FLOAT Logical type
> --
>
> Key: PARQUET-758
> URL: https://issues.apache.org/jira/browse/PARQUET-758
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-format
>Reporter: Julien Le Dem
>Priority: Minor
>




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


[jira] [Commented] (PARQUET-2276) ParquetReader reads do not work with Hadoop version 2.8.5

2023-04-18 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-2276:
---

I think it is fine to drop support of older systems from time to time. It is 
unfortunate though, it was not properly advertised in PARQUET-2158 that we did 
not simply upgrade hadoop version in our build but made it incompatible with 
hadoop2. 
Meanwhile, I think it is fine to re-add support for hadoop2 if it is 
practically feasible and won't break the hadoop3 support. 

> ParquetReader reads do not work with Hadoop version 2.8.5
> -
>
> Key: PARQUET-2276
> URL: https://issues.apache.org/jira/browse/PARQUET-2276
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.13.0
>Reporter: Atul Mohan
>Priority: Major
>
> {{ParquetReader.read() fails with the following exception on parquet-mr 
> version 1.13.0 when using hadoop version 2.8.5:}}
> {code:java}
>  java.lang.NoSuchMethodError: 'boolean 
> org.apache.hadoop.fs.FSDataInputStream.hasCapability(java.lang.String)' 
> at 
> org.apache.parquet.hadoop.util.HadoopStreams.isWrappedStreamByteBufferReadable(HadoopStreams.java:74)
>  
> at org.apache.parquet.hadoop.util.HadoopStreams.wrap(HadoopStreams.java:49) 
> at 
> org.apache.parquet.hadoop.util.HadoopInputFile.newStream(HadoopInputFile.java:69)
>  
> at 
> org.apache.parquet.hadoop.ParquetFileReader.(ParquetFileReader.java:787)
>  
> at 
> org.apache.parquet.hadoop.ParquetFileReader.open(ParquetFileReader.java:657) 
> at org.apache.parquet.hadoop.ParquetReader.initReader(ParquetReader.java:162) 
> org.apache.parquet.hadoop.ParquetReader.read(ParquetReader.java:135)
> {code}
>  
>  
>  
> From an initial investigation, it looks like HadoopStreams has started using 
> [FSDataInputStream.hasCapability|https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L74]
>  but _FSDataInputStream_ does not have the _hasCapability_ API in [hadoop 
> 2.8.x|https://hadoop.apache.org/docs/r2.8.3/api/org/apache/hadoop/fs/FSDataInputStream.html].



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


[jira] [Commented] (PARQUET-2256) Adding Compression for BloomFilter

2023-03-17 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-2256:
---

[~mwish], would you mind to do some investigations before this update? Let's 
get the binary data of a mentioned 2M bloom filter and compress with some 
codecs to see the gain. If the ratio is good, it might worth adding this 
features. It is also worth to mention that compressing bloom filter might hit 
filtering from performance point of view.

> Adding Compression for BloomFilter
> --
>
> Key: PARQUET-2256
> URL: https://issues.apache.org/jira/browse/PARQUET-2256
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-format
>Affects Versions: format-2.9.0
>Reporter: Xuwei Fu
>Assignee: Xuwei Fu
>Priority: Major
>
> In Current Parquet implementions, if BloomFilter doesn't set the ndv, most 
> implementions will guess the 1M as the ndv. And use it for fpp. So, if fpp is 
> 0.01, the BloomFilter size may grows to 2M for each column, which is really 
> huge. Should we support compression for BloomFilter, like:
>  
> ```
>  /**
>  * The compression used in the Bloom filter.
>  **/
> struct Uncompressed {}
> union BloomFilterCompression {
>   1: Uncompressed UNCOMPRESSED;
> +2: CompressionCodec COMPRESSION;
> }
> ```



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


[jira] [Assigned] (PARQUET-2256) Adding Compression for BloomFilter

2023-03-17 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky reassigned PARQUET-2256:
-

Assignee: Xuwei Fu

> Adding Compression for BloomFilter
> --
>
> Key: PARQUET-2256
> URL: https://issues.apache.org/jira/browse/PARQUET-2256
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-format
>Affects Versions: format-2.9.0
>Reporter: Xuwei Fu
>Assignee: Xuwei Fu
>Priority: Major
>
> In Current Parquet implementions, if BloomFilter doesn't set the ndv, most 
> implementions will guess the 1M as the ndv. And use it for fpp. So, if fpp is 
> 0.01, the BloomFilter size may grows to 2M for each column, which is really 
> huge. Should we support compression for BloomFilter, like:
>  
> ```
>  /**
>  * The compression used in the Bloom filter.
>  **/
> struct Uncompressed {}
> union BloomFilterCompression {
>   1: Uncompressed UNCOMPRESSED;
> +2: CompressionCodec COMPRESSION;
> }
> ```



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


[jira] [Commented] (PARQUET-2258) Storing toString fields in FilterPredicate instances can lead to memory pressure

2023-03-17 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-2258:
---

Thanks for fixing this, [~abstractdog]!
As far as I understood this is not a serious issue so I don't think we need to 
include it in a patch release. If you agree please update the version number to 
{{1.13.0}}. (I usually don't bother selecting version numbers which is targeted 
by {{master}}. We'll set them in a bulk update based on the changelog.)

> Storing toString fields in FilterPredicate instances can lead to memory 
> pressure
> 
>
> Key: PARQUET-2258
> URL: https://issues.apache.org/jira/browse/PARQUET-2258
> Project: Parquet
>  Issue Type: Improvement
>Reporter: László Bodor
>Assignee: László Bodor
>Priority: Major
> Fix For: 1.12.3
>
> Attachments: Parquet_Predicate_toString_memory.png, 
> image-2023-03-14-13-27-54-008.png
>
>
> It happens with Hive (HiveServer2), a certain amount of predicate instances 
> can make HiveServer2 OOM. According to the heapdump and background 
> information, the predicates must have been simplified a bit, but still, 
> storing toString in the objects looks very weird.



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


[jira] [Commented] (PARQUET-1690) Integer Overflow of BinaryStatistics#isSmallerThan()

2023-03-17 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-1690:
---

[~humanoid], I don't know/remember the background of this issue and the closed 
PRs. I think it would be the best to start over with a new PR.
[~sha...@uber.com], do you remember why the last PR was closed and not 
reviewed/submitted?

> Integer Overflow of BinaryStatistics#isSmallerThan()
> 
>
> Key: PARQUET-1690
> URL: https://issues.apache.org/jira/browse/PARQUET-1690
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Affects Versions: 1.11.0
>Reporter: Xinli Shang
>Assignee: Xinli Shang
>Priority: Major
>  Labels: pull-request-available
>
> "(min.length() + max.length()) < size" didn't handle integer overflow 
> [https://github.com/apache/parquet-mr/blob/master/parquet-column/src/main/java/org/apache/parquet/column/statistics/BinaryStatistics.java#L103]



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


[jira] [Commented] (PARQUET-2255) BloomFilter and float point is ambiguous

2023-03-13 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-2255:
---

But we don't build the dictionary for filtering but for encoding. We should not 
add anything else than what we have in the pages. So anything should be added 
to the read path.

Maybe we do not need to handle +0.0 and -0.0 differently from the other values. 
(We needed to handle them separately for min/max values because the comparison 
is not trivial and there were actual issues.) If someone deals with FP numbers 
they should know about the difference between +0.0 and -0.0. 

Because the FP spec allows to have multiple NaN values (even though java use 
one actual bitmap for it) we need to avoid using Bloom filter in this case. 
Dictionary is a different thing because we deserialize it to java Double/Float 
values in a Set so we will have one NaN value that is the very same one we are 
searching for. (It is more for the other implementations to deal with NaN if 
the language has several NaN values.)

> BloomFilter and float point is ambiguous
> 
>
> Key: PARQUET-2255
> URL: https://issues.apache.org/jira/browse/PARQUET-2255
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-format
>Reporter: Xuwei Fu
>Priority: Major
> Fix For: format-2.9.0
>
>
> Currently, our Parquet can use BloomFilter for any physical types. However, 
> when BloomFilter apply on float:
>  # What does +0 -0 means? Are they equal?
>  # Should qNaN sNaN written in BloomFilter? Are they equal?
>  



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


[jira] [Commented] (PARQUET-2255) BloomFilter and float point is ambiguous

2023-03-13 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-2255:
---

Bloom filters are for searching for exact values. Exact checking of floating 
point numbers are usually code smell. Usually checking if the difference is 
below an epsilon value is suggested over using exact equality. I am wondering 
if there is a real usecase for searching for an exact floating point number. 
Maybe disabling bloom filters completely for FP numbers is the simplest choice 
and probably won't bother anyone.

If we still want to handle FP bloom filters I agree with [~wgtmac]'s proposal. 
(It is a similar approach we implemented for min/max values.) Keep in mind that 
we need to handle the case when someone wants to filter on a NaN.



> BloomFilter and float point is ambiguous
> 
>
> Key: PARQUET-2255
> URL: https://issues.apache.org/jira/browse/PARQUET-2255
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-format
>Reporter: Xuwei Fu
>Priority: Major
> Fix For: format-2.9.0
>
>
> Currently, our Parquet can use BloomFilter for any physical types. However, 
> when BloomFilter apply on float:
>  # What does +0 -0 means? Are they equal?
>  # Should qNaN sNaN written in BloomFilter? Are they equal?
>  



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


[jira] [Commented] (PARQUET-2254) Build a BloomFilter with a more precise size

2023-03-07 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-2254:
---

1) I think, for creating bloom filters we have the statistics to decide how 
much space the bloom filter shall occupy (we have the actual data). What we 
don't know if the bloom filter in itself will be useful or not. (Whould there 
be filtering on the related column and would it be Eq/NotEq/IsIn etc. like 
predicates.) This one shall be decided by the client by the already introduced 
properties. We do not write bloom filters by default anyway.
2) Of course it is hard to be smart for PPD since we don't know the actual data 
(we are just before reading it). But there is an actual order of checking the 
row group filters: statistics, dictionary, bloom filter. Checking the 
statistics first is obviously correct. What I am not sure about is if we want 
to check dictionary first and then the bloom filter or the other way around. 
Because of that question I am also unsure if it is a good practice to not write 
bloom filters if the whole column chunk is dictionary encoded.

> Build a BloomFilter with a more precise size
> 
>
> Key: PARQUET-2254
> URL: https://issues.apache.org/jira/browse/PARQUET-2254
> Project: Parquet
>  Issue Type: Improvement
>Reporter: Mars
>Assignee: Mars
>Priority: Major
>
> Now the usage is to specify the size, and then build BloomFilter. In general 
> scenarios, it is actually not sure how much the distinct value is. 
> If BloomFilter can be automatically generated according to the data, the file 
> size can be reduced and the reading efficiency can also be improved.
> I have an idea that the user can specify a maximum BloomFilter filter size, 
> then we build multiple BloomFilter at the same time, we can use the largest 
> BloomFilter as a counting tool( If there is no hit when inserting a value, 
> the counter will be +1, of course this may be imprecise but enough)
> Then at the end of the write, choose a BloomFilter of a more appropriate size 
> when the file is finally written.
> I want to implement this feature and hope to get your opinions, thank you



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


[jira] [Commented] (PARQUET-2254) Build a BloomFilter with a more precise size

2023-03-07 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-2254:
---

I think this is a good idea. Meanwhile, it would increase the memory footprint 
of the writer. However, if you plan to keep the current logic that the user 
decides the columns which bloom filters are generated for, it should be 
acceptable.
However, I think, we need to take one step back and investigate/synchronize the 
efforts around row group filtering. Or maybe it is only me for whom the 
following questions are not obvious? :)
* Is it always true that reading the dictionary for filtering is cheaper than 
reading the bloom filter? Bloom filters should be usually smaller than 
dictionaries and faster to be scanned for a value.
* Based on the previous one if we decide that it might worth reading the bloom 
filter before the dictionary it also questions the logic of not writing bloom 
filters in case of the whole column chunk is dictionary encoded.
* Meanwhile, if the whole column chunk is dictionary encoded but the dictionary 
is still small (the redundancy is high) then it might not worth writing a bloom 
filter since checking the dictionary might be cheaper.
What do you think?

> Build a BloomFilter with a more precise size
> 
>
> Key: PARQUET-2254
> URL: https://issues.apache.org/jira/browse/PARQUET-2254
> Project: Parquet
>  Issue Type: Improvement
>Reporter: Mars
>Assignee: Mars
>Priority: Major
>
> Now the usage is to specify the size, and then build BloomFilter. In general 
> scenarios, it is actually not sure how much the distinct value is. 
> If BloomFilter can be automatically generated according to the data, the file 
> size can be reduced and the reading efficiency can also be improved.
> I have an idea that the user can specify a maximum BloomFilter filter size, 
> then we build multiple BloomFilter at the same time, we can use the largest 
> BloomFilter as a counting tool( If there is no hit when inserting a value, 
> the counter will be +1, of course this may be imprecise but enough)
> Then at the end of the write, choose a BloomFilter of a more appropriate size 
> when the file is finally written.
> I want to implement this feature and hope to get your opinions, thank you



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


[jira] [Assigned] (PARQUET-2254) Build a BloomFilter with a more precise size

2023-03-07 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky reassigned PARQUET-2254:
-

Assignee: Mars

> Build a BloomFilter with a more precise size
> 
>
> Key: PARQUET-2254
> URL: https://issues.apache.org/jira/browse/PARQUET-2254
> Project: Parquet
>  Issue Type: Improvement
>Reporter: Mars
>Assignee: Mars
>Priority: Major
>
> Now the usage is to specify the size, and then build BloomFilter. In general 
> scenarios, it is actually not sure how much the distinct value is. 
> If BloomFilter can be automatically generated according to the data, the file 
> size can be reduced and the reading efficiency can also be improved.
> I have an idea that the user can specify a maximum BloomFilter filter size, 
> then we build multiple BloomFilter at the same time, we can use the largest 
> BloomFilter as a counting tool( If there is no hit when inserting a value, 
> the counter will be +1, of course this may be imprecise but enough)
> Then at the end of the write, choose a BloomFilter of a more appropriate size 
> when the file is finally written.
> I want to implement this feature and hope to get your opinions, thank you



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


[jira] [Resolved] (PARQUET-2246) Add short circuit logic to column index filter

2023-02-23 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky resolved PARQUET-2246.
---
Resolution: Fixed

> Add short circuit logic to column index filter
> --
>
> Key: PARQUET-2246
> URL: https://issues.apache.org/jira/browse/PARQUET-2246
> Project: Parquet
>  Issue Type: Improvement
>Reporter: Yujiang Zhong
>Assignee: Yujiang Zhong
>Priority: Minor
>
> ColumnIndexFilter can be optimized by adding short-circuit logic to `AND` and 
> `OR` operations. It's not necessary to evaluating the right node in some 
> cases:
>  * If the left result row ranges of `AND` is empty
>  * If the left result row ranges of `OR` is full range of the row-group



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


[jira] [Assigned] (PARQUET-2246) Add short circuit logic to column index filter

2023-02-23 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky reassigned PARQUET-2246:
-

Assignee: Yujiang Zhong

> Add short circuit logic to column index filter
> --
>
> Key: PARQUET-2246
> URL: https://issues.apache.org/jira/browse/PARQUET-2246
> Project: Parquet
>  Issue Type: Improvement
>Reporter: Yujiang Zhong
>Assignee: Yujiang Zhong
>Priority: Minor
>
> ColumnIndexFilter can be optimized by adding short-circuit logic to `AND` and 
> `OR` operations. It's not necessary to evaluating the right node in some 
> cases:
>  * If the left result row ranges of `AND` is empty
>  * If the left result row ranges of `OR` is full range of the row-group



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


[jira] [Resolved] (PARQUET-2243) Support zstd-jni in DirectCodecFactory

2023-02-22 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky resolved PARQUET-2243.
---
Resolution: Fixed

> Support zstd-jni in DirectCodecFactory
> --
>
> Key: PARQUET-2243
> URL: https://issues.apache.org/jira/browse/PARQUET-2243
> Project: Parquet
>  Issue Type: Bug
>    Reporter: Gabor Szadovszky
>    Assignee: Gabor Szadovszky
>Priority: Major
>
> During switching to zstd-jni (from the Hadoop native zstd codec) we missed to 
> add proper implementations for {{DirectCodecFactory}}. Currently, NPE occurs 
> in case of the {{DirectCodecFactory}} is used while reading/writing.



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


[jira] [Assigned] (PARQUET-2247) Fail-fast if CapacityByteArrayOutputStream write overflow

2023-02-22 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky reassigned PARQUET-2247:
-

Assignee: dzcxzl  (was: Gabor Szadovszky)

> Fail-fast if CapacityByteArrayOutputStream write overflow
> -
>
> Key: PARQUET-2247
> URL: https://issues.apache.org/jira/browse/PARQUET-2247
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Reporter: dzcxzl
>Assignee: dzcxzl
>Priority: Critical
>
> The bytesUsed of CapacityByteArrayOutputStream may overflow when writing some 
> large byte data, resulting in parquet file write corruption.



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


[jira] [Resolved] (PARQUET-2247) Fail-fast if CapacityByteArrayOutputStream write overflow

2023-02-22 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky resolved PARQUET-2247.
---
Resolution: Fixed

> Fail-fast if CapacityByteArrayOutputStream write overflow
> -
>
> Key: PARQUET-2247
> URL: https://issues.apache.org/jira/browse/PARQUET-2247
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Reporter: dzcxzl
>Assignee: dzcxzl
>Priority: Critical
>
> The bytesUsed of CapacityByteArrayOutputStream may overflow when writing some 
> large byte data, resulting in parquet file write corruption.



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


[jira] [Assigned] (PARQUET-2247) Fail-fast if CapacityByteArrayOutputStream write overflow

2023-02-22 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky reassigned PARQUET-2247:
-

Assignee: Gabor Szadovszky

> Fail-fast if CapacityByteArrayOutputStream write overflow
> -
>
> Key: PARQUET-2247
> URL: https://issues.apache.org/jira/browse/PARQUET-2247
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Reporter: dzcxzl
>Assignee: Gabor Szadovszky
>Priority: Critical
>
> The bytesUsed of CapacityByteArrayOutputStream may overflow when writing some 
> large byte data, resulting in parquet file write corruption.



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


[jira] [Resolved] (PARQUET-2241) ByteStreamSplitDecoder broken in presence of nulls

2023-02-21 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky resolved PARQUET-2241.
---
Resolution: Fixed

> ByteStreamSplitDecoder broken in presence of nulls
> --
>
> Key: PARQUET-2241
> URL: https://issues.apache.org/jira/browse/PARQUET-2241
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-format, parquet-mr
>Affects Versions: format-2.8.0
>Reporter: Xuwei Fu
>Assignee: Gang Wu
>Priority: Major
>
>  
> This problem is shown in this issue: 
> [https://github.com/apache/arrow/issues/15173|https://github.com/apache/arrow/issues/15173Let]
> Let me talk about it briefly:
> * Encoder doesn't write "num_values" on Page payload for BYTE_STREAM_SPLIT, 
> but using "num_values" as stride in BYTE_STREAM_SPLIT
> * When decoding, for DATA_PAGE_V2, it can now the num_values and num_nulls in 
> the page, however, in DATA_PAGE_V1, without statistics, we should read 
> def-levels and rep-levels to get the real num-of-values. And without the 
> num-of-values, we aren't able to decode BYTE_STREAM_SPLIT correctly
>  
> The bug-reproducing code is in the issue.



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


[jira] [Resolved] (PARQUET-2228) ParquetRewriter supports more than one input file

2023-02-21 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky resolved PARQUET-2228.
---
Resolution: Fixed

> ParquetRewriter supports more than one input file
> -
>
> Key: PARQUET-2228
> URL: https://issues.apache.org/jira/browse/PARQUET-2228
> Project: Parquet
>  Issue Type: Sub-task
>  Components: parquet-mr
>Reporter: Gang Wu
>Assignee: Gang Wu
>Priority: Major
>
> ParquetRewriter currently supports only one input file. The scope of this 
> task is to support multiple input files and the rewriter merges them into a 
> single one w/o some rewrite options specified.



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


[jira] [Assigned] (PARQUET-2244) Dictionary filter may skip row-groups incorrectly when evaluating notIn

2023-02-15 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky reassigned PARQUET-2244:
-

Assignee: Yujiang Zhong

> Dictionary filter may skip row-groups incorrectly when evaluating notIn
> ---
>
> Key: PARQUET-2244
> URL: https://issues.apache.org/jira/browse/PARQUET-2244
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.2
>Reporter: Yujiang Zhong
>Assignee: Yujiang Zhong
>Priority: Major
>
> Dictionary filter may skip row-groups incorrectly when evaluating `notIn` on 
> optional columns with null values. Here is an example:
> Say there is a optional column `c1` with all pages dict encoded, `c1` has and 
> only has two distinct values: ['foo', null],  and the predicate is  `c1 not 
> in ('foo', 'bar')`. 
> Now dictionary filter may skip this row-group that is actually should not be 
> skipped, because there are nulls in the column.
>  
> This is a bug similar to #1510.



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


[jira] [Resolved] (PARQUET-2244) Dictionary filter may skip row-groups incorrectly when evaluating notIn

2023-02-15 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky resolved PARQUET-2244.
---
Resolution: Fixed

> Dictionary filter may skip row-groups incorrectly when evaluating notIn
> ---
>
> Key: PARQUET-2244
> URL: https://issues.apache.org/jira/browse/PARQUET-2244
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.2
>Reporter: Yujiang Zhong
>Assignee: Yujiang Zhong
>Priority: Major
>
> Dictionary filter may skip row-groups incorrectly when evaluating `notIn` on 
> optional columns with null values. Here is an example:
> Say there is a optional column `c1` with all pages dict encoded, `c1` has and 
> only has two distinct values: ['foo', null],  and the predicate is  `c1 not 
> in ('foo', 'bar')`. 
> Now dictionary filter may skip this row-group that is actually should not be 
> skipped, because there are nulls in the column.
>  
> This is a bug similar to #1510.



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


[jira] [Created] (PARQUET-2243) Support zstd-jni in DirectCodecFactory

2023-02-14 Thread Gabor Szadovszky (Jira)
Gabor Szadovszky created PARQUET-2243:
-

 Summary: Support zstd-jni in DirectCodecFactory
 Key: PARQUET-2243
 URL: https://issues.apache.org/jira/browse/PARQUET-2243
 Project: Parquet
  Issue Type: Bug
Reporter: Gabor Szadovszky
Assignee: Gabor Szadovszky


During switching to zstd-jni (from the Hadoop native zstd codec) we missed to 
add proper implementations for {{DirectCodecFactory}}. Currently, NPE occurs in 
case of the {{DirectCodecFactory}} is used while reading/writing.



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


[jira] [Commented] (PARQUET-2241) ByteStreamSplitDecoder broken in presence of nulls

2023-02-14 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-2241:
---

[~wgtmac], realted to your question about production. I haven't seen any usage 
of BYTE_STREAM_SPLIT in prod. Production envrionments I have been working on 
was stuck to Parquet v1 encodings.

> ByteStreamSplitDecoder broken in presence of nulls
> --
>
> Key: PARQUET-2241
> URL: https://issues.apache.org/jira/browse/PARQUET-2241
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-format, parquet-mr
>Affects Versions: format-2.8.0
>Reporter: Xuwei Fu
>Assignee: Gang Wu
>Priority: Major
>
>  
> This problem is shown in this issue: 
> [https://github.com/apache/arrow/issues/15173|https://github.com/apache/arrow/issues/15173Let]
> Let me talk about it briefly:
> * Encoder doesn't write "num_values" on Page payload for BYTE_STREAM_SPLIT, 
> but using "num_values" as stride in BYTE_STREAM_SPLIT
> * When decoding, for DATA_PAGE_V2, it can now the num_values and num_nulls in 
> the page, however, in DATA_PAGE_V1, without statistics, we should read 
> def-levels and rep-levels to get the real num-of-values. And without the 
> num-of-values, we aren't able to decode BYTE_STREAM_SPLIT correctly
>  
> The bug-reproducing code is in the issue.



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


[jira] [Comment Edited] (PARQUET-2241) ByteStreamSplitDecoder broken in presence of nulls

2023-02-14 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky edited comment on PARQUET-2241 at 2/14/23 8:37 AM:


[~wgtmac], realted to your question about production. I haven't seen any usage 
of BYTE_STREAM_SPLIT in prod. Production envrionments I have been working on 
were stuck to Parquet v1 encodings.


was (Author: gszadovszky):
[~wgtmac], realted to your question about production. I haven't seen any usage 
of BYTE_STREAM_SPLIT in prod. Production envrionments I have been working on 
was stuck to Parquet v1 encodings.

> ByteStreamSplitDecoder broken in presence of nulls
> --
>
> Key: PARQUET-2241
> URL: https://issues.apache.org/jira/browse/PARQUET-2241
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-format, parquet-mr
>Affects Versions: format-2.8.0
>Reporter: Xuwei Fu
>Assignee: Gang Wu
>Priority: Major
>
>  
> This problem is shown in this issue: 
> [https://github.com/apache/arrow/issues/15173|https://github.com/apache/arrow/issues/15173Let]
> Let me talk about it briefly:
> * Encoder doesn't write "num_values" on Page payload for BYTE_STREAM_SPLIT, 
> but using "num_values" as stride in BYTE_STREAM_SPLIT
> * When decoding, for DATA_PAGE_V2, it can now the num_values and num_nulls in 
> the page, however, in DATA_PAGE_V1, without statistics, we should read 
> def-levels and rep-levels to get the real num-of-values. And without the 
> num-of-values, we aren't able to decode BYTE_STREAM_SPLIT correctly
>  
> The bug-reproducing code is in the issue.



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


[jira] [Resolved] (PARQUET-2226) Support merge Bloom Filter

2023-01-16 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky resolved PARQUET-2226.
---
Resolution: Fixed

> Support merge Bloom Filter
> --
>
> Key: PARQUET-2226
> URL: https://issues.apache.org/jira/browse/PARQUET-2226
> Project: Parquet
>  Issue Type: Improvement
>Reporter: Mars
>Assignee: miracle
>Priority: Major
>
> We need to collect Parquet's bloom filter of multiple files, and then 
> synthesize a more comprehensive bloom filter for common use. 
> Guava supports similar api operations
> https://guava.dev/releases/31.0.1-jre/api/docs/src-html/com/google/common/hash/BloomFilter.html#line.252



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


[jira] [Assigned] (PARQUET-2226) Support merge Bloom Filter

2023-01-16 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky reassigned PARQUET-2226:
-

Assignee: miracle

> Support merge Bloom Filter
> --
>
> Key: PARQUET-2226
> URL: https://issues.apache.org/jira/browse/PARQUET-2226
> Project: Parquet
>  Issue Type: Improvement
>Reporter: Mars
>Assignee: miracle
>Priority: Major
>
> We need to collect Parquet's bloom filter of multiple files, and then 
> synthesize a more comprehensive bloom filter for common use. 
> Guava supports similar api operations
> https://guava.dev/releases/31.0.1-jre/api/docs/src-html/com/google/common/hash/BloomFilter.html#line.252



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


[jira] [Assigned] (PARQUET-2226) Support merge Bloom Filter

2023-01-16 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky reassigned PARQUET-2226:
-

Assignee: (was: miracle)

> Support merge Bloom Filter
> --
>
> Key: PARQUET-2226
> URL: https://issues.apache.org/jira/browse/PARQUET-2226
> Project: Parquet
>  Issue Type: Improvement
>Reporter: Mars
>Priority: Major
>
> We need to collect Parquet's bloom filter of multiple files, and then 
> synthesize a more comprehensive bloom filter for common use. 
> Guava supports similar api operations
> https://guava.dev/releases/31.0.1-jre/api/docs/src-html/com/google/common/hash/BloomFilter.html#line.252



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


[jira] [Assigned] (PARQUET-2226) Support merge Bloom Filter

2023-01-16 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky reassigned PARQUET-2226:
-

Assignee: miracle

> Support merge Bloom Filter
> --
>
> Key: PARQUET-2226
> URL: https://issues.apache.org/jira/browse/PARQUET-2226
> Project: Parquet
>  Issue Type: Improvement
>Reporter: Mars
>Assignee: miracle
>Priority: Major
>
> We need to collect Parquet's bloom filter of multiple files, and then 
> synthesize a more comprehensive bloom filter for common use. 
> Guava supports similar api operations
> https://guava.dev/releases/31.0.1-jre/api/docs/src-html/com/google/common/hash/BloomFilter.html#line.252



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


[jira] [Commented] (PARQUET-1980) Build and test Apache Parquet on ARM64 CPU architecture

2023-01-10 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-1980:
---

Perfect. Thank you, [~mgrigorov]!

> Build and test Apache Parquet on ARM64 CPU architecture
> ---
>
> Key: PARQUET-1980
> URL: https://issues.apache.org/jira/browse/PARQUET-1980
> Project: Parquet
>  Issue Type: Test
>  Components: parquet-format
>Reporter: Martin Tzvetanov Grigorov
>Assignee: Martin Tzvetanov Grigorov
>Priority: Minor
>  Labels: pull-request-available
> Fix For: 1.12.0
>
>
> More and more deployments are being done on ARM64 machines.
> It would be good to make sure Parquet MR project builds fine on it.
> The project moved from TravisCI to GitHub Actions recently (PARQUET-1969) but 
> .travis.yml could be re-intorduced for ARM64 until GitHub Actions provide 
> aarch64 nodes!



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


[jira] [Reopened] (PARQUET-1980) Build and test Apache Parquet on ARM64 CPU architecture

2023-01-08 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky reopened PARQUET-1980:
---

[~mgrigorov],

PMC just got a note from Apache IT that they are about to "move away from 
Travis at the beginning of 2023". I don't know if Github actions are now 
suitable for ARM64 or there any other solutions for this. If you have time, 
could you please take a look?

> Build and test Apache Parquet on ARM64 CPU architecture
> ---
>
> Key: PARQUET-1980
> URL: https://issues.apache.org/jira/browse/PARQUET-1980
> Project: Parquet
>  Issue Type: Test
>  Components: parquet-format
>Reporter: Martin Tzvetanov Grigorov
>Assignee: Martin Tzvetanov Grigorov
>Priority: Minor
>  Labels: pull-request-available
> Fix For: 1.12.0
>
>
> More and more deployments are being done on ARM64 machines.
> It would be good to make sure Parquet MR project builds fine on it.
> The project moved from TravisCI to GitHub Actions recently (PARQUET-1969) but 
> .travis.yml could be re-intorduced for ARM64 until GitHub Actions provide 
> aarch64 nodes!



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


[jira] [Commented] (PARQUET-2220) Parquet Filter predicate storing nested string causing OOM's

2022-12-31 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-2220:
---

[~abhiSumo304], I agree eagerly storing the toString value is not a good idea. 
I don't think it has proper use case either. toString should be used for 
debugging purposes anyway so eagerly storing the value does not really make 
sense. Unfortunately, I don't work on the Parquet code base actively anymore. 
Feel free to put up a PR to fix this and I'll try to review it in time.

> Parquet Filter predicate storing nested string causing OOM's
> 
>
> Key: PARQUET-2220
> URL: https://issues.apache.org/jira/browse/PARQUET-2220
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-format
>Reporter: Abhishek Jain
>Priority: Critical
>
> Each Instance of ColumnFilterPredicate stores the filter values in toString 
> variable eagerly. Which is not useful
> {code:java}
> static abstract class ColumnFilterPredicate> 
> implements FilterPredicate, Serializable  {
>   private final Column column;
>   private final T value;
>   private final String toString; 
> protected ColumnFilterPredicate(Column column, T value) {
>   this.column = Objects.requireNonNull(column, "column cannot be null");
>   // Eq and NotEq allow value to be null, Lt, Gt, LtEq, GtEq however do not, 
> so they guard against
>   // null in their own constructors.
>   this.value = value;
>   String name = getClass().getSimpleName().toLowerCase(Locale.ENGLISH);
>   this.toString = name + "(" + column.getColumnPath().toDotString() + ", " + 
> value + ")";
> }{code}
>  
>  
> If your filter predicate is too long/nested this can take a lot of memory 
> while creating Filter.
> We have seen in our productions this can go upto 4gbs of space while opening 
> multiple parquet readers
> Same thing is replicated in BinaryLogicalFilterPredicate. Where toString is 
> eagerly calculated and stored in string and lot of duplication is happening 
> while making And/or filter.
> I did not find use case of storing it so eagerly



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


[jira] [Assigned] (PARQUET-2159) Parquet bit-packing de/encode optimization

2022-11-25 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky reassigned PARQUET-2159:
-

Assignee: Fang-Xie

> Parquet bit-packing de/encode optimization
> --
>
> Key: PARQUET-2159
> URL: https://issues.apache.org/jira/browse/PARQUET-2159
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Affects Versions: 1.13.0
>Reporter: Fang-Xie
>Assignee: Fang-Xie
>Priority: Major
> Fix For: 1.13.0
>
> Attachments: image-2022-06-15-22-56-08-396.png, 
> image-2022-06-15-22-57-15-964.png, image-2022-06-15-22-58-01-442.png, 
> image-2022-06-15-22-58-40-704.png
>
>
> Current Spark use Parquet-mr as parquet reader/writer library, but the 
> built-in bit-packing en/decode is not efficient enough. 
> Our optimization for Parquet bit-packing en/decode with jdk.incubator.vector 
> in Open JDK18 brings prominent performance improvement.
> Due to Vector API is added to OpenJDK since 16, So this optimization request 
> JDK16 or higher.
> *Below are our test results*
> Functional test is based on open-source parquet-mr Bit-pack decoding 
> function: *_public final void unpack8Values(final byte[] in, final int inPos, 
> final int[] out, final int outPos)_* __
> compared with our implementation with vector API *_public final void 
> unpack8Values_vec(final byte[] in, final int inPos, final int[] out, final 
> int outPos)_*
> We tested 10 pairs (open source parquet bit unpacking vs ours optimized 
> vectorized SIMD implementation) decode function with bit 
> width=\{1,2,3,4,5,6,7,8,9,10}, below are test results:
> !image-2022-06-15-22-56-08-396.png|width=437,height=223!
> We integrated our bit-packing decode implementation into parquet-mr, tested 
> the parquet batch reader ability from Spark VectorizedParquetRecordReader 
> which get parquet column data by the batch way. We construct parquet file 
> with different row count and column count, the column data type is Int32, the 
> maximum int value is 127 which satisfies bit pack encode with bit width=7,   
> the count of the row is from 10k to 100 million and the count of the column 
> is from 1 to 4.
> !image-2022-06-15-22-57-15-964.png|width=453,height=229!
> !image-2022-06-15-22-58-01-442.png|width=439,height=217!
> !image-2022-06-15-22-58-40-704.png|width=415,height=208!



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


[jira] [Commented] (PARQUET-2020) Remove deprecated modules

2022-10-13 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-2020:
---

[~Unsta], the module {{parquet-cli}} is meant to substitute the functionality 
of {{parquet-tools}}. {{parquet-cli}} might have the functionality you need. 
However, neither of them was designed to have its classes used publicly. (There 
are no guarantees the changes will be backward compatible.)
I don't think that {{parquet-format-structures}} would be a good fit to place 
such functionality either. This module is for reading/writing the footer and 
also not designed to be used by our clients. 
The question is if you need this json representation for production use or for 
debugging purposes. In case of the latter one we might want to create a new 
module inside parquet-mr for tools to be used from the java API. We might 
factor out some existing implementation from {{parquet-cli}} and maybe having 
back something from {{parquet-tools}} if required.
If however you need the reading to json (and maybe writing from it) for 
production use I would suggest having a new binding for json just like we have 
{{parquet-avro}}, {{parquet-protobuf}}, {{parquet-thrift}} etc.
Unfortunately, I won't have time to guide you with any of these choices. I 
would suggest bringing up this topic on [mailto:dev@parquet.apache.org] to have 
broader audience.


> Remove deprecated modules
> -
>
> Key: PARQUET-2020
> URL: https://issues.apache.org/jira/browse/PARQUET-2020
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-cascading
>Affects Versions: 1.12.0
>Reporter: Fokko Driesprong
>Assignee: Fokko Driesprong
>Priority: Major
> Fix For: 1.13.0
>
>
> Removes: 
>  * parquet-tools-deprecated
>  * parquet-scrooge-deprecated
>  * parquet-cascading-common23-deprecated
>  * parquet-cascading-deprecated
>  * parquet-cascading3-deprecated



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


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

2022-10-10 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-1222:
---

[~emkornfield],

There are a couple of docs in the parquet-format repo. The related ones are 
[about logical 
types|[https://github.com/apache/parquet-format/blob/master/LogicalTypes.md]] 
and the main one that contains the description of the [primitive 
types|https://github.com/apache/parquet-format/blob/master/README.md#types]. 
Unfortunately, the latter one does not contain anything about sorting order.
So, I think, we need to do the following:
* Define the sorting order for the primitive types or reference the logical 
types description for it. (In most cases it would be referencing since the 
ordering depends on the related logical types e.g. signed/unsigned sorting of 
integral types)
* After defining the sorting order of the primitive floating point numbers 
based on what we've discussed above reference it from the new half-precision FP 
logical type.

(Another unfortunate thing is that we have some specification-like docs at the 
[parquet site|https://parquet.apache.org] as well. I think we should propagate 
the parquet-format docs to there automatically or simply link them from the 
site. But it is clearly a different topic.)

> 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 Gabor Szadovszky (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-1222?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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] [Created] (PARQUET-2182) Handle unknown logical types

2022-08-30 Thread Gabor Szadovszky (Jira)
Gabor Szadovszky created PARQUET-2182:
-

 Summary: Handle unknown logical types
 Key: PARQUET-2182
 URL: https://issues.apache.org/jira/browse/PARQUET-2182
 Project: Parquet
  Issue Type: Bug
Reporter: Gabor Szadovszky


New logical types introduced in parquet-format shall be properly handled in 
parquet-mr releases that are not aware of this new type. In this case we shall 
read the data as if only the primitive type would be defined (without a logical 
type) with one exception: We shall not use min/max based statistics (including 
column indexes) since we don't know the proper ordering of that type.




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


[jira] [Updated] (PARQUET-2094) Handle negative values in page headers

2021-12-20 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky updated PARQUET-2094:
--
 External issue ID: CVE-2021-41561
External issue URL: 
https://cve.mitre.org/cgi-bin/cvename.cgi?name=CVE-2021-41561

> Handle negative values in page headers
> --
>
> Key: PARQUET-2094
> URL: https://issues.apache.org/jira/browse/PARQUET-2094
> Project: Parquet
>  Issue Type: Bug
>    Reporter: Gabor Szadovszky
>    Assignee: Gabor Szadovszky
>Priority: Major
> Fix For: 1.11.2, 1.12.2
>
>
> There are integer values in the page headers that should be always positive 
> (e.g. length). I am not sure if we properly handle the cases if they are not 
> positive.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Updated] (PARQUET-2106) BinaryComparator should avoid doing ByteBuffer.wrap in the hot-path

2021-12-09 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky updated PARQUET-2106:
--
Issue Type: Improvement  (was: Task)

> BinaryComparator should avoid doing ByteBuffer.wrap in the hot-path
> ---
>
> Key: PARQUET-2106
> URL: https://issues.apache.org/jira/browse/PARQUET-2106
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Affects Versions: 1.12.2
>Reporter: Alexey Kudinkin
>Assignee: Alexey Kudinkin
>Priority: Major
> Attachments: Screen Shot 2021-12-03 at 3.26.31 PM.png, 
> profile_48449_alloc_1638494450_sort_by.html
>
>
> *Background*
> While writing out large Parquet tables using Spark, we've noticed that 
> BinaryComparator is the source of substantial churn of extremely short-lived 
> `HeapByteBuffer` objects – It's taking up to *16%* of total amount of 
> allocations in our benchmarks, putting substantial pressure on a Garbage 
> Collector:
> !Screen Shot 2021-12-03 at 3.26.31 PM.png|width=828,height=521!
> [^profile_48449_alloc_1638494450_sort_by.html]
>  
> *Proposal*
> We're proposing to adjust lexicographical comparison (at least) to avoid 
> doing any allocations, since this code lies on the hot-path of every Parquet 
> write, therefore causing substantial churn amplification.
>  
>  
>  



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Assigned] (PARQUET-2106) BinaryComparator should avoid doing ByteBuffer.wrap in the hot-path

2021-12-09 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky reassigned PARQUET-2106:
-

Assignee: Alexey Kudinkin

> BinaryComparator should avoid doing ByteBuffer.wrap in the hot-path
> ---
>
> Key: PARQUET-2106
> URL: https://issues.apache.org/jira/browse/PARQUET-2106
> Project: Parquet
>  Issue Type: Task
>  Components: parquet-mr
>Affects Versions: 1.12.2
>Reporter: Alexey Kudinkin
>Assignee: Alexey Kudinkin
>Priority: Major
> Attachments: Screen Shot 2021-12-03 at 3.26.31 PM.png, 
> profile_48449_alloc_1638494450_sort_by.html
>
>
> *Background*
> While writing out large Parquet tables using Spark, we've noticed that 
> BinaryComparator is the source of substantial churn of extremely short-lived 
> `HeapByteBuffer` objects – It's taking up to *16%* of total amount of 
> allocations in our benchmarks, putting substantial pressure on a Garbage 
> Collector:
> !Screen Shot 2021-12-03 at 3.26.31 PM.png|width=828,height=521!
> [^profile_48449_alloc_1638494450_sort_by.html]
>  
> *Proposal*
> We're proposing to adjust lexicographical comparison (at least) to avoid 
> doing any allocations, since this code lies on the hot-path of every Parquet 
> write, therefore causing substantial churn amplification.
>  
>  
>  



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Resolved] (PARQUET-2107) Travis failures

2021-12-08 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky resolved PARQUET-2107.
---
Resolution: Fixed

> Travis failures
> ---
>
> Key: PARQUET-2107
> URL: https://issues.apache.org/jira/browse/PARQUET-2107
> Project: Parquet
>  Issue Type: Bug
>    Reporter: Gabor Szadovszky
>    Assignee: Gabor Szadovszky
>Priority: Major
>
> There are Travis failures since a while in our PRs. See e.g. 
> https://app.travis-ci.com/github/apache/parquet-mr/jobs/550598285 or 
> https://app.travis-ci.com/github/apache/parquet-mr/jobs/550598286



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Created] (PARQUET-2107) Travis failures

2021-12-07 Thread Gabor Szadovszky (Jira)
Gabor Szadovszky created PARQUET-2107:
-

 Summary: Travis failures
 Key: PARQUET-2107
 URL: https://issues.apache.org/jira/browse/PARQUET-2107
 Project: Parquet
  Issue Type: Bug
Reporter: Gabor Szadovszky
Assignee: Gabor Szadovszky


There are Travis failures since a while in our PRs. See e.g. 
https://app.travis-ci.com/github/apache/parquet-mr/jobs/550598285 or 
https://app.travis-ci.com/github/apache/parquet-mr/jobs/550598286



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (PARQUET-2104) parquet-cli broken in master

2021-11-24 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-2104:
---

[~gamaken], I am not sure about a workaround. I've tried this on master as well 
as on the tags of the releases 1.12.2 and 1.11.2. All works the same way. :(

One idea is to use parquet-tools instead of parquet-cli. It has similar 
functionality. However, parquet-tools has been deprecated in 1.12.0 and removed 
in the current master. You may want to try it with an older tag (e.g. 
apache-parquet-1.11.2).

> parquet-cli broken in master
> 
>
> Key: PARQUET-2104
> URL: https://issues.apache.org/jira/browse/PARQUET-2104
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-cli
>Affects Versions: 1.12.2
> Environment: ubuntu 18.04 and ubuntu 20.04
>Reporter: Balaji K
>Priority: Major
>
> Creating a Jira per this thread:
> [https://lists.apache.org/thread/k233838g010lvbp81s99floqjmm7nnvs]
>  # clone parquet-mr and build the repo locally
>  # run parquet-cli without Hadoop (according to this ReadMe 
> <[https://github.com/apache/parquet-mr/tree/master/parquet-cli#running-without-hadoop]>
>  )
>  # try a command that deserializes data such as cat or head
>  # observe NoSuchMethodError being thrown
> *Error stack:* ~/repos/parquet-mr/parquet-cli$ parquet cat 
> ../../testdata/dictionaryEncodingSample.parquet WARNING: An illegal 
> reflective access operation has occurred .. 
> Exception in thread "main" java.lang.NoSuchMethodError: 
> 'org.apache.avro.Schema 
> org.apache.parquet.avro.AvroSchemaConverter.convert(org.apache.parquet.schema.MessageType)'
>  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)



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


Re: unable to get rid of NoSuchMethodError with parquet-cli

2021-11-23 Thread Gabor Szadovszky
Hey,

I can reproduce the same on master. It seems the same issue happens with
older versions as well. I don't know how we did not find it yet. (Or I am
making the same mistake as you :) ).
Could you please create a jira about it and continue the discussion there?

Thanks a lot,
Gabor

On Tue, Nov 23, 2021 at 1:08 AM gamaken k  wrote:

> Hello,
>
> I'm relatively new to parquet and to the community, kindly bear with me if
> I'm making simple mistakes.
>
> I cloned parquet-mr and built the repo locally on my ubuntu 20.04, When
> running parquet-cli, particularly when trying to read values (cat, head)
> from a parquet file, I run into NoSuchMethodError (full stack at the end of
> message).
>
> *Here is what I have tried so far to understand this problem:*
> 1. clean the repo and clone again and retry
> 2. metadata inspection commands such as `meta`, `schema`, `pages`
> `dictionary` - they all work perfectly. check-stats command reports no
> corruption or errors,
> 3. I'm running parquet-cli without Hadoop (according to this ReadMe
> <
> https://github.com/apache/parquet-mr/tree/master/parquet-cli#running-without-hadoop
> >
> )
> 4. I have tried this on two different machines (ubuntu 18.04 and
> ubuntu 20.04) and gotten the same result.
> 5. The input file is produced by a library that I am writing to serialize
> parquet data in .net (i.e, not produced by parquet-mr).
> 6. parquet-cpp is able to read the file correctly and deserialize the data.
> parquet-mr compatibility is important to me, so I'm trying to get this
> working.
>
> I could not solve this problem, so, I would really appreciate it if someone
> who understands what's wrong here could kindly share insights.
>
> *Error stack:*
> ~/repos/parquet-mr/parquet-cli$ parquet cat
> ../../testdata/dictionaryEncodingSample.parquet
> WARNING: An illegal reflective access operation has occurred
> ..
> Exception in thread "main" java.lang.NoSuchMethodError:
> 'org.apache.avro.Schema
>
> org.apache.parquet.avro.AvroSchemaConverter.convert(org.apache.parquet.schema.MessageType)'
> 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)
>
> *mvn dependency:tree output for parquet-cli*
> [INFO] ---< org.apache.parquet:parquet-cli
> >---
> [INFO] Building Apache Parquet Command-line 1.13.0-SNAPSHOT
> [INFO] [ jar
> ]-
> [INFO]
> [INFO] --- maven-dependency-plugin:3.1.1:tree (default-cli) @ parquet-cli
> ---
> [INFO] org.apache.parquet:parquet-cli:jar:1.13.0-SNAPSHOT
> [INFO] +- org.apache.parquet:parquet-avro:jar:1.13.0-SNAPSHOT:compile
> [INFO] +-
> org.apache.parquet:parquet-format-structures:jar:1.13.0-SNAPSHOT:compile
> [INFO] +- org.apache.parquet:parquet-common:jar:1.13.0-SNAPSHOT:compile
> [INFO] +- org.apache.parquet:parquet-column:jar:1.13.0-SNAPSHOT:compile
> [INFO] |  +-
> org.apache.parquet:parquet-encoding:jar:1.13.0-SNAPSHOT:compile
> [INFO] |  \- org.apache.yetus:audience-annotations:jar:0.13.0:compile
> [INFO] +- org.apache.parquet:parquet-hadoop:jar:1.13.0-SNAPSHOT:compile
> [INFO] |  +- org.xerial.snappy:snappy-java:jar:1.1.8.3:compile
> [INFO] |  \- commons-pool:commons-pool:jar:1.6:compile
> [INFO] +- org.apache.avro:avro:jar:1.10.2:compile
> [INFO] |  \- org.apache.commons:commons-compress:jar:1.20:compile
> [INFO] +- com.github.luben:zstd-jni:jar:1.5.0-1:runtime
> [INFO] +- org.slf4j:slf4j-api:jar:1.7.22:compile
> [INFO] +- net.sf.opencsv:opencsv:jar:2.3:compile
> [INFO] +- org.apache.commons:commons-text:jar:1.8:compile
> [INFO] |  \- org.apache.commons:commons-lang3:jar:3.9:compile
> [INFO] +- org.apache.parquet:parquet-jackson:jar:1.13.0-SNAPSHOT:runtime
> [INFO] +- com.fasterxml.jackson.core:jackson-databind:jar:2.12.2:compile
> [INFO] +- com.fasterxml.jackson.core:jackson-core:jar:2.12.2:compile
> [INFO] +- com.fasterxml.jackson.core:jackson-annotations:jar:2.12.2:compile
> [INFO] +- com.beust:jcommander:jar:1.72:compile
> [INFO] +- org.slf4j:slf4j-log4j12:jar:1.7.22:provided
> [INFO] +- com.google.guava:guava:jar:27.0.1-jre:provided
> [INFO] |  +- com.google.guava:failureaccess:jar:1.0.1:provided
> [INFO] |  +-
>
> com.google.guava:listenablefuture:jar:.0-empty-to-avoid-conflict-with-guava:provided
> [INFO] |  +- org.checkerframework:checker-qual:jar:2.5.2:provided
> [INFO] |  +-
> com.google.errorprone:error_prone_annotations:jar:2.2.0:provided
> [INFO] |  +- com.google.j2objc:j2objc-annotations:jar:1.1:provided
> [INFO] |  \- org.codehaus.mojo:animal-sniffer-annotations:jar:1.17:provided
> [INFO] +- org.apache.hadoop:hadoop-client:jar:2.10.1:provided
> [INFO] |  +- 

[jira] [Commented] (PARQUET-2103) crypto exception in print toPrettyJSON

2021-11-22 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-2103:
---

I think, we need to update 
[ParquetMetadata.toJSON|https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ParquetMetadata.java#L67-L71].
 Jackson shall be able to be configured to not to look for getter methods but 
the private fields. I am not sure if it is a good idea or if it will work in 
every environment. Another option would be to refactor 
EncryptedColumnChunkMetaData to not to call "decrypt" for a getter but it might 
not worth the efforts. The easiest way would be to simply detect if the 
metadata contains encrypted data and do not log anything. I don't know how 
urgent might it be to log the metadata in case of debugging.

> crypto exception in print toPrettyJSON
> --
>
> Key: PARQUET-2103
> URL: https://issues.apache.org/jira/browse/PARQUET-2103
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.0, 1.12.1, 1.12.2
>Reporter: Gidon Gershinsky
>Priority: Major
>
> In debug mode, this code 
> {{if (LOG.isDebugEnabled()) {}}
> {{  LOG.debug(ParquetMetadata.toPrettyJSON(parquetMetadata));}}
> {{}}}
> called in 
> {{org.apache.parquet.format.converter.ParquetMetadataConverter.readParquetMetadata()}}
>  
> _*for unencrypted files*_ 
> triggers an exception:
>  
> {{Caused by: org.apache.parquet.crypto.ParquetCryptoRuntimeException: [id]. 
> Null File Decryptor     }}
> {{    at 
> org.apache.parquet.hadoop.metadata.EncryptedColumnChunkMetaData.decryptIfNeeded(ColumnChunkMetaData.java:602)
>  ~[parquet-hadoop-1.12.0jar:1.12.0]}}
> {{    at 
> org.apache.parquet.hadoop.metadata.ColumnChunkMetaData.getEncodingStats(ColumnChunkMetaData.java:353)
>  ~[parquet-hadoop-1.12.0jar:1.12.0]}}
> {{    at jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) 
> ~[?:?]}}
> {{    at 
> jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>  ~[?:?]}}
> {{    at 
> jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>  ~[?:?]}}
> {{    at java.lang.reflect.Method.invoke(Method.java:566) ~[?:?]}}
> {{    at 
> shaded.parquet.com.fasterxml.jackson.databind.ser.BeanPropertyWriter.serializeAsField(BeanPropertyWriter.java:689)
>  ~[parquet-jackson-1.12.0jar:1.12.0]}}
> {{    at 
> shaded.parquet.com.fasterxml.jackson.databind.ser.std.BeanSerializerBase.serializeFields(BeanSerializerBase.java:755)
>  ~[parquet-jackson-1.12.0jar:1.12.0]}}
> {{    at 
> shaded.parquet.com.fasterxml.jackson.databind.ser.BeanSerializer.serialize(BeanSerializer.java:178)
>  ~[parquet-jackson-1.12.0jar:1.12.0]}}
> {{    at 
> shaded.parquet.com.fasterxml.jackson.databind.ser.impl.IndexedListSerializer.serializeContents(IndexedListSerializer.java:119)
>  ~[parquet-jackson-1.12.0jar:1.12.0]}}
> {{    at 
> shaded.parquet.com.fasterxml.jackson.databind.ser.impl.IndexedListSerializer.serialize(IndexedListSerializer.java:79)
>  ~[parquet-jackson-1.12.0jar:1.12.0]}}
> {{    at 
> shaded.parquet.com.fasterxml.jackson.databind.ser.impl.IndexedListSerializer.serialize(IndexedListSerializer.java:18)
>  ~[parquet-jackson-1.12.0jar:1.12.0]}}
> {{    at 
> shaded.parquet.com.fasterxml.jackson.databind.ser.BeanPropertyWriter.serializeAsField(BeanPropertyWriter.java:728)
>  ~[parquet-jackson-1.12.0jar:1.12.0]}}
> {{    at 
> shaded.parquet.com.fasterxml.jackson.databind.ser.std.BeanSerializerBase.serializeFields(BeanSerializerBase.java:755)
>  ~[parquet-jackson-1.12.0jar:1.12.0]}}
> {{    at 
> shaded.parquet.com.fasterxml.jackson.databind.ser.BeanSerializer.serialize(BeanSerializer.java:178)
>  ~[parquet-jackson-1.12.0jar:1.12.0]}}
> {{    at 
> shaded.parquet.com.fasterxml.jackson.databind.ser.impl.IndexedListSerializer.serializeContents(IndexedListSerializer.java:119)
>  ~[parquet-jackson-1.12.0jar:1.12.0]}}
> {{    at 
> shaded.parquet.com.fasterxml.jackson.databind.ser.impl.IndexedListSerializer.serialize(IndexedListSerializer.java:79)
>  ~[parquet-jackson-1.12.0jar:1.12.0]}}
> {{    at 
> shaded.parquet.com.fasterxml.jackson.databind.ser.impl.IndexedListSerializer.serialize(IndexedListSerializer.java:18)
>  ~[parquet-jackson-1.12.0jar:1.12.0]}}
> {{    at 
> shaded.parquet.com.fasterxml.jackson.databind.ser.BeanPropertyWriter.serializeAsField(BeanPropertyWriter.java:728)
>  ~[parquet-jackson-1.12.0jar:1.12.0]}}
> {{    at 
> shaded.parquet.com.fasterxml.jackso

[jira] [Resolved] (PARQUET-2101) Fix wrong descriptions about the default block size

2021-11-02 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky resolved PARQUET-2101.
---
Resolution: Fixed

> Fix wrong descriptions about the default block size
> ---
>
> Key: PARQUET-2101
> URL: https://issues.apache.org/jira/browse/PARQUET-2101
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-avro, parquet-mr, parquet-protobuf
>Reporter: Kengo Seki
>Assignee: Kengo Seki
>Priority: Trivial
>
> https://github.com/apache/parquet-mr/blob/master/parquet-avro/src/main/java/org/apache/parquet/avro/AvroParquetWriter.java#L90
> https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java#L240
> https://github.com/apache/parquet-mr/blob/master/parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoParquetWriter.java#L80
> These javadocs say the default block size is 50 MB but it's actually 128MB.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


Re: Map Type duplicate keys

2021-10-26 Thread Gabor Szadovszky
Hi Micah,

Parquet-MR does not have its own data model (except an example
implementation used for unit tests). So it is up to the data model how the
values are handled. I think it is possible to store key-value pairs with
the same key using the example implementation but there are no such tests.
I am not sure how the other bindings (arrow, avro, thrift, protobuf, pig)
work.

I think the specification is written this way to cover how parquet-mr works
because there are no checks implemented to store/read only one key-value
pair for a key. So technically it is possible to use Parquet MAP as a
multi-map but it might not be interoperable with other implementations or
models in parquet-mr.

Regards,
Gabor

On Mon, Oct 25, 2021 at 9:07 PM Micah Kornfield 
wrote:

> Hi dev@parquet,
> The Logical Type Specification [1] has the following to say about duplicate
> keys.
>
> If there are multiple key-value pairs for the same key, then the final
> > value for that key must be the last value. Other values may be ignored or
> > may be added with replacement to the map container in the order that they
> > are encoded. The MAP annotation should not be used to encode multi-maps
> > using duplicate keys.
>
>
> I was wondering if anybody was aware of systems that use this in practice
> (i.e. write out duplicate keys and rely on the reader to deduplicate them).
>
> Thanks,
> Micah
>
>
> [1]
> https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#maps
>


[jira] [Updated] (PARQUET-2094) Handle negative values in page headers

2021-09-30 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky updated PARQUET-2094:
--
Fix Version/s: 1.12.2
   1.11.2

> Handle negative values in page headers
> --
>
> Key: PARQUET-2094
> URL: https://issues.apache.org/jira/browse/PARQUET-2094
> Project: Parquet
>  Issue Type: Bug
>    Reporter: Gabor Szadovszky
>    Assignee: Gabor Szadovszky
>Priority: Major
> Fix For: 1.11.2, 1.12.2
>
>
> There are integer values in the page headers that should be always positive 
> (e.g. length). I am not sure if we properly handle the cases if they are not 
> positive.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Resolved] (PARQUET-2094) Handle negative values in page headers

2021-09-30 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky resolved PARQUET-2094.
---
Resolution: Fixed

> Handle negative values in page headers
> --
>
> Key: PARQUET-2094
> URL: https://issues.apache.org/jira/browse/PARQUET-2094
> Project: Parquet
>  Issue Type: Bug
>    Reporter: Gabor Szadovszky
>    Assignee: Gabor Szadovszky
>Priority: Major
>
> There are integer values in the page headers that should be always positive 
> (e.g. length). I am not sure if we properly handle the cases if they are not 
> positive.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Resolved] (PARQUET-1968) FilterApi support In predicate

2021-09-30 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky resolved PARQUET-1968.
---
Resolution: Fixed

> FilterApi support In predicate
> --
>
> Key: PARQUET-1968
> URL: https://issues.apache.org/jira/browse/PARQUET-1968
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Affects Versions: 1.12.0
>Reporter: Yuming Wang
>Assignee: Huaxin Gao
>Priority: Major
>
> FilterApi should support native In predicate.
> Spark:
> https://github.com/apache/spark/blob/d6a68e0b67ff7de58073c176dd097070e88ac831/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala#L600-L605
> Impala:
> https://issues.apache.org/jira/browse/IMPALA-3654



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Assigned] (PARQUET-1968) FilterApi support In predicate

2021-09-30 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky reassigned PARQUET-1968:
-

Assignee: Huaxin Gao

> FilterApi support In predicate
> --
>
> Key: PARQUET-1968
> URL: https://issues.apache.org/jira/browse/PARQUET-1968
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Affects Versions: 1.12.0
>Reporter: Yuming Wang
>Assignee: Huaxin Gao
>Priority: Major
>
> FilterApi should support native In predicate.
> Spark:
> https://github.com/apache/spark/blob/d6a68e0b67ff7de58073c176dd097070e88ac831/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala#L600-L605
> Impala:
> https://issues.apache.org/jira/browse/IMPALA-3654



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Resolved] (PARQUET-2096) Upgrade Thrift to 0.15.0

2021-09-28 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky resolved PARQUET-2096.
---
Resolution: Fixed

> Upgrade Thrift to 0.15.0
> 
>
> Key: PARQUET-2096
> URL: https://issues.apache.org/jira/browse/PARQUET-2096
> Project: Parquet
>  Issue Type: Improvement
>Reporter: Vinoo Ganesh
>Assignee: Vinoo Ganesh
>Priority: Minor
>
> Thrift 0.15.0 is currently the default in brew: 
> [https://github.com/Homebrew/homebrew-core/blob/82d03f657371e1541a9a5e5de57c5e1aa00acd45/Formula/thrift.rb#L4.|https://github.com/Homebrew/homebrew-core/blob/master/Formula/thrift.rb#L4.]
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Assigned] (PARQUET-2096) Upgrade Thrift to 0.15.0

2021-09-28 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky reassigned PARQUET-2096:
-

Assignee: Vinoo Ganesh

> Upgrade Thrift to 0.15.0
> 
>
> Key: PARQUET-2096
> URL: https://issues.apache.org/jira/browse/PARQUET-2096
> Project: Parquet
>  Issue Type: Improvement
>Reporter: Vinoo Ganesh
>Assignee: Vinoo Ganesh
>Priority: Minor
>
> Thrift 0.15.0 is currently the default in brew: 
> [https://github.com/Homebrew/homebrew-core/blob/82d03f657371e1541a9a5e5de57c5e1aa00acd45/Formula/thrift.rb#L4.|https://github.com/Homebrew/homebrew-core/blob/master/Formula/thrift.rb#L4.]
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (PARQUET-2080) Deprecate RowGroup.file_offset

2021-09-28 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-2080:
---

[~gershinsky], could you make the doc available for comments?

> Deprecate RowGroup.file_offset
> --
>
> Key: PARQUET-2080
> URL: https://issues.apache.org/jira/browse/PARQUET-2080
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-format
>    Reporter: Gabor Szadovszky
>Assignee: Gidon Gershinsky
>Priority: Major
>
> Due to PARQUET-2078 RowGroup.file_offset is not reliable.
> This field is also wrongly calculated in the C++ oss parquet implementation 
> PARQUET-2089



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


Re: Guidelines for Thrift max message size? (Thrift 0.14+)

2021-09-27 Thread Gabor Szadovszky
Hi Antoine,

I do not have too much to add just hate getting no replies on the dev list.
Parquet-mr doesn't have a release with thrift 0.14+ yet. (The latest
release 1.12.1 went out with 0.13.0.) I don't know how common a >100MB file
footer is. Since we read the whole footer at once to memory and pass it to
Thrift to parse it does not seem to be a memory allocation issue but a CPU
processing issue. If the footer is too large the parsing might require
significant efforts. But if it is a valid Parquet file we have to do it
anyway if not we don't know anything without parsing the footer. Also, I am
not a Thrift expert but I think if the purpose is to forge a Thrift
structure to overload CPU the simple limit of the size would not be enough.
So, I would say we should extend this limit to the maximum (~2GB in java)
if we think the 100MB is not enough.

Cheers,
Gabor

On Mon, Sep 27, 2021 at 11:43 AM Antoine Pitrou  wrote:

>
> Ping.  Does nobody really have any experience with or opinion on this?
>
> I'm going to assume it's ok to disable this security check.
>
> Regards
>
> Antoine.
>
>
>
> On Thu, 16 Sep 2021 17:54:23 +0200
> Antoine Pitrou  wrote:
> > Hello,
> >
> > On Mon, 13 Sep 2021 16:08:19 +0200
> > Antoine Pitrou  wrote:
> > >
> > > My initial fix is to simply remove the limitation.  That is based on
> > > the interpretation that the "message size" is simply the encoded size
> > > of a Thrift payload.  Since we load the Thrift message entirely in
> > > memory from the Parquet file, based on what the Parquet metadata says,
> > > the fact that another size is recorded in the Thrift message shouldn't
> > > ideally be a problem.  But of course, that feels a bit unsatisfactory
> > > (I cannot say for sure whether a problem exists or not).
> > > https://github.com/apache/arrow/pull/11123
> >
> > I'm following up now that I've read through the relevant Thrift C++
> > transport implementations.  I'm reasonably convinced that my analysis
> > is correct, as the max message size applies to encoded Thrift bytes,
> > and we already know the encoded.  I still hope to receive an answer
> > from the Thrift community on
> > https://issues.apache.org/jira/browse/THRIFT-5237.
> >
> > Did nobody experience this issue with other Parquet implementations?
> >
> > Regards
> >
> > Antoine.
> >
> >
> >
>
>
>
>


[jira] [Created] (PARQUET-2094) Handle negative values in page headers

2021-09-22 Thread Gabor Szadovszky (Jira)
Gabor Szadovszky created PARQUET-2094:
-

 Summary: Handle negative values in page headers
 Key: PARQUET-2094
 URL: https://issues.apache.org/jira/browse/PARQUET-2094
 Project: Parquet
  Issue Type: Bug
Reporter: Gabor Szadovszky
Assignee: Gabor Szadovszky


There are integer values in the page headers that should be always positive 
(e.g. length). I am not sure if we properly handle the cases if they are not 
positive.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (PARQUET-118) Provide option to use on-heap buffers for Snappy compression/decompression

2021-09-21 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-118:
--

[~MasterDDT], Unfortunately I can only say something similar that Julien add at 
the first comment. I'm happy to review any PRs about this topic. :)

> Provide option to use on-heap buffers for Snappy compression/decompression
> --
>
> Key: PARQUET-118
> URL: https://issues.apache.org/jira/browse/PARQUET-118
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Affects Versions: 1.6.0
>Reporter: Patrick Wendell
>Priority: Major
>
> The current code uses direct off-heap buffers for decompression. If many 
> decompressors are instantiated across multiple threads, and/or the objects 
> being decompressed are large, this can lead to a huge amount of off-heap 
> allocation by the JVM. This can be exacerbated if overall, there is not heap 
> contention, since no GC will be performed to reclaim the space used by these 
> buffers.
> It would be nice if there was a flag we cold use to simply allocate on-heap 
> buffers here:
> https://github.com/apache/incubator-parquet-mr/blob/master/parquet-hadoop/src/main/java/parquet/hadoop/codec/SnappyDecompressor.java#L28
> We ran into an issue today where these buffers totaled a very large amount of 
> storage and caused our Java processes (running within containers) to be 
> terminated by the kernel OOM-killer.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (PARQUET-2091) Fix release build error introduced by PARQUET-2043

2021-09-20 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-2091:
---

Strange to me because the release command should not do anything more (related 
to dependencies) than a {{mvn verify}}.
Isn't it possible that this issue occurred only on the 1.12.x branch and the 
master doesn't have this issue?

> Fix release build error introduced by PARQUET-2043
> --
>
> Key: PARQUET-2091
> URL: https://issues.apache.org/jira/browse/PARQUET-2091
> Project: Parquet
>  Issue Type: Task
>Reporter: Xinli Shang
>Assignee: Xinli Shang
>Priority: Major
>
> After PARQUET-2043 when building for a release like 1.12.1, there is build 
> error complaining 'used undeclared dependency'. 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (PARQUET-2088) Different created_by field values for application and library

2021-09-15 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-2088:
---

parquet-mr automatically fills the {{created_by}} field by using FULL_VERSION. 
The components using it (Hive/Spark) do not have to populate anything. So if 
parquet-mr writes a file the proper full version string of parquet-mr will be 
written to the field every time.

You are right that there is no separate field to fill the version of the 
"higher level" application. (I remember some discussions about this topic but 
could not find it in the jiras :( ) The issue here is which application version 
should we store? For example there is a customer code that uses a tool written 
for Spark that writes the parquet file. We can make mistakes at any level that 
may cause invalid values (from a certain point of view). So how should we 
handle this and how can we formalize it? Also, how can we enforce the client 
codes to fill these fields?
Anyway, if you have a proposal feel free to write to the dev list.

> Different created_by field values for application and library
> -
>
> Key: PARQUET-2088
> URL: https://issues.apache.org/jira/browse/PARQUET-2088
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Affects Versions: format-2.9.0
>Reporter: Joshua Howard
>Priority: Minor
>
> There seems to be a discrepancy in the Parquet format created_by field 
> regarding how it should be filled out. The parquet-mr library uses this value 
> to enable/disable features based on the parquet-mr version 
> [here|https://github.com/apache/parquet-mr/blob/5f403501e9de05b6aa48f028191b4e78bb97fb12/parquet-column/src/main/java/org/apache/parquet/CorruptDeltaByteArrays.java#L64-L68].
>  Meanwhile, users are encouraged to make use of the application version 
> [here|https://www.javadoc.io/doc/org.apache.parquet/parquet-format/latest/org/apache/parquet/format/FileMetaData.html].
>  It seems like there are multiple fields needed for an application and 
> library version. 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (PARQUET-2091) Fix release build error introduced by PARQUET-2043

2021-09-14 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-2091:
---

[~sha...@uber.com], do you have issues with building on master? Just checked 
and it is working fine on my environment. (Also seems to be working at the PR 
checks.)

> Fix release build error introduced by PARQUET-2043
> --
>
> Key: PARQUET-2091
> URL: https://issues.apache.org/jira/browse/PARQUET-2091
> Project: Parquet
>  Issue Type: Task
>Reporter: Xinli Shang
>Assignee: Xinli Shang
>Priority: Major
>
> After PARQUET-2043 when building for a release like 1.12.1, there is build 
> error complaining 'used undeclared dependency'. 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Resolved] (PARQUET-2084) Upgrade Thrift to 0.14.2

2021-09-14 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky resolved PARQUET-2084.
---
Resolution: Fixed

> Upgrade Thrift to 0.14.2
> 
>
> Key: PARQUET-2084
> URL: https://issues.apache.org/jira/browse/PARQUET-2084
> Project: Parquet
>  Issue Type: Improvement
>Reporter: Chao Sun
>Assignee: Chao Sun
>Priority: Minor
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Resolved] (PARQUET-2083) Expose getFieldPath from ColumnIO

2021-09-14 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky resolved PARQUET-2083.
---
Resolution: Fixed

> Expose getFieldPath from ColumnIO
> -
>
> Key: PARQUET-2083
> URL: https://issues.apache.org/jira/browse/PARQUET-2083
> Project: Parquet
>  Issue Type: Improvement
>Reporter: Chao Sun
>Assignee: Chao Sun
>Priority: Minor
>
> Similar to PARUQET-2050, this exposes {{getFieldPath}} from {{ColumnIO}} so 
> downstream apps such as Spark can use it to assemble nested records.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


Re: [VOTE] Release Apache Parquet 1.12.1 RC1

2021-09-14 Thread Gabor Szadovszky
Thanks for the new RC, Xinli.

The content seems correct to me. The checksum and sign are correct. Unit
tests pass.

My vote is +1 (binding)

On Mon, Sep 13, 2021 at 8:11 PM Xinli shang  wrote:

> Hi everyone,
>
>
> I propose the following RC to be released as the official Apache Parquet
> 1.12.1 release.
>
>
> The commit id is 2a5c06c58fa987f85aa22170be14d927d5ff6e7d
>
> * This corresponds to the tag: apache-parquet-1.12.1-rc1
>
> *
>
> https://github.com/apache/parquet-mr/tree/2a5c06c58fa987f85aa22170be14d927d5ff6e7d
>
>
> The release tarball, signature, and checksums are here:
>
> *
> https://dist.apache.org/repos/dist/dev/parquet/apache-parquet-1.12.1-rc1/
>
>
> You can find the KEYS file here:
>
> * *https://dist.apache.org/repos/dist/release/parquet/KEYS
> *
>
>
> Binary artifacts are staged in Nexus here:
>
> * https://repository.apache.org/content/groups/staging/org/apache/parquet/
>
>
> This release includes important changes listed
> https://github.com/apache/parquet-mr/blob/parquet-1.12.x/CHANGES.md
>
>
> Please download, verify, and test.
>
>
> Please vote in the next 72 hours.
>
>
> [ ] +1 Release this as Apache Parquet 1.12.1
>
> [ ] +0
>
> [ ] -1 Do not release this because...
>
> --
> Xinli Shang | Tech Lead Manager @ Uber Data Infra
>


[jira] [Commented] (PARQUET-2088) Different created_by field values for application and library

2021-09-14 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-2088:
---

Ah, I see. So, that code part is not about a feature but a bug fix. It is the 
pain in file format implementations that you not only have to fix issues in the 
code but you have to deal with invalid files written by that faulty code (if it 
was released). This time we've had to implement a workaround for those invalid 
files written by parquet-mr releases before 1.8.0.
I am not sure how the Impala reader/writer works. I work on parquet-mr and 
Impala is not tightly part of the Parquet community. It is more an example that 
the created_by field has to be filled by the application actually implements 
the writing of the parquet files. So e.g. Hive, Spark etc. won't be listed here 
ever as they are using parquet-mr to write/read the files. Impala has its own 
writer/reader implementation.

> Different created_by field values for application and library
> -
>
> Key: PARQUET-2088
> URL: https://issues.apache.org/jira/browse/PARQUET-2088
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Affects Versions: format-2.9.0
>Reporter: Joshua Howard
>Priority: Minor
>
> There seems to be a discrepancy in the Parquet format created_by field 
> regarding how it should be filled out. The parquet-mr library uses this value 
> to enable/disable features based on the parquet-mr version 
> [here|https://github.com/apache/parquet-mr/blob/5f403501e9de05b6aa48f028191b4e78bb97fb12/parquet-column/src/main/java/org/apache/parquet/CorruptDeltaByteArrays.java#L64-L68].
>  Meanwhile, users are encouraged to make use of the application version 
> [here|https://www.javadoc.io/doc/org.apache.parquet/parquet-format/latest/org/apache/parquet/format/FileMetaData.html].
>  It seems like there are multiple fields needed for an application and 
> library version. 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (PARQUET-2085) Formatting is broken for description of BIT_PACKED

2021-09-14 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-2085:
---

[~alexott], I got it now. You are talking about the [Parquet 
site|http://parquet.apache.org/documentation/latest/]. I was confused because 
the PR is in the parquet-format repo. The Offical site has a separate 
repository: https://github.com/apache/parquet-site. It is a bit tricky to 
update (need to install old ruby libs and generate the htmls manually) but if 
you would like to give a try feel free to create a new PR on the site repo.

> Formatting is broken for description of BIT_PACKED
> --
>
> Key: PARQUET-2085
> URL: https://issues.apache.org/jira/browse/PARQUET-2085
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-format
>Reporter: Alex Ott
>Priority: Minor
>
> The Nested Encoding section of documentation doesn't escape the {{_}} 
> character, so it looks as following:
> Two encodings for the levels are supported BIT_PACKED and RLE. Only RLE is 
> now used as it supersedes BIT_PACKED.
> instead of
> Two encodings for the levels are supported BIT_PACKED and RLE. Only RLE is 
> now used as it supersedes BIT_PACKED.
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


Re: Concatenation of parquet files

2021-09-14 Thread Gabor Szadovszky
Hi Pau,

I guess attachments are not allowed in the apache lists so we cannot see
the image.

If the two row groups contain the very same data in the same order and
encoded with the same encoding, compressed with the same codec I think,
they should be the same binary. I am not sure why you have different binary
streams for these row groups but if the proper data can be decoded from
both row groups I would not spend too much time on it.

About merging row groups. It is a tough issue and far not that simple as
concatenating the row groups (files) and creating a new footer. There are
statistics in the footer that you have to take care about as well as column
indexes and bloom filters that are not part of the footer and neither the
row groups. (They are written in separate data structures before the
footer.)
If you don't want to decode the row groups these statistics can be updated
(with the new offsets) as well as the new footer can be created by reading
the original footers only. The problem here is creating such a parquet file
is not very useful in most cases. Most of the problems come from many small
row groups (in small files) which cannot be solved this way. To solve the
small files problem we need to merge the row groups and for that we need to
decode the original data so we can re-create the statistics (at least for
bloom filters).

Long story short, theoretically it is solvable but it is a feature we
haven't implemented properly so far.

Cheers,
Gabor

On Tue, Sep 14, 2021 at 10:08 AM Pau Tallada  wrote:

> Hi,
>
> I am a developer of cosmohub.pic.es, a science platform that provides
> interactive analysis and exploration of large scientific datasets. Working
> with Hive, users are able to generate the subset of data they are
> interested in, and this result set is stored as a set of files. When users
> want to download this dataset, we combine/concatenate all the files
> on-the-fly to generate a single stream that gets downloaded. Done right,
> this is very efficient, avoids materializing the combined file and the
> stream is even seekable so downloads can be resumed. We are able to do this
> for csv.bz2 and FITS formats.
>
> I am trying to do the same with parquet. Looking at the format
> specification, it seems that it could be done by simply concatenating the
> binary blobs of the set of row groups and generating a new footer for the
> merged file. The problem is that the same data, written twice in the same
> file (in two row groups), is represented with some differences in the
> binary stream produced (see attached image). Why is the binary
> representation of a row group different if the data is the same? Is the
> order or position of a row group codified inside its metadata?
>
> I attach the image of a parquet file with the same data (a single integer
> column named 'c' with a single value 0) written twice, with at least two
> differences marked in red and blue.
> [image: image.png]
>
>
> A little diagram to show what I'm trying to accomplish:
>
> *contents of parquet file A:*
> PAR1
> ROW GROUP A1
> ROW GROUP A2
> FOOTER A
>
> *contents of parquet file B:*
> PAR1
> ROW GROUP B1
> ROW GROUP B2
> FOOTER B
>
> If I'm not mistaken, there is no metadata in each row group that refers to
> its file or its position, so they should be relocatable. The final
> file/stream would look like this:
>
> *contents of combined parquet file:*
> PAR1
> ROW GROUP A1
> ROW GROUP A2
> ROW GROUP B1
> ROW GROUP B2
> NEW FOOTER A+B
>
> Thanks a lot in advance for the help understanding this,
>
> Best regards,
>
> Pau.
> --
> --
> Pau Tallada Crespí
> Departament de Serveis
> Port d'Informació Científica (PIC)
> Tel: +34 93 170 2729
> --
>
>


Re: [VOTE] Release Apache Parquet 1.12.1 RC0

2021-09-13 Thread Gabor Szadovszky
Thanks a lot for working on this, Xinli. Do not forget that you also have a
vote :)

I have some issues with the content of the release. I would not include the
change PARQUET-2043. It is not a bugfix and contains a lot of changes
around dependencies. I feel it a bit risky to include it in a patch
release. I also don't get why the change
261e320329d719ef3c7f05ff2c3feb788d6ffd4f

was necessary. We should not remove modules in a patch release.

Because of these issues I my vote is -1 (binding)

Please, also take care about the release
 in the
jira. There are only 2 issues registered for 1.12.1.

Cheers,
Gabor

On Mon, Sep 13, 2021 at 9:42 AM Gidon Gershinsky  wrote:

> +1 (non-binding)
>
> - checked the sum
> - ran build and test, everything was ok
> - ran additional framework tests with the built jars, passed
>
> Cheers, Gidon
>
>
> On Sun, Sep 12, 2021 at 12:05 AM Xinli shang 
> wrote:
>
> > Hi everyone,
> >
> >
> > I propose the following RC to be released as the official Apache Parquet
> >  release.
> >
> >
> > The commit id is d1dccf6e680d86e94ce97005f5ac51848ba6d794
> >
> > * This corresponds to the tag: apache-parquet-1.12.1-rc0
> >
> > * https://github.com/apache/parquet-mr/tree/
> > d1dccf6e680d86e94ce97005f5ac51848ba6d794
> >
> >
> > The release tarball, signature, and checksums are here:
> >
> > *
> >
> https://dist.apache.org/repos/dist/dev/parquet/apache-parquet-1.12.1-rc0/
> >
> >
> > You can find the KEYS file here:
> >
> > * *https://dist.apache.org/repos/dist/release/parquet/KEYS
> > *
> >
> >
> > Binary artifacts are staged in Nexus here:
> >
> > *
> https://repository.apache.org/content/groups/staging/org/apache/parquet/
> >
> >
> > This release includes important changes listed
> > https://github.com/apache/parquet-mr/blob/parquet-1.12.x/CHANGES.md
> >
> >
> > Please download, verify, and test.
> >
> >
> > Please vote in the next 72 hours.
> >
> >
> > [ ] +1 Release this as Apache Parquet 1.12.1
> >
> > [ ] +0
> >
> > [ ] -1 Do not release this because...
> >
> > --
> > Xinli Shang
> >
>


[jira] [Resolved] (PARQUET-2078) Failed to read parquet file after writing with the same parquet version

2021-09-13 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky resolved PARQUET-2078.
---
Resolution: Fixed

Since the PR is merged I am resolving this.

> Failed to read parquet file after writing with the same parquet version
> ---
>
> Key: PARQUET-2078
> URL: https://issues.apache.org/jira/browse/PARQUET-2078
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.0
>Reporter: Nemon Lou
>Assignee: Nemon Lou
>Priority: Critical
> Fix For: 1.13.0, 1.12.1
>
> Attachments: 
> PARQUET_2078_how_to_fix_rowgroup_fileoffset_for_branch_1.12.x.patch, 
> tpcds_customer_footer.json
>
>
> Writing parquet  file with version 1.12.0 in Apache Hive, then read that 
> file, returns the following error:
> {noformat}
> Caused by: java.lang.IllegalStateException: All of the offsets in the split 
> should be found in the file. expected: [4, 133961161] found: 
> [BlockMetaData{1530100, 133961157 [ColumnMetaData{UNCOMPRESSED 
> [c_customer_sk] optional int64 c_customer_sk  [PLAIN, RLE, BIT_PACKED], 4}, 
> ColumnMetaData{UNCOMPRESSED [c_customer_id] optional binary c_customer_id 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 12243647}, ColumnMetaData{UNCOMPRESSED 
> [c_current_cdemo_sk] optional int64 c_current_cdemo_sk  [PLAIN, RLE, 
> BIT_PACKED], 42848491}, ColumnMetaData{UNCOMPRESSED [c_current_hdemo_sk] 
> optional int64 c_current_hdemo_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 54868535}, ColumnMetaData{UNCOMPRESSED [c_current_addr_sk] optional int64 
> c_current_addr_sk  [PLAIN, RLE, BIT_PACKED], 57421932}, 
> ColumnMetaData{UNCOMPRESSED [c_first_shipto_date_sk] optional int64 
> c_first_shipto_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 69694809}, 
> ColumnMetaData{UNCOMPRESSED [c_first_sales_date_sk] optional int64 
> c_first_sales_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 72093040}, 
> ColumnMetaData{UNCOMPRESSED [c_salutation] optional binary c_salutation 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 74461508}, 
> ColumnMetaData{UNCOMPRESSED [c_first_name] optional binary c_first_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 75092758}, 
> ColumnMetaData{UNCOMPRESSED [c_last_name] optional binary c_last_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 77626525}, 
> ColumnMetaData{UNCOMPRESSED [c_preferred_cust_flag] optional binary 
> c_preferred_cust_flag (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 80116456}, ColumnMetaData{UNCOMPRESSED [c_birth_day] optional int32 
> c_birth_day  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 80505351}, 
> ColumnMetaData{UNCOMPRESSED [c_birth_month] optional int32 c_birth_month  
> [RLE, PLAIN_DICTIONARY, BIT_PACKED], 81581772}, ColumnMetaData{UNCOMPRESSED 
> [c_birth_year] optional int32 c_birth_year  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 82473740}, ColumnMetaData{UNCOMPRESSED [c_birth_country] 
> optional binary c_birth_country (STRING)  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 83921564}, ColumnMetaData{UNCOMPRESSED [c_login] optional binary 
> c_login (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 85457674}, 
> ColumnMetaData{UNCOMPRESSED [c_email_address] optional binary c_email_address 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 85460523}, ColumnMetaData{UNCOMPRESSED 
> [c_last_review_date_sk] optional int64 c_last_review_date_sk  [RLE, 
> PLAIN_DICTIONARY, BIT_PACKED], 132146109}]}]
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.initializeInternalReader(ParquetRecordReader.java:172)
>  ~[parquet-hadoop-bundle-1.12.0.jar:1.12.0]
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.initialize(ParquetRecordReader.java:140)
>  ~[parquet-hadoop-bundle-1.12.0.jar:1.12.0]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.(ParquetRecordReaderWrapper.java:95)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.(ParquetRecordReaderWrapper.java:60)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat.getRecordReader(MapredParquetInputFormat.java:89)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at 
> org.apache.hadoop.hive.ql.io.CombineHiveRecordReader.(CombineHiveRecordReader.java:96)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native 
> Method) ~[?:1.8.0_292]
>   at 
> su

[jira] [Commented] (PARQUET-2088) Different created_by field values for application and library

2021-09-13 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-2088:
---

Could you please list what exact features do you think parquet-mr is 
enabling/disabling based on {{created_by}}? This field is used by the actual 
writer implementations (e.g. Impala, parquet-mr, parquet-cpp etc.). The example 
already explains how to use it: {{impala version 1.0 (build 
6cf94d29b2b7115df4de2c06e2ab4326d721eb55)}}

> Different created_by field values for application and library
> -
>
> Key: PARQUET-2088
> URL: https://issues.apache.org/jira/browse/PARQUET-2088
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Affects Versions: format-2.9.0
>Reporter: Joshua Howard
>Priority: Minor
>
> There seems to be a discrepancy in the Parquet format created_by field 
> regarding how it should be filled out. The parquet-mr library uses this value 
> to enable/disable features based on the parquet-mr version [here|#L64-L68]. 
> Meanwhile, users are encouraged to make use of the application version 
> [here|[https://www.javadoc.io/doc/org.apache.parquet/parquet-format/latest/org/apache/parquet/format/FileMetaData.html]].
>  It seems like there are multiple fields needed for an application and 
> library version. 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (PARQUET-2080) Deprecate RowGroup.file_offset

2021-09-13 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-2080:
---

[~gershinsky], however the original topic of this jira is invalid we still need 
to add proper comments to {{RowGroup.file_offset}} describing the situation of 
PARQUET-2078 and helping the implementations to handle the potential wrong 
value. Would you like to handle this?

> Deprecate RowGroup.file_offset
> --
>
> Key: PARQUET-2080
> URL: https://issues.apache.org/jira/browse/PARQUET-2080
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-format
>    Reporter: Gabor Szadovszky
>Assignee: Gabor Szadovszky
>Priority: Major
>
> Due to PARQUET-2078 RowGroup.file_offset is not reliable. We shall deprecate 
> the field and add suggestions how to calculate the value.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


Re: Any Parquet implementations might be impacted by PARQUET-2078

2021-08-30 Thread Gabor Szadovszky
It turned out that ColumnMetaData.dictionary_page_offset is not impacted by
this issue so it is much easier to handle. It seems that 1.12.0 is the
first parquet_mr release which writes ColumnChunk.file_offset and according
to PARQUET-2078
<https://issues.apache.org/jira/browse/PARQUET-2078?focusedCommentId=17405527=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17405527>
it
is invalid in certain cases. So any implementations need to have a way to
calculate/gather this offset without using the actual field. What we need
to ensure is that no one relies on the value of ColumnChunk.file_offset at
least in cases when the file was written by parquet-mr 1.12.0.

I've also created PARQUET-2080
<https://issues.apache.org/jira/browse/PARQUET-2080> to deprecate the field
in the format.

Regards,
Gabor

On Fri, Aug 27, 2021 at 11:11 AM Gabor Szadovszky  wrote:

> Hi everyone,
>
> It turned out that since parquet-mr 1.12.0 in certain conditions we write
> wrong values into ColumnMetaData.dictionary_page_offset
> <https://github.com/apache/parquet-format/blob/master/src/main/thrift/parquet.thrift#L753>
>  and
> ColumnChunk.file_offset
> <https://github.com/apache/parquet-format/blob/master/src/main/thrift/parquet.thrift#L790>.
> See details in PARQUET-2078
> <https://issues.apache.org/jira/browse/PARQUET-2078?focusedCommentId=17405527=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17405527>.
> Because of that any implementations that use these values have to be
> prepared for potential invalid values in case the file is written by
> parquet-mr 1.12.0.
>
> As per my understanding of the issue (to be verified) the
> distinguish between valid and invalid values of these offsets is quite
> easy: dictionary_page_offset is set to a value while the column chunk is
> not dictionary encoded (as per ColumnMetaData.encodings
> <https://github.com/apache/parquet-format/blob/master/src/main/thrift/parquet.thrift#L725>).
> In this case we have to use the offset of the first data page
> <https://github.com/apache/parquet-format/blob/master/src/main/thrift/parquet.thrift#L747>
> in the first column chunk of the row group.
>
> Regards,
> Gabor
>


[jira] [Created] (PARQUET-2080) Deprecate RowGroup.file_offset

2021-08-30 Thread Gabor Szadovszky (Jira)
Gabor Szadovszky created PARQUET-2080:
-

 Summary: Deprecate RowGroup.file_offset
 Key: PARQUET-2080
 URL: https://issues.apache.org/jira/browse/PARQUET-2080
 Project: Parquet
  Issue Type: Bug
  Components: parquet-format
Reporter: Gabor Szadovszky
Assignee: Gabor Szadovszky


Due to PARQUET-2078 RowGroup.file_offset is not reliable. We shall deprecate 
the field and add suggestions how to calculate the value.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Assigned] (PARQUET-2078) Failed to read parquet file after writing with the same parquet version

2021-08-30 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky reassigned PARQUET-2078:
-

Assignee: Nemon Lou

> Failed to read parquet file after writing with the same parquet version
> ---
>
> Key: PARQUET-2078
> URL: https://issues.apache.org/jira/browse/PARQUET-2078
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.0
>Reporter: Nemon Lou
>Assignee: Nemon Lou
>Priority: Critical
> Fix For: 1.13.0, 1.12.1
>
> Attachments: 
> PARQUET_2078_how_to_fix_rowgroup_fileoffset_for_branch_1.12.x.patch, 
> tpcds_customer_footer.json
>
>
> Writing parquet  file with version 1.12.0 in Apache Hive, then read that 
> file, returns the following error:
> {noformat}
> Caused by: java.lang.IllegalStateException: All of the offsets in the split 
> should be found in the file. expected: [4, 133961161] found: 
> [BlockMetaData{1530100, 133961157 [ColumnMetaData{UNCOMPRESSED 
> [c_customer_sk] optional int64 c_customer_sk  [PLAIN, RLE, BIT_PACKED], 4}, 
> ColumnMetaData{UNCOMPRESSED [c_customer_id] optional binary c_customer_id 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 12243647}, ColumnMetaData{UNCOMPRESSED 
> [c_current_cdemo_sk] optional int64 c_current_cdemo_sk  [PLAIN, RLE, 
> BIT_PACKED], 42848491}, ColumnMetaData{UNCOMPRESSED [c_current_hdemo_sk] 
> optional int64 c_current_hdemo_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 54868535}, ColumnMetaData{UNCOMPRESSED [c_current_addr_sk] optional int64 
> c_current_addr_sk  [PLAIN, RLE, BIT_PACKED], 57421932}, 
> ColumnMetaData{UNCOMPRESSED [c_first_shipto_date_sk] optional int64 
> c_first_shipto_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 69694809}, 
> ColumnMetaData{UNCOMPRESSED [c_first_sales_date_sk] optional int64 
> c_first_sales_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 72093040}, 
> ColumnMetaData{UNCOMPRESSED [c_salutation] optional binary c_salutation 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 74461508}, 
> ColumnMetaData{UNCOMPRESSED [c_first_name] optional binary c_first_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 75092758}, 
> ColumnMetaData{UNCOMPRESSED [c_last_name] optional binary c_last_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 77626525}, 
> ColumnMetaData{UNCOMPRESSED [c_preferred_cust_flag] optional binary 
> c_preferred_cust_flag (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 80116456}, ColumnMetaData{UNCOMPRESSED [c_birth_day] optional int32 
> c_birth_day  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 80505351}, 
> ColumnMetaData{UNCOMPRESSED [c_birth_month] optional int32 c_birth_month  
> [RLE, PLAIN_DICTIONARY, BIT_PACKED], 81581772}, ColumnMetaData{UNCOMPRESSED 
> [c_birth_year] optional int32 c_birth_year  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 82473740}, ColumnMetaData{UNCOMPRESSED [c_birth_country] 
> optional binary c_birth_country (STRING)  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 83921564}, ColumnMetaData{UNCOMPRESSED [c_login] optional binary 
> c_login (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 85457674}, 
> ColumnMetaData{UNCOMPRESSED [c_email_address] optional binary c_email_address 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 85460523}, ColumnMetaData{UNCOMPRESSED 
> [c_last_review_date_sk] optional int64 c_last_review_date_sk  [RLE, 
> PLAIN_DICTIONARY, BIT_PACKED], 132146109}]}]
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.initializeInternalReader(ParquetRecordReader.java:172)
>  ~[parquet-hadoop-bundle-1.12.0.jar:1.12.0]
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.initialize(ParquetRecordReader.java:140)
>  ~[parquet-hadoop-bundle-1.12.0.jar:1.12.0]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.(ParquetRecordReaderWrapper.java:95)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.(ParquetRecordReaderWrapper.java:60)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat.getRecordReader(MapredParquetInputFormat.java:89)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at 
> org.apache.hadoop.hive.ql.io.CombineHiveRecordReader.(CombineHiveRecordReader.java:96)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native 
> Method) ~[?:1.8.0_292]
>   at 
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccess

[jira] [Commented] (PARQUET-2078) Failed to read parquet file after writing with the same parquet version

2021-08-30 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-2078:
---

[~nemon], you are right, so {{dictionaryPageOffset}} is not impacted. Great 
news. 

After the second look it is not required to have the first column being 
dictionary encoded before the invalid row group. It is enough that there are 
dictionary encoded column chunks in the previous row groups and that the first 
column chunk is not dictionary encoded in the invalid row group. So, [~nemon], 
you also right with your PR. 

> Failed to read parquet file after writing with the same parquet version
> ---
>
> Key: PARQUET-2078
> URL: https://issues.apache.org/jira/browse/PARQUET-2078
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.0
>Reporter: Nemon Lou
>Priority: Critical
> Fix For: 1.13.0, 1.12.1
>
> Attachments: 
> PARQUET_2078_how_to_fix_rowgroup_fileoffset_for_branch_1.12.x.patch, 
> tpcds_customer_footer.json
>
>
> Writing parquet  file with version 1.12.0 in Apache Hive, then read that 
> file, returns the following error:
> {noformat}
> Caused by: java.lang.IllegalStateException: All of the offsets in the split 
> should be found in the file. expected: [4, 133961161] found: 
> [BlockMetaData{1530100, 133961157 [ColumnMetaData{UNCOMPRESSED 
> [c_customer_sk] optional int64 c_customer_sk  [PLAIN, RLE, BIT_PACKED], 4}, 
> ColumnMetaData{UNCOMPRESSED [c_customer_id] optional binary c_customer_id 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 12243647}, ColumnMetaData{UNCOMPRESSED 
> [c_current_cdemo_sk] optional int64 c_current_cdemo_sk  [PLAIN, RLE, 
> BIT_PACKED], 42848491}, ColumnMetaData{UNCOMPRESSED [c_current_hdemo_sk] 
> optional int64 c_current_hdemo_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 54868535}, ColumnMetaData{UNCOMPRESSED [c_current_addr_sk] optional int64 
> c_current_addr_sk  [PLAIN, RLE, BIT_PACKED], 57421932}, 
> ColumnMetaData{UNCOMPRESSED [c_first_shipto_date_sk] optional int64 
> c_first_shipto_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 69694809}, 
> ColumnMetaData{UNCOMPRESSED [c_first_sales_date_sk] optional int64 
> c_first_sales_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 72093040}, 
> ColumnMetaData{UNCOMPRESSED [c_salutation] optional binary c_salutation 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 74461508}, 
> ColumnMetaData{UNCOMPRESSED [c_first_name] optional binary c_first_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 75092758}, 
> ColumnMetaData{UNCOMPRESSED [c_last_name] optional binary c_last_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 77626525}, 
> ColumnMetaData{UNCOMPRESSED [c_preferred_cust_flag] optional binary 
> c_preferred_cust_flag (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 80116456}, ColumnMetaData{UNCOMPRESSED [c_birth_day] optional int32 
> c_birth_day  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 80505351}, 
> ColumnMetaData{UNCOMPRESSED [c_birth_month] optional int32 c_birth_month  
> [RLE, PLAIN_DICTIONARY, BIT_PACKED], 81581772}, ColumnMetaData{UNCOMPRESSED 
> [c_birth_year] optional int32 c_birth_year  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 82473740}, ColumnMetaData{UNCOMPRESSED [c_birth_country] 
> optional binary c_birth_country (STRING)  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 83921564}, ColumnMetaData{UNCOMPRESSED [c_login] optional binary 
> c_login (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 85457674}, 
> ColumnMetaData{UNCOMPRESSED [c_email_address] optional binary c_email_address 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 85460523}, ColumnMetaData{UNCOMPRESSED 
> [c_last_review_date_sk] optional int64 c_last_review_date_sk  [RLE, 
> PLAIN_DICTIONARY, BIT_PACKED], 132146109}]}]
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.initializeInternalReader(ParquetRecordReader.java:172)
>  ~[parquet-hadoop-bundle-1.12.0.jar:1.12.0]
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.initialize(ParquetRecordReader.java:140)
>  ~[parquet-hadoop-bundle-1.12.0.jar:1.12.0]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.(ParquetRecordReaderWrapper.java:95)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.(ParquetRecordReaderWrapper.java:60)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat.getRecordReader(MapredParquetInputFo

[jira] [Commented] (PARQUET-2078) Failed to read parquet file after writing with the same parquet version

2021-08-30 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-2078:
---

[~nemon], I am not sure how it would be possible. RowGroup.file_offset is set 
by using the dictionary page offset of the first column chunk (if there is any):
 * 
[rowGroup.setFile_offset(block.getStartingPos())|https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java#L580]
 * 
[BlockMetaData.getStartingPos()|https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/BlockMetaData.java#L102-L104]
 * 
[ColumnChunkMetaData.getStartingPos()|https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ColumnChunkMetaData.java#L184-L193]

As per my understanding the issue is based on the following to have the wrong 
offsets of {{rowGroup ~n~}} (where we have {{k}} columns):
* {{columnChunk ~n-1, 1~}} (first column chunk of {{rowGroup ~n-1~}}) is 
dictionary encoded as well as {{columnChunk ~n-1, k~}}
* {{columnChunk ~n, 1~}} is not dictionary encoded
In this case {{fileOffset ~n~ = dictionaryOffset ~n, 1~ = dictionaryOffset 
~n-1, k~}}

To discover this issue we should check if a column chunk is dictionary encoded 
before using the dictionary offset of it. Unfortunately, we have to do the same 
before using the file offset of a row group, or simply ignore this value and 
use the offsets of the first column chunk with the check.

> Failed to read parquet file after writing with the same parquet version
> ---
>
> Key: PARQUET-2078
> URL: https://issues.apache.org/jira/browse/PARQUET-2078
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.0
>Reporter: Nemon Lou
>Priority: Critical
> Fix For: 1.13.0, 1.12.1
>
> Attachments: 
> PARQUET_2078_how_to_fix_rowgroup_fileoffset_for_branch_1.12.x.patch, 
> tpcds_customer_footer.json
>
>
> Writing parquet  file with version 1.12.0 in Apache Hive, then read that 
> file, returns the following error:
> {noformat}
> Caused by: java.lang.IllegalStateException: All of the offsets in the split 
> should be found in the file. expected: [4, 133961161] found: 
> [BlockMetaData{1530100, 133961157 [ColumnMetaData{UNCOMPRESSED 
> [c_customer_sk] optional int64 c_customer_sk  [PLAIN, RLE, BIT_PACKED], 4}, 
> ColumnMetaData{UNCOMPRESSED [c_customer_id] optional binary c_customer_id 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 12243647}, ColumnMetaData{UNCOMPRESSED 
> [c_current_cdemo_sk] optional int64 c_current_cdemo_sk  [PLAIN, RLE, 
> BIT_PACKED], 42848491}, ColumnMetaData{UNCOMPRESSED [c_current_hdemo_sk] 
> optional int64 c_current_hdemo_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 54868535}, ColumnMetaData{UNCOMPRESSED [c_current_addr_sk] optional int64 
> c_current_addr_sk  [PLAIN, RLE, BIT_PACKED], 57421932}, 
> ColumnMetaData{UNCOMPRESSED [c_first_shipto_date_sk] optional int64 
> c_first_shipto_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 69694809}, 
> ColumnMetaData{UNCOMPRESSED [c_first_sales_date_sk] optional int64 
> c_first_sales_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 72093040}, 
> ColumnMetaData{UNCOMPRESSED [c_salutation] optional binary c_salutation 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 74461508}, 
> ColumnMetaData{UNCOMPRESSED [c_first_name] optional binary c_first_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 75092758}, 
> ColumnMetaData{UNCOMPRESSED [c_last_name] optional binary c_last_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 77626525}, 
> ColumnMetaData{UNCOMPRESSED [c_preferred_cust_flag] optional binary 
> c_preferred_cust_flag (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 80116456}, ColumnMetaData{UNCOMPRESSED [c_birth_day] optional int32 
> c_birth_day  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 80505351}, 
> ColumnMetaData{UNCOMPRESSED [c_birth_month] optional int32 c_birth_month  
> [RLE, PLAIN_DICTIONARY, BIT_PACKED], 81581772}, ColumnMetaData{UNCOMPRESSED 
> [c_birth_year] optional int32 c_birth_year  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 82473740}, ColumnMetaData{UNCOMPRESSED [c_birth_country] 
> optional binary c_birth_country (STRING)  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 83921564}, ColumnMetaData{UNCOMPRESSED [c_login] optional binary 
> c_login (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 85457674}, 
> ColumnMetaData{UNCOMPRESSED [c_email_address] optional binary c_email_address 
> (STRING)  [PLAIN, RLE, B

[jira] [Commented] (PARQUET-2078) Failed to read parquet file after writing with the same parquet version

2021-08-27 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-2078:
---

Added the dev list thread link here to keep both sides in the loop.

> Failed to read parquet file after writing with the same parquet version
> ---
>
> Key: PARQUET-2078
> URL: https://issues.apache.org/jira/browse/PARQUET-2078
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.0
>Reporter: Nemon Lou
>Priority: Critical
> Fix For: 1.13.0, 1.12.1
>
> Attachments: 
> PARQUET_2078_how_to_fix_rowgroup_fileoffset_for_branch_1.12.x.patch, 
> tpcds_customer_footer.json
>
>
> Writing parquet  file with version 1.12.0 in Apache Hive, then read that 
> file, returns the following error:
> {noformat}
> Caused by: java.lang.IllegalStateException: All of the offsets in the split 
> should be found in the file. expected: [4, 133961161] found: 
> [BlockMetaData{1530100, 133961157 [ColumnMetaData{UNCOMPRESSED 
> [c_customer_sk] optional int64 c_customer_sk  [PLAIN, RLE, BIT_PACKED], 4}, 
> ColumnMetaData{UNCOMPRESSED [c_customer_id] optional binary c_customer_id 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 12243647}, ColumnMetaData{UNCOMPRESSED 
> [c_current_cdemo_sk] optional int64 c_current_cdemo_sk  [PLAIN, RLE, 
> BIT_PACKED], 42848491}, ColumnMetaData{UNCOMPRESSED [c_current_hdemo_sk] 
> optional int64 c_current_hdemo_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 54868535}, ColumnMetaData{UNCOMPRESSED [c_current_addr_sk] optional int64 
> c_current_addr_sk  [PLAIN, RLE, BIT_PACKED], 57421932}, 
> ColumnMetaData{UNCOMPRESSED [c_first_shipto_date_sk] optional int64 
> c_first_shipto_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 69694809}, 
> ColumnMetaData{UNCOMPRESSED [c_first_sales_date_sk] optional int64 
> c_first_sales_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 72093040}, 
> ColumnMetaData{UNCOMPRESSED [c_salutation] optional binary c_salutation 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 74461508}, 
> ColumnMetaData{UNCOMPRESSED [c_first_name] optional binary c_first_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 75092758}, 
> ColumnMetaData{UNCOMPRESSED [c_last_name] optional binary c_last_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 77626525}, 
> ColumnMetaData{UNCOMPRESSED [c_preferred_cust_flag] optional binary 
> c_preferred_cust_flag (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 80116456}, ColumnMetaData{UNCOMPRESSED [c_birth_day] optional int32 
> c_birth_day  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 80505351}, 
> ColumnMetaData{UNCOMPRESSED [c_birth_month] optional int32 c_birth_month  
> [RLE, PLAIN_DICTIONARY, BIT_PACKED], 81581772}, ColumnMetaData{UNCOMPRESSED 
> [c_birth_year] optional int32 c_birth_year  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 82473740}, ColumnMetaData{UNCOMPRESSED [c_birth_country] 
> optional binary c_birth_country (STRING)  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 83921564}, ColumnMetaData{UNCOMPRESSED [c_login] optional binary 
> c_login (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 85457674}, 
> ColumnMetaData{UNCOMPRESSED [c_email_address] optional binary c_email_address 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 85460523}, ColumnMetaData{UNCOMPRESSED 
> [c_last_review_date_sk] optional int64 c_last_review_date_sk  [RLE, 
> PLAIN_DICTIONARY, BIT_PACKED], 132146109}]}]
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.initializeInternalReader(ParquetRecordReader.java:172)
>  ~[parquet-hadoop-bundle-1.12.0.jar:1.12.0]
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.initialize(ParquetRecordReader.java:140)
>  ~[parquet-hadoop-bundle-1.12.0.jar:1.12.0]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.(ParquetRecordReaderWrapper.java:95)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.(ParquetRecordReaderWrapper.java:60)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat.getRecordReader(MapredParquetInputFormat.java:89)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at 
> org.apache.hadoop.hive.ql.io.CombineHiveRecordReader.(CombineHiveRecordReader.java:96)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native 
> Method) ~[?:1.8.0_292]
>   at 
> sun.r

Any Parquet implementations might be impacted by PARQUET-2078

2021-08-27 Thread Gabor Szadovszky
Hi everyone,

It turned out that since parquet-mr 1.12.0 in certain conditions we write
wrong values into ColumnMetaData.dictionary_page_offset

and
ColumnChunk.file_offset
.
See details in PARQUET-2078
.
Because of that any implementations that use these values have to be
prepared for potential invalid values in case the file is written by
parquet-mr 1.12.0.

As per my understanding of the issue (to be verified) the
distinguish between valid and invalid values of these offsets is quite
easy: dictionary_page_offset is set to a value while the column chunk is
not dictionary encoded (as per ColumnMetaData.encodings
).
In this case we have to use the offset of the first data page

in the first column chunk of the row group.

Regards,
Gabor


[jira] [Comment Edited] (PARQUET-2078) Failed to read parquet file after writing with the same parquet version

2021-08-27 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky edited comment on PARQUET-2078 at 8/27/21, 8:50 AM:
-

[~nemon], thanks a lot for the detailed explanation and the patch! So what I 
have written before stands. Before 1.12.0 we did not write the dictionary 
offset to the column chunk metadata (see PARQUET-1850) even though the 
calculation was wrong since the beginning. Since we released 1.12.0 already it 
means we have to prepare for the invalid dictionary offset values.

What we need to handle in a fix:
* Fix the calculation issue (see the attached patch)
* Add unit test for this issue to ensure it works properly and won't happen 
again
* Investigate all code parts where the dictionary offset and file offset are 
used and prepare for invalid values

[~nemon], would you like to work on this by opening a PR on github?


was (Author: gszadovszky):
[~nemon], thanks a lot for the detailed explanation and the patch! So what I 
have written before stands. Before 1.12.0 we did not write the dictionary 
offset to the column chunk metadata (see PARQUET-1850) even though the 
calculation was wrong since the beginning. Since we released 1.12.0 already it 
means we have to prepare for the invalid dictionary offset values.

What we need to handle in a fix:
* Fix the calculation issue (see the attached patch)
* Add unit test for this issue to ensure it works properly and won't happen 
again
* Investigate all code parts where the dictionary offset is used and prepare 
for invalid values

[~nemon], would you like to work on this by opening a PR on github?

> Failed to read parquet file after writing with the same parquet version
> ---
>
> Key: PARQUET-2078
> URL: https://issues.apache.org/jira/browse/PARQUET-2078
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.0
>Reporter: Nemon Lou
>Priority: Critical
> Fix For: 1.13.0, 1.12.1
>
> Attachments: 
> PARQUET_2078_how_to_fix_rowgroup_fileoffset_for_branch_1.12.x.patch, 
> tpcds_customer_footer.json
>
>
> Writing parquet  file with version 1.12.0 in Apache Hive, then read that 
> file, returns the following error:
> {noformat}
> Caused by: java.lang.IllegalStateException: All of the offsets in the split 
> should be found in the file. expected: [4, 133961161] found: 
> [BlockMetaData{1530100, 133961157 [ColumnMetaData{UNCOMPRESSED 
> [c_customer_sk] optional int64 c_customer_sk  [PLAIN, RLE, BIT_PACKED], 4}, 
> ColumnMetaData{UNCOMPRESSED [c_customer_id] optional binary c_customer_id 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 12243647}, ColumnMetaData{UNCOMPRESSED 
> [c_current_cdemo_sk] optional int64 c_current_cdemo_sk  [PLAIN, RLE, 
> BIT_PACKED], 42848491}, ColumnMetaData{UNCOMPRESSED [c_current_hdemo_sk] 
> optional int64 c_current_hdemo_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 54868535}, ColumnMetaData{UNCOMPRESSED [c_current_addr_sk] optional int64 
> c_current_addr_sk  [PLAIN, RLE, BIT_PACKED], 57421932}, 
> ColumnMetaData{UNCOMPRESSED [c_first_shipto_date_sk] optional int64 
> c_first_shipto_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 69694809}, 
> ColumnMetaData{UNCOMPRESSED [c_first_sales_date_sk] optional int64 
> c_first_sales_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 72093040}, 
> ColumnMetaData{UNCOMPRESSED [c_salutation] optional binary c_salutation 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 74461508}, 
> ColumnMetaData{UNCOMPRESSED [c_first_name] optional binary c_first_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 75092758}, 
> ColumnMetaData{UNCOMPRESSED [c_last_name] optional binary c_last_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 77626525}, 
> ColumnMetaData{UNCOMPRESSED [c_preferred_cust_flag] optional binary 
> c_preferred_cust_flag (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 80116456}, ColumnMetaData{UNCOMPRESSED [c_birth_day] optional int32 
> c_birth_day  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 80505351}, 
> ColumnMetaData{UNCOMPRESSED [c_birth_month] optional int32 c_birth_month  
> [RLE, PLAIN_DICTIONARY, BIT_PACKED], 81581772}, ColumnMetaData{UNCOMPRESSED 
> [c_birth_year] optional int32 c_birth_year  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 82473740}, ColumnMetaData{UNCOMPRESSED [c_birth_country] 
> optional binary c_birth_country (STRING)  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 83921564}, ColumnMetaData{UNCOMPRESSED [c_login] optional binary 
> c_login (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 85457674}, 
> ColumnMetaData{UNCOMPRESSED [

[jira] [Commented] (PARQUET-2078) Failed to read parquet file after writing with the same parquet version

2021-08-27 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-2078:
---

[~nemon], thanks a lot for the detailed explanation and the patch! So what I 
have written before stands. Before 1.12.0 we did not write the dictionary 
offset to the column chunk metadata (see PARQUET-1850) even though the 
calculation was wrong since the beginning. Since we released 1.12.0 already it 
means we have to prepare for the invalid dictionary offset values.

What we need to handle in a fix:
* Fix the calculation issue (see the attached patch)
* Add unit test for this issue to ensure it works properly and won't happen 
again
* Investigate all code parts where the dictionary offset is used and prepare 
for invalid values

[~nemon], would you like to work on this by opening a PR on github?

> Failed to read parquet file after writing with the same parquet version
> ---
>
> Key: PARQUET-2078
> URL: https://issues.apache.org/jira/browse/PARQUET-2078
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.0
>Reporter: Nemon Lou
>Priority: Critical
> Fix For: 1.13.0, 1.12.1
>
> Attachments: 
> PARQUET_2078_how_to_fix_rowgroup_fileoffset_for_branch_1.12.x.patch, 
> tpcds_customer_footer.json
>
>
> Writing parquet  file with version 1.12.0 in Apache Hive, then read that 
> file, returns the following error:
> {noformat}
> Caused by: java.lang.IllegalStateException: All of the offsets in the split 
> should be found in the file. expected: [4, 133961161] found: 
> [BlockMetaData{1530100, 133961157 [ColumnMetaData{UNCOMPRESSED 
> [c_customer_sk] optional int64 c_customer_sk  [PLAIN, RLE, BIT_PACKED], 4}, 
> ColumnMetaData{UNCOMPRESSED [c_customer_id] optional binary c_customer_id 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 12243647}, ColumnMetaData{UNCOMPRESSED 
> [c_current_cdemo_sk] optional int64 c_current_cdemo_sk  [PLAIN, RLE, 
> BIT_PACKED], 42848491}, ColumnMetaData{UNCOMPRESSED [c_current_hdemo_sk] 
> optional int64 c_current_hdemo_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 54868535}, ColumnMetaData{UNCOMPRESSED [c_current_addr_sk] optional int64 
> c_current_addr_sk  [PLAIN, RLE, BIT_PACKED], 57421932}, 
> ColumnMetaData{UNCOMPRESSED [c_first_shipto_date_sk] optional int64 
> c_first_shipto_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 69694809}, 
> ColumnMetaData{UNCOMPRESSED [c_first_sales_date_sk] optional int64 
> c_first_sales_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 72093040}, 
> ColumnMetaData{UNCOMPRESSED [c_salutation] optional binary c_salutation 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 74461508}, 
> ColumnMetaData{UNCOMPRESSED [c_first_name] optional binary c_first_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 75092758}, 
> ColumnMetaData{UNCOMPRESSED [c_last_name] optional binary c_last_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 77626525}, 
> ColumnMetaData{UNCOMPRESSED [c_preferred_cust_flag] optional binary 
> c_preferred_cust_flag (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 80116456}, ColumnMetaData{UNCOMPRESSED [c_birth_day] optional int32 
> c_birth_day  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 80505351}, 
> ColumnMetaData{UNCOMPRESSED [c_birth_month] optional int32 c_birth_month  
> [RLE, PLAIN_DICTIONARY, BIT_PACKED], 81581772}, ColumnMetaData{UNCOMPRESSED 
> [c_birth_year] optional int32 c_birth_year  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 82473740}, ColumnMetaData{UNCOMPRESSED [c_birth_country] 
> optional binary c_birth_country (STRING)  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 83921564}, ColumnMetaData{UNCOMPRESSED [c_login] optional binary 
> c_login (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 85457674}, 
> ColumnMetaData{UNCOMPRESSED [c_email_address] optional binary c_email_address 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 85460523}, ColumnMetaData{UNCOMPRESSED 
> [c_last_review_date_sk] optional int64 c_last_review_date_sk  [RLE, 
> PLAIN_DICTIONARY, BIT_PACKED], 132146109}]}]
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.initializeInternalReader(ParquetRecordReader.java:172)
>  ~[parquet-hadoop-bundle-1.12.0.jar:1.12.0]
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.initialize(ParquetRecordReader.java:140)
>  ~[parquet-hadoop-bundle-1.12.0.jar:1.12.0]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.(ParquetRecordReaderWrapper.java:95)
>  ~[hive-exec-4.0.0

[jira] [Commented] (PARQUET-2078) Failed to read parquet file after writing with the same parquet version

2021-08-26 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-2078:
---

[~nemon], thanks a lot for the investigation. What is not clear to me how it 
could happen that we set the wrong value to {{RowGroup.file_offset}}. Based on 
the code in 
[ParquetMetadataConverter|https://github.com/apache/parquet-mr/blob/apache-parquet-1.12.0/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java#L580]
 we use the starting position of the first column chunk of the actual row 
group. The starting position of the column chunk is the dictionary page offset 
or the first data page offset, whatever is the smaller (because dictionary page 
is always at the starting position of the column chunk.) If the dictionary page 
offset or the first data page offset would be wrong we should have other issues 
as well. Can you read the file content without using InputSplits (e.g. 
parquet-tools, parquet-cli or java code that reads the whole file)? There is a 
new parquet-cli tool called footer that can list the raw footer of the file. It 
would be interesting to see the output of it on the related parquet file. 
Unfortunately, this feature is not released yet so it have to be built from 
master. If you are interested to do so please check the 
[readme|https://github.com/apache/parquet-mr/blob/master/parquet-cli/README.md] 
for details.

If you are right and we write invalid offsets to the file since 1.12.0 that it 
is a serious issue. We not only have to fix the writing path but the reading as 
well since we will have files already written by 1.12.0.

> Failed to read parquet file after writing with the same parquet version
> ---
>
> Key: PARQUET-2078
> URL: https://issues.apache.org/jira/browse/PARQUET-2078
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.0
>Reporter: Nemon Lou
>Priority: Critical
>
> Writing parquet  file with version 1.12.0 in Apache Hive, then read that 
> file, returns the following error:
> {noformat}
> Caused by: java.lang.IllegalStateException: All of the offsets in the split 
> should be found in the file. expected: [4, 133961161] found: 
> [BlockMetaData{1530100, 133961157 [ColumnMetaData{UNCOMPRESSED 
> [c_customer_sk] optional int64 c_customer_sk  [PLAIN, RLE, BIT_PACKED], 4}, 
> ColumnMetaData{UNCOMPRESSED [c_customer_id] optional binary c_customer_id 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 12243647}, ColumnMetaData{UNCOMPRESSED 
> [c_current_cdemo_sk] optional int64 c_current_cdemo_sk  [PLAIN, RLE, 
> BIT_PACKED], 42848491}, ColumnMetaData{UNCOMPRESSED [c_current_hdemo_sk] 
> optional int64 c_current_hdemo_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 54868535}, ColumnMetaData{UNCOMPRESSED [c_current_addr_sk] optional int64 
> c_current_addr_sk  [PLAIN, RLE, BIT_PACKED], 57421932}, 
> ColumnMetaData{UNCOMPRESSED [c_first_shipto_date_sk] optional int64 
> c_first_shipto_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 69694809}, 
> ColumnMetaData{UNCOMPRESSED [c_first_sales_date_sk] optional int64 
> c_first_sales_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 72093040}, 
> ColumnMetaData{UNCOMPRESSED [c_salutation] optional binary c_salutation 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 74461508}, 
> ColumnMetaData{UNCOMPRESSED [c_first_name] optional binary c_first_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 75092758}, 
> ColumnMetaData{UNCOMPRESSED [c_last_name] optional binary c_last_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 77626525}, 
> ColumnMetaData{UNCOMPRESSED [c_preferred_cust_flag] optional binary 
> c_preferred_cust_flag (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 80116456}, ColumnMetaData{UNCOMPRESSED [c_birth_day] optional int32 
> c_birth_day  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 80505351}, 
> ColumnMetaData{UNCOMPRESSED [c_birth_month] optional int32 c_birth_month  
> [RLE, PLAIN_DICTIONARY, BIT_PACKED], 81581772}, ColumnMetaData{UNCOMPRESSED 
> [c_birth_year] optional int32 c_birth_year  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 82473740}, ColumnMetaData{UNCOMPRESSED [c_birth_country] 
> optional binary c_birth_country (STRING)  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 83921564}, ColumnMetaData{UNCOMPRESSED [c_login] optional binary 
> c_login (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 85457674}, 
> ColumnMetaData{UNCOMPRESSED [c_email_address] optional binary c_email_address 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 85460523}, ColumnMetaData{UNCOMPRESSED 
> [c_last_review_date_sk] optional int64 c_last_revi

[jira] [Updated] (PARQUET-2078) Failed to read parquet file after writing with the same parquet version

2021-08-26 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky updated PARQUET-2078:
--
Fix Version/s: 1.12.1
   1.13.0

> Failed to read parquet file after writing with the same parquet version
> ---
>
> Key: PARQUET-2078
> URL: https://issues.apache.org/jira/browse/PARQUET-2078
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.0
>Reporter: Nemon Lou
>Priority: Critical
> Fix For: 1.13.0, 1.12.1
>
>
> Writing parquet  file with version 1.12.0 in Apache Hive, then read that 
> file, returns the following error:
> {noformat}
> Caused by: java.lang.IllegalStateException: All of the offsets in the split 
> should be found in the file. expected: [4, 133961161] found: 
> [BlockMetaData{1530100, 133961157 [ColumnMetaData{UNCOMPRESSED 
> [c_customer_sk] optional int64 c_customer_sk  [PLAIN, RLE, BIT_PACKED], 4}, 
> ColumnMetaData{UNCOMPRESSED [c_customer_id] optional binary c_customer_id 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 12243647}, ColumnMetaData{UNCOMPRESSED 
> [c_current_cdemo_sk] optional int64 c_current_cdemo_sk  [PLAIN, RLE, 
> BIT_PACKED], 42848491}, ColumnMetaData{UNCOMPRESSED [c_current_hdemo_sk] 
> optional int64 c_current_hdemo_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 54868535}, ColumnMetaData{UNCOMPRESSED [c_current_addr_sk] optional int64 
> c_current_addr_sk  [PLAIN, RLE, BIT_PACKED], 57421932}, 
> ColumnMetaData{UNCOMPRESSED [c_first_shipto_date_sk] optional int64 
> c_first_shipto_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 69694809}, 
> ColumnMetaData{UNCOMPRESSED [c_first_sales_date_sk] optional int64 
> c_first_sales_date_sk  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 72093040}, 
> ColumnMetaData{UNCOMPRESSED [c_salutation] optional binary c_salutation 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 74461508}, 
> ColumnMetaData{UNCOMPRESSED [c_first_name] optional binary c_first_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 75092758}, 
> ColumnMetaData{UNCOMPRESSED [c_last_name] optional binary c_last_name 
> (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 77626525}, 
> ColumnMetaData{UNCOMPRESSED [c_preferred_cust_flag] optional binary 
> c_preferred_cust_flag (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 
> 80116456}, ColumnMetaData{UNCOMPRESSED [c_birth_day] optional int32 
> c_birth_day  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 80505351}, 
> ColumnMetaData{UNCOMPRESSED [c_birth_month] optional int32 c_birth_month  
> [RLE, PLAIN_DICTIONARY, BIT_PACKED], 81581772}, ColumnMetaData{UNCOMPRESSED 
> [c_birth_year] optional int32 c_birth_year  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 82473740}, ColumnMetaData{UNCOMPRESSED [c_birth_country] 
> optional binary c_birth_country (STRING)  [RLE, PLAIN_DICTIONARY, 
> BIT_PACKED], 83921564}, ColumnMetaData{UNCOMPRESSED [c_login] optional binary 
> c_login (STRING)  [RLE, PLAIN_DICTIONARY, BIT_PACKED], 85457674}, 
> ColumnMetaData{UNCOMPRESSED [c_email_address] optional binary c_email_address 
> (STRING)  [PLAIN, RLE, BIT_PACKED], 85460523}, ColumnMetaData{UNCOMPRESSED 
> [c_last_review_date_sk] optional int64 c_last_review_date_sk  [RLE, 
> PLAIN_DICTIONARY, BIT_PACKED], 132146109}]}]
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.initializeInternalReader(ParquetRecordReader.java:172)
>  ~[parquet-hadoop-bundle-1.12.0.jar:1.12.0]
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.initialize(ParquetRecordReader.java:140)
>  ~[parquet-hadoop-bundle-1.12.0.jar:1.12.0]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.(ParquetRecordReaderWrapper.java:95)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.(ParquetRecordReaderWrapper.java:60)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at 
> org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat.getRecordReader(MapredParquetInputFormat.java:89)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at 
> org.apache.hadoop.hive.ql.io.CombineHiveRecordReader.(CombineHiveRecordReader.java:96)
>  ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT]
>   at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native 
> Method) ~[?:1.8.0_292]
>   at 
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
>  ~[?:1.8.0_292]
>   at 
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAcce

[jira] [Commented] (PARQUET-2071) Encryption translation tool

2021-08-23 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-2071:
---

[~sha...@uber.com], sure, I am fine with having the "universal tool" and the 
required refactors be handled under the separate jira.

> Encryption translation tool 
> 
>
> Key: PARQUET-2071
> URL: https://issues.apache.org/jira/browse/PARQUET-2071
> Project: Parquet
>  Issue Type: New Feature
>  Components: parquet-mr
>Reporter: Xinli Shang
>Assignee: Xinli Shang
>Priority: Major
>
> When translating existing data to encryption state, we could develop a tool 
> like TransCompression to translate the data at page level to encryption state 
> without reading to record and rewrite. This will speed up the process a lot. 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


Re: 【vulnerability confirmation】parquet-format-structures-1.12.0

2021-08-17 Thread Gabor Szadovszky
Hi,

It is required to shade the thrift library into paquet-format-structures
because we use thrift to serialize/deserialize the metadata structures in
the parquet files. So, you really don't have any way to change it at
runtime. If it is urgent you may build your parquet-mr on your own with an
upgraded thrift version. (The upgrade to 0.14.1 is already in master. See
PARQUET-2005 for details.)
It is unfortunate that we missed (or was not able) to upgrade the thrift
library for the 1.12.0 release.

I think there are no big risks to do a thrift upgrade in a bugfix release
(1.12.1) but I would like to hear opinions from the community. I cannot say
any ETA for this release but there are other jiras in the queue already.

Cheers,
Gabor

On Mon, Aug 16, 2021 at 7:21 PM huhaiyang (C)  wrote:

> Hi all,
> To whom it may concerned, when I introduce Parquet-format-structure
> package in my project, there is a problem I have to deal with.
> It was a vulnerability of shaded component libthrift 0.13.0 found in the
> latest release version 1.12.0, which the CVE No. is CVE-2020-13949. It
> disturbed me so much that I have no idea how to avoid this vulnerability as
> there is no bug-fixed version since Mar 25.
> Now I am sincerely asking you when the new version will be available or is
> there a solution to handle the vulnerability.
>


[jira] [Resolved] (PARQUET-2064) Make Range public accessible in RowRanges

2021-08-16 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky resolved PARQUET-2064.
---
Resolution: Fixed

> Make Range public accessible in RowRanges
> -
>
> Key: PARQUET-2064
> URL: https://issues.apache.org/jira/browse/PARQUET-2064
> Project: Parquet
>  Issue Type: New Feature
>Reporter: Xinli Shang
>Assignee: Xinli Shang
>Priority: Major
>
> When rolling out to Presto, I found we need to know the boundaries of each 
> Range in RowRanges. It is still doable with Iterator but Presto has. batch 
> reader, we cannot use iterator for each row. 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Resolved] (PARQUET-2073) Is there something wrong calculate usedMem in ColumnWriteStoreBase.java

2021-08-16 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky resolved PARQUET-2073.
---
Resolution: Fixed

> Is there something wrong calculate usedMem in ColumnWriteStoreBase.java
> ---
>
> Key: PARQUET-2073
> URL: https://issues.apache.org/jira/browse/PARQUET-2073
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.0
>Reporter: JiangYang
>Assignee: JiangYang
>Priority: Critical
> Attachments: image-2021-08-05-14-37-51-299.png
>
>
> !image-2021-08-05-14-37-51-299.png!



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Resolved] (PARQUET-2059) Tests require too much memory

2021-08-16 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky resolved PARQUET-2059.
---
Resolution: Fixed

> Tests require too much memory
> -
>
> Key: PARQUET-2059
> URL: https://issues.apache.org/jira/browse/PARQUET-2059
> Project: Parquet
>  Issue Type: Test
>    Reporter: Gabor Szadovszky
>    Assignee: Gabor Szadovszky
>Priority: Major
>
> For testing the solution of PARQUET-1633 we require ~3GB memory that is not 
> always available. To solve this issue we temporarily disabled the implemented 
> unit test.
> We need to ensure somehow that [this 
> test|https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestLargeColumnChunk.java]
>  (and maybe some other similar ones) are executed regularly. Some options we 
> might have:
> * Execute this test separately with a maven profile. I am not sure if the CI 
> allows allocating such large memory but with Xmx options we might give a try 
> and create a separate check for this test only.
> * Similar to the previous with the profile but not executing in the CI ever. 
> Instead, we add some comments to the release doc so this test will be 
> executed at least once per release.
> * Configuring the CI profile to skip this test but have it in the normal 
> scenario meaning the devs will execute it locally. There are a couple of cons 
> though. There is no guarantee that devs executes all the tests including this 
> one. It also can cause issues if the dev doesn't have enough memory and don't 
> know that the test failure is not related to the current change.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Resolved] (PARQUET-2043) Fail build for used but not declared direct dependencies

2021-08-16 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky resolved PARQUET-2043.
---
Resolution: Fixed

> Fail build for used but not declared direct dependencies
> 
>
> Key: PARQUET-2043
> URL: https://issues.apache.org/jira/browse/PARQUET-2043
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>    Reporter: Gabor Szadovszky
>Assignee: Gabor Szadovszky
>Priority: Major
>
> It is always a good practice to specify all the dependencies directly used 
> (classes are imported from) by our modules. We have a couple of issues where 
> classes are imported from transitive dependencies. It makes hard to validate 
> the actual dependency tree and also may result in using wrong versions of 
> classes (see PARQUET-2038 for example).
> It would be good to enforce to reference such dependencies directly in the 
> module poms. The [maven-dependency-plugin analyze-only 
> goal|http://maven.apache.org/plugins/maven-dependency-plugin/analyze-only-mojo.html]
>  can be used for this purpose.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Resolved] (PARQUET-2063) Remove Compile Warnings from MemoryManager

2021-08-10 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky resolved PARQUET-2063.
---
Resolution: Fixed

> Remove Compile Warnings from MemoryManager
> --
>
> Key: PARQUET-2063
> URL: https://issues.apache.org/jira/browse/PARQUET-2063
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Reporter: David Mollitor
>Assignee: David Mollitor
>Priority: Minor
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (PARQUET-2074) Upgrade to JDK 9+

2021-08-09 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky commented on PARQUET-2074:
---

[~belugabehr], it sounds good to me but also keep in mind that switching to 
JDK9 and using its new capabilities would make parqet-mr incompatible with 
certain environments. Also, this would require a community agreement.

I would suggest bringing up this topic in the next parquet sync (Aug 24) and/or 
start a formal vote in the dev list.

> Upgrade to JDK 9+
> -
>
> Key: PARQUET-2074
> URL: https://issues.apache.org/jira/browse/PARQUET-2074
> Project: Parquet
>  Issue Type: Improvement
>Reporter: David Mollitor
>Priority: Major
>
> Moving to JDK 9 will provide a plethora of new compares/equals capabilities 
> on arrays that are all based on vectorization and implement 
> {{\@IntrinsicCandidate}}
> https://docs.oracle.com/javase/9/docs/api/java/util/Arrays.html



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Assigned] (PARQUET-2073) Is there something wrong calculate usedMem in ColumnWriteStoreBase.java

2021-08-09 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky reassigned PARQUET-2073:
-

Assignee: JiangYang

> Is there something wrong calculate usedMem in ColumnWriteStoreBase.java
> ---
>
> Key: PARQUET-2073
> URL: https://issues.apache.org/jira/browse/PARQUET-2073
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.0
>Reporter: JiangYang
>Assignee: JiangYang
>Priority: Critical
> Attachments: image-2021-08-05-14-37-51-299.png
>
>
> !image-2021-08-05-14-37-51-299.png!



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


  1   2   3   4   5   6   7   8   9   10   >