If you emit multiple tuples from nextTuple at once, it doesn't block
nextTuple to exceed max spout pending. For example you can set
max.spout.pending to 100, and emit 1000 tuples in nextTuple. Max spout
pending is for handling whether nextTuple should be called or not, not
restricting exact count of root tuples in flight.

Regarding second question I'm not sure. I'm working based on 1.x for months
so don't remember what 0.9.x supports. I'd suggest you to upgrade your
Storm cluster to latest 1.x. I expect release vote for 1.0.2 RC4 and 1.1.0
RC2 will be open soon, so wait for one or two week or so and upgrade latest.

Hope this helps.

2016년 7월 22일 (금) 오전 1:34, Girish Reddy <[email protected]>님이 작성:

> Hi,
>
> You are a TimeSaVIoR :). I looked at the spout stats and found the numbers
> are close enough.
>
> The default stream numbers are here:
>
> *Stream Emitted Transferred Acked Failed*
>
> *default 463960 463960 463060 0*
> 2 questions:
>
>    - Why is difference between emitted - acked > max.spout.pending? i.e
>    900 v/s 500(max.spout.pending). Is it because of UI reporting them slowly?
>    - I see that the ticket is for REST API. Are those numbers exposed
>    through NimbusClient Java API? (At least in Storm 1.0.1?)
>
>
> Thank You!!
>
>
> On Thu, Jul 21, 2016 at 9:02 AM, Jungtaek Lim <[email protected]> wrote:
>
>> You can click 'Show topology visualization' with 'Show System Stats', and
>> see where the Spout emits the tuples. Emit value could include sent tuples
>> for ackers, so you might fail to compare emit count == ack count.
>>
>> There's an API to show topology statistics via stream level, but it's
>> planned to be included to Storm 1.1.0.
>> https://issues.apache.org/jira/browse/STORM-1719
>>
>> Hope this helps.
>>
>> Thanks,
>> Jungtaek Lim (HeartSaVIoR)
>>
>> 2016년 7월 22일 (금) 오전 12:48, Girish Reddy <[email protected]>님이 작성:
>>
>>> Hello,
>>>
>>> I'm using storm-0.9.6 and have a topology with KafkaSpout and 3 other
>>> bolts. I've max.spout.pending : 500, message.timeout : 30, acker.executors:
>>> 10.
>>>
>>> I see the following numbers for kafka-spout in UI :
>>>
>>> *Executors Tasks Emitted Transferred Acked Failed*
>>> *3* *3* *834380* *834380* *414420* *0*
>>>
>>> My question here is why do I see a huge difference between emitted and
>>> acked + failed (834380 - 414420 = 419960) when I set the max.spout.pending
>>> as 500. Does that mean there are still 419960 tuples in flight? Or does it
>>> mean they still need to be acked? I'm asking this because I want to
>>> understand if my spout processed all the tuples and once I'm done I want to
>>> kick off some action like for example killing topology/sending email or
>>> whatever. Or to be precise, I want to understand if there are any tuples in
>>> flight.
>>>
>>> I'm forming tuple tree by anchoring tuples and ultimately acking/failing
>>> the tuple tree at the sink bolt.
>>>
>>> Please help!
>>>
>>> Thank You.
>>>
>>
>

Reply via email to