I just created a JIRA and will put up a PR shortly: https://issues.apache.org/jira/browse/NIFI-4391
The processor is catching the exception while attempting to obtain a connection, and then logs an error and transfers to failure which is where we see this message: 2017-09-17 14:20:20,860 ERROR [Timer-Driven Process Thread-10] o.apache.nifi.processors.standard.PutTCP PutTCP[id=80231a39-1008-1159-a6fa-1f9e3751d608] No available connections, and unable to create a new one, transferring https://github.com/apache/nifi/blob/master/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/put/AbstractPutEventProcessor.java#L330-L347 On Mon, Sep 18, 2017 at 9:22 AM, Joe Witt <[email protected]> wrote: > Good catch. Can you please be sure to cover in a JIRA? > > That said, wouldn't we see that in the stack trace during the > problematic condition? > > On Mon, Sep 18, 2017 at 9:16 AM, Bryan Bende <[email protected]> wrote: >> The code in SocketChannelSender that Davy pointed out could definitely >> be the problem... >> >> It makes a non-blocking channel and calls connect, then goes into a >> loop waiting for finishConnect() to return true, but if that doesn't >> happen before the configured timeout, then it throws an exception, but >> it doesn't first close the channel, and the processor also doesn't >> close the sender. >> >> As an example comparison, this code which is being used by PutTCP: >> >> https://github.com/apache/nifi/blob/master/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/put/sender/SocketChannelSender.java#L45-L78 >> >> Should be doing something like this where it catches, closes, and rethrows: >> >> https://github.com/apache/nifi/blob/master/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannel.java#L140-L179 >> >> >> On Mon, Sep 18, 2017 at 9:09 AM, Joe Witt <[email protected]> wrote: >>> Davy >>> >>> Interesting. So in looking through the stack trace I don't see >>> anything related to sockets nifi has initiated to another service and >>> nothing for PutTCP. I'm not saying that means there is nothing but >>> the stack traces only show the custom GetTCP processors. >>> >>> You can use netstat to show open sockets from the nifi process. Can >>> you try that and share those? >>> >>> Does the NiFi UI show the processor as having stuck threads? I'm >>> guessing not since there is nothing in the stack traces. >>> >>> Thanks >>> >>> On Mon, Sep 18, 2017 at 1:54 AM, ddewaele <[email protected]> wrote: >>>> Stopping the processor doesn't cleanup the tcp connection. It remains >>>> ESTABLISHED. >>>> >>>> There are 2 ways of getting out of it (none of them are ideal). >>>> >>>> - Restarting Nifi >>>> - Restarting the Moxa serial ports >>>> >>>> I've dumped the output in the following gist : >>>> https://gist.github.com/ddewaele/83705003740674962c1e133fb617f68c >>>> >>>> The GetTCP processor you'll see in the thread dump also interacts with the >>>> moxa. It is a Netty based custom processor we created (because there was no >>>> GetTCP at the time). However, we log all interactions (including client >>>> ports) with this processor and all of them end up getting closed correctly. >>>> >>>> So the "hanging" connection originated from the built-in PutTCP processor. >>>> >>>> >>>> Joe Witt wrote >>>>> If you stop the processor manually does it clean them up? >>>>> >>>>> When the connections appear stuck can you please get a thread dump? >>>>> >>>>> bin/nifi.sh dump >>>>> >>>>> The results end up in bootstrap.log. >>>>> >>>>> Thanks >>>>> Joe >>>>> >>>>> On Sep 17, 2017 2:22 PM, "ddewaele" < >>>> >>>>> ddewaele@ >>>> >>>>> > wrote: >>>>> >>>>>> We are using NiFi PutTCP processors to send messages to a number of Moxa >>>>>> onCell ip gateway devices. >>>>>> >>>>>> These Moxa devices are running on a cellular network with not always the >>>>>> most ideal connection. The Moxa only allows for a maximum of 2 >>>>>> simultaneous >>>>>> client connections. >>>>>> >>>>>> What we notice is that although we specify connection / read timeouts on >>>>>> both PutTCP and the Moxa, that sometimes a connection get "stuck". (In >>>>>> the >>>>>> moxa network monitoring we see 2 client sockets coming from PutTCP in the >>>>>> ESTABLISHED state that never go away). >>>>>> >>>>>> This doesn't always happen, but often enough for it to be considered a >>>>>> problem, as it requires a restart of the moxa ports to clear the >>>>>> connections >>>>>> (manual step). It typically happens when PutTCP experiences a Timeout. >>>>>> >>>>>> On the PutTCP processors we have the following settings : >>>>>> >>>>>> - Idle Connection Expiration : 30 seconds (we've set this higher due to >>>>>> bad >>>>>> gprs connection) >>>>>> - Timeout : 10 seconds (this is only used as a timeout for establishing >>>>>> the >>>>>> connection) >>>>>> >>>>>> On the Moxas we have >>>>>> >>>>>> - TCP alive check time : 2min (this should force the Moxa to close the >>>>>> socket) >>>>>> >>>>>> Yet for some reason the connection remains established. >>>>>> >>>>>> Here's what I found out : >>>>>> >>>>>> On the moxa I noticed a connection (with client port 48440) that is in >>>>>> ESTABLISHED mode for 4+ hours. (blocking other connections). On the Moxa >>>>>> I >>>>>> can see when the connection was established : >>>>>> >>>>>> 2017/09/17 14:20:29 [OpMode] Port01 Connect 10.192.2.90:48440 >>>>>> >>>>>> I can track that down in Nifi via the logs (unfortunately PutTCP doesn't >>>>>> log >>>>>> client ports, but from the timestamp I'm sure it's this connection : >>>>>> >>>>>> 2017-09-17 14:20:10,837 DEBUG [Timer-Driven Process Thread-10] >>>>>> o.apache.nifi.processors.standard.PutTCP >>>>>> PutTCP[id=80231a39-1008-1159-a6fa-1f9e3751d608] No available connections, >>>>>> creating a new one... >>>>>> 2017-09-17 14:20:20,860 ERROR [Timer-Driven Process Thread-10] >>>>>> o.apache.nifi.processors.standard.PutTCP >>>>>> PutTCP[id=80231a39-1008-1159-a6fa-1f9e3751d608] No available connections, >>>>>> and unable to create a new one, transferring >>>>>> StandardFlowFileRecord[uuid=79f2a166-5211-4d2d-9275-03f0ce4d5b29,claim= >>>>>> StandardContentClaim >>>>>> [resourceClaim=StandardResourceClaim[id=1505641210025-1, >>>>>> container=default, >>>>>> section=1], offset=84519, length=9],offset=0,name= >>>>>> 23934743676390659,size=9] >>>>>> to failure: java.net.SocketTimeoutException: Timed out connecting to >>>>>> 10.32.133.40:4001 >>>>>> 2017-09-17 14:20:20,860 ERROR [Timer-Driven Process Thread-10] >>>>>> o.apache.nifi.processors.standard.PutTCP >>>>>> java.net.SocketTimeoutException: Timed out connecting to >>>>>> 10.32.133.40:4001 >>>>>> at >>>>>> org.apache.nifi.processor.util.put.sender.SocketChannelSender.open( >>>>>> SocketChannelSender.java:66) >>>>>> ~[nifi-processor-utils-1.1.0.jar:1.1.0] >>>>>> at >>>>>> org.apache.nifi.processor.util.put.AbstractPutEventProcessor.createSender( >>>>>> AbstractPutEventProcessor.java:312) >>>>>> ~[nifi-processor-utils-1.1.0.jar:1.1.0] >>>>>> at >>>>>> org.apache.nifi.processors.standard.PutTCP.createSender(PutTCP.java:121) >>>>>> [nifi-standard-processors-1.1.0.jar:1.1.0] >>>>>> at >>>>>> org.apache.nifi.processor.util.put.AbstractPutEventProcessor. >>>>>> acquireSender(AbstractPutEventProcessor.java:334) >>>>>> ~[nifi-processor-utils-1.1.0.jar:1.1.0] >>>>>> at >>>>>> org.apache.nifi.processors.standard.PutTCP.onTrigger(PutTCP.java:176) >>>>>> [nifi-standard-processors-1.1.0.jar:1.1.0] >>>>>> at >>>>>> org.apache.nifi.controller.StandardProcessorNode.onTrigger( >>>>>> StandardProcessorNode.java:1099) >>>>>> [nifi-framework-core-1.1.0.jar:1.1.0] >>>>>> at >>>>>> org.apache.nifi.controller.tasks.ContinuallyRunProcessorTask.call( >>>>>> ContinuallyRunProcessorTask.java:136) >>>>>> [nifi-framework-core-1.1.0.jar:1.1.0] >>>>>> at >>>>>> org.apache.nifi.controller.tasks.ContinuallyRunProcessorTask.call( >>>>>> ContinuallyRunProcessorTask.java:47) >>>>>> [nifi-framework-core-1.1.0.jar:1.1.0] >>>>>> at >>>>>> org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent$1.run( >>>>>> TimerDrivenSchedulingAgent.java:132) >>>>>> [nifi-framework-core-1.1.0.jar:1.1.0] >>>>>> at >>>>>> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) >>>>>> [na:1.8.0_111] >>>>>> at java.util.concurrent.FutureTask.runAndReset( >>>>>> FutureTask.java:308) >>>>>> [na:1.8.0_111] >>>>>> at >>>>>> java.util.concurrent.ScheduledThreadPoolExecutor$ >>>>>> ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) >>>>>> [na:1.8.0_111] >>>>>> at >>>>>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run( >>>>>> ScheduledThreadPoolExecutor.java:294) >>>>>> [na:1.8.0_111] >>>>>> at >>>>>> java.util.concurrent.ThreadPoolExecutor.runWorker( >>>>>> ThreadPoolExecutor.java:1142) >>>>>> [na:1.8.0_111] >>>>>> at >>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run( >>>>>> ThreadPoolExecutor.java:617) >>>>>> [na:1.8.0_111] >>>>>> at java.lang.Thread.run(Thread.java:745) [na:1.8.0_111] >>>>>> >>>>>> >>>>>> On an OS level I indeed see the TCP connection originated from Nifi : >>>>>> >>>>>> netstat -tn | grep 48440 >>>>>> >>>>>> tcp 711 0 10.192.2.90:48440 10.32.133.40:4001 >>>>>> ESTABLISHED >>>>>> >>>>>> >>>>>> lsof -i TCP:48440 >>>>>> >>>>>> COMMAND PID USER FD TYPE DEVICE SIZE/OFF NODE NAME >>>>>> java 3424 root 1864u IPv4 404675057 0t0 TCP >>>>>> NifiServer:48440->10.32.133.40:newoak (ESTABLISHED) >>>>>> >>>>>> ps -ef | grep 3424 >>>>>> >>>>>> root 3424 3390 8 11:32 ? 00:44:33 java -classpath >>>>>> /opt/nifi-1.1.0/./conf..... >>>>>> >>>>>> Any ideas on how to debug this further ? >>>>>> >>>>>> I looked at the code in SocketChannelSender.java:66 and was wondering it >>>>>> no >>>>>> additional cleanup is necessary when throwing the SocketTimeoutException. >>>>>> If >>>>>> the connection ends up getting established after the timeout, doesn't >>>>>> PutTCP >>>>>> need to clean that up ? >>>>>> >>>>>> Thx >>>>>> >>>>>> >>>>>> >>>>>> >>>>>> -- >>>>>> Sent from: http://apache-nifi-users-list.2361937.n4.nabble.com/ >>>>>> >>>> >>>> >>>> >>>> >>>> >>>> -- >>>> Sent from: http://apache-nifi-users-list.2361937.n4.nabble.com/
