I agree, the groupBy, in the batch API is misleading, since a
ds.groupBy().reduce() does not really build any groups, it is really a
ds.keyBy().reduceByKey(). In the streaming API we can still fix this, IMHO.

On Tue, 14 Jul 2015 at 10:56 Stephan Ewen <se...@apache.org> wrote:

> It is not a bit different than the batch API, because streaming semantics
> are a bit different ;-)
>
> One good thing is that we can make things better that were sub-optimal in
> the Batch API.
>
> On Tue, Jul 14, 2015 at 10:55 AM, Stephan Ewen <se...@apache.org> wrote:
>
> > keyBy() does not do any grouping. Grouping in streams in not defined
> > without windows.
> >
> > On Tue, Jul 14, 2015 at 10:48 AM, Gyula Fóra <gyula.f...@gmail.com>
> wrote:
> >
> >> If we only want to have either keyBy or groupBy, why not keep groupBy?
> >> That
> >> would be more consistent with the batch api.
> >> On Tue, Jul 14, 2015 at 10:35 AM Stephan Ewen <se...@apache.org> wrote:
> >>
> >> > Concerning your comments:
> >> >
> >> > 1) In the new design, there is no grouping without windowing. The
> >> > KeyedDataStream subsumes the grouping and key-ing for partitioned
> state.
> >> >
> >> >     The keyBy() + window() makes a parallel grouped window
> >> >     keyBy() alone allows access to partitioned state.
> >> >
> >> >     My thought was that this is simpler, because it needs not
> groupBy()
> >> and
> >> > keyBy(), but one construct to handle both cases.
> >> >
> >> > 2) The discretization is a rough thought and is nothing for the short
> >> term.
> >> > It totally needs more thoughts. I put it there to have it as a sketch
> >> for
> >> > how to evolve this.
> >> >
> >> >     The idea is of course to not have a single data set, but a series
> of
> >> > data set. In each discrete time slice, the data set can be treated
> like
> >> a
> >> > regular data set.
> >> >
> >> >     Let's kick off a separate design for the discretization. Joins are
> >> good
> >> > to talk about (data sets can be joined with data set), and I am sure
> >> there
> >> > are more questions coming up.
> >> >
> >> >
> >> > Does that make sense?
> >> >
> >> >
> >> >
> >> >
> >> >
> >> > On Tue, Jul 14, 2015 at 10:05 AM, Gyula Fóra <gyula.f...@gmail.com>
> >> wrote:
> >> >
> >> > > I think Marton has some good points here.
> >> > >
> >> > > 1) Is KeyedDataStream a better name if this is only a renaming?
> >> > >
> >> > > 2) the discretize semantics is unclear indeed. Are we operating on a
> >> > single
> >> > > or sequence of datasets? If the latter why not call it something
> else
> >> > > (dstream). How are joins and other binary operators defined for
> >> different
> >> > > discretizations etc.
> >> > > On Mon, Jul 13, 2015 at 7:37 PM Márton Balassi <mbala...@apache.org
> >
> >> > > wrote:
> >> > >
> >> > > > Generally I agree with the new design. Two concerns:
> >> > > >
> >> > > > 1) Does KeyedDataStream replace GroupedDataStream or is it the
> >> latter a
> >> > > > special case of the former?
> >> > > >
> >> > > > The KeyedDataStream as described in the design document is a bit
> >> > unclear
> >> > > > for me. It lists the following usages:
> >> > > >   a) It is the first step in building a window stream, on top of
> >> which
> >> > > the
> >> > > > grouped/windowed aggregation and reduce-style function can be
> >> applied
> >> > > >   b) It allows to use the "by-key" state of functions. Here, every
> >> > record
> >> > > > has access to a state that is scoped by its key. Key-scoped state
> >> can
> >> > be
> >> > > > automatically redistributed and repartitioned.
> >> > > >
> >> > > > The code snippet describes a use case where the computation and
> the
> >> > > access
> >> > > > of the state is used the way currently the GroupedDataStream
> should
> >> > > work. I
> >> > > > suppose this is the example for case b). Would case a) also window
> >> > > elements
> >> > > > by key? If yes, then this is practically a renaming and
> enhancement
> >> of
> >> > > the
> >> > > > GroupedDataStream functionality with keyed state. Then the
> >> > > > StreamExecutionEnvironment.createKeyedStream(Partitioner,
> >> > > > KeySelector)construction does not make much sense as the user only
> >> > > operates
> >> > > > within the scope of the keyselector and not the partitioner
> anyway.
> >> > > >
> >> > > > I personally think KeyedDataStream as a name does not necessarily
> >> > suggest
> >> > > > that the records are grouped by key, it only suggests partitioning
> >> by
> >> > > key -
> >> > > > at least for me. :)
> >> > > >
> >> > > > 2) The API for discretization is not convenient IMHO
> >> > > >
> >> > > > The discretization part declares that the output of
> >> > > DataStream.discretize()
> >> > > > is a sequence of DataSets. I love this approach, but then in the
> >> code
> >> > > > snippet the return value of this function is simply a DataSet and
> >> uses
> >> > it
> >> > > > as such. The take home message of that code is the following: this
> >> is
> >> > > > actually the way you would like to program on these sequence of
> >> > DataSets,
> >> > > > most probably you would like to do the same with each of them. If
> >> that
> >> > is
> >> > > > the case we should provide a nice utility for that. I think Spark
> >> > > > Streaming's DStream.foreachRDD() is fairly useful for this
> purpose.
> >> > > >
> >> > > > On Mon, Jul 13, 2015 at 6:30 PM, Gyula Fóra <gyula.f...@gmail.com
> >
> >> > > wrote:
> >> > > >
> >> > > > > +1
> >> > > > > On Mon, Jul 13, 2015 at 6:23 PM Stephan Ewen <se...@apache.org>
> >> > wrote:
> >> > > > >
> >> > > > > > If naming is the only concern, then we should go ahead,
> because
> >> we
> >> > > can
> >> > > > > > change names easily (before the release).
> >> > > > > >
> >> > > > > > In fact, I don't think it leaves a bad impression. Global
> >> windows
> >> > are
> >> > > > > > non-parallel windows. There are also parallel windows. Pick
> what
> >> > you
> >> > > > need
> >> > > > > > and what works.
> >> > > > > >
> >> > > > > >
> >> > > > > > On Mon, Jul 13, 2015 at 6:13 PM, Gyula Fóra <
> >> gyula.f...@gmail.com>
> >> > > > > wrote:
> >> > > > > >
> >> > > > > > > I think we agree on everything its more of a naming issue :)
> >> > > > > > >
> >> > > > > > > I thought it might be misleading that global time windows
> are
> >> > > > > > > "non-parallel" windows. We dont want to give a bad
> impression.
> >> > > (Also
> >> > > > we
> >> > > > > > > dont want them to think that every global window is parallel
> >> but
> >> > > > thats
> >> > > > > > not
> >> > > > > > > a problem here)
> >> > > > > > >
> >> > > > > > > Gyula
> >> > > > > > > On Mon, Jul 13, 2015 at 5:22 PM Stephan Ewen <
> >> se...@apache.org>
> >> > > > wrote:
> >> > > > > > >
> >> > > > > > > > Okay, what is missing about the windowing in your opinion?
> >> > > > > > > >
> >> > > > > > > > The core points of the document are:
> >> > > > > > > >
> >> > > > > > > >   - The parallel windows are per group only.
> >> > > > > > > >
> >> > > > > > > >   - The implementation of the parallel windows holds
> window
> >> > data
> >> > > in
> >> > > > > the
> >> > > > > > > > group buffers.
> >> > > > > > > >
> >> > > > > > > >   - The global windows are non-parallel. May have parallel
> >> > > > > > > pre-aggregation,
> >> > > > > > > > if they are time windows.
> >> > > > > > > >
> >> > > > > > > >   - Time may be operator time (timer thread), or watermark
> >> > time.
> >> > > > > > > Watermark
> >> > > > > > > > time can refer to ingress or event time.
> >> > > > > > > >
> >> > > > > > > >   - Windows that do not pre-aggregate may require elements
> >> in
> >> > > > order.
> >> > > > > > Not
> >> > > > > > > > part of the first prototype.
> >> > > > > > > >
> >> > > > > > > > Do we agree on those points?
> >> > > > > > > >
> >> > > > > > > >
> >> > > > > > > > On Mon, Jul 13, 2015 at 4:50 PM, Gyula Fóra <
> >> > > gyula.f...@gmail.com>
> >> > > > > > > wrote:
> >> > > > > > > >
> >> > > > > > > > > In general I like it, although the main difference
> between
> >> > the
> >> > > > > > current
> >> > > > > > > > and
> >> > > > > > > > > the new one is the windowing and that is still not very
> >> > clear.
> >> > > > > > > > >
> >> > > > > > > > > Where do we have the full stream time windows for
> >> > > instance?(which
> >> > > > > is
> >> > > > > > > > > parallel but not keyed)
> >> > > > > > > > > On Mon, Jul 13, 2015 at 4:28 PM Aljoscha Krettek <
> >> > > > > > aljos...@apache.org>
> >> > > > > > > > > wrote:
> >> > > > > > > > >
> >> > > > > > > > > > +1 I like it as well.
> >> > > > > > > > > >
> >> > > > > > > > > > On Mon, 13 Jul 2015 at 16:17 Kostas Tzoumas <
> >> > > > ktzou...@apache.org
> >> > > > > >
> >> > > > > > > > wrote:
> >> > > > > > > > > >
> >> > > > > > > > > > > +1 from my side
> >> > > > > > > > > > >
> >> > > > > > > > > > > On Mon, Jul 13, 2015 at 4:15 PM, Stephan Ewen <
> >> > > > > se...@apache.org>
> >> > > > > > > > > wrote:
> >> > > > > > > > > > >
> >> > > > > > > > > > > > Do we have consensus on these designs?
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > If we have, we should get to implementing this
> soon,
> >> > > > because
> >> > > > > > > > > basically
> >> > > > > > > > > > > all
> >> > > > > > > > > > > > streaming patches will have to be revisited in
> >> light of
> >> > > > > this...
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > On Tue, Jul 7, 2015 at 3:41 PM, Gyula Fóra <
> >> > > > > > gyula.f...@gmail.com
> >> > > > > > > >
> >> > > > > > > > > > wrote:
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > > You are right thats an important issue.
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > And I think we should also do some renaming with
> >> the
> >> > > > > > > "iterations"
> >> > > > > > > > > > > because
> >> > > > > > > > > > > > > they are not really iterations like in the batch
> >> case
> >> > > and
> >> > > > > it
> >> > > > > > > > might
> >> > > > > > > > > > > > confuse
> >> > > > > > > > > > > > > some users.
> >> > > > > > > > > > > > > Maybe we can call them loops or cycles and
> rename
> >> the
> >> > > api
> >> > > > > > calls
> >> > > > > > > > to
> >> > > > > > > > > > make
> >> > > > > > > > > > > > it
> >> > > > > > > > > > > > > more intuitive what happens. It is really just a
> >> > cyclic
> >> > > > > > > dataflow.
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > Aljoscha Krettek <aljos...@apache.org> ezt írta
> >> > > > (időpont:
> >> > > > > > > 2015.
> >> > > > > > > > > júl.
> >> > > > > > > > > > > 7.,
> >> > > > > > > > > > > > > K,
> >> > > > > > > > > > > > > 15:35):
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > > Hi,
> >> > > > > > > > > > > > > > I just noticed that we don't have anything
> about
> >> > how
> >> > > > > > > iterations
> >> > > > > > > > > and
> >> > > > > > > > > > > > > > timestamps/watermarks should interact.
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > Cheers,
> >> > > > > > > > > > > > > > Aljoscha
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > On Mon, 6 Jul 2015 at 23:56 Stephan Ewen <
> >> > > > > se...@apache.org
> >> > > > > > >
> >> > > > > > > > > wrote:
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > Hi all!
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > As many of you know, there are a ongoing
> >> efforts
> >> > to
> >> > > > > > > > consolidate
> >> > > > > > > > > > the
> >> > > > > > > > > > > > > > > streaming API for the next release, and then
> >> > > graduate
> >> > > > > it
> >> > > > > > > > (from
> >> > > > > > > > > > beta
> >> > > > > > > > > > > > > > > status).
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > In the process of this consolidation, we
> want
> >> to
> >> > > > > achieve
> >> > > > > > > the
> >> > > > > > > > > > > > following
> >> > > > > > > > > > > > > > > goals.
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > >  - Make the code more robust and simplify it
> >> in
> >> > > parts
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > >  - Clearly define the semantics of the
> >> > constructs.
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > >  - Prepare it for support of more advanced
> >> > > concepts,
> >> > > > > like
> >> > > > > > > > > > > > partitionable
> >> > > > > > > > > > > > > > > state, and event time.
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > >  - Cut support for certain corner cases that
> >> were
> >> > > > > > > prototyped,
> >> > > > > > > > > but
> >> > > > > > > > > > > > > turned
> >> > > > > > > > > > > > > > > out to be not efficiently doable
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > Based on prior discussions on the mailing
> >> list,
> >> > > > > Aljoscha
> >> > > > > > > and
> >> > > > > > > > me
> >> > > > > > > > > > > > drafted
> >> > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > design documents below, which outline how
> the
> >> > > > > > consolidated
> >> > > > > > > > API
> >> > > > > > > > > > > would
> >> > > > > > > > > > > > > > like.
> >> > > > > > > > > > > > > > > We focused in constructs, time, and window
> >> > > semantics.
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > Design document on how to restructure the
> >> > Streaming
> >> > > > > API:
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > >
> >> > > > > > > > > > >
> >> > > > > > > > > >
> >> > > > > > > > >
> >> > > > > > > >
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> https://cwiki.apache.org/confluence/display/FLINK/Streams+and+Operations+on+Streams
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > Design document on definitions of time,
> order,
> >> > and
> >> > > > the
> >> > > > > > > > > resulting
> >> > > > > > > > > > > > > > semantics:
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > >
> >> > > > > > > > > > >
> >> > > > > > > > > >
> >> > > > > > > > >
> >> > > > > > > >
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> https://cwiki.apache.org/confluence/display/FLINK/Time+and+Order+in+Streams
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > Note: The design of the interfaces and
> >> concepts
> >> > for
> >> > > > > > > advanced
> >> > > > > > > > > > state
> >> > > > > > > > > > > in
> >> > > > > > > > > > > > > > > functions is not in here. That is part of a
> >> > > separate
> >> > > > > > design
> >> > > > > > > > > > > > discussion
> >> > > > > > > > > > > > > > and
> >> > > > > > > > > > > > > > > orthogonal to the designs drafted here.
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > Please have a look and voice questions and
> >> > > concerns.
> >> > > > > > Since
> >> > > > > > > we
> >> > > > > > > > > > > should
> >> > > > > > > > > > > > > not
> >> > > > > > > > > > > > > > > break the streaming API more than once, we
> >> should
> >> > > > make
> >> > > > > > sure
> >> > > > > > > > > this
> >> > > > > > > > > > > > > > > consolidation brings it into the shape we
> >> want it
> >> > > to
> >> > > > be
> >> > > > > > in.
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > Greetings,
> >> > > > > > > > > > > > > > > Stephan
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > >
> >> > > > > > > > > > >
> >> > > > > > > > > >
> >> > > > > > > > >
> >> > > > > > > >
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >
> >
>

Reply via email to