You are viewing a plain text version of this content. The canonical link for it is here.
Posted to users@nifi.apache.org by Man on the Moon <ma...@gmail.com> on 2021/08/05 07:02:34 UTC

Problems with Load Balancing - NiFi 1.14.0

Hi all,

we are running a NiFi 1.14.0 cluster with two nodes:
Node1:
hostname: nifi1.mydomain
ip: 172.16.0.1

Node 2:
hostname: nifi2.mydomain
ip: 172.16.0.2

Lately, we are persistingly seeing errors in the logs related to the Load
Balancing, and the flowfiles are processed only in the Primary Node. The
configuration we have for the load balancing is:

nifi.cluster.load.balance.comms.timeout=30 sec
nifi.cluster.load.balance.connections.per.node=8
nifi.cluster.load.balance.host=172.16.0.1
nifi.cluster.load.balance.max.thread.count=16
nifi.cluster.load.balance.port=6342

As seen in several posts in the list, we tried using the IPs in the
nifi.cluster.load.balance.host property, with no visible effect.

The logs in node 1 show the following exceptions (continously):

2021-08-05 06:28:25,811 ERROR [Load-Balanced Client Thread-14]
o.a.n.c.q.c.c.a.n.NioAsyncLoadBalanceClient Failed to communicate with Peer
nifi2.mydomain:8443
java.io.IOException: Failed to decrypt data from Peer nifi2.mydomain:8443
because Peer unexpectedly closed connection
at
org.apache.nifi.controller.queue.clustered.client.async.nio.PeerChannel.decrypt(PeerChannel.java:269)
at
org.apache.nifi.controller.queue.clustered.client.async.nio.PeerChannel.read(PeerChannel.java:159)
at
org.apache.nifi.controller.queue.clustered.client.async.nio.PeerChannel.read(PeerChannel.java:80)
at
org.apache.nifi.controller.queue.clustered.client.async.nio.LoadBalanceSession.receiveSpaceAvailableResponse(LoadBalanceSession.java:573)
at
org.apache.nifi.controller.queue.clustered.client.async.nio.LoadBalanceSession.communicate(LoadBalanceSession.java:151)
at
org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClient.communicate(NioAsyncLoadBalanceClient.java:264)
at
org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClientTask.run(NioAsyncLoadBalanceClientTask.java:81)
at org.apache.nifi.engine.FlowEngine$2.run(FlowEngine.java:110)
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at java.lang.Thread.run(Thread.java:748)
2021-08-05 06:28:25,811 ERROR [Load-Balanced Client Thread-14]
o.a.n.c.queue.SwappablePriorityQueue Updated Size of Queue Unacknowledged
from FlowFile Queue Size[ ActiveQueue=[0, 0 Bytes], Swap Queue=[1293,
2575557 Bytes], Swap Files=[0], Unacknowledged=[-1293, -2575557 Bytes] ] to
FlowFile Queue Size[ ActiveQueue=[0, 0 Bytes], Swap Queue=[1293, 2575557
Bytes], Swap Files=[0], Unacknowledged=[-1293, -2575557 Bytes] ]
java.lang.RuntimeException: Cannot create negative queue size
at
org.apache.nifi.controller.queue.SwappablePriorityQueue.logIfNegative(SwappablePriorityQueue.java:1068)
at
org.apache.nifi.controller.queue.SwappablePriorityQueue.incrementUnacknowledgedQueueSize(SwappablePriorityQueue.java:1058)
at
org.apache.nifi.controller.queue.SwappablePriorityQueue.acknowledge(SwappablePriorityQueue.java:478)
at
org.apache.nifi.controller.queue.clustered.partition.RemoteQueuePartition$1.onTransactionFailed(RemoteQueuePartition.java:171)
at
org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClient.communicate(NioAsyncLoadBalanceClient.java:271)
at
org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClientTask.run(NioAsyncLoadBalanceClientTask.java:81)
at org.apache.nifi.engine.FlowEngine$2.run(FlowEngine.java:110)
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at java.lang.Thread.run(Thread.java:748)


While in Node 2 the exceptions are also continuous, but different, like
this:

2021-08-05 06:31:31,775 ERROR [Load-Balance Server Thread-165207]
o.a.n.c.q.c.s.ConnectionLoadBalanceServer Failed to communicate with Peer /
172.16.0.1:42968
org.apache.nifi.controller.queue.clustered.server.TransactionAbortedException:
Attempted to receive FlowFiles from Peer nifi1.mydomain for Connection with
ID 0feea45e-017b-1000-0000-0000645cab5f but no Connection exists with that
ID
at
org.apache.nifi.controller.queue.clustered.server.StandardLoadBalanceProtocol.receiveFlowFiles(StandardLoadBalanceProtocol.java:205)
at
org.apache.nifi.controller.queue.clustered.server.StandardLoadBalanceProtocol.receiveFlowFiles(StandardLoadBalanceProtocol.java:134)
at
org.apache.nifi.controller.queue.clustered.server.ConnectionLoadBalanceServer$CommunicateAction.run(ConnectionLoadBalanceServer.java:162)
at java.lang.Thread.run(Thread.java:748)

I am running out of ideas, and have tried everything I found online to
tackle load balancing problems. Apparently, there are no inter-node
communication issues.

Any clues or guidance would be highly appreciated!!

Luis.

Re: Problems with Load Balancing - NiFi 1.14.0

Posted by "Robert R. Bruno" <rb...@gmail.com>.
Just wanted to resurrect this thread.  We saw a similar error happening to
our nifi when we upgraded to 1.14.0.  In our case we would get this error
when we would delete a non load balanced connection between two
processors.  Sadly the only way to remove the error was to restart the
nifis, and we were able to reproduce the error once the nifi came back up.
We decided for now to downgrade to 1.13.2 and were not able to reproduce
the issue.  We are happy to try upgrading again once a fix is in place if
one is found.

Thanks,
Robert

On Fri, Aug 6, 2021 at 11:32 AM Man on the Moon <
manonthedarksideofthemoon@gmail.com> wrote:

