[
https://issues.apache.org/jira/browse/STORM-886?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14717197#comment-14717197
]
ASF GitHub Bot commented on STORM-886:
--------------------------------------
Github user knusbaum commented on a diff in the pull request:
https://github.com/apache/storm/pull/700#discussion_r38126936
--- Diff: storm-core/src/clj/backtype/storm/daemon/executor.clj ---
@@ -811,9 +841,18 @@
(setup-metrics! executor-data)
(let [receive-queue (:receive-queue executor-data)
- event-handler (mk-task-receiver executor-data
tuple-action-fn)]
+ event-handler (mk-task-receiver executor-data
tuple-action-fn)
+ high-watermark ((:storm-conf executor-data)
BACKPRESSURE-EXECUTOR-HIGH-WATERMARK)
+ low-watermark ((:storm-conf executor-data)
BACKPRESSURE-EXECUTOR-LOW-WATERMARK)
+ receive-queue-size ((:storm-conf executor-data)
TOPOLOGY-EXECUTOR-RECEIVE-BUFFER-SIZE)
+ high-watermark (int (* high-watermark receive-queue-size))
+ low-watermark (int (* low-watermark receive-queue-size))]
(disruptor/consumer-started! receive-queue)
(fn []
+ ;; this additional check is necessary because rec-q can be 0
while the executor backpressure flag is forever set
+ (if (and ((:storm-conf executor-data)
TOPOLOGY-BACKPRESSURE-ENABLE) (< (.population receive-queue) low-watermark)
@(:backpressure executor-data))
--- End diff --
Can we move the `and` form's second condition to a new line?
> 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)