You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by "ASF subversion and git services (Jira)" <ji...@apache.org> on 2023/06/02 19:34:00 UTC

[jira] [Commented] (NIFI-11636) ParquetReader buffers up to 2 GB of content into heap unnecessarily

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

ASF subversion and git services commented on NIFI-11636:
--------------------------------------------------------

Commit 5da77e8e74343f04ebe7da34ba47cbc3cbe0d4fc in nifi's branch refs/heads/support/nifi-1.x from Mark Payne
[ https://gitbox.apache.org/repos/asf?p=nifi.git;h=5da77e8e74 ]

NIFI-11636: Do not buffer Parquet content into memory unnecessarily

NIFI-11636: Change default log level from parquet internal reader to WARN as it logs excessively at INFO level
Signed-off-by: Matt Burgess <ma...@apache.org>


> ParquetReader buffers up to 2 GB of content into heap unnecessarily
> -------------------------------------------------------------------
>
>                 Key: NIFI-11636
>                 URL: https://issues.apache.org/jira/browse/NIFI-11636
>             Project: Apache NiFi
>          Issue Type: Bug
>          Components: Extensions
>            Reporter: Mark Payne
>            Assignee: Mark Payne
>            Priority: Major
>             Fix For: 1.latest, 2.latest
>
>          Time Spent: 0.5h
>  Remaining Estimate: 0h
>
> The Parquet Record Reader uses the NiFiSeekableInputStream. Because Parquet requires reading the footer first, this class is intended to use {{mark/reset}} so that we can read the footer and then reset back to the beginning.
> To achieve this, it calls {{InputStream.mark(Integer.MAX_VALUE)}} which will buffer up to 2 GB onto heap. However, the underlying InputStream is the ContentClaimInputStream. The ContentClaimInputStream has smarts built into it to allow resetting without having to buffer content into memory. In particular, if you read over the {{limit}} provided and then call {{reset}} it will close the InputStream and open a new InputStream from the beginning of the FlowFIle content and seek to the desired offset.
> Because of this, we don't need to use {{InputStream.mark(Integer.MAX_VALUE)}} and can instead use {{InputStream.mark(8192)}} or some similarly small value.



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