Hi Jan,
Your proposal has merit, but I think using the TimerFamily specification is
more consistent with the existing API. I think that a StateFamily can also
have domains just like timers.
Luke's suggestion for the proto changes sound good.
Reuven
On Tue, Oct 29, 2019 at 2:43 AM Jan Lukavský
Based upon the current description, from the portability perspective we
could:
Update the timer spec map comment[1] to be:
// (Optional) A mapping of local timer families to timer specifications.
map timer_specs = 5;
And update the timer coder to have the timer id[2]:
// Encodes a timer
Hi Reuven,
I didn't propose to restrict the model. Model can (and should have)
multiple timers per key and even dynamic. The question was if this can
be made efficiently by using single timer (after all, the runner will
probably have single "timer service" so no matter what we expose on the
Just to circle back around, after the discussion on this thread I propose
modifying the proposed API as follows:
class MyDoFn extends DoFn {
@TimerFamily("timers") TimerSpec timers =
TimerSpecs.timerFamily(TimeDomain(EVENT_TIME));
@ProcessElement
public void process(@Element String e,
On Wed, Oct 23, 2019 at 1:21 AM Jan Lukavský wrote:
> Hi Reuven,
>
> yes, if this change is intended to be used by end users, then
> DoFnSignatures cannot be used, agree on that. Regarding the relationship
> with dynamic state - I agree that this is separate problem, but because it
> is close
+1 on this, having the ability to create timers based on data would make a
bunch of use cases easier to write.
Any thoughts on having a isSet() / read() / setMinimum(timeStamp) type
ability?
On Wed, 23 Oct 2019 at 00:52, Reuven Lax wrote:
> Kenn:
> +1 to using TimerFamily instead of TimerId
Kenn:
+1 to using TimerFamily instead of TimerId and TimerMap.
Jan:
This is definitely not just for DSLs. I've definitely seen cases where the
user wants different timers based on input data, so they cannot be defined
statically. As a thought experiment: one stated goal of state + timers was
to
Hi Max,
wouldn't that be actually the same as
class MyDoFn extends DoFn {
@ProcessElement
public void process(
ProcessContext context) {
// "get" would register a new TimerSpec
Timer timer1 = context.getTimer("timer1");
Timer timer2 = context.getTimer("timer2");
The idea makes sense to me. I really like that Beam gives upfront specs
for timer and state, but it is not flexible enough for timer-based
libraries or for users which want to dynamically generate timers.
I'm not sure about the proposed API yet. Shouldn't we separate the timer
specs from
Hi Kenn,
On 10/22/19 2:48 AM, Kenneth Knowles wrote:
This seems extremely useful.
I assume you mean `@OnTimer("timers")` in your example. I would
suggest that the parameter annotation be something other than @TimerId
since that annotation is already used for a very similar but different
Hi Reuven,
first of all big +1 for this.
Next, couple of questions that arise. Do you target DSLs only, or do you
suppose that this would be used by end-users as well? If only DSLs would
be in concern, then I think:
a) it is not only about timers, but state has to be managed in the
same
This seems extremely useful.
I assume you mean `@OnTimer("timers")` in your example. I would suggest
that the parameter annotation be something other than @TimerId since that
annotation is already used for a very similar but different purpose; they
are close enough that it is tempting to pun
BEAM-6857 documents the need for dynamic timer support in the Beam API. I
wanted to make a proposal for what this API would look like, and how to
express it in the portability protos.
Background: Today Beam (especially BeamJava) requires a ParDo to statically
declare all timers it accesses at
13 matches
Mail list logo