Re: [PROPOSAL] "Requires deterministic input"

2017-12-11 Thread Reuven Lax
On Mon, Dec 11, 2017 at 8:47 PM, Kenneth Knowles wrote: > In discussion on https://github.com/apache/beam/pull/4135 and offline it > came up how this should interact with stateful/timely DoFns. > > Highlights: > > - It was suggested to support it for @OnTimer which would mean that the > state is

Re: [PROPOSAL] "Requires deterministic input"

2017-12-11 Thread Kenneth Knowles
In discussion on https://github.com/apache/beam/pull/4135 and offline it came up how this should interact with stateful/timely DoFns. Highlights: - It was suggested to support it for @OnTimer which would mean that the state is stable on retry - But an unfortunate corollary is that for stateful

Re: [PROPOSAL] "Requires deterministic input"

2017-11-15 Thread Aljoscha Krettek
+1 > On 15. Nov 2017, at 14:07, Jean-Baptiste Onofré wrote: > > Agree ! > > Thanks Kenn, > Regards > JB > > On 11/15/2017 02:05 PM, Kenneth Knowles wrote: >> Reviving this again, since it came up again today in yet another context. I >> think it is time to add this as an experimental annotatio

Re: [PROPOSAL] "Requires deterministic input"

2017-11-15 Thread Jean-Baptiste Onofré
Agree ! Thanks Kenn, Regards JB On 11/15/2017 02:05 PM, Kenneth Knowles wrote: Reviving this again, since it came up again today in yet another context. I think it is time to add this as an experimental annotation. I think we know that we need it, and roughly how it should work, while there are

Re: [PROPOSAL] "Requires deterministic input"

2017-11-15 Thread Kenneth Knowles
Reviving this again, since it came up again today in yet another context. I think it is time to add this as an experimental annotation. I think we know that we need it, and roughly how it should work, while there are still finer points to discuss about what it means for input to be stable. So I fi

Re: [PROPOSAL] "Requires deterministic input"

2017-08-15 Thread Reuven Lax
Well the Fn API is still being designed, so this is something we'd have to think about. On Tue, Aug 15, 2017 at 2:19 PM, Robert Bradshaw < rober...@google.com.invalid> wrote: > On Tue, Aug 15, 2017 at 2:14 PM, Reuven Lax > wrote: > > On Tue, Aug 15, 2017 at 1:59 PM, Robert Bradshaw < > > rober..

Re: [PROPOSAL] "Requires deterministic input"

2017-08-15 Thread Robert Bradshaw
On Tue, Aug 15, 2017 at 2:14 PM, Reuven Lax wrote: > On Tue, Aug 15, 2017 at 1:59 PM, Robert Bradshaw < > rober...@google.com.invalid> wrote: > >> On Sat, Aug 12, 2017 at 1:13 AM, Reuven Lax >> wrote: >> > On Fri, Aug 11, 2017 at 10:52 PM, Robert Bradshaw < >> >> The question here is whether the

Re: [PROPOSAL] "Requires deterministic input"

2017-08-15 Thread Reuven Lax
On Tue, Aug 15, 2017 at 1:59 PM, Robert Bradshaw < rober...@google.com.invalid> wrote: > On Sat, Aug 12, 2017 at 1:13 AM, Reuven Lax > wrote: > > On Fri, Aug 11, 2017 at 10:52 PM, Robert Bradshaw < > >> The question here is whether the ordering is part of the "content" of > >> an iterable. > > >

Re: [PROPOSAL] "Requires deterministic input"

2017-08-15 Thread Robert Bradshaw
On Sat, Aug 12, 2017 at 1:13 AM, Reuven Lax wrote: > On Fri, Aug 11, 2017 at 10:52 PM, Robert Bradshaw < >> The question here is whether the ordering is part of the "content" of >> an iterable. > > My initial instinct was to say yes - but maybe it should not be until Beam > has a first-class notio

Re: [PROPOSAL] "Requires deterministic input"

2017-08-12 Thread Reuven Lax
On Fri, Aug 11, 2017 at 10:52 PM, Robert Bradshaw < rober...@google.com.invalid> wrote: > On Thu, Aug 10, 2017 at 1:53 PM, Reuven Lax > wrote: > > On Thu, Aug 10, 2017 at 1:07 PM, Kenneth Knowles > > > wrote: > > > >> > > >- Does it also imply fixed length and content for value > >> iterato

