Would it be worth considering turning it off by default and making it
opt-in? I can imagine it's very useful, but a runner needs to implement
checking the generated requirements file, right? In which case, the default
behavior incurs a cost while not fully implementing the debugging behavior.

Another example, the TrivialRunner[1] will run this pip freeze subprocess
on every pipeline run but I don't think it checks the generated file.

(I don't feel strongly either way FWIW)

[1]
https://github.com/apache/beam/blob/b7f2e1611556cf2dab7e9a901d3477023cd71294/sdks/python/apache_beam/runners/trivial_runner.py#L47

On Thu, Jun 5, 2025 at 9:59 AM Danny McCormick via dev <dev@beam.apache.org>
wrote:

> Thanks for calling this out. I generally agree with you. I've found this
> feature to be generally quite useful for production jobs running in
> distributed environments. I have seen several issues which have been solved
> because of it (and similarly I have seen issues which would have benefited
> from it before its introduction). At the same time, I agree it is not worth
> the cost when running locally since you're not at the same risk of
> diverging environments.
>
> I'd vote we disable this for Prism and can take that on as part of
> enabling prism as the default runner [1].
>
> [1] WIP - https://github.com/apache/beam/pull/34612
>
> On Wed, Jun 4, 2025 at 10:48 AM Joey Tran <joey.t...@schrodinger.com>
> wrote:
>
>> Hey all,
>>
>> We recently upgraded to Beam 2.63 from 2.50. After the upgrade, our unit
>> tests testing our runner saw a 4x-5x performance hit. It turned out it was
>> because for every pipeline run, the default `PipelineRunner.run_pipeline
>> invokes `PIpelineRunner.default_environment`[1] which eventually results in
>> a subprocess call to `pip freeze` to gather python requirements for later
>> logging [2].
>>
>> This caught me by surprise and was very hard to debug since the massive
>> slowdown was due to using 2x more subprocesses than I specified for my unit
>> test runner, which resulted in my python processes thrashing. I've turned
>> off this logging feature for our runner, but just wanted to give a heads up
>> as any runner that uses the default `run_pipeline` method will incur this
>> cost. May be relevant to using the PrismRunner to replace the python
>> directrunner (or maybe y'all do want this check?)
>>
>> Cheers,
>> Joey
>>
>> [1]
>> https://github.com/apache/beam/blob/dd51c4cba108a0c425c37dfc28a81b3caf80d215/sdks/python/apache_beam/runners/runner.py#L182
>> [2]
>> https://github.com/apache/beam/blob/dd51c4cba108a0c425c37dfc28a81b3caf80d215/sdks/python/apache_beam/runners/portability/stager.py#L906
>>
>

Reply via email to