Hello!

AFAIK, Spout uses tick tuple to trigger check tuple timeout.
Tick interval is 'topology.message.timeout.secs'. Spout handles it to
override Storm conf 'topology.tick.tuple.freq.secs' to
'topology.message.timeout.secs' on the fly.

Spout also uses RotatingMap, which seems to be like a LRU.
It stores all ack tuples but not completed yet.
When acker tells Spout that root tuple is complete, Spout removes tuple
from RotatingMap.
When tick tuple occurs, Spout calls RotatingMap.rotate() to remove last
bucket and tuples in removed bucket are treated to timed-out.

Since RotatingMap has two bucket, so your situation could be occurred.
Why RotatingMap has two bucket is, Storm has a scenario that some tuples
are inserted to RotatingMap, and tick tuple occurred in a row.

tl;dr. Tuple can be treated to timeout between
(topology.message.timeout.secs + a) ~ topology.message.timeout.secs * 2.

You can find out these from storm-core source code,
- mk-threads :spout in executor.clj
- RotatingMap.java

Thanks.

Regards.
Jungtaek Lim (HeartSaVioR)


2015-04-14 7:25 GMT+09:00 <[email protected]>:

> Hello guys,
>
>
>
> Quite weird, but I have found actual timeout of message processing is
> always twice more than I have put in configuration.
>
> If I set up 2 minutes, I would get fail in spout in 4 minutes, etc. It's
> quite easy to reproduce in local mode if comment ack in one of bolts.
>
>
>
> Am I crazy or just miss something?
>
> Storm 0.9.3.
>
>
>
> Thanks,
>
> Aleksey.
>
>
>
> Please visit our website at
> http://financialservicesinc.ubs.com/wealth/E-maildisclaimer.html
> for important disclosures and information about our e-mail
> policies. For your protection, please do not transmit orders
> or instructions by e-mail or include account numbers, Social
> Security numbers, credit card numbers, passwords, or other
> personal information.
>



-- 
Name : 임 정택
Blog : http://www.heartsavior.net / http://dev.heartsavior.net
Twitter : http://twitter.com/heartsavior
LinkedIn : http://www.linkedin.com/in/heartsavior

Reply via email to