> Hi Mark,
>
> we restarted Node 2, and those six connections were not in the logs any
> more, but when we deleted some more Process Groups we started seeing the
> same situation. Strangely, some of those connections were "DISABLED" before
> being deleted.
>
> We have finished the deletions, so we will be safe for a while  :-D
>
> Nevertheless, we are facing what I think are some connectivity problems
> between the two nodes, so I will look for some help from the sys/network
> admins. I do not know if the previous situation could be affected by a
> faulty inter-node connection.
>
> Thanks!
>
>
> El vie, 6 ago 2021 a las 16:56, Mark Payne (<ma...@hotmail.com>)
> escribió:
>
>> Luis,
>>
>> Have you tried restarting the cluster since this occurred? I.e., does it
>> attempt to send to a missing connection even after restart?
>>
>> Thanks
>> -Mark
>>
>>
>> On Aug 6, 2021, at 10:51 AM, Man on the Moon <
>> manonthedarksideofthemoon@gmail.com> wrote:
>>
>> Hi again Mark,
>>
>> just one more clue, in case it helps. We have listed all the connections
>> that are referenced as "missing" in the logs. At the moment of the research
>> there were a total of six. None of them was present in the flow.xml.gz, but
>> they were indeed present in some of the flows present in the "archive"
>> folder. If I am not mistaken, this would suggest that they correspond to
>> connections that have been recently deleted. Two of them were not present
>> in any of the flows of the "archive" folder, so I suspect they would have
>> been removed longer ago.
>>
>> Why a NiFi node should want to balance something in a connection that had
>> been deleted at least some minutes ago, I cannot figure out.
>>
>> Thanks again!
>>
>>
>> El vie, 6 ago 2021 a las 8:27, Man on the Moon (<
>> manonthedarksideofthemoon@gmail.com>) escribió:
>>
>>> Hi Mark,
>>>
>>> I just did the zgrep and found that none of the nodes has a reference to
>>> that uuid in its flow.xml.gz file. What other things could I try to gather
>>> more information?
>>>
>>> Thanks!
>>>
>>>
>>>
>>>
>>> El jue, 5 ago 2021 a las 17:40, Mark Payne (<ma...@hotmail.com>)
>>> escribió:
>>>
>>>> Luis,
>>>>
>>>> That is certainly unexpected. Does the connection
>>>> 0feea45e-017b-1000-0000-0000645cab5f exist in your flow?
>>>>
>>>> If so, I would recommend using grep to check both nodes to see if the
>>>> connection is there. I.e., on both nodes run
>>>>
>>>> zgrep "0feea45e-017b-1000-0000-0000645cab5f” conf/flow.xml.gz
>>>>
>>>> Do both nodes show the connection as existing?
>>>>
>>>> On Aug 5, 2021, at 3:02 AM, Man on the Moon <
>>>> manonthedarksideofthemoon@gmail.com> wrote:
>>>>
>>>> Hi all,
>>>>
>>>> we are running a NiFi 1.14.0 cluster with two nodes:
>>>> Node1:
>>>> hostname: nifi1.mydomain
>>>> ip: 172.16.0.1
>>>>
>>>> Node 2:
>>>> hostname: nifi2.mydomain
>>>> ip: 172.16.0.2
>>>>
>>>> Lately, we are persistingly seeing errors in the logs related to the
>>>> Load Balancing, and the flowfiles are processed only in the Primary Node.
>>>> The configuration we have for the load balancing is:
>>>>
>>>> nifi.cluster.load.balance.comms.timeout=30 sec
>>>> nifi.cluster.load.balance.connections.per.node=8
>>>> nifi.cluster.load.balance.host=172.16.0.1
>>>> nifi.cluster.load.balance.max.thread.count=16
>>>> nifi.cluster.load.balance.port=6342
>>>>
>>>> As seen in several posts in the list, we tried using the IPs in the
>>>> nifi.cluster.load.balance.host property, with no visible effect.
>>>>
>>>> The logs in node 1 show the following exceptions (continously):
>>>>
>>>> 2021-08-05 06:28:25,811 ERROR [Load-Balanced Client Thread-14]
>>>> o.a.n.c.q.c.c.a.n.NioAsyncLoadBalanceClient Failed to communicate with Peer
>>>> nifi2.mydomain:8443
>>>> java.io.IOException: Failed to decrypt data from Peer
>>>> nifi2.mydomain:8443 because Peer unexpectedly closed connection
>>>> at
>>>> org.apache.nifi.controller.queue.clustered.client.async.nio.PeerChannel.decrypt(PeerChannel.java:269)
>>>> at
>>>> org.apache.nifi.controller.queue.clustered.client.async.nio.PeerChannel.read(PeerChannel.java:159)
>>>> at
>>>> org.apache.nifi.controller.queue.clustered.client.async.nio.PeerChannel.read(PeerChannel.java:80)
>>>> at
>>>> org.apache.nifi.controller.queue.clustered.client.async.nio.LoadBalanceSession.receiveSpaceAvailableResponse(LoadBalanceSession.java:573)
>>>> at
>>>> org.apache.nifi.controller.queue.clustered.client.async.nio.LoadBalanceSession.communicate(LoadBalanceSession.java:151)
>>>> at
>>>> org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClient.communicate(NioAsyncLoadBalanceClient.java:264)
>>>> at
>>>> org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClientTask.run(NioAsyncLoadBalanceClientTask.java:81)
>>>> at org.apache.nifi.engine.FlowEngine$2.run(FlowEngine.java:110)
>>>> at
>>>> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>>>> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>>>> at
>>>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
>>>> at
>>>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>>>> at
>>>> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>>>> at
>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>>>> at java.lang.Thread.run(Thread.java:748)
>>>> 2021-08-05 06:28:25,811 ERROR [Load-Balanced Client Thread-14]
>>>> o.a.n.c.queue.SwappablePriorityQueue Updated Size of Queue Unacknowledged
>>>> from FlowFile Queue Size[ ActiveQueue=[0, 0 Bytes], Swap Queue=[1293,
>>>> 2575557 Bytes], Swap Files=[0], Unacknowledged=[-1293, -2575557 Bytes] ] to
>>>> FlowFile Queue Size[ ActiveQueue=[0, 0 Bytes], Swap Queue=[1293, 2575557
>>>> Bytes], Swap Files=[0], Unacknowledged=[-1293, -2575557 Bytes] ]
>>>> java.lang.RuntimeException: Cannot create negative queue size
>>>> at
>>>> org.apache.nifi.controller.queue.SwappablePriorityQueue.logIfNegative(SwappablePriorityQueue.java:1068)
>>>> at
>>>> org.apache.nifi.controller.queue.SwappablePriorityQueue.incrementUnacknowledgedQueueSize(SwappablePriorityQueue.java:1058)
>>>> at
>>>> org.apache.nifi.controller.queue.SwappablePriorityQueue.acknowledge(SwappablePriorityQueue.java:478)
>>>> at
>>>> org.apache.nifi.controller.queue.clustered.partition.RemoteQueuePartition$1.onTransactionFailed(RemoteQueuePartition.java:171)
>>>> at
>>>> org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClient.communicate(NioAsyncLoadBalanceClient.java:271)
>>>> at
>>>> org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClientTask.run(NioAsyncLoadBalanceClientTask.java:81)
>>>> at org.apache.nifi.engine.FlowEngine$2.run(FlowEngine.java:110)
>>>> at
>>>> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>>>> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>>>> at
>>>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
>>>> at
>>>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>>>> at
>>>> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>>>> at
>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>>>> at java.lang.Thread.run(Thread.java:748)
>>>>
>>>>
>>>> While in Node 2 the exceptions are also continuous, but different, like
>>>> this:
>>>>
>>>> 2021-08-05 06:31:31,775 ERROR [Load-Balance Server Thread-165207]
>>>> o.a.n.c.q.c.s.ConnectionLoadBalanceServer Failed to communicate with Peer /
>>>> 172.16.0.1:42968
>>>> org.apache.nifi.controller.queue.clustered.server.TransactionAbortedException:
>>>> Attempted to receive FlowFiles from Peer nifi1.mydomain for Connection with
>>>> ID 0feea45e-017b-1000-0000-0000645cab5f but no Connection exists with that
>>>> ID
>>>> at
>>>> org.apache.nifi.controller.queue.clustered.server.StandardLoadBalanceProtocol.receiveFlowFiles(StandardLoadBalanceProtocol.java:205)
>>>> at
>>>> org.apache.nifi.controller.queue.clustered.server.StandardLoadBalanceProtocol.receiveFlowFiles(StandardLoadBalanceProtocol.java:134)
>>>> at
>>>> org.apache.nifi.controller.queue.clustered.server.ConnectionLoadBalanceServer$CommunicateAction.run(ConnectionLoadBalanceServer.java:162)
>>>> at java.lang.Thread.run(Thread.java:748)
>>>>
>>>> I am running out of ideas, and have tried everything I found online to
>>>> tackle load balancing problems. Apparently, there are no inter-node
>>>> communication issues.
>>>>
>>>> Any clues or guidance would be highly appreciated!!
>>>>
>>>> Luis.
>>>>
>>>>
>>>>
>>

