The idea behind the DELETE mechanism is that in some environments there were timeouts that would occur quite frequently between PostHTTP / ListenHTTP and this resulted in quite a lot of data duplication. By adding in the two-phase commit, we were able to drastically reduce the amount of data duplication, as a timeout anywhere in the first (typically MUCH longer) phase would result in the data on the receiving side being dropped because the receiving side would not delete the hold that it placed on the FlowFiles.
It would be reasonable to add an option for PostHTTP so that it requests not to perform a two-phase commit. Alternatively, you could use either PostHTTP with 'Send as FlowFile' set to 'false' or you could use InvokeHTTP. These do not send the attributes, though, so you would need to precede this with a MergeContent with Merge Format of "FlowFile Stream, v3". Then, on the receiving side, you could use UnpackContent to unpack these FlowFile Packages back into their 'native' form. Or, a simpler option, if Amazon's ELB supports it, is to configure the ELB such that HTTP Requests that contain the same value for the "x-nifi-transaction-id" header will go to the same node. This header was added specifically to allow for this functionality through Load Balancers, but I don't know if ELB specifically supports this or not. Thanks -Mark > On Jun 7, 2016, at 2:16 PM, Aldrin Piri <[email protected]> wrote: > > InvokeHTTP may be the better option if the user is not interested in > transmitting content _packaged as_ FlowFiles. Someone with a bit more > history than myself can provide some additional context if I have strayed > off the path, but PostHTTP and ListenHTTP were precursors to Site to Site. > While they can transmit arbitrary content, were created for this > inter-instance communication to aid in the guaranteed delivery semantics. > The listed hold, in this case, is part of that transaction occurring where > a response is returned to acknowledge receipt via ListenHTTP [1] and the > ContentAcknowledgementServlet [2]. > > [1] > https://github.com/apache/nifi/blob/1bd2cf0d09a7111bcecffd0f473aa71c25a69845/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java > [2] > https://github.com/apache/nifi/blob/1bd2cf0d09a7111bcecffd0f473aa71c25a69845/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/servlets/ContentAcknowledgmentServlet.java > > On Tue, Jun 7, 2016 at 2:10 PM, Bryan Bende <[email protected]> wrote: > >> Looks like PostHttp interprets the response, and based on a series of >> conditions can intentionally issue a delete. >> >> I can't fully understand what is happening, but the code is here: >> >> https://github.com/apache/nifi/blob/1bd2cf0d09a7111bcecffd0f473aa71c25a69845/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java#L754 >> >> Unless someone understands what is happening there, maybe InvokeHttp could >> be used to make the post instead? >> >> -Bryan >> >> >> On Tue, Jun 7, 2016 at 1:53 PM, Edgardo Vega <[email protected]> >> wrote: >> >>> Joe, >>> >>> We were testing with another nifi machine outside the elb to post a >>> flowfile in using the PostHttp processor. It seems that on each post >> there >>> is an immediate delete call. Behind the load balancer it goes haywire. >>> >>> Cheers,, >>> >>> Edgardo >>> >>> On Tue, Jun 7, 2016 at 8:40 AM, Joe Witt <[email protected]> wrote: >>> >>>> Edgardo >>>> >>>> Are you saying the clients are posting and then calling delete? >>>> >>>> Also the more complex but flexible options are handle http request and >>>> response. >>>> >>>> Thanks >>>> Joe >>>> On Jun 7, 2016 7:04 AM, "Edgardo Vega" <[email protected]> wrote: >>>> >>>>> I wanted to throw Nifi behind a AWS ELB. I then have the ELB pointing >>> at >>>>> the nifi cluster. On the cluster I have a ListenHttp. I want to allow >>>>> people to post flow files to that url. >>>>> >>>>> When testing this setup, it seems I am getting an error due to the >> fact >>>> it >>>>> seems like first there is a post and then a delete to confirm that it >>> was >>>>> posted. How do I get Nifi to stop doing that? I just want nifi to >> post >>>> and >>>>> if its get something larger than a 400 fail otherwise succeed. >>>>> >>>>> >>>>> -- >>>>> Cheers, >>>>> >>>>> Edgardo >>>>> >>>> >>> >>> >>> >>> -- >>> Cheers, >>> >>> Edgardo >>> >>
