Hi Joe - yes - /data/4, /data/5 are separate spindles, and yes /data/5
is where the flowfile repo is; which is large.
ls -lh
-rw-r--r-- 1 root root 6.5G Jul 12 12:36 checkpoint
-rw-r--r-- 1 root root 5.2G Jul 12 12:46 checkpoint.partial
drwxr-xr-x 4 root root 132 Jul 12 12:46 journals
Joe,
The way that the processor works is that it adds an attribute for every
“Capturing Group” in the regular expression.
This includes a “Capturing Group” 0, which contains the entire value that the
regex was run against.
You can actually disable capturing this as an attribute by setting the
Thank you Mark - it looks like attributes is to blame. I'm adding lots
of UpdateAttribute to delete them as soon as they are not needed and
disk IO has dropped.
Right now, it's all going to 'spinning rust' - soon to all new SSDs, but
either way, this needed addressing.
One oddity, is when I
Some thoughts… putting 10kb of text into an attribute probably isn’t ideal.
Is there another way perhaps to accomplish what you’re doing?
Also your flowfile.repo.checkpoint.interval is pretty high. I’d consider
lowering this considerably…
On Jul 12, 2023 at 11:18 AM -0400, Joe Obernberger
,
Ah ok. And 'data/5' is its own partition (same physical disk as data/4?).
And data/5 is where you see those large files? Can you show what you see
there in terms of files/sizes?
For the checkpoint period the default is 20 seconds. Am curious to
know what benefit moving to 300 seconds was
Joe,
How many FlowFiles are you processing here? Let’s say, per second? How many
processors are in those flows?
Is the FlowFile Repo a spinning disk, SSD, or NAS?
You said you’re using ExtractText to pull 10 KB into an attribute. I presume
you’re then doing something with it. So maybe you’re
Thank you Joe -
The content repo doesn't seem to be the issue - it's the flowfile repo.
Here is the section from one of the nodes:
nifi.content.repository.implementation=org.apache.nifi.controller.repository.FileSystemRepository
nifi.content.claim.max.appendable.size=50 KB
Joe
I dont recall the specific version in which we got it truly sorted but
there was an issue with our default settings for an important content repo
property and how we handled mixture of large/small flowfiles written within
the same underlying slab/claim in the content repository.
Please check
Raising this thread from the dead...
Having issues with IO to the flowfile repository. NiFi will show 500k
flow files and a size of ~1.7G - but the size on disk on each of the 4
nodes is massive - over 100G, and disk IO to the flowfile spindle is
just pegged doing writes.
I do have
Apologies in advance if I've got this completely wrong, but I recall that
error if I forget to increase the limit of open files for a heavily loaded
install. It is more obvious via the UI but the logs will have error
messages about too many open files.
On Wed, 22 Mar 2023, 16:49 Mark Payne,
OK. So changing the checkpoint internal to 300 seconds might help reduce IO a
bit. But it will cause the repo to become much larger, and it will take much
longer to startup whenever you restart NiFi.
The variance in size between nodes is likely due to how recently it’s
checkpointed. If it
Thanks for this Mark. I'm not seeing any large attributes at the moment
but will go through this and verify - but I did have one queue that was
set to 100k instead of 10k.
I set the nifi.cluster.node.connection.timeout to 30 seconds (up from 5)
and the
Joe,
The errors noted are indicating that NiFi cannot communicate with registry.
Either the registry is offline, NiFi’s Registry Client is not configured
properly, there’s a firewall in the way, etc.
A FlowFile repo of 35 GB is rather huge. This would imply one of 3 things:
- You have a huge
Thank you Mark. These are SATA drives - but there's no way for the
flowfile repo to be on multiple spindles. It's not huge - maybe 35G per
node.
I do see a lot of messages like this in the log:
2023-03-22 10:52:13,960 ERROR [Timer-Driven Process Thread-62]
I've since brought the node back up - no change. Looks like IO is all
related to flowfile repository. When it's running, CPU is pretty high -
usually ~12 cores (ie top will show 1200%) per node. I'm using the XFS
filesystem; maybe some FS parameters would help?
The big change is that I was
Joe,
1.8 million FlowFiles is not a concern. But when you say “Should I reduce the
queue sizes?” it makes me wonder if they’re all in a single queue?
Generally, you should leave the backpressure threshold at the default 10,000
FlowFile max. Increasing this can lead to huge amounts of swapping,
Thank you. Was able to get in.
Currently there are 1.8 million flow files and 3.2G. Is this too much
for a 3 node cluster with mutliple spindles each (SATA drives)?
Should I reduce the queue sizes?
-Joe
On 3/22/2023 10:23 AM, Phillip Lord wrote:
Joe,
If you need the UI to come back up,
Joe,
If you need the UI to come back up, try setting the autoresume setting in
nifi.properties to false and restart node(s).
This will bring up every component/controllerService up stopped/disabled and
may provide some breathing room for the UI to become available again.
Phil
On Mar 22, 2023
atop shows the disk as being all red with IO - 100% utilization. There
are a lot of flowfiles currently trying to run through, but I can't
monitor it becauseUI wont' load.
-Joe
On 3/22/2023 10:16 AM, Mark Payne wrote:
Joe,
I’d recommend taking a look at garbage collection. It is far
Joe,
I’d recommend taking a look at garbage collection. It is far more likely the
culprit than disk I/O.
Thanks
-Mark
> On Mar 22, 2023, at 10:12 AM, Joe Obernberger
> wrote:
>
> I'm getting "java.net.SocketTimeoutException: timeout" from the user
> interface of NiFi when load is heavy.
I'm getting "java.net.SocketTimeoutException: timeout" from the user
interface of NiFi when load is heavy. This is 1.18.0 running on a 3
node cluster. Disk IO is high and when that happens, I can't get into
the UI to stop any of the processors.
Any ideas?
I have put the flowfile repository
21 matches
Mail list logo