PS for the elements that flow through when < x you will need to add a data driven trigger to after the global window.
On Thu, 26 Jan 2023 at 20:11, Reza Ardeshir Rokni <raro...@gmail.com> wrote: > So it sounds like the timestamp of the event is not important here? If > that is correct then order is not important and you can do something like > this ( not tried it out so I might have missed detail).. > > Use a CombiningState to keep a count of elements that are being passed > through. > On a new element check the value, if its < x then output the element if > > x then set an EventTimer for time now+something and add the element to a > bagstate > In the timercode clear the bag state and reset the combinestate > > > On Thu, 26 Jan 2023 at 12:08, Vignesh Kumar Kathiresan < > vkath...@yahooinc.com> wrote: > >> Thanks Reza. Does this mean having a global window and triggering for >> every new event? The use case I was mentioning was a throttle kind of >> application using the key based state. Events flow though and the keyed >> state is increased by 1 but once a particular key reaches a certain max >> count, the newer events are skipped until the state is cleared after the >> throttle period. I am looking at something similar to a stateful keyed >> parDo so that all events of same key go to the same worker (assuming state >> is local to worker as in flink) >> >> On Thu, Jan 26, 2023 at 8:13 AM Reza Ardeshir Rokni <raro...@gmail.com> >> wrote: >> >>> Hi, >>> >>> For these types of use cases, folks will generally make use of the >>> Global Window which is -/+ inf and Timers. Some key considerations when >>> using the Global Window: >>> >>> 1- GC is not done by the system as the window will never close. >>> 2- There are no order guarantees, so you will often need to make use of >>> looping >>> timer >>> <https://urldefense.com/v3/__https://beam.apache.org/blog/looping-timers/__;!!Op6eflyXZCqGR5I!H02HPpgqudbKt18Xo0IC7a0LSrWD3znlzIBATW1EMItF4iCSrhnLM-ziCgsZd14hFgKlQEKBi4GQULrH$> >>> patterns. >>> >>> Cheers >>> >>> Reza >>> >>> On Thu, 26 Jan 2023 at 00:09, Vignesh Kumar Kathiresan via user < >>> user@beam.apache.org> wrote: >>> >>>> Hi Community, >>>> >>>> I am new to beam coming from flink. In flink state can be scoped to >>>> only key. A datasteam(similar to Pcollections in beam) can be converted to >>>> a keyed data stream. And a process function on this keyed stream can access >>>> state scoped to only key. It also has state scoped to key+window. In beam >>>> though I see only state scoped to key+window combination. Is my >>>> understanding correct? How are use cases using state scoped to key without >>>> windows solved in beam. >>>> >>>> thanks >>>> Vignesh >>>> >>>