Re: Problems with Load Balancing - NiFi 1.14.0

Posted by Man on the Moon <ma...@gmail.com>.
Hi Mark,

we restarted Node 2, and those six connections were not in the logs any
more, but when we deleted some more Process Groups we started seeing the
same situation. Strangely, some of those connections were "DISABLED" before
being deleted.

We have finished the deletions, so we will be safe for a while  :-D

Nevertheless, we are facing what I think are some connectivity problems
between the two nodes, so I will look for some help from the sys/network
admins. I do not know if the previous situation could be affected by a
faulty inter-node connection.

Thanks!


El vie, 6 ago 2021 a las 16:56, Mark Payne (<ma...@hotmail.com>)
escribió:

> Luis,
>
> Have you tried restarting the cluster since this occurred? I.e., does it
> attempt to send to a missing connection even after restart?
>
> Thanks
> -Mark
>
>
> On Aug 6, 2021, at 10:51 AM, Man on the Moon <
> manonthedarksideofthemoon@gmail.com> wrote:
>
> Hi again Mark,
>
> just one more clue, in case it helps. We have listed all the connections
> that are referenced as "missing" in the logs. At the moment of the research
> there were a total of six. None of them was present in the flow.xml.gz, but
> they were indeed present in some of the flows present in the "archive"
> folder. If I am not mistaken, this would suggest that they correspond to
> connections that have been recently deleted. Two of them were not present
> in any of the flows of the "archive" folder, so I suspect they would have
> been removed longer ago.
>
> Why a NiFi node should want to balance something in a connection that had
> been deleted at least some minutes ago, I cannot figure out.
>
> Thanks again!
>
>
> El vie, 6 ago 2021 a las 8:27, Man on the Moon (<
> manonthedarksideofthemoon@gmail.com>) escribió:
>
>> Hi Mark,
>>
>> I just did the zgrep and found that none of the nodes has a reference to
>> that uuid in its flow.xml.gz file. What other things could I try to gather
>> more information?
>>
>> Thanks!
>>
>>
>>
>>
>> El jue, 5 ago 2021 a las 17:40, Mark Payne (<ma...@hotmail.com>)
>> escribió:
>>
>>> Luis,
>>>
>>> That is certainly unexpected. Does the connection
>>> 0feea45e-017b-1000-0000-0000645cab5f exist in your flow?
>>>
>>> If so, I would recommend using grep to check both nodes to see if the
>>> connection is there. I.e., on both nodes run
>>>
>>> zgrep "0feea45e-017b-1000-0000-0000645cab5f” conf/flow.xml.gz
>>>
>>> Do both nodes show the connection as existing?
>>>
>>> On Aug 5, 2021, at 3:02 AM, Man on the Moon <
>>> manonthedarksideofthemoon@gmail.com> wrote:
>>>
>>> Hi all,
>>>
>>> we are running a NiFi 1.14.0 cluster with two nodes:
>>> Node1:
>>> hostname: nifi1.mydomain
>>> ip: 172.16.0.1
>>>
>>> Node 2:
>>> hostname: nifi2.mydomain
>>> ip: 172.16.0.2
>>>
>>> Lately, we are persistingly seeing errors in the logs related to the
>>> Load Balancing, and the flowfiles are processed only in the Primary Node.
>>> The configuration we have for the load balancing is:
>>>
>>> nifi.cluster.load.balance.comms.timeout=30 sec
>>> nifi.cluster.load.balance.connections.per.node=8
>>> nifi.cluster.load.balance.host=172.16.0.1
>>> nifi.cluster.load.balance.max.thread.count=16
>>> nifi.cluster.load.balance.port=6342
>>>
>>> As seen in several posts in the list, we tried using the IPs in the
>>> nifi.cluster.load.balance.host property, with no visible effect.
>>>
>>> The logs in node 1 show the following exceptions (continously):
>>>
>>> 2021-08-05 06:28:25,811 ERROR [Load-Balanced Client Thread-14]
>>> o.a.n.c.q.c.c.a.n.NioAsyncLoadBalanceClient Failed to communicate with Peer
>>> nifi2.mydomain:8443
>>> java.io.IOException: Failed to decrypt data from Peer
>>> nifi2.mydomain:8443 because Peer unexpectedly closed connection
>>> at
>>> org.apache.nifi.controller.queue.clustered.client.async.nio.PeerChannel.decrypt(PeerChannel.java:269)
>>> at
>>> org.apache.nifi.controller.queue.clustered.client.async.nio.PeerChannel.read(PeerChannel.java:159)
>>> at
>>> org.apache.nifi.controller.queue.clustered.client.async.nio.PeerChannel.read(PeerChannel.java:80)
>>> at
>>> org.apache.nifi.controller.queue.clustered.client.async.nio.LoadBalanceSession.receiveSpaceAvailableResponse(LoadBalanceSession.java:573)
>>> at
>>> org.apache.nifi.controller.queue.clustered.client.async.nio.LoadBalanceSession.communicate(LoadBalanceSession.java:151)
>>> at
>>> org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClient.communicate(NioAsyncLoadBalanceClient.java:264)
>>> at
>>> org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClientTask.run(NioAsyncLoadBalanceClientTask.java:81)
>>> at org.apache.nifi.engine.FlowEngine$2.run(FlowEngine.java:110)
>>> at
>>> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>>> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>>> at
>>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
>>> at
>>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>>> at
>>> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>>> at
>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>>> at java.lang.Thread.run(Thread.java:748)
>>> 2021-08-05 06:28:25,811 ERROR [Load-Balanced Client Thread-14]
>>> o.a.n.c.queue.SwappablePriorityQueue Updated Size of Queue Unacknowledged
>>> from FlowFile Queue Size[ ActiveQueue=[0, 0 Bytes], Swap Queue=[1293,
>>> 2575557 Bytes], Swap Files=[0], Unacknowledged=[-1293, -2575557 Bytes] ] to
>>> FlowFile Queue Size[ ActiveQueue=[0, 0 Bytes], Swap Queue=[1293, 2575557
>>> Bytes], Swap Files=[0], Unacknowledged=[-1293, -2575557 Bytes] ]
>>> java.lang.RuntimeException: Cannot create negative queue size
>>> at
>>> org.apache.nifi.controller.queue.SwappablePriorityQueue.logIfNegative(SwappablePriorityQueue.java:1068)
>>> at
>>> org.apache.nifi.controller.queue.SwappablePriorityQueue.incrementUnacknowledgedQueueSize(SwappablePriorityQueue.java:1058)
>>> at
>>> org.apache.nifi.controller.queue.SwappablePriorityQueue.acknowledge(SwappablePriorityQueue.java:478)
>>> at
>>> org.apache.nifi.controller.queue.clustered.partition.RemoteQueuePartition$1.onTransactionFailed(RemoteQueuePartition.java:171)
>>> at
>>> org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClient.communicate(NioAsyncLoadBalanceClient.java:271)
>>> at
>>> org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClientTask.run(NioAsyncLoadBalanceClientTask.java:81)
>>> at org.apache.nifi.engine.FlowEngine$2.run(FlowEngine.java:110)
>>> at
>>> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>>> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>>> at
>>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
>>> at
>>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>>> at
>>> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>>> at
>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>>> at java.lang.Thread.run(Thread.java:748)
>>>
>>>
>>> While in Node 2 the exceptions are also continuous, but different, like
>>> this:
>>>
>>> 2021-08-05 06:31:31,775 ERROR [Load-Balance Server Thread-165207]
>>> o.a.n.c.q.c.s.ConnectionLoadBalanceServer Failed to communicate with Peer /
>>> 172.16.0.1:42968
>>> org.apache.nifi.controller.queue.clustered.server.TransactionAbortedException:
>>> Attempted to receive FlowFiles from Peer nifi1.mydomain for Connection with
>>> ID 0feea45e-017b-1000-0000-0000645cab5f but no Connection exists with that
>>> ID
>>> at
>>> org.apache.nifi.controller.queue.clustered.server.StandardLoadBalanceProtocol.receiveFlowFiles(StandardLoadBalanceProtocol.java:205)
>>> at
>>> org.apache.nifi.controller.queue.clustered.server.StandardLoadBalanceProtocol.receiveFlowFiles(StandardLoadBalanceProtocol.java:134)
>>> at
>>> org.apache.nifi.controller.queue.clustered.server.ConnectionLoadBalanceServer$CommunicateAction.run(ConnectionLoadBalanceServer.java:162)
>>> at java.lang.Thread.run(Thread.java:748)
>>>
>>> I am running out of ideas, and have tried everything I found online to
>>> tackle load balancing problems. Apparently, there are no inter-node
>>> communication issues.
>>>
>>> Any clues or guidance would be highly appreciated!!
>>>
>>> Luis.
>>>
>>>
>>>
>

Re: Problems with Load Balancing - NiFi 1.14.0

Posted by Mark Payne <ma...@hotmail.com>.
Luis,

Have you tried restarting the cluster since this occurred? I.e., does it attempt to send to a missing connection even after restart?

Thanks
-Mark


On Aug 6, 2021, at 10:51 AM, Man on the Moon <ma...@gmail.com>> wrote:

Hi again Mark,

just one more clue, in case it helps. We have listed all the connections that are referenced as "missing" in the logs. At the moment of the research there were a total of six. None of them was present in the flow.xml.gz, but they were indeed present in some of the flows present in the "archive" folder. If I am not mistaken, this would suggest that they correspond to connections that have been recently deleted. Two of them were not present in any of the flows of the "archive" folder, so I suspect they would have been removed longer ago.

Why a NiFi node should want to balance something in a connection that had been deleted at least some minutes ago, I cannot figure out.

Thanks again!


El vie, 6 ago 2021 a las 8:27, Man on the Moon (<ma...@gmail.com>>) escribió:
Hi Mark,

I just did the zgrep and found that none of the nodes has a reference to that uuid in its flow.xml.gz file. What other things could I try to gather more information?

Thanks!




El jue, 5 ago 2021 a las 17:40, Mark Payne (<ma...@hotmail.com>>) escribió:
Luis,

