Guille, Thanks for the extra details.
I just tried again. In my case, all worked as expected when I had a symlink to a directory. But when I had a symlink to a file, I got the same error and stack trace as you. So looks like we are handling the case properly for symlinked directories but not symlinked files. Thanks -Mark On Feb 3, 2022, at 11:43 AM, Guillermo Muñoz <guillermo.munoz.salg...@gmail.com<mailto:guillermo.munoz.salg...@gmail.com>> wrote: Hi, David. Sorry for the misunderstanding, my fault. Firstly, we tried using ListSFTP and FetchSFTP, and when it didn't work, we tried another option (GetSFTP), and I pasted the wrong stack trace. So, i've done the following tests: * ListSFTP + FetchSFTP: Error in ListSFTP [1] * GetSFTP: Error [2] * Generate flowfile + FetchSFTP with the name of the symlink in the Remote File property: OK, the file is downloaded. So, it seems the issue is in ListSFTP and GetSFTP, but FetchSFTP works fine. Thanks. Regards -- Guille [1] 2022-02-03 17:36:45,466 ERROR [Timer-Driven Process Thread-8] o.a.nifi.processors.standard.ListSFTP ListSFTP[id=64443154-ac76-1736-9e49-f2ca388dfbdf] Unable to get listing from *****.gz; skipping: java.io.FileNotFoundException: Could not perform listing on *****.gz because could not find the file on the remote server java.io.FileNotFoundException: Could not perform listing on *****.gz because could not find the file on the remote server at org.apache.nifi.processors.standard.util.SFTPTransfer.getListing(SFTPTransfer.java:350) at org.apache.nifi.processors.standard.util.SFTPTransfer.getListing(SFTPTransfer.java:365) at org.apache.nifi.processors.standard.util.SFTPTransfer.getListing(SFTPTransfer.java:262) at org.apache.nifi.processors.standard.ListFileTransfer.performListing(ListFileTransfer.java:120) at org.apache.nifi.processors.standard.ListSFTP.performListing(ListSFTP.java:150) at org.apache.nifi.processors.standard.ListFileTransfer.performListing(ListFileTransfer.java:112) at org.apache.nifi.processor.util.list.AbstractListProcessor.listByTrackingTimestamps(AbstractListProcessor.java:750) at org.apache.nifi.processor.util.list.AbstractListProcessor.onTrigger(AbstractListProcessor.java:525) at org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:27) at org.apache.nifi.controller.StandardProcessorNode.onTrigger(StandardProcessorNode.java:1273) at org.apache.nifi.controller.tasks.ConnectableTask.invoke(ConnectableTask.java:214) at org.apache.nifi.controller.scheduling.AbstractTimeBasedSchedulingAgent.lambda$doScheduleOnce$0(AbstractTimeBasedSchedulingAgent.java:63) 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) [2] 2022-02-03 17:39:14,714 ERROR [Timer-Driven Process Thread-27] o.a.nifi.processors.standard.GetSFTP GetSFTP[id=c0300c77-017e-1000-ffff-fffffa9c1f31] Unable to get listing from *****.gz; skipping: java.io.FileNotFoundException: Could not perform listing on *****.gz because could not find the file on the remote server java.io.FileNotFoundException: Could not perform listing on *****.gz because could not find the file on the remote server at org.apache.nifi.processors.standard.util.SFTPTransfer.getListing(SFTPTransfer.java:350) at org.apache.nifi.processors.standard.util.SFTPTransfer.getListing(SFTPTransfer.java:365) at org.apache.nifi.processors.standard.util.SFTPTransfer.getListing(SFTPTransfer.java:262) at org.apache.nifi.processors.standard.GetFileTransfer.fetchListing(GetFileTransfer.java:299) at org.apache.nifi.processors.standard.GetFileTransfer.onTrigger(GetFileTransfer.java:126) at org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:27) at org.apache.nifi.controller.StandardProcessorNode.onTrigger(StandardProcessorNode.java:1273) at org.apache.nifi.controller.tasks.ConnectableTask.invoke(ConnectableTask.java:214) at org.apache.nifi.controller.scheduling.AbstractTimeBasedSchedulingAgent.lambda$doScheduleOnce$0(AbstractTimeBasedSchedulingAgent.java:63) 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) El jue, 3 feb 2022 a las 17:09, David Handermann (<exceptionfact...@apache.org<mailto:exceptionfact...@apache.org>>) escribió: Hi Guille, Thanks for raising this issue and providing a stack trace. You mentioned using ListSFTP and FetchSFTP, but the stack references GetFileTransfer, which corresponds to GetSFTP. Can you confirm the same error using FetchSFTP? If you can confirm the same issue with FetchSFTP, it would be very helpful to add those details on the newer Jira issue NIFI-6699. NiFi SFTP processors switched to a different SSH library after the resolution of NIFI-5560, so it is possible that some changes may be necessary. However, it would be helpful to confirm whether this is an issue with FetchSFTP, GetSFTP, or both processors. Regards, David Handermann On Thu, Feb 3, 2022 at 9:57 AM Guillermo Muñoz Salgado <mun...@gmail.com<mailto:mun...@gmail.com>> wrote: Hi all, We are developing a use case in which we have to get some files from a server. We have implemented it by the listSFTP + FetchSFTP way in a 3 nodes cluster running nifi 1.15.3. But we are having some issues when what we want to get are symlinks instead of files. We have set true the property "Follow symlink" but we get the same results. Are we doing something wrong? Or is it a bug or a known issue? We have found this issue [1] but it is old and resolved and this other one [2], that is older and unresolved. We're not sure if they are related to this behaviour or not. I paste our error log: 2022-02-03 16:27:41,002 ERROR [Timer-Driven Process Thread-18] o.a.nifi.processors.standard.GetSFTP GetSFTP[id=c0300c77-017e-1000-ffff-fff-ffa9c1f31] Unable to get listing from testfile.gz; skipping: java.io.FileNotFoundException: Could not perform listing on testfile.gz because could not find the file on the remote server java.io.FileNotFoundException: Could not perform listing on testfile.gz because could not find the file on the remote server at org.apache.nifi.processors.standard.util.SFTPTransfer.getListing(SFTPTransfer.java:350) at org.apache.nifi.processors.standard.util.SFTPTransfer.getListing(SFTPTransfer.java:365) at org.apache.nifi.processors.standard.util.SFTPTransfer.getListing(SFTPTransfer.java:262) at org.apache.nifi.processors.standard.GetFileTransfer.fetchListing(GetFileTransfer.java:299) at org.apache.nifi.processors.standard.GetFileTransfer.onTrigger(GetFileTransfer.java:126) at org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:27) at org.apache.nifi.controller.StandardProcessorNode.onTrigger(StandardProcessorNode.java:1273) at org.apache.nifi.controller.tasks.ConnectableTask.invoke(ConnectableTask.java:214) at org.apache.nifi.controller.scheduling.AbstractTimeBasedSchedulingAgent.lambda$doScheduleOnce$0(AbstractTimeBasedSchedulingAgent.java:63) 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) Thanks in advance -- Guille [1] https://issues.apache.org/jira/browse/NIFI-5560 [2] https://issues.apache.org/jira/browse/NIFI-6699 -- Guillermo Muñoz Salgado