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

ASF GitHub Bot commented on PARQUET-2160:
-----------------------------------------

zhongyujiang opened a new pull request, #982:
URL: https://github.com/apache/parquet-mr/pull/982

   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [ ] My PR addresses the following [Parquet 
Jira](https://issues.apache.org/jira/browse/PARQUET-2160) issues and references 
them in the PR title. For example, "PARQUET-1234: My Parquet PR"
     - https://issues.apache.org/jira/browse/PARQUET-2160
     - In case you are adding a dependency, check if the license complies with 
the [ASF 3rd Party License 
Policy](https://www.apache.org/legal/resolved.html#category-x).
   
   ### Tests
   
   - [ ] My PR adds the following unit tests __OR__ does not need testing for 
this extremely good reason:
   
   ### Commits
   
   - [ ] My commits all reference Jira issues in their subject lines. In 
addition, my commits follow the guidelines from "[How to write a good git 
commit message](http://chris.beams.io/posts/git-commit/)":
     1. Subject is separated from body by a blank line
     1. Subject is limited to 50 characters (not including Jira issue reference)
     1. Subject does not end with a period
     1. Subject uses the imperative mood ("add", not "adding")
     1. Body wraps at 72 characters
     1. Body explains "what" and "why", not "how"
   
   ### Documentation
   
   - [ ] In case of new functionality, my PR adds documentation that describes 
how to use it.
     - All the public functions and the classes in the PR contain Javadoc that 
explain what it does
   




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

Reply via email to