That is certainly unexpected. Does the connection 0feea45e-017b-1000-0000-0000645cab5f exist in your flow?

If so, I would recommend using grep to check both nodes to see if the connection is there. I.e., on both nodes run

zgrep "0feea45e-017b-1000-0000-0000645cab5f” conf/flow.xml.gz

Do both nodes show the connection as existing?

On Aug 5, 2021, at 3:02 AM, Man on the Moon <ma...@gmail.com>> wrote:

Hi all,

we are running a NiFi 1.14.0 cluster with two nodes:
Node1:
hostname: nifi1.mydomain
ip: 172.16.0.1

Node 2:
hostname: nifi2.mydomain
ip: 172.16.0.2

Lately, we are persistingly seeing errors in the logs related to the Load Balancing, and the flowfiles are processed only in the Primary Node. The configuration we have for the load balancing is:

nifi.cluster.load.balance.comms.timeout=30 sec
nifi.cluster.load.balance.connections.per.node=8
nifi.cluster.load.balance.host=172.16.0.1
nifi.cluster.load.balance.max.thread.count=16
nifi.cluster.load.balance.port=6342

As seen in several posts in the list, we tried using the IPs in the nifi.cluster.load.balance.host property, with no visible effect.

The logs in node 1 show the following exceptions (continously):

2021-08-05 06:28:25,811 ERROR [Load-Balanced Client Thread-14] o.a.n.c.q.c.c.a.n.NioAsyncLoadBalanceClient Failed to communicate with Peer nifi2.mydomain:8443
java.io.IOException: Failed to decrypt data from Peer nifi2.mydomain:8443 because Peer unexpectedly closed connection
at org.apache.nifi.controller.queue.clustered.client.async.nio.PeerChannel.decrypt(PeerChannel.java:269)
at org.apache.nifi.controller.queue.clustered.client.async.nio.PeerChannel.read(PeerChannel.java:159)
at org.apache.nifi.controller.queue.clustered.client.async.nio.PeerChannel.read(PeerChannel.java:80)
at org.apache.nifi.controller.queue.clustered.client.async.nio.LoadBalanceSession.receiveSpaceAvailableResponse(LoadBalanceSession.java:573)
at org.apache.nifi.controller.queue.clustered.client.async.nio.LoadBalanceSession.communicate(LoadBalanceSession.java:151)
at org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClient.communicate(NioAsyncLoadBalanceClient.java:264)
at org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClientTask.run(NioAsyncLoadBalanceClientTask.java:81)
at org.apache.nifi.engine.FlowEngine$2.run(FlowEngine.java:110)
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at java.lang.Thread.run(Thread.java:748)
2021-08-05 06:28:25,811 ERROR [Load-Balanced Client Thread-14] o.a.n.c.queue.SwappablePriorityQueue Updated Size of Queue Unacknowledged from FlowFile Queue Size[ ActiveQueue=[0, 0 Bytes], Swap Queue=[1293, 2575557 Bytes], Swap Files=[0], Unacknowledged=[-1293, -2575557 Bytes] ] to FlowFile Queue Size[ ActiveQueue=[0, 0 Bytes], Swap Queue=[1293, 2575557 Bytes], Swap Files=[0], Unacknowledged=[-1293, -2575557 Bytes] ]
java.lang.RuntimeException: Cannot create negative queue size
at org.apache.nifi.controller.queue.SwappablePriorityQueue.logIfNegative(SwappablePriorityQueue.java:1068)
at org.apache.nifi.controller.queue.SwappablePriorityQueue.incrementUnacknowledgedQueueSize(SwappablePriorityQueue.java:1058)
at org.apache.nifi.controller.queue.SwappablePriorityQueue.acknowledge(SwappablePriorityQueue.java:478)
at org.apache.nifi.controller.queue.clustered.partition.RemoteQueuePartition$1.onTransactionFailed(RemoteQueuePartition.java:171)
at org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClient.communicate(NioAsyncLoadBalanceClient.java:271)
at org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClientTask.run(NioAsyncLoadBalanceClientTask.java:81)
at org.apache.nifi.engine.FlowEngine$2.run(FlowEngine.java:110)
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at java.lang.Thread.run(Thread.java:748)


While in Node 2 the exceptions are also continuous, but different, like this:

2021-08-05 06:31:31,775 ERROR [Load-Balance Server Thread-165207] o.a.n.c.q.c.s.ConnectionLoadBalanceServer Failed to communicate with Peer /172.16.0.1:42968<http://172.16.0.1:42968/>
org.apache.nifi.controller.queue.clustered.server.TransactionAbortedException: Attempted to receive FlowFiles from Peer nifi1.mydomain for Connection with ID 0feea45e-017b-1000-0000-0000645cab5f but no Connection exists with that ID
at org.apache.nifi.controller.queue.clustered.server.StandardLoadBalanceProtocol.receiveFlowFiles(StandardLoadBalanceProtocol.java:205)
at org.apache.nifi.controller.queue.clustered.server.StandardLoadBalanceProtocol.receiveFlowFiles(StandardLoadBalanceProtocol.java:134)
at org.apache.nifi.controller.queue.clustered.server.ConnectionLoadBalanceServer$CommunicateAction.run(ConnectionLoadBalanceServer.java:162)
at java.lang.Thread.run(Thread.java:748)

I am running out of ideas, and have tried everything I found online to tackle load balancing problems. Apparently, there are no inter-node communication issues.

Any clues or guidance would be highly appreciated!!

Luis.




Re: Problems with Load Balancing - NiFi 1.14.0

Posted by Man on the Moon <ma...@gmail.com>.
Hi again Mark,

just one more clue, in case it helps. We have listed all the connections
that are referenced as "missing" in the logs. At the moment of the research
there were a total of six. None of them was present in the flow.xml.gz, but
they were indeed present in some of the flows present in the "archive"
folder. If I am not mistaken, this would suggest that they correspond to
connections that have been recently deleted. Two of them were not present
in any of the flows of the "archive" folder, so I suspect they would have
been removed longer ago.

