[ 
https://issues.apache.org/jira/browse/BEAM-9487?focusedWorklogId=664089&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-664089
 ]

ASF GitHub Bot logged work on BEAM-9487:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 12/Oct/21 14:23
            Start Date: 12/Oct/21 14:23
    Worklog Time Spent: 10m 
      Work Description: kennknowles commented on a change in pull request 
#15603:
URL: https://github.com/apache/beam/pull/15603#discussion_r727189577



##########
File path: sdks/python/apache_beam/transforms/trigger.py
##########
@@ -833,13 +839,12 @@ class AfterAny(_ParallelTriggerFn):
   combine_op = any
 
   def may_lose_data(self, windowing):
-    reason = DataLossReason.NO_POTENTIAL_LOSS
-    for trigger in self.triggers:
-      t_reason = trigger.may_lose_data(windowing)
-      if t_reason == DataLossReason.NO_POTENTIAL_LOSS:
-        return t_reason
-      reason |= t_reason
-    return reason
+    """If any sub-trigger may finish, this one may finish."""

Review comment:
       That actually means that its semantics have diverged from spec, so on 
portable runners where triggers are not executed in the Python we would get 
different results. Hrmm.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 664089)
    Time Spent: 36h 40m  (was: 36.5h)

> GBKs on unbounded pcolls with global windows and no triggers should fail
> ------------------------------------------------------------------------
>
>                 Key: BEAM-9487
>                 URL: https://issues.apache.org/jira/browse/BEAM-9487
>             Project: Beam
>          Issue Type: Bug
>          Components: sdk-py-core
>            Reporter: Udi Meiri
>            Assignee: Zachary Houfek
>            Priority: P1
>              Labels: EaseOfUse, starter
>             Fix For: 2.34.0
>
>          Time Spent: 36h 40m
>  Remaining Estimate: 0h
>
> This, according to "4.2.2.1 GroupByKey and unbounded PCollections" in 
> https://beam.apache.org/documentation/programming-guide/.
> bq. If you do apply GroupByKey or CoGroupByKey to a group of unbounded 
> PCollections without setting either a non-global windowing strategy, a 
> trigger strategy, or both for each collection, Beam generates an 
> IllegalStateException error at pipeline construction time.
> Example where this doesn't happen in Python SDK: 
> https://stackoverflow.com/questions/60623246/merge-pcollection-with-apache-beam
> I also believe that this unit test should fail, since test_stream is 
> unbounded, uses global window, and has no triggers.
> {code}
>   def test_global_window_gbk_fail(self):
>     with TestPipeline() as p:
>       test_stream = TestStream()
>       _ = p | test_stream | GroupByKey()
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to