Yes, spot on, thanks David. I forgot how the mime-type would be updated coming out of MergeContent. That's definitely going to be Tom's issue!
/Adam On Thu, Feb 9, 2023 at 7:23 AM David Handermann <[email protected]> wrote: > Tom and Adam, > > The UnpackContent Processor is not necessary after ListenHTTP in this flow. > > When MergeContent runs with FlowFile Version 3, it sets > application/flowfile-v3 as the mime.type FlowFile attribute. InvokeHTTP in > the default configuration uses the mime.type attribute to set the > Content-Type header. On the receiving side, ListenHTTP reads the > Content-Type header and extracts the FlowFile attributes and content from > the packaged binary. In essence, ListenHTTP also performs the function of > UnpackContent when it receives a message with the FlowFile Version 3. > > Removing the UnpackContent Processor from the flow configuration should > allow everything else to work as expected. > > Regards, > David Handermann > > > > On Thu, Feb 9, 2023 at 7:57 AM Tom Horvath via users < > [email protected]> wrote: > >> Adam, >> >> Thanks for the quick response. I am having an issue getting this to work. >> Option 1 is my preferred method as we don't want to have to pass >> certificates bidirectionally for this to work but providing the cert from >> the receiving instance of NiFi to the sending side is acceptable. I added >> the MergeContent processor as you described into the flow. The properties I >> chose on this one were Bin-Packing Algorithm for the Merge Strategy and >> FlowFile Stream, v3 for the Merge Format. I left the other properties as >> their default values. Now, on the receiving side, I added an UnpackContent >> processor into the flow and chose flowfile-stream-v3 as the Packing Format >> and left the other properties at their default values. The error I am >> receiving is >> >> 2023-02-09 13:44:48,317 ERROR [Timer-Driven Process Thread-10] >> o.a.n.processors.standard.UnpackContent >> UnpackContent[id=3661a39d-0186-1000-7de6-6a9f83b69400] Unable to unpack >> StandardFlowFileRecord[uuid=8fd3961b-04b2-47d6-9678-d50d6c1ad1fc,claim=StandardContentClaim >> [resourceClaim=StandardResourceClaim[id=1675868391223-1, container=default, >> section=1], offset=208, length=29],offset=0,name=test9,size=29]; routing to >> failure >> org.apache.nifi.processor.exception.ProcessException: IOException thrown >> from UnpackContent[id=3661a39d-0186-1000-7de6-6a9f83b69400]: >> java.io.IOException: Not in FlowFile-v3 format >> at >> org.apache.nifi.controller.repository.StandardProcessSession.write(StandardProcessSession.java:3148) >> at >> org.apache.nifi.processors.standard.UnpackContent$FlowFileStreamUnpacker.lambda$unpack$1(UnpackContent.java:533) >> at >> org.apache.nifi.controller.repository.StandardProcessSession.read(StandardProcessSession.java:2694) >> at >> org.apache.nifi.controller.repository.StandardProcessSession.read(StandardProcessSession.java:2662) >> at >> org.apache.nifi.processors.standard.UnpackContent$FlowFileStreamUnpacker.unpack(UnpackContent.java:527) >> at >> org.apache.nifi.processors.standard.UnpackContent.onTrigger(UnpackContent.java:294) >> at >> org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:27) >> at >> org.apache.nifi.controller.StandardProcessorNode.onTrigger(StandardProcessorNode.java:1356) >> at >> org.apache.nifi.controller.tasks.ConnectableTask.invoke(ConnectableTask.java:246) >> at >> org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent$1.run(TimerDrivenSchedulingAgent.java:102) >> 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.runAndReset(FutureTask.java:308) >> at >> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) >> at >> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) >> 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:750) >> Caused by: java.io.IOException: Not in FlowFile-v3 format >> at >> org.apache.nifi.util.FlowFileUnpackagerV3.unpackageFlowFile(FlowFileUnpackagerV3.java:59) >> at >> org.apache.nifi.processors.standard.UnpackContent$FlowFileStreamUnpacker.lambda$null$0(UnpackContent.java:535) >> at >> org.apache.nifi.controller.repository.StandardProcessSession.write(StandardProcessSession.java:3133) >> ... 17 common frames omitted >> >> I double checked the properties on each of new processors I added in the >> flow and they are exactly as described above. What am I doing wrong? >> >> >> --Tom >> >> >> On Wednesday, February 8, 2023 at 04:11:25 PM EST, Adam Taft < >> [email protected]> wrote: >> >> >> Tom, >> >> Acknowledged a bit of pain here. This was recently discussed in the Dev >> mailing list about this functionality going away, and yes, it's definitely >> going away. So you're aware, this deprecation will be locked in when NiFi >> 2.0 is released (in the near future). That being said, there are a few >> solutions for you... >> >> 1. You can bundle up your flowfiles using the MergeContent processor. You >> will generally want to choose the "FlowFile Stream v3" format. With this >> configuration, MergeContent will grab a batch of flowfiles and serialize >> them into a format that captures all of the flowfile attributes and >> payload. It's a very simple protocol underneath the hood, basically just a >> binary concatenation style of format. It does capture all of the >> attributes, which can then be extracted on the receiving side. >> >> On the receiving side, you will want to configure the UnpackContent >> processor, again with the "FlowFile Stream v3" configuration. This will >> effectively unwind the bundle and recreate all of your original flowfiles. >> Note that there are a couple of attributes that will be completely lost in >> this process. Specifically, the flowfile uuid will not be updated to the >> original value. So if you need to maintain the original flowfile id, you >> will need to copy it (using UpdateAttribute) before running through >> MergeContent on the sender. >> >> You would use InvokeHTTP and ListenHTTP as your transport for this >> option. The complete workflow would look something like: >> >> [Sender] ... -> MergeContent -> InvokeHTTP (using POST) -> { ... } -> >> [Receiver] ListenHTTP -> UnpackContent -> ... >> >> 2. You can also configure NiFi to talk to another NiFi using the "site to >> site" capability. The setup for this is a bit more complicated and may not >> necessarily work for your network topology. But it's there as an option and >> used by many people successfully. And it's pretty cool, because it's >> configured more like a processor group rather than a processor chain (as in >> #1 above). Site-to-site supports both "raw" (raw socket) and "http" >> transport protocols. There's plenty of good documentation about this, but >> here's the user manual entry for it: >> >> https://nifi.apache.org/docs/nifi-docs/html/user-guide.html#site-to-site >> >> I think in general, the second approach (site-to-site) would be >> preferred. But after experimenting with it or based on some characteristics >> of your network, you might find favor with the first method. Additionally, >> the first approach is probably closer to your existing dataflow >> configuration today, if you're just simply trying to model the behavior you >> had with deprecated PostHTTP. >> >> Hope this helps. >> >> /Adam >> >> >> On Wed, Feb 8, 2023 at 11:23 AM Tom Horvath via users < >> [email protected]> wrote: >> >> Hello, >> >> I’d like to use the InvokeHTTP processor instead of the PostHTTP >> processor because the PostHTTP processor is deprecated but I cannot figure >> out how to configure it to send the flow file attributes along with the >> content similar to how the PostHTTP processor does, when the send as flow >> file option is set to true. I want to maintain the attributes across the >> sites. Can someone assist? >> >> Regards, >> Tom >> >>
