Open kennknowles opened 2 years ago
Just ran into this on 2.43.
@tvalentyn @robertwb any insights into this case where the _clock
in the triggering logic is None
?
I'm not sure. Presumably the runner (the old python runner in this case) should have provided it?
seems like a bug in portable runner, still reproducible as of 2.45.0, not reproducible in Dataflow runner.
Repro: https://gist.github.com/tvalentyn/8a1f10e4de5a337d025f9bae2dba04be
I'll try to get some eyes on this and investigate if this looks like an easy fix.
I'm also experiencing this issue when testing locally. I haven't tried it running with any other runner other than the DirectRunner though.
test_stream = TestStream()
test_stream.add_elements(input_data)
with TestPipeline() as testpipeline:
results = (
testpipeline
| test_stream
| "Add info"
>> beam.ParDo(MyDoFn()
)
(
results
| "Form datapoints" >> beam.ParDo(FormDataPoints())
| "Emit datapoints on over x num elems or after y time elapsed"
>> beam.WindowInto(
window.GlobalWindows(),
trigger=trigger.Repeatedly(
trigger.AfterAny(trigger.AfterCount(1), trigger.AfterProcessingTime(2))
),
accumulation_mode=trigger.AccumulationMode.DISCARDING,
)
| "Group by table" >> beam.GroupBy(lambda x: x.table)
| "print" >> Map(print)
)
There's some good clues from the user Subhash in this jira issue: https://issues.apache.org/jira/browse/BEAM-5132 if that helps
I've just realised that I didn't have the pipeline options set to streaming:
pipeline_options = PipelineOptions()
pipeline_options.view_as(StandardOptions).streaming = True
with TestPipeline(options=pipeline_options) as testpipeline:
It's now working fine
Glad you were able to find a workaround, though it is still a bug in the sense that it should not fail with such an error even if streaming isn't set.
On Thu, Apr 13, 2023 at 8:40 AM RobMcKiernan @.***> wrote:
I've just realised that I didn't have the pipeline options set to streaming:
pipeline_options = PipelineOptions() pipeline_options.view_as(StandardOptions).streaming = True with TestPipeline(options=pipeline_options) as testpipeline:
It's now working fine
— Reply to this email directly, view it on GitHub https://github.com/apache/beam/issues/19070#issuecomment-1507189914, or unsubscribe https://github.com/notifications/unsubscribe-auth/AADWVAK6UX5M4JR3NC3TMUDXBAM5VANCNFSM5X2JJLWA . You are receiving this because you were mentioned.Message ID: @.***>
Tried to apply a window function to the existing pipeline that was running fine. Got the following error:
the composite window function is copied from official documents: https://beam.apache.org/documentation/programming-guide/#composite-triggers Please refer to pipeline relevant source code below:
Imported from Jira BEAM-5132. Original Jira may contain additional context. Reported by: thangbui.