Taepper opened a new issue, #47641: URL: https://github.com/apache/arrow/issues/47641
### Describe the bug, including details regarding any error messages, version, and platform. The `SourceNode` is susceptible to backpressure. This means that a `SourceNode` should stop emitting data once `PauseProducing` is invoked. Instead, at that point a future is awaited to resume the emitting of data. The limitation of the current implementation is that the `SourceNode` only checks and awaits this backpressure future after invoking the `generator` lambda to only resume invoking it again when backpressure is removed. This concept would work great if the output of a `SourceNode` were synchronized, but the method `SliceAndDeliverMorsel` which is invoked after each call to `generator` only _asynchronously schedules_ the emitting of the data. Therefore, it can happen (and consistently does in my case), that the entire source has already been consumed and ALL batches are scheduled for further processing before backpressure is applied (in my case more concretely, before the `StartProducing` method even returns). This is a problem as subsequently invoked `ProjectNode::InputReceived` methods have no concept of backpressure anymore. The solution could be to make the lambda scheduled by the `SourceNode` susceptible to backpressure by awaiting the corresponding lambda. I also saw that there exists a `ThrottledAsyncTaskScheduler` that performs similar tasks, but using this might be dangerous regarding the introduction of dead-locks. I would also be interested whether there are other to circumvent the memory spikes in such cases. ### Component(s) C++ -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
