[
https://issues.apache.org/jira/browse/NIFI-8726?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17367467#comment-17367467
]
Mark Payne commented on NIFI-8726:
----------------------------------
This isn't truly a bug - it is operating as designed. But it is something that
we can improve. Whenever the content repository reaches a certain threshold, it
applies backpressure to prevent new data from being written to it until the
archive/pruning process is completed. The reason for this is that in certain
environments, when there are a very large number of files in the content
repository, we can actually create files and write to them faster than the
operating system is able to delete them. When this happens, we end up in a
situation where the content repository runs out of disk space, which causes
even more problems, just because file deletion takes longer than file creation.
To avoid that, we have mechanism by which the content repository can apply
backpressure to those trying to write to it. This gives the background process
a chance to archive or remove expired data. By default, the backpressure is
applied when the disk hits about 50% full. This can be controlled, though, by
updating the value in nifi.properties:
nifi.content.repository.archive.max.usage.percentage=50%
If you change that to say 90% and restart you should be in good shape -
assuming that your disk is not almost full. But it means that if you have a
dataflow that is super active and deals with tons of splitting, etc. NiFi could
end up using almost all of your disk space for short periods of time.
> Nifi hangs when running for some time
> -------------------------------------
>
> Key: NIFI-8726
> URL: https://issues.apache.org/jira/browse/NIFI-8726
> Project: Apache NiFi
> Issue Type: Bug
> Components: Core Framework
> Affects Versions: 1.13.2
> Environment: Windows
> Reporter: Roman Kurmanowytsch
> Priority: Blocker
> Attachments: nifi.log
>
>
> When running for some hours, Nifi seems to stop processing flowfiles. When
> trying to stop the process to have a look at the queues, it seems that some
> processors are stuck and cannot be terminated.
> In my particular case, I could not stop (among others) a simple ReplaceText
> processor, which just has to replace the complete content of a the flowfile
> by an empty String (so, no special wizardry here).
>
> The thread dump shows this:
> {noformat}
> Timer-Driven Process Thread-2" Id=56 WAITING on
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@32f60190
> at [email protected]/jdk.internal.misc.Unsafe.park(Native Method)
> at
> [email protected]/java.util.concurrent.locks.LockSupport.park(LockSupport.java:194)
> at
> [email protected]/java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2081)
> at
> org.apache.nifi.controller.repository.FileSystemRepository$ContainerState.waitForArchiveExpiration(FileSystemRepository.java:1660)
> at
> org.apache.nifi.controller.repository.FileSystemRepository.create(FileSystemRepository.java:609)
> at
> org.apache.nifi.controller.repository.claim.StandardContentClaimWriteCache.getContentClaim(StandardContentClaimWriteCache.java:63)
> at
> org.apache.nifi.controller.repository.StandardProcessSession.write(StandardProcessSession.java:2782)
> at
> org.apache.nifi.processors.standard.ReplaceText$AlwaysReplace.replace(ReplaceText.java:384)
> at
> org.apache.nifi.processors.standard.ReplaceText.onTrigger(ReplaceText.java:318)
> at
> app//org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:27)
> at
> org.apache.nifi.controller.StandardProcessorNode.onTrigger(StandardProcessorNode.java:1173)
> at
> org.apache.nifi.controller.tasks.ConnectableTask.invoke(ConnectableTask.java:214)
> at
> org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent$1.run(TimerDrivenSchedulingAgent.java:117)
> at org.apache.nifi.engine.FlowEngine$2.run(FlowEngine.java:110)
> [...]{noformat}
> So, it seems to wait on some lock. Interestingly, according to the thread
> dump, there are in total 8 threads waiting for ConditionObject@32f60190.
> Could you please check if I am doing something wrong here?
> Thank you!
>
>
--
This message was sent by Atlassian Jira
(v8.3.4#803005)