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

ASF GitHub Bot commented on NIFI-3736:
--------------------------------------

Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2010#discussion_r128609849
  
    --- Diff: 
nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java
 ---
    @@ -149,6 +154,10 @@ public FileSystemRepository(final NiFiProperties 
nifiProperties) throws IOExcept
             for (final Path path : fileRespositoryPaths.values()) {
                 Files.createDirectories(path);
             }
    +        this.maxFlowFilesPerClaim = 
nifiProperties.getMaxFlowFilesPerClaim();
    +        this.writableClaimQueue  = new 
LinkedBlockingQueue<>(maxFlowFilesPerClaim);
    +        final String maxAppendableClaimSize = 
nifiProperties.getMaxAppendableClaimSize();
    +        this.maxAppendableClaimLength = 
DataUnit.parseDataSize(maxAppendableClaimSize, DataUnit.B).intValue();
    --- End diff --
    
    If this value gets set to something like "10 GB" this could cause some 
really problematic (and difficult to track down) problems because the value 
would overflow to a negative value. Probably is best to use a longValue() and 
then perhaps even cap it at something like 100 MB or 10 MB and if the value is 
larger than that, just emit a WARN log event and use the max value.


> NiFi not honoring the "nifi.content.claim.max.appendable.size" and 
> "nifi.content.claim.max.flow.files" properties
> -----------------------------------------------------------------------------------------------------------------
>
>                 Key: NIFI-3736
>                 URL: https://issues.apache.org/jira/browse/NIFI-3736
>             Project: Apache NiFi
>          Issue Type: Bug
>          Components: Core Framework
>            Reporter: Mark Payne
>            Assignee: Michael Hogue
>
> The nifi.properties file has two properties for controlling how many 
> FlowFiles to jam into one Content Claim. Unfortunately, it looks like this is 
> no longer honored in FileSystemRepository.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to