Re: [PROPOSAL] "Requires deterministic input"

2017-08-11 Thread Robert Bradshaw
On Thu, Aug 10, 2017 at 1:53 PM, Reuven Lax wrote: > On Thu, Aug 10, 2017 at 1:07 PM, Kenneth Knowles > wrote: > >> > > >- Does it also imply fixed length and content for value >> iterators? >> > > > >> >> The concept of "value iterator" brings up a nit. >> >> First, there is no such concept

Re: [PROPOSAL] "Requires deterministic input"

2017-08-10 Thread Reuven Lax
On Thu, Aug 10, 2017 at 1:07 PM, Kenneth Knowles wrote: > > > >- Does it also imply fixed length and content for value > iterators? > > > > > > The concept of "value iterator" brings up a nit. > > First, there is no such concept in the Beam model, and I don't think there > should be. I don't

Re: [PROPOSAL] "Requires deterministic input"

2017-08-10 Thread Kenneth Knowles
> > >- Does it also imply fixed length and content for value iterators? > > > The concept of "value iterator" brings up a nit. First, there is no such concept in the Beam model, and I don't think there should be. I don't think we should special case GBK if we can avoid it. If a PCollection c

Re: [PROPOSAL] "Requires deterministic input"

2017-08-10 Thread Reuven Lax
On Thu, Aug 10, 2017 at 11:18 AM, Thomas Groh wrote: > I think it must imply fixed content >s - making a decision based > on the contents of an iterable assuming the Iterable is deterministic seems > an acceptable use of the API, and that requires the contents to be > identical through failures.

Re: [PROPOSAL] "Requires deterministic input"

