Thanks Mark. I see what you're saying. I'll try out your ideas tomorrow and see how it goes. Thanks a lot!
On Thu, Jan 28, 2021 at 8:55 PM Mark Payne <[email protected]> wrote: > … And now that I’m reading more about how your flow is setup (I didn’t see > that the source of the data was SQS at first)… I would not recommend using > load-balanced connections at all. > > Instead, you should be able to rely solely on backpressure to allow for > evenly distributing the data. If you configure backpressure appropriately, > you will really only have the amount of data on each node that it can work > on at once (plus maybe a little extra to avoid higher latency). > > So if PublishKafka is your bottleneck and has 4 threads, the connection > coming into PublishKafka should have a max queue size of maybe 8 FlowFiles. > This will cause backpressure to flow back up the chain, and each queue > should have very small backpressure limits. As a result, the nodes will > only pull the SQS messages are they are able to handle it. You really only > want to pull the data as you’re able to process it. > > On Jan 28, 2021, at 3:48 PM, Mark Payne <[email protected]> wrote: > > Zilvinas, > > That is accurate - when a connection is load balanced, the data is pushed > to a particular node based on the selected algorithm. It is not continually > rebalanced. > > So for a flow like this, my recommendation would be: > > 1) Set the backpressure threshold from FetchS3Object -> PublishKafka to > something “small.” Probably 1 or 2x the number of concurrent tasks that you > have for PublishKafka. That means that you’ll queue up little data there. > That will cause the data to instead build up on the connection between > EvaluateJsonPath -> FetchS3Object. > > 2) Set the backpressure threshold from EvaluateJsonPath -> FetchS3Object > to something small also. Maybe 2x-4x the number of nodes in the cluster. > This will result in data not queuing up here. As a result, as the size of > this queue gets smaller on one node, the data will begin to flow in and get > distributed to one of the nodes in the cluster. > > By keeping these queues small, essentially this will cause the “load > balanced” data to stay small. As the faster nodes work off their queue, it > will allow more data to flow in and be load balanced. The nodes that are > not performing well will still have backpressure applied so they won’t get > much of the data as it flows in. > > As your flow is right now, there’s no backpressure being hit in the > load-balanced queue. As a result, data streams in as fast as it can and > gets distributed across the cluster. And the nodes that can’t keep up > already have a huge backlog of SQS messages. So making this adjustment will > help to distribute the data as to the nodes as they become able to handle > it. > > Thanks > -Mark > > On Jan 28, 2021, at 3:27 PM, Zilvinas Saltys < > [email protected]> wrote: > > My other issue is that the balancing is not rebalancing the queue? Perhaps > I misunderstand how balancing should work and it only balances round robin > new incoming files? I can easily manually rebalance by disabling balancing > and enabling it again but after a while it gets back to the same situation > where some nodes are getting worse and worse delayed more and more and some > remain fine. > > On Thu, Jan 28, 2021 at 8:22 PM Zilvinas Saltys < > [email protected]> wrote: > >> Hi Joe, >> >> Yes it is the same issue. We have used your advice and reduced the amount >> of threads on our large processors: fetch/compress/publish to a minimum and >> then increased gradually to 4 until the processing rate became acceptable >> (about 2000 files per 5 min). This is a cluster of 25 nodes of 36 cores >> each. >> >> On Thu, Jan 28, 2021 at 8:19 PM Joe Witt <[email protected]> wrote: >> >>> I'm assuming also this is the same thing Maksym was asking about >>> yesterday. Let's try to keep the thread together as this gets discussed. >>> >>> On Thu, Jan 28, 2021 at 1:10 PM Pierre Villard < >>> [email protected]> wrote: >>> >>>> Hi Zilvinas, >>>> >>>> I'm afraid we would need more details to help you out here. >>>> >>>> My first question by quickly looking at the graph would be: there is a >>>> host (green line) where the number of queued flow files is more or less >>>> constantly growing. Where in the flow are the flow files accumulating for >>>> this node? What processor is creating back pressure? Do we have anything in >>>> the log for this node around the time where flow files start accumulating? >>>> >>>> Thanks, >>>> Pierre >>>> >>>> Le ven. 29 janv. 2021 à 00:02, Zilvinas Saltys < >>>> [email protected]> a écrit : >>>> >>>>> Hi, >>>>> >>>>> We run a 25 node Nifi cluster on version 1.12. We're processing about >>>>> 2000 files per 5 mins where each file is from 100 to 500 megabytes. >>>>> >>>>> What I notice is that some workers degrade in performance and keep >>>>> accumulating a queued files delay. See attached screenshots where it shows >>>>> two hosts where one is degraded. >>>>> >>>>> One seemingly dead give away is that the degraded node starts doing >>>>> heavy and intensive disk read io while the other node keeps doing none. I >>>>> ran iostat on those nodes and I know that the read IOs are on the >>>>> content_repository directory. But it makes no sense to me how some of the >>>>> nodes who are doing these heavy tasks are doing no disk read io. In this >>>>> example I know that both nodes are processing roughly the same amount of >>>>> files and of same size. >>>>> >>>>> The pipeline is somewhat simple: >>>>> 1) Read from SQS 2) Fetch file contents from S3 3) Publish file >>>>> contents to Kafka 4) Compress file contents 5) Put compressed contents >>>>> back >>>>> to S3 >>>>> >>>>> All of these operations to my understanding should require heavy reads >>>>> from local disk to fetch file contents from content repository? How is >>>>> such >>>>> a thing possible that some nodes are processing lots of files and are not >>>>> showing any disk reads and then suddenly spike in disk reads and degrade? >>>>> >>>>> Any clues would be really helpful. >>>>> Thanks. >>>>> >>>> > >
