[
https://issues.apache.org/jira/browse/STORM-886?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14741644#comment-14741644
]
ASF GitHub Bot commented on STORM-886:
--------------------------------------
Github user zhuoliu commented on the pull request:
https://github.com/apache/storm/pull/700#issuecomment-139678312
2. Test with a congested topology.
WordCountTopology2 (3 times more workers and executors than original
WordCount; in every minute, for the wordcount bolt: in the first 30 second,
there is a 50 ms sleep before each tuple emits"if (time/30000 %2 == 0)
Thread.sleep(50);").
Interestingly, without ABP, this 12 worker (87 executors) topology can not
run successfully in Storm since workers frequently crash because overflowing of
tuples in the bolt executors.
And the ZK receive workload is 20.6 to 21 pack/second.
On the contrast, with ABP enabled, this topology runs pretty well.
And we see about 22.5 pack/sec receive workloads on Zookeeper nodes.
This test shows the great advantage of Backpressure when dealing with
topologies that may have congested or slow components. Since (1) ABP makes sure
this topology can run successfully; (2) ABP causes small overheads to Zookeeper.
@revans2 @d2r @knusbaum
> Automatic Back Pressure
> -----------------------
>
> Key: STORM-886
> URL: https://issues.apache.org/jira/browse/STORM-886
> Project: Apache Storm
> Issue Type: Improvement
> Reporter: Robert Joseph Evans
> Assignee: Zhuo Liu
> Attachments: an simple example for backpressure.png, backpressure.png
>
>
> This new feature is aimed for automatic flow control through the topology DAG
> since different components may have unmatched tuple processing speed.
> Currently, the tuples may get dropped if the downstream components can not
> process as quickly, thereby causing a waste of network bandwidth and
> processing capability. In addition, it is difficult to tune the
> max.spout.pending parameter for best backpressure performance. Therefore, an
> automatic back pressure scheme is highly desirable.
> Heron proposed a form of back pressure that does not rely on acking or max
> spout pending. Instead spouts throttle not only when max.spout.pending is
> hit, but also if any bolt has gone over a high water mark in their input
> queue, and has not yet gone below a low water mark again. There is a lot of
> room for potential improvement here around control theory and having spouts
> only respond to downstream bolts backing up, but a simple bang-bang
> controller like this is a great start.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)