You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@parquet.apache.org by "Yujiang Zhong (Jira)" <ji...@apache.org> on 2022/08/05 02:51:00 UTC

[jira] [Comment Edited] (PARQUET-2160) Close decompression stream to free off-heap memory in time

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

Yujiang Zhong edited comment on PARQUET-2160 at 8/5/22 2:50 AM:
----------------------------------------------------------------

{quote}What about query performance? was it affected too since it now incurs an extra copy.
{quote}
[~csun] 

It does not add an extra copy actually,  BytesInput#copy  reads bytes from the input stream and constructs a ByteArrayBytesInput as return,  these bytes will be wrapped as ByteByffer later to construct a ByteBufferInputStream. This is the same as the current process, just that bytes are read in advance. I had run some queries to test (not strict benchmark) and the results show that there is no performance loss.


was (Author: JIRAUSER291005):
??What about query performance? was it affected too since it now incurs an extra copy.??

[~csun] 

It does not add an extra copy actually,  BytesInput#copy  reads bytes from the input stream and constructs a ByteArrayBytesInput as return,  these bytes will be wrapped as ByteByffer later to construct a ByteBufferInputStream. This is the same as the current process, just that bytes are read in advance. I had run some queries to test (not strict benchmark) and the results show that there is no performance loss.

> Close decompression stream to free off-heap memory in time
> ----------------------------------------------------------
>
>                 Key: PARQUET-2160
>                 URL: https://issues.apache.org/jira/browse/PARQUET-2160
>             Project: Parquet
>          Issue Type: Improvement
>         Environment: Spark 3.1.2 + Iceberg 0.12 + Parquet 1.12.3 + zstd-jni 1.4.9.1 + glibc
>            Reporter: Yujiang Zhong
>            Priority: Major
>
> The decompressed stream in HeapBytesDecompressor$decompress now relies on the JVM GC to close. When reading parquet in zstd compressed format, sometimes I ran into OOM cause high off-heap usage. I think the reason is that the GC is not timely and causes off-heap memory fragmentation. I had to set  lower MALLOC_TRIM_THRESHOLD_ to make glibc give back memory to system quickly. There is a [thread|[https://apache-iceberg.slack.com/archives/C025PH0G1D4/p1650928750269869?thread_ts=1650927062.590789&cid=C025PH0G1D4]] of this zstd parquet issus in Iceberg community slack:  some people had the same problem. 
> I think maybe we can use ByteArrayBytesInput as decompressed bytes input and close decompressed stream in time to solve this problem:
> {code:java}
> InputStream is = codec.createInputStream(bytes.toInputStream(), decompressor);
> decompressed = BytesInput.from(is, uncompressedSize); {code}
> ->
> {code:java}
> InputStream is = codec.createInputStream(bytes.toInputStream(), decompressor);
> decompressed = BytesInput.copy(BytesInput.from(is, uncompressedSize));
> is.close(); {code}
> After I made this change to decompress, I found off-heap memory is significantly reduced (with same query on same data).



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