Open damianr13 opened 1 year ago
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 _is_final_bundle()
returns true, AKA the watermark was advanced to infinity prematurely for the next invocation of a GBK. This seems to be the case if batch size > num elements, with the watermark still being at positive infinity on the next invocation instead of being reset to negative infinity for the next DoFn.
I can demonstrate this with this pipeline + some extra logging within the GBK transform evaluator:
def run(argv=None, save_main_session=True):
parser = argparse.ArgumentParser()
# Parse the arguments from the command line as defined in the options class
known_args, pipeline_args = parser.parse_known_args(argv)
pipeline_options = PipelineOptions(pipeline_args)
with (beam.Pipeline(options=pipeline_options) as p):
(
p
| "Read Vertex embedder output" >> beam.Create(["a", "b", "c"])
| "Assign dummy keys" >> beam.Map(lambda x: ("key", x))
| "Batch up to 30 elements" >> beam.GroupIntoBatches(3)
| "Flat map" >> beam.FlatMap(lambda x: x[1])
| "re-key" >> beam.Map(lambda x: ("key2", x))
| "gbk" >> beam.GroupByKey()
)
if __name__ == "__main__":
run()
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 (apache_beam/runners/direct/executor.py
).
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 https://github.com/apache/beam/blob/223dded769df48270df317868dc32144ec2fb353/sdks/python/apache_beam/runners/direct/direct_runner.py#L115-L126
We should have used the FnApi runner.
@tvalentyn
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):
This happens while it checks _GroupIntoBatchesDoFn
.
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: https://github.com/apache/beam/blob/3a45ecf4b271997b5ce03e1181676356eaa351e1/sdks/python/apache_beam/transforms/util.py#L1067 , you can find more information about timers in https://beam.apache.org/documentation/programming-guide/#processing-time-timers.
This logic was added in https://github.com/apache/beam/pull/13144
@robertwb do you know if this restriction still applies: https://github.com/apache/beam/blob/223dded769df48270df317868dc32144ec2fb353/sdks/python/apache_beam/runners/direct/direct_runner.py#L110-L111 ?
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 build-requirements.txt
file is missing. I am guessing that one can get the dependencies using the gradle build file.
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 build-requirements.txt
anymore if you rebase/merge onto master branch.
GroupIntoBatches
seems to still cause this issue with beam 2.58.0
when using DirectRunner
and even TestPipeline
.
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