Why a NiFi node should want to balance something in a connection that had
been deleted at least some minutes ago, I cannot figure out.

Thanks again!


El vie, 6 ago 2021 a las 8:27, Man on the Moon (<
manonthedarksideofthemoon@gmail.com>) escribió:

> Hi Mark,
>
> I just did the zgrep and found that none of the nodes has a reference to
> that uuid in its flow.xml.gz file. What other things could I try to gather
> more information?
>
> Thanks!
>
>
>
>
> El jue, 5 ago 2021 a las 17:40, Mark Payne (<ma...@hotmail.com>)
> escribió:
>
>> Luis,
>>
>> That is certainly unexpected. Does the connection
>> 0feea45e-017b-1000-0000-0000645cab5f exist in your flow?
>>
>> If so, I would recommend using grep to check both nodes to see if the
>> connection is there. I.e., on both nodes run
>>
>> zgrep "0feea45e-017b-1000-0000-0000645cab5f” conf/flow.xml.gz
>>
>> Do both nodes show the connection as existing?
>>
>> On Aug 5, 2021, at 3:02 AM, Man on the Moon <
>> manonthedarksideofthemoon@gmail.com> wrote:
>>
>> Hi all,
>>
>> we are running a NiFi 1.14.0 cluster with two nodes:
>> Node1:
>> hostname: nifi1.mydomain
>> ip: 172.16.0.1
>>
>> Node 2:
>> hostname: nifi2.mydomain
>> ip: 172.16.0.2
>>
>> Lately, we are persistingly seeing errors in the logs related to the Load
>> Balancing, and the flowfiles are processed only in the Primary Node. The
>> configuration we have for the load balancing is:
>>
>> nifi.cluster.load.balance.comms.timeout=30 sec
>> nifi.cluster.load.balance.connections.per.node=8
>> nifi.cluster.load.balance.host=172.16.0.1
>> nifi.cluster.load.balance.max.thread.count=16
>> nifi.cluster.load.balance.port=6342
>>
>> As seen in several posts in the list, we tried using the IPs in the
>> nifi.cluster.load.balance.host property, with no visible effect.
>>
>> The logs in node 1 show the following exceptions (continously):
>>
>> 2021-08-05 06:28:25,811 ERROR [Load-Balanced Client Thread-14]
>> o.a.n.c.q.c.c.a.n.NioAsyncLoadBalanceClient Failed to communicate with Peer
>> nifi2.mydomain:8443
>> java.io.IOException: Failed to decrypt data from Peer nifi2.mydomain:8443
>> because Peer unexpectedly closed connection
>> at
>> org.apache.nifi.controller.queue.clustered.client.async.nio.PeerChannel.decrypt(PeerChannel.java:269)
>> at
>> org.apache.nifi.controller.queue.clustered.client.async.nio.PeerChannel.read(PeerChannel.java:159)
>> at
>> org.apache.nifi.controller.queue.clustered.client.async.nio.PeerChannel.read(PeerChannel.java:80)
>> at
>> org.apache.nifi.controller.queue.clustered.client.async.nio.LoadBalanceSession.receiveSpaceAvailableResponse(LoadBalanceSession.java:573)
>> at
>> org.apache.nifi.controller.queue.clustered.client.async.nio.LoadBalanceSession.communicate(LoadBalanceSession.java:151)
>> at
>> org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClient.communicate(NioAsyncLoadBalanceClient.java:264)
>> at
>> org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClientTask.run(NioAsyncLoadBalanceClientTask.java:81)
>> at org.apache.nifi.engine.FlowEngine$2.run(FlowEngine.java:110)
>> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>> at
>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
>> at
>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>> at
>> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>> at
>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>> at java.lang.Thread.run(Thread.java:748)
>> 2021-08-05 06:28:25,811 ERROR [Load-Balanced Client Thread-14]
>> o.a.n.c.queue.SwappablePriorityQueue Updated Size of Queue Unacknowledged
>> from FlowFile Queue Size[ ActiveQueue=[0, 0 Bytes], Swap Queue=[1293,
>> 2575557 Bytes], Swap Files=[0], Unacknowledged=[-1293, -2575557 Bytes] ] to
>> FlowFile Queue Size[ ActiveQueue=[0, 0 Bytes], Swap Queue=[1293, 2575557
>> Bytes], Swap Files=[0], Unacknowledged=[-1293, -2575557 Bytes] ]
>> java.lang.RuntimeException: Cannot create negative queue size
>> at
>> org.apache.nifi.controller.queue.SwappablePriorityQueue.logIfNegative(SwappablePriorityQueue.java:1068)
>> at
>> org.apache.nifi.controller.queue.SwappablePriorityQueue.incrementUnacknowledgedQueueSize(SwappablePriorityQueue.java:1058)
>> at
>> org.apache.nifi.controller.queue.SwappablePriorityQueue.acknowledge(SwappablePriorityQueue.java:478)
>> at
>> org.apache.nifi.controller.queue.clustered.partition.RemoteQueuePartition$1.onTransactionFailed(RemoteQueuePartition.java:171)
>> at
>> org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClient.communicate(NioAsyncLoadBalanceClient.java:271)
>> at
>> org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClientTask.run(NioAsyncLoadBalanceClientTask.java:81)
>> at org.apache.nifi.engine.FlowEngine$2.run(FlowEngine.java:110)
>> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>> at
>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
>> at
>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>> at
>> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>> at
>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>> at java.lang.Thread.run(Thread.java:748)
>>
>>
>> While in Node 2 the exceptions are also continuous, but different, like
>> this:
>>
>> 2021-08-05 06:31:31,775 ERROR [Load-Balance Server Thread-165207]
>> o.a.n.c.q.c.s.ConnectionLoadBalanceServer Failed to communicate with Peer /
>> 172.16.0.1:42968
>> org.apache.nifi.controller.queue.clustered.server.TransactionAbortedException:
>> Attempted to receive FlowFiles from Peer nifi1.mydomain for Connection with
>> ID 0feea45e-017b-1000-0000-0000645cab5f but no Connection exists with that
>> ID
>> at
>> org.apache.nifi.controller.queue.clustered.server.StandardLoadBalanceProtocol.receiveFlowFiles(StandardLoadBalanceProtocol.java:205)
>> at
>> org.apache.nifi.controller.queue.clustered.server.StandardLoadBalanceProtocol.receiveFlowFiles(StandardLoadBalanceProtocol.java:134)
>> at
>> org.apache.nifi.controller.queue.clustered.server.ConnectionLoadBalanceServer$CommunicateAction.run(ConnectionLoadBalanceServer.java:162)
>> at java.lang.Thread.run(Thread.java:748)
>>
>> I am running out of ideas, and have tried everything I found online to
>> tackle load balancing problems. Apparently, there are no inter-node
>> communication issues.
>>
>> Any clues or guidance would be highly appreciated!!
>>
>> Luis.
>>
>>
>>