2017-08-10 Thread Thomas Groh
I think it must imply fixed content >s - making a decision based on the contents of an iterable assuming the Iterable is deterministic seems an acceptable use of the API, and that requires the contents to be identical through failures. This does imply that (assuming this is reading directly from th

Re: [PROPOSAL] "Requires deterministic input"

2017-08-10 Thread Reuven Lax
It means that single element replay is stable. On Thu, Aug 10, 2017 at 10:56 AM, Raghu Angadi wrote: > Can we define what exactly is meant by deterministic/stable/replayable > etc? > >- Does it imply a fixed order? If yes, it implies fixed order of >processElement() invocations, right? A

Re: [PROPOSAL] "Requires deterministic input"

2017-08-10 Thread Raghu Angadi
Can we define what exactly is meant by deterministic/stable/replayable etc? - Does it imply a fixed order? If yes, it implies fixed order of processElement() invocations, right? Are there any qualifiers (within a window+key etc)? - Does it also imply fixed length and content for value

Re: [PROPOSAL] "Requires deterministic input"

2017-08-10 Thread Ben Chambers
I think it only makes sense in places where a user might reasonable require stable input to ensure idempotency of side-effects. It also only makes sense in places where a runner could reasonably provide such a guarantee. A given Combine is unlikely to have side effects so it is less likely to bene

Re: [PROPOSAL] "Requires deterministic input"

2017-08-10 Thread Reuven Lax
I don't think it really makes sense to to do this on Combine. And I agree with you, it doesn't make sense on composites either. On Thu, Aug 10, 2017 at 9:19 AM, Scott Wegner wrote: > Does requires-stable-input only apply to ParDo transforms? > > I don't think it would make sense to annotate to c

Re: [PROPOSAL] "Requires deterministic input"

2017-08-10 Thread Scott Wegner
Does requires-stable-input only apply to ParDo transforms? I don't think it would make sense to annotate to composite, because checkpointing should happen as close to the side-effecting operation as possible, since upstream transforms within a composite could introduce non-determinism. So it's the

Re: [PROPOSAL] "Requires deterministic input"

2017-08-10 Thread Tyler Akidau
+1 to the annotation idea, and to having it on processTimer. -Tyler On Thu, Aug 10, 2017 at 2:15 AM Aljoscha Krettek wrote: > +1 to the annotation approach. I outlined how implementing this would work > in the Flink runner in the Thread about the exactly-once Kafka Sink. > > > On 9. Aug 2017, a

Re: [PROPOSAL] "Requires deterministic input"

2017-08-10 Thread Aljoscha Krettek
+1 to the annotation approach. I outlined how implementing this would work in the Flink runner in the Thread about the exactly-once Kafka Sink. > On 9. Aug 2017, at 23:03, Reuven Lax wrote: > > Yes - I don't think we should try and make any deterministic guarantees > about what is in a bundle.

Re: [PROPOSAL] "Requires deterministic input"

2017-08-09 Thread Reuven Lax
Yes - I don't think we should try and make any deterministic guarantees about what is in a bundle. Stability guarantees are per element only. On Wed, Aug 9, 2017 at 1:30 PM, Thomas Groh wrote: > +1 to the annotation-on-ProcessElement approach. ProcessElement is the > minimum implementation requi

Re: [PROPOSAL] "Requires deterministic input"

2017-08-09 Thread Thomas Groh
As I said, a minor concern; we should be explicit in our documentation that it is only the input _elements_ that are deterministic/stable/replayable/etc, and not operational concerns surrounding them (such as bundling). I'd generally avoid making the actual annotation more verbose. On Wed, Aug 9,

Re: [PROPOSAL] "Requires deterministic input"

2017-08-09 Thread Kenneth Knowles
On Wed, Aug 9, 2017 at 1:30 PM, Thomas Groh wrote: > I have a minor concern that this may not work as expected for users that > try to batch remote calls in `FinishBundle` - we should make sure we > document that it is explicitly the input elements that will be replayed, > and bundles and other o

Re: [PROPOSAL] "Requires deterministic input"

2017-08-09 Thread Thomas Groh
+1 to the annotation-on-ProcessElement approach. ProcessElement is the minimum implementation requirement of a DoFn, and should be where the processing logic which depends on characteristics of the inputs lie. It's a good way of signalling the requirements of the Fn, and letting the runner decide.

Re: [PROPOSAL] "Requires deterministic input"

2017-08-09 Thread Reuven Lax
I think deterministic here means deterministically replayable. i.e. no matter how many times the element is retried, it will always be the same. I think we should also allow specifying this on processTimer. This would mean that any keyed state written in a previous processElement must be guarantee

Re: [PROPOSAL] "Requires deterministic input"

2017-08-09 Thread Kenneth Knowles
I like "Stable" too. I can try to make up other scenarios to try out different vocabulary. Here are a couple: - redundant processing to mitigate stragglers - duplication in the course of optimizations* This expands the scope of the feature to be not just agreement on the PCollection contents b

Re: [PROPOSAL] "Requires deterministic input"

2017-08-09 Thread Ben Chambers
I strongly agree with this proposal. I think moving away from "just insert a GroupByKey for one of the 3 different reasons you may want it" towards APIs that allow code to express the requirements they have and the runner to choose the best way to meet this is a major step forwards in terms of port

Re: [PROPOSAL] "Requires deterministic input"

2017-08-09 Thread Kenneth Knowles
This came up again, so I wanted to push it along by proposing a specific API for Java that could have a derived API in Python. I am writing this quickly to get something out there, so I welcome suggestions for revision. Today a DoFn has a @ProcessElement annotated method with various automated par

Re: [PROPOSAL] "Requires deterministic input"

2017-03-21 Thread Ben Chambers
Allowing an annotation on DoFn's that produce deterministic output could be added in the future, but doesn't seem like a great option. 1. It is a correctness issue to assume a DoFn is deterministic and be wrong, so we would need to assume all transform outputs are non-deterministic unless annotate

Re: [PROPOSAL] "Requires deterministic input"

2017-03-21 Thread Kenneth Knowles
Good points & questions. I'll try to be more clear. > On 21 March 2017 at 13:52, Stephen Sisk wrote: > > > Hey Kenn- > > > > this seems important, but I don't have all the context on what the > problem > > is. > > > > Can you explain this sentence "Specifically, there is pseudorandom data > > g

Re: [PROPOSAL] "Requires deterministic input"

2017-03-21 Thread vikas rk
+1 for the general idea of runners handling it over hard-coded implementation strategy. For the Write transform I believe you are talking about ApplyShardingKey

Re: [PROPOSAL] "Requires deterministic input"

2017-03-21 Thread Stephen Sisk
Hey Kenn- this seems important, but I don't have all the context on what the problem is. Can you explain this sentence "Specifically, there is pseudorandom data generated and once it has been observed and used to produce a side effect, it cannot be regenerated without erroneous results." ? Where