Thank you guys for the discussion. What if I want exact-once processing for all nodes (bolts), even when failure happens, will Trident be the one?
On Wed, Sep 14, 2016 at 3:49 PM, Ravi Sharma <[email protected]> wrote: > Hi T.I. > Few things why Spout is responsible for replay rather then Various Bolts. > > 1. ack and fail messages carry only message ID, Usually your spouts > generate messaged Id and knows what tuple/message is linked to it(via > source i.e. jms etc). If ack or fail happens then Spout can do various > things like on ack delete from queue, on fail put in some dead letter > queue. intermediate Bolt Wont know what message it sent, unless you > implement something of your own. Technically you can put Delete message > from JMS in bolts but then your whole topology knows from where you are > getting data, what if tommorow you start processing data from JMS, Http > rest service, Database and file system etc. > > 2. BoltB fails, it tells BoltA, BoltA retry 3 times, it fails 3 times, now > what BoltA should do,? Send it to another bolt(say BoltPreA exists between > him and spout) or send it to Spout.? > If it sends to BoltPreA that means BoltPreA will retry 3 > times(just using 3 number consider as N), that means for each try to > BoltPreA, BoltA will retry again 3 times, so total 9 retries.(basically > total retries will be based on Total bolt from Spout to Failure Bolt TB and > total Retries TR, it will be like TR + Power(TR,2) ..... + Power(TR,TB) > If you send back from failure from BoltA to Spout then we can > argue why not send it to Spout from BoltB, as a framework i shouldnt be > looking into if BoltB is really costly or BoltA is really costly. > > 3. Also failure scenario are suppose to be really really low, and if your > database is down(means 100% tuple will fail), then performance wont be your > only concern. your concern will be to make sure database comes up and > reprocess all failed tuple. > > 4. Also you will have to take care of retry logic in every Bolt. Currently > its only at one place. > > > > *There is one thing i am looking forward from Storm is to inform Spout > about what kind of failure it was*. i.e. if it was ConnectionTimeout or > ReadTimeout etc, that means if i retry it may pass. But say it was null > pointer exception(java world) , i know the data which is being expected is > not there and my code is not handling that scenario, so either i will have > to change code or ask data provider to send that field, but retry wont help > me. > > Currently only way to do is use a outside datastore like Redis, whichever > Bolt you fail add a key with mesageId and Exception/error detail in redis > before calling fail. and then let Spout read that data from redis with > messageId received in onFail call and then spout can decide if i want to > retry or not. I would usually Create two wrappers Retry-able Exception and > *non* Retry-able Exception, so each bolt can inform whether retry can > help or not. Its upto you where you put this decision making logic. > > > > Thanks > Ravi. > > > > > > > On Wed, Sep 14, 2016 at 6:43 AM, Tech Id <[email protected]> wrote: > >> Thanks Ambud, >> >> I did read some very good things about acking mechanism in Storm but I am >> not sure it explains why point to point checking is expensive. >> >> Consider the example: Spout--> BoltA--->BoltB. >> >> If BoltB fails, it will report failure to the acker. >> If the acker can ask the Spout to replay, then why can't the acker ask >> the parent of BoltB to replay at this point? >> I don't think keeping parent of a bolt could be expensive. >> >> >> On a related note, I am a little confused about a statement "When a new >> tupletree is born, the spout sends the XORed edge-ids of each tuple >> recipient, which the acker records in its pending ledger" in >> Acking-framework-implementation.html >> <http://storm.apache.org/releases/current/Acking-framework-implementation.html> >> . >> How does the spout know before hand which bolts would receive the tuple? >> Bolts forward tuples to other bolts based on groupings and dynamically >> generated fields. How does spout know what fields will be generated and >> which bolts will receive the tuples? If it does not know that, then how >> does it send the XOR of each tuple recipient in a tuple's path because each >> tuple's path will be different (I think, not sure though). >> >> >> Thx, >> T.I. >> >> >> On Tue, Sep 13, 2016 at 6:37 PM, Ambud Sharma <[email protected]> >> wrote: >> >>> Here is a post on it https://bryantsai.com/fault-to >>> lerant-message-processing-in-storm/. >>> >>> Point to point tracking is expensive unless you are using transactions. >>> Flume does point to point transfers using transactions. >>> >>> On Sep 13, 2016 3:27 PM, "Tech Id" <[email protected]> wrote: >>> >>>> I agree with this statement about code/architecture but in case of some >>>> system outages, like one of the end-points (Solr, Couchbase, Elastic-Search >>>> etc.) being down temporarily, a very large number of other fully-functional >>>> and healthy systems will receive a large number of duplicate replays >>>> (especially in heavy throughput topologies). >>>> >>>> If you can elaborate a little more on the performance cost of tracking >>>> tuples or point to a document reflecting the same, that will be of great >>>> help. >>>> >>>> Best, >>>> T.I. >>>> >>>> On Tue, Sep 13, 2016 at 12:26 PM, Hart, James W. <[email protected]> >>>> wrote: >>>> >>>>> Failures should be very infrequent, if they are not then rethink the >>>>> code and architecture. The performance cost of tracking tuples in the way >>>>> that would be required to replay at the failure is large, basically that >>>>> method would slow everything way down for very infrequent failures. >>>>> >>>>> >>>>> >>>>> *From:* S G [mailto:[email protected]] >>>>> *Sent:* Tuesday, September 13, 2016 3:17 PM >>>>> *To:* [email protected] >>>>> *Subject:* Re: How will storm replay the tuple tree? >>>>> >>>>> >>>>> >>>>> Hi, >>>>> >>>>> >>>>> >>>>> I am a little curious to know why we begin at the spout level for case >>>>> 1. >>>>> >>>>> If we replay at the failing bolt's parent level (BoltA in this case), >>>>> then it should be more performant due to a decrease in duplicate >>>>> processing >>>>> (as compared to whole tuple tree replays). >>>>> >>>>> >>>>> >>>>> If BoltA crashes due to some reason while replaying, only then the >>>>> Spout should receive this as a failure and whole tuple tree should be >>>>> replayed. >>>>> >>>>> >>>>> >>>>> This saving in duplicate processing will be more visible with several >>>>> layers of bolts. >>>>> >>>>> >>>>> >>>>> I am sure there is a good reason to replay the whole tuple-tree, and >>>>> want to know the same. >>>>> >>>>> >>>>> >>>>> Thanks >>>>> >>>>> SG >>>>> >>>>> >>>>> >>>>> On Tue, Sep 13, 2016 at 10:22 AM, P. Taylor Goetz <[email protected]> >>>>> wrote: >>>>> >>>>> Hi Cheney, >>>>> >>>>> >>>>> >>>>> Replays happen at the spout level. So if there is a failure at any >>>>> point in the tuple tree (the tuple tree being the anchored emits, >>>>> unanchored emits don’t count), the original spout tuple will be replayed. >>>>> So the replayed tuple will traverse the topology again, including >>>>> unanchored points. >>>>> >>>>> >>>>> >>>>> If an unanchored tuple fails downstream, it will not trigger a replay. >>>>> >>>>> >>>>> >>>>> Hope this helps. >>>>> >>>>> >>>>> >>>>> -Taylor >>>>> >>>>> >>>>> >>>>> >>>>> >>>>> On Sep 13, 2016, at 4:42 AM, Cheney Chen <[email protected]> wrote: >>>>> >>>>> >>>>> >>>>> Hi there, >>>>> >>>>> >>>>> >>>>> We're using storm 1.0.1, and I'm checking through >>>>> http://storm.apache.org/releases/1.0.1/Guaranteeing- >>>>> message-processing.html >>>>> >>>>> >>>>> >>>>> Got questions for below two scenarios. >>>>> >>>>> Assume topology: S (spout) --> BoltA --> BoltB >>>>> >>>>> 1. S: anchored emit, BoltA: anchored emit >>>>> >>>>> Suppose BoltB processing failed w/ ack, what will the replay be, will >>>>> it execute both BoltA and BoltB or only failed BoltB processing? >>>>> >>>>> >>>>> >>>>> 2. S: anchored emit, BoltA: unanchored emit >>>>> >>>>> Suppose BoltB processing failed w/ ack, replay will not happen, >>>>> correct? >>>>> >>>>> >>>>> >>>>> -- >>>>> >>>>> Regards, >>>>> Qili Chen (Cheney) >>>>> >>>>> E-mail: [email protected] >>>>> MP: (+1) 4086217503 >>>>> >>>>> >>>>> >>>>> >>>>> >>>> >>>> >> > -- Regards, Qili Chen (Cheney) E-mail: [email protected] MP: (+1) 4086217503