Re: Problems with Load Balancing - NiFi 1.14.0

Posted by Man on the Moon <ma...@gmail.com>.
Hi Mark,

I just did the zgrep and found that none of the nodes has a reference to
that uuid in its flow.xml.gz file. What other things could I try to gather
more information?

Thanks!




El jue, 5 ago 2021 a las 17:40, Mark Payne (<ma...@hotmail.com>)
escribió:

> Luis,
>
> That is certainly unexpected. Does the connection
> 0feea45e-017b-1000-0000-0000645cab5f exist in your flow?
>
> If so, I would recommend using grep to check both nodes to see if the
> connection is there. I.e., on both nodes run
>
> zgrep "0feea45e-017b-1000-0000-0000645cab5f” conf/flow.xml.gz
>
> Do both nodes show the connection as existing?
>
> On Aug 5, 2021, at 3:02 AM, Man on the Moon <
> manonthedarksideofthemoon@gmail.com> wrote:
>
> Hi all,
>
> we are running a NiFi 1.14.0 cluster with two nodes:
> Node1:
> hostname: nifi1.mydomain
> ip: 172.16.0.1
>
> Node 2:
> hostname: nifi2.mydomain
> ip: 172.16.0.2
>
> Lately, we are persistingly seeing errors in the logs related to the Load
> Balancing, and the flowfiles are processed only in the Primary Node. The
> configuration we have for the load balancing is:
>
> nifi.cluster.load.balance.comms.timeout=30 sec
> nifi.cluster.load.balance.connections.per.node=8
> nifi.cluster.load.balance.host=172.16.0.1
> nifi.cluster.load.balance.max.thread.count=16
> nifi.cluster.load.balance.port=6342
>
> As seen in several posts in the list, we tried using the IPs in the
> nifi.cluster.load.balance.host property, with no visible effect.
>
> The logs in node 1 show the following exceptions (continously):
>
> 2021-08-05 06:28:25,811 ERROR [Load-Balanced Client Thread-14]
> o.a.n.c.q.c.c.a.n.NioAsyncLoadBalanceClient Failed to communicate with Peer
> nifi2.mydomain:8443
> java.io.IOException: Failed to decrypt data from Peer nifi2.mydomain:8443
> because Peer unexpectedly closed connection
> at
> org.apache.nifi.controller.queue.clustered.client.async.nio.PeerChannel.decrypt(PeerChannel.java:269)
> at
> org.apache.nifi.controller.queue.clustered.client.async.nio.PeerChannel.read(PeerChannel.java:159)
> at
> org.apache.nifi.controller.queue.clustered.client.async.nio.PeerChannel.read(PeerChannel.java:80)
> at
> org.apache.nifi.controller.queue.clustered.client.async.nio.LoadBalanceSession.receiveSpaceAvailableResponse(LoadBalanceSession.java:573)
> at
> org.apache.nifi.controller.queue.clustered.client.async.nio.LoadBalanceSession.communicate(LoadBalanceSession.java:151)
> at
> org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClient.communicate(NioAsyncLoadBalanceClient.java:264)
> at
> org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClientTask.run(NioAsyncLoadBalanceClientTask.java:81)
> at org.apache.nifi.engine.FlowEngine$2.run(FlowEngine.java:110)
> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
> at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at java.lang.Thread.run(Thread.java:748)
> 2021-08-05 06:28:25,811 ERROR [Load-Balanced Client Thread-14]
> o.a.n.c.queue.SwappablePriorityQueue Updated Size of Queue Unacknowledged
> from FlowFile Queue Size[ ActiveQueue=[0, 0 Bytes], Swap Queue=[1293,
> 2575557 Bytes], Swap Files=[0], Unacknowledged=[-1293, -2575557 Bytes] ] to
> FlowFile Queue Size[ ActiveQueue=[0, 0 Bytes], Swap Queue=[1293, 2575557
> Bytes], Swap Files=[0], Unacknowledged=[-1293, -2575557 Bytes] ]
> java.lang.RuntimeException: Cannot create negative queue size
> at
> org.apache.nifi.controller.queue.SwappablePriorityQueue.logIfNegative(SwappablePriorityQueue.java:1068)
> at
> org.apache.nifi.controller.queue.SwappablePriorityQueue.incrementUnacknowledgedQueueSize(SwappablePriorityQueue.java:1058)
> at
> org.apache.nifi.controller.queue.SwappablePriorityQueue.acknowledge(SwappablePriorityQueue.java:478)
> at
> org.apache.nifi.controller.queue.clustered.partition.RemoteQueuePartition$1.onTransactionFailed(RemoteQueuePartition.java:171)
> at
> org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClient.communicate(NioAsyncLoadBalanceClient.java:271)
> at
> org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClientTask.run(NioAsyncLoadBalanceClientTask.java:81)
> at org.apache.nifi.engine.FlowEngine$2.run(FlowEngine.java:110)
> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
> at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at java.lang.Thread.run(Thread.java:748)
>
>
> While in Node 2 the exceptions are also continuous, but different, like
> this:
>
> 2021-08-05 06:31:31,775 ERROR [Load-Balance Server Thread-165207]
> o.a.n.c.q.c.s.ConnectionLoadBalanceServer Failed to communicate with Peer /
> 172.16.0.1:42968
> org.apache.nifi.controller.queue.clustered.server.TransactionAbortedException:
> Attempted to receive FlowFiles from Peer nifi1.mydomain for Connection with
> ID 0feea45e-017b-1000-0000-0000645cab5f but no Connection exists with that
> ID
> at
> org.apache.nifi.controller.queue.clustered.server.StandardLoadBalanceProtocol.receiveFlowFiles(StandardLoadBalanceProtocol.java:205)
> at
> org.apache.nifi.controller.queue.clustered.server.StandardLoadBalanceProtocol.receiveFlowFiles(StandardLoadBalanceProtocol.java:134)
> at
> org.apache.nifi.controller.queue.clustered.server.ConnectionLoadBalanceServer$CommunicateAction.run(ConnectionLoadBalanceServer.java:162)
> at java.lang.Thread.run(Thread.java:748)
>
> I am running out of ideas, and have tried everything I found online to
> tackle load balancing problems. Apparently, there are no inter-node
> communication issues.
>
> Any clues or guidance would be highly appreciated!!
>
> Luis.
>
>
>

