Oh, there was another issue there - heartbeat. Thanks for the information. Btw, can you share stack trace so that we can try to find out next issue?
2015년 6월 3일 수요일, charlie quillard<[email protected]>님이 작성한 메시지: > Ok, i resolved my error "Anchored onto '' after ack/fail" with the add > of "is_heartbeat_tuple method"(when i receive a heartbeat tuple, i call my > sync method) so i forget to update the code. > > But now, i have a new error : *backtype.storm.multilang.NoOutputException: > Pipe to subprocess seems to be broken! No output read.* > > > ------------------------------ > *De :* charlie quillard <[email protected] > <javascript:_e(%7B%7D,'cvml','[email protected]');>> > *Envoyé :* mercredi 3 juin 2015 16:20 > *À :* [email protected] > <javascript:_e(%7B%7D,'cvml','[email protected]');> > *Objet :* RE: ShellBolt Error - Anchored onto “” after ack/fail > > > Sorry i am a newbie in storm and i don't understand what do you want to > means with your sentence "clean Topology" ? I have to clean Spout/Bolt of > topology ? Kill the topology before the shutdown of LocalCluster ? Or send > a timeout message in my cppBolt when this one don't receive a new tuple > from the spout ? > > > And i saw a strange case in my cppBolt running, my spout sent the tuple > to my cppBolt and from a time, the last one did not receive the tuple. > > > ------------------------------ > *De :* 임정택 <[email protected] > <javascript:_e(%7B%7D,'cvml','[email protected]');>> > *Envoyé :* mercredi 3 juin 2015 15:13 > *À :* [email protected] > <javascript:_e(%7B%7D,'cvml','[email protected]');> > *Objet :* Re: ShellBolt Error - Anchored onto “” after ack/fail > > No, I mean "kill topology with enough waiting time". > Spout will be deactivated first and after processing pending tuples, > eventually Bolt has no tuple to process. Then it would be safe to clean > Topology. > It is a way to shutdown topology gracefully. > > 2015-06-03 21:34 GMT+09:00 charlie quillard <[email protected] > <javascript:_e(%7B%7D,'cvml','[email protected]');>>: > >> Thanks for answering, i will add message timeout sec and will keep you >> updated. >> >> >> ------------------------------ >> *De :* 임정택 <[email protected] >> <javascript:_e(%7B%7D,'cvml','[email protected]');>> >> *Envoyé :* mercredi 3 juin 2015 12:33 >> *À :* [email protected] >> <javascript:_e(%7B%7D,'cvml','[email protected]');> >> *Objet :* Re: ShellBolt Error - Anchored onto “” after ack/fail >> >> Hi. >> >> IMO there seems to be race condition between BoltReaderRunnable.run() >> and ShellBolt.cleanup(). >> ShellBolt.cleanup() interrupts ReaderThread but actually few of methods >> inside BoltReaderRunnable.run() throws InterruptedException. (Accessing >> _pendingWrites throws InterruptedException, but I cannot find any others.) >> So we assume that interrupting ReaderThread doesn't stop its flow at that >> time before encountering _pendingWrite. >> >> 1. ReaderThread.run() passes "while(_running)" check and dive into loop. >> 2. ShellBolt.cleanup() interrupts ReaderThread. It has marked as >> 'interrupted'. >> 3. ShellBolt.cleanup() clears _inputs >> 4. ReaderThread.run() reads shell message, and it is emit, so it calls >> handleEmit(). >> 5. ShellBolt.handleEmit() checks _inputs but nothing found. So it throws >> RuntimeException, as you stated. >> >> I'll file relevant issue to Storm JIRA. >> >> Normally topologies are killed with deactivating spouts first, and >> having enough times to process any pending tuples, so when Bolt is cleared, >> Bolt doesn't process any tuples at that time. >> >> You may want to kill topology with enough times, maybe message timeout >> sec or twice would be fine. >> >> Thanks. >> Jungtaek Lim (HeartSaVioR) >> >> >> 2015-06-03 18:20 GMT+09:00 charlie quillard <[email protected] >> <javascript:_e(%7B%7D,'cvml','[email protected]');>>: >> >>> Hello, >>> >>> >>> I have a problem with my shellbolt, which uses a cpp bolt(with this >>> wrapper : http://demeter.inf.ed.ac.uk/cross/stormcpp.html ) with the >>> multilang module. This one dies because the shellBolt's attribute "_inputs" >>> is empty and call the next runtime exception : "Anchored onto #Anchor after >>> ack/fail" . >>> >>> >>> The log: http://pastebin.com/pG9mHt8X >>> >>> >>> I used storm 0.9.4, and i think that my error is connected to this >>> reported issue : https://issues.apache.org/jira/browse/STORM-531 >>> >>> If someone can help me , i would appreciate it. >>> >>> >>> Best regards, >>> >>> Charlie >>> >>> >>> >>> >>> >> >> >> -- >> Name : 임 정택 >> Blog : http://www.heartsavior.net / http://dev.heartsavior.net >> Twitter : http://twitter.com/heartsavior >> LinkedIn : http://www.linkedin.com/in/heartsavior >> > > > > -- > Name : 임 정택 > Blog : http://www.heartsavior.net / http://dev.heartsavior.net > Twitter : http://twitter.com/heartsavior > LinkedIn : http://www.linkedin.com/in/heartsavior > -- Name : 임 정택 Blog : http://www.heartsavior.net / http://dev.heartsavior.net Twitter : http://twitter.com/heartsavior LinkedIn : http://www.linkedin.com/in/heartsavior
