-
Notifications
You must be signed in to change notification settings - Fork 4.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[Bug]: DirectRunner fails after grouping into batches, and applying flat map with assertion error #28716
Comments
Okay so I can repro the issue, specifically when the batch size specified is > the number of elements in the input PCollection and the another GBK happens in the pipeline after the GroupIntoBatches call. (GBK is marked as complete here and then re-invoked here.) This would happen if I can demonstrate this with this pipeline + some extra logging within the GBK transform evaluator:
With batch size = 3 we get each element through both GBKs, but with batch size > 3 the second GBK invocation fails on the first element because the Completion Tag and watermark are not reset. I haven't quite found where the reset between DoFn invocations happens yet, but this is where the problem is. |
Somehow this pipeline is using a wrong direct runner ( the old BundleBased direct runner is no longer maintained, and it is not expected that it is still being used for this batch pipeline. The relevant code is in beam/sdks/python/apache_beam/runners/direct/direct_runner.py Lines 115 to 126 in 223dded
We should have used the FnApi runner. |
Thanks for your reply. Based on what you said I added some breakpoints and I discovered that this is where the decision to not use FnApi runner is taken (line 111): beam/sdks/python/apache_beam/runners/direct/direct_runner.py Lines 107 to 112 in 0586161
This happens while it checks Here is a screenshot of the timer data I took in my IDE: I could investigate further and come up with a PR maybe, but I will need a few pointers:
|
Thanks! The implementation of GroupIntoBatches includes a processing-time timer: beam/sdks/python/apache_beam/transforms/util.py Line 1067 in 3a45ecf
This logic was added in #13144 @robertwb do you know if this restriction still applies: beam/sdks/python/apache_beam/runners/direct/direct_runner.py Lines 110 to 111 in 223dded
We could also try to lift it, run the tests and see what fails. |
I was trying to set up my environment using the following guide: https://cwiki.apache.org/confluence/display/BEAM/Python+Tips I noticed that the Should I open another issue that is about updating the documentation? |
@damianr13 I am working in the updating the wiki page. Thanks. You don't need to use |
|
What happened?
I am running the following code:
As you can see this code generates 3 dummy values. Let's say I want to take my elements in batches of 30, so I assign the same dummy key to all of them. After working with my batches, I want to go back to single elements, and then publish the results to Google Cloud Storage.
Running the code above fails with the following error:
I am running the code in a pipenv environent, with python version 3.11. I attached the requirements.
requirements.txt
Here is my Pipfile:
Issue Priority
Priority: 3 (minor)
Issue Components
The text was updated successfully, but these errors were encountered: