Vibhath, We can see from the thread dump in the diagnostics report several threads like the following:
"Timer-Driven Process Thread-6" Id=59 WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@6e35ff81 at [email protected]<mailto:[email protected]>/jdk.internal.misc.Unsafe.park(Native Method) at [email protected]<mailto:[email protected]>/java.util.concurrent.locks.LockSupport.park(LockSupport.java:194) at [email protected]<mailto:[email protected]>/java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2081) at org.apache.nifi.controller.repository.FileSystemRepository$ContainerState.waitForArchiveExpiration(FileSystemRepository.java:1698) at org.apache.nifi.controller.repository.FileSystemRepository.create(FileSystemRepository.java:634) at org.apache.nifi.controller.repository.claim.StandardContentClaimWriteCache.getContentClaim(StandardContentClaimWriteCache.java:63) at org.apache.nifi.controller.repository.StandardProcessSession.write(StandardProcessSession.java:2886) at edu.test_grp.test.processors.test_pack.ExtractDB.onTrigger(ExtractDB.java:192) at app//org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:27) at org.apache.nifi.controller.StandardProcessorNode.onTrigger(StandardProcessorNode.java:1202) at org.apache.nifi.controller.tasks.ConnectableTask.invoke(ConnectableTask.java:214) at org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent$1.run(TimerDrivenSchedulingAgent.java:103) at org.apache.nifi.engine.FlowEngine$2.run(FlowEngine.java:110) at [email protected]<mailto:[email protected]>/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515) at [email protected]<mailto:[email protected]>/java.util.concurrent.FutureTask.runAndReset(FutureTask.java:305) at [email protected]<mailto:[email protected]>/java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:305) at [email protected]<mailto:[email protected]>/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) at [email protected]<mailto:[email protected]>/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) at [email protected]<mailto:[email protected]>/java.lang.Thread.run(Thread.java:834) Number of Locked Synchronizers: 1 - java.util.concurrent.ThreadPoolExecutor$Worker@ad7cc49 Note the line: at org.apache.nifi.controller.repository.FileSystemRepository$ContainerState.waitForArchiveExpiration(FileSystemRepository.java:1698) We can also see from the properties in the diagnostics dump: nifi.content.repository.archive.max.retention.period : 7 days nifi.content.repository.archive.max.usage.percentage : 50% So this tells us that the Content Repository wants to ensure that the disk is used no more than 50%. When the disk is 50% full, it will start applying backpressure in order to prevent processors from writing to the content repository until a background thread performs cleanup of the Content Repo archive. And the content repository is configured as; Content Repository Implementation: org.apache.nifi.controller.repository.FileSystemRepository Content Repository <default> Storage Capacity: 98.31 GB Content Repository <default> Usable Space: 37.47 GB So here we can see that the amount of Usable Space (free space) on the Content Repo is 37.47 GB, or 38%. I.e., used space is 62%. So this confirms that the content repo should be applying backpressure. If you look in your logs, you’ll also see logs that say something like: Unable to write to container default due to archive file size constraints; waiting for archive cleanup The reason for the content repository doing this is that depending on your disk(s) and your operating system, the OS can likely create smallish files (1 MB or so) faster than the OS can delete them. As a result, when NiFi is under heavy load, it can be in a situation where it’s creating new data in the content repository faster than it can delete old data that is no longer needed. And this can cause you to run out of disk space. So this check is in place in order to give the background process (of deleting old data from the repository) a chance to catch up. So. What to do about it? The easiest thing to do is to change nifi.content.repository.archive.max.usage.percentage=50% to nifi.content.repository.archive.max.usage.percentage=85% in nifi.properties. That will allow the system to keep writing to the content repository until it reaches 85% full instead of 50% full. Of course, that then has its own set of concerns. It allows the repo to grow much larger, which could result in running out of disk space. Ideally, you’ll have each repository (flowfile, provenance, content) on its own physical partition, or at least its own logical volume. Your config shows that all are writing to the same location. Hope this is helpful! -Mark On Aug 3, 2021, at 1:15 AM, Vibhath Ileperuma <[email protected]<mailto:[email protected]>> wrote: Hi Mark, Please find the attached diagnostics file. Thank You. Vibhath. On Mon, Aug 2, 2021 at 8:36 PM Mark Payne <[email protected]<mailto:[email protected]>> wrote: Vibhath, If you get into that state again, recommend you run: bin/nifi.sh diagnostics diag1.txt And then provide the generated file, diag1.txt - that will include a lot of diagnostic information that should help understand exactly what is happening. Thanks -Mark On Aug 2, 2021, at 10:01 AM, Vibhath Ileperuma <[email protected]<mailto:[email protected]>> wrote: Hi Joe, My scripts directly interact with NIFI through the REST API. As you mentioned, after updating the jdk version, I was able to use NIFI 1.14 with the http protocol. However, even with 1.14, NIFI gets stuck in the middle of execution. In the below image, a flow file is there in the input queue of a processor group. <image001 (1).png> However, when the processor group is opened, the queue after the input port is empty. <image002 (1).png> Note that, I have configured the processor group flow file concurrency to 'Unbounded'. Even Though I waited for about 30 minutes, the flow file was not transferred into the processor group. At the same time, a putS3 processor showed that there are 2 concurrent tasks running. But it was unable to upload files to S3 within 30 minutes. I believe that, even though it showed 2 threads, they were stuck since this processor usually finishes uploading files with same size within several minutes (less than 10 minutes) <image005.png> After about 30 minutes of waiting, I stopped the NIFI node and restarted it. Then, 1. The flow file which was stuck in the input of a processor group was transferred into the processor group. 2. The PutS3 processor routed the flow files into the failure relationship. I would be grateful if you could suggest a way to avoid this issue. Thank you. Vibhath, On Sat, Jul 31, 2021 at 11:22 PM Joe Gresock <[email protected]<mailto:[email protected]>> wrote: Vibhath, I believe this exception is thrown when using an encryption key of at least 16 bytes without the JCE Unlimited Strength Jurisdiction Policy Files installed. Since NiFi now automatically generates a nifi.sensitive.props.key in 1.14.0, you may be encountering this situation. What version of Java are you running? I believe this is no longer required for Java 8 u162, so it's possible an update could fix the issue. Are your scripts interacting directly with the NiFi REST API, or just with a processor that listens for an HTTP connection (like ListenHTTP or HandleHttpRequest)? If the latter, you shouldn't have to disable the security settings on NiFi, since your processor settings are not tied to the NiFi security settings. Joe [https://ipmcdn.avast.com/images/icons/icon-envelope-tick-round-orange-animated-no-repeat-v1.gif]<https://www.avast.com/sig-email?utm_medium=email&utm_source=link&utm_campaign=sig-email&utm_content=webmail&utm_term=icon> Virus-free. www.avast.com<https://www.avast.com/sig-email?utm_medium=email&utm_source=link&utm_campaign=sig-email&utm_content=webmail&utm_term=link> On Sat, Jul 31, 2021 at 12:52 PM Vibhath Ileperuma <[email protected]<mailto:[email protected]>> wrote: Hi Joe and Mark, Thank you for the information. We have developed a few custom scripts which use http to communicate with NIFI. Hence, we need to enable http in NIFI instead of https. To disable security, I set the keystore and truststore properties to empty values as below. <image003.png> With the above configurations, even though I can login to the web frontend with http, when I set the value to a sensitive field I get the below error. <image001.png> nifi-app,log <image002.png> Could you please advise me on resolving this error. Thank You. Vibhath. On Wed, Jul 28, 2021 at 7:12 PM Mark Payne <[email protected]<mailto:[email protected]>> wrote: Vibhath, You can change the configuration to be http based instead of https. But I would high recommend against doing so. Thanks -Mark On Jul 27, 2021, at 10:46 PM, Vibhath Ileperuma <[email protected]<mailto:[email protected]>> wrote: Hi Mark, As you mentioned I tried to use NIFI 1.14.0. However, our current implementation uses http protocol. In the release note of 1.14.0, it is mentioned that it comes with security enabled as default. Is it possible to use http with 1.14.0 disabling security features. Thank You Vibhath. On Mon, Jul 26, 2021 at 7:11 PM Mark Payne <[email protected]<mailto:[email protected]>> wrote: Vibhath, This was a known issue in 1.13. It has been fixed in 1.14. Please update to the latest, and you should be good to go! Thanks -Mark On Jul 26, 2021, at 8:01 AM, Vibhath Ileperuma <[email protected]<mailto:[email protected]>> wrote: Hi All, I'm using an ExecuteScript processor to write a set of flow file attributes to a csv file located in the local file system. Note that I'm writing attribute values of all the flow files in a single csv file. Even Though this works fine most of the times, in some times, the processor does not start though there are flow files in the incoming queue. I have attached a screenshot of such scenario below. Though a flow file was shown in the queue, when I tried to list the queue it notified that the queue was empty. When I restarted NIFI, processor started to work properly. I'm using NIFI 1.13.2. I'll be grateful is someone can suggest a way to avoid this issue. <image.png> Thank You, Vibhath. [https://ipmcdn.avast.com/images/icons/icon-envelope-tick-round-orange-animated-no-repeat-v1.gif]<https://www.avast.com/sig-email?utm_medium=email&utm_source=link&utm_campaign=sig-email&utm_content=webmail&utm_term=icon> Virus-free. www.avast.com<https://www.avast.com/sig-email?utm_medium=email&utm_source=link&utm_campaign=sig-email&utm_content=webmail&utm_term=link> <diag1.txt>