Re: Problems with Load Balancing - NiFi 1.14.0

Posted by Mark Payne <ma...@hotmail.com>.
Luis,

That is certainly unexpected. Does the connection 0feea45e-017b-1000-0000-0000645cab5f exist in your flow?

If so, I would recommend using grep to check both nodes to see if the connection is there. I.e., on both nodes run

zgrep "0feea45e-017b-1000-0000-0000645cab5f” conf/flow.xml.gz

Do both nodes show the connection as existing?

On Aug 5, 2021, at 3:02 AM, Man on the Moon <ma...@gmail.com>> wrote:

Hi all,

we are running a NiFi 1.14.0 cluster with two nodes:
Node1:
hostname: nifi1.mydomain
ip: 172.16.0.1

Node 2:
hostname: nifi2.mydomain
ip: 172.16.0.2

Lately, we are persistingly seeing errors in the logs related to the Load Balancing, and the flowfiles are processed only in the Primary Node. The configuration we have for the load balancing is:

nifi.cluster.load.balance.comms.timeout=30 sec
nifi.cluster.load.balance.connections.per.node=8
nifi.cluster.load.balance.host=172.16.0.1
nifi.cluster.load.balance.max.thread.count=16
nifi.cluster.load.balance.port=6342

As seen in several posts in the list, we tried using the IPs in the nifi.cluster.load.balance.host property, with no visible effect.

The logs in node 1 show the following exceptions (continously):

2021-08-05 06:28:25,811 ERROR [Load-Balanced Client Thread-14] o.a.n.c.q.c.c.a.n.NioAsyncLoadBalanceClient Failed to communicate with Peer nifi2.mydomain:8443
java.io.IOException: Failed to decrypt data from Peer nifi2.mydomain:8443 because Peer unexpectedly closed connection
at org.apache.nifi.controller.queue.clustered.client.async.nio.PeerChannel.decrypt(PeerChannel.java:269)
at org.apache.nifi.controller.queue.clustered.client.async.nio.PeerChannel.read(PeerChannel.java:159)
at org.apache.nifi.controller.queue.clustered.client.async.nio.PeerChannel.read(PeerChannel.java:80)
at org.apache.nifi.controller.queue.clustered.client.async.nio.LoadBalanceSession.receiveSpaceAvailableResponse(LoadBalanceSession.java:573)
at org.apache.nifi.controller.queue.clustered.client.async.nio.LoadBalanceSession.communicate(LoadBalanceSession.java:151)
at org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClient.communicate(NioAsyncLoadBalanceClient.java:264)
at org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClientTask.run(NioAsyncLoadBalanceClientTask.java:81)
at org.apache.nifi.engine.FlowEngine$2.run(FlowEngine.java:110)
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at java.lang.Thread.run(Thread.java:748)
2021-08-05 06:28:25,811 ERROR [Load-Balanced Client Thread-14] o.a.n.c.queue.SwappablePriorityQueue Updated Size of Queue Unacknowledged from FlowFile Queue Size[ ActiveQueue=[0, 0 Bytes], Swap Queue=[1293, 2575557 Bytes], Swap Files=[0], Unacknowledged=[-1293, -2575557 Bytes] ] to FlowFile Queue Size[ ActiveQueue=[0, 0 Bytes], Swap Queue=[1293, 2575557 Bytes], Swap Files=[0], Unacknowledged=[-1293, -2575557 Bytes] ]
java.lang.RuntimeException: Cannot create negative queue size
at org.apache.nifi.controller.queue.SwappablePriorityQueue.logIfNegative(SwappablePriorityQueue.java:1068)
at org.apache.nifi.controller.queue.SwappablePriorityQueue.incrementUnacknowledgedQueueSize(SwappablePriorityQueue.java:1058)
at org.apache.nifi.controller.queue.SwappablePriorityQueue.acknowledge(SwappablePriorityQueue.java:478)
at org.apache.nifi.controller.queue.clustered.partition.RemoteQueuePartition$1.onTransactionFailed(RemoteQueuePartition.java:171)
at org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClient.communicate(NioAsyncLoadBalanceClient.java:271)
at org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClientTask.run(NioAsyncLoadBalanceClientTask.java:81)
at org.apache.nifi.engine.FlowEngine$2.run(FlowEngine.java:110)
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at java.lang.Thread.run(Thread.java:748)


While in Node 2 the exceptions are also continuous, but different, like this:

2021-08-05 06:31:31,775 ERROR [Load-Balance Server Thread-165207] o.a.n.c.q.c.s.ConnectionLoadBalanceServer Failed to communicate with Peer /172.16.0.1:42968<http://172.16.0.1:42968/>
org.apache.nifi.controller.queue.clustered.server.TransactionAbortedException: Attempted to receive FlowFiles from Peer nifi1.mydomain for Connection with ID 0feea45e-017b-1000-0000-0000645cab5f but no Connection exists with that ID
at org.apache.nifi.controller.queue.clustered.server.StandardLoadBalanceProtocol.receiveFlowFiles(StandardLoadBalanceProtocol.java:205)
at org.apache.nifi.controller.queue.clustered.server.StandardLoadBalanceProtocol.receiveFlowFiles(StandardLoadBalanceProtocol.java:134)
at org.apache.nifi.controller.queue.clustered.server.ConnectionLoadBalanceServer$CommunicateAction.run(ConnectionLoadBalanceServer.java:162)
at java.lang.Thread.run(Thread.java:748)

I am running out of ideas, and have tried everything I found online to tackle load balancing problems. Apparently, there are no inter-node communication issues.

Any clues or guidance would be highly appreciated!!

Luis.