-
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]: Python AfterProcessingTime
behaves different than Java
#23071
Comments
I have made a fix for this, changing it to how Java works, will submit the PR tomorrow |
Is there a possibility this will be a breaking change for people that rely on the old behavior? |
hm there is. However this code change only affects DirectRunner pipelines. For other runners, the trigger code is executed by the runners.... |
Just for clarification, I tested the trigger in Dataflow (both Legacy and UW) and it works as in Java (as it should). I am not aware how this works in other runners, but I think users would expect it to work as in Java and how it's explained in the documentation. Do you have any suggestion on how to approach this @TheNeuralBit ? Maybe adding notes in the release? Let me know if you want to discuss this offline |
Do you mean it worked as in Java on Dataflow, even before your change?
Yes but users sometimes come to rely on buggy behavior :) (obligatory xkcd: https://xkcd.com/1172/)
I don't think I have enough context here to advise on a specific action. If the change really is just affecting the DirectRunner (non-production) I think it makes sense to just mention this in the release notes. If there's a possibility it will affect production pipelines on Dataflow or other runners we might consider a deprecation cycle (add a warning that behavior will change in a future release, possibly with a new trigger option that preserves existing behavior). |
Yes, it works as Java on Dataflow (both Legacy and UW)
I don't have the capacity to test on other runners besides DF and DirectRunner, so not sure how to proceed. To be fair, the documentation marks the trigger as experimental. Considering the description, how Java works and what most user would actually want (*) and this being experimental, I think adding this change will help more than can hinder users. I'd say it's worse if a customer relies on it working as described but it doesn't, than a customer relying on something buggy. (*) As of now (with no fix), the trigger just works as a session window, which doesn't really add value to the poll of triggers, while having something as "wait X time and then trigger" does. Of course, this is just my personal opinion and I'm nobody to decide what to do. Let me know what you want to do and i'd do my best :D |
Got it, maybe let's just record it as a bugfix in CHANGES.md |
Created this PR |
What happened?
The Python trigger
AfterProcessingTime
behaves different than Java'sAfterProcessingTime.pastFirstElementInPane().plusDelayOf
.While Java behaves as "wait X time since the first element to trigger", Python behaves similar to a Session Window, where the wait is since the previous element instead of the first element in pane:
https://github.com/apache/beam/blob/master/sdks/python/apache_beam/transforms/trigger.py#L387
You can see this in this example:
Python
Java
The output in Python is two panes
['1', '2', '3', '4'], ['5', '6', '7', '8', '9', '10']
and Java is the "right" output['1', '2'], ['3', '4'], ['5', '6'], ['7', '8'], ['9', '10']
.The fix doesn't seem hard (worse thing to say ever), but given that users may be using this trigger already, I am not sure how to proceed.
Issue Priority
Priority: 2
Issue Component
Component: sdk-py-core
The text was updated successfully, but these errors were encountered: