[ https://issues.apache.org/jira/browse/STORM-886?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14715347#comment-14715347 ]
ASF GitHub Bot commented on STORM-886: -------------------------------------- Github user revans2 commented on a diff in the pull request: https://github.com/apache/storm/pull/700#discussion_r38021586 --- Diff: storm-core/src/clj/backtype/storm/daemon/executor.clj --- @@ -602,7 +607,15 @@ (log-message "Activating spout " component-id ":" (keys task-datas)) (fast-list-iter [^ISpout spout spouts] (.activate spout))) - (fast-list-iter [^ISpout spout spouts] (.nextTuple spout))) + (if (and + ((:storm-conf executor-data) TOPOLOGY-BACKPRESSURE-ENABLE) + @(:throttle-on (:worker executor-data)) + suspend-time + (not= suspend-time 0)) + (do --- End diff -- Can we move this conditional up to be part of the overflow-buffer/max-spout-pending check? It feels like it fits better there, and that we don't want to output anything when back-pressure is on, instead of just slowing down how quickly we output. Also instead of logging something, can we look at having a metric instead that shows how may times we called paused for a given reason. Logging is not going to really give us the picture we want and is going to be difficult to follow. > 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)