Ok, so it seems we have to go with the OutputTag variant for windows as
well, for now.

For Flink 2.0 we can change that. Would everyone be OK with that?

On Thu, Mar 2, 2017 at 12:05 PM, Robert Metzger <rmetz...@apache.org> wrote:

> Flink enforces binary compatibility for all classes tagged with the @Public
> annotation.
> Binary compatibility allows users to execute a job against a newer Flink
> version without recompiling their job jar.
> Your change alters the return type of some methods (apply()). I think
> there's no way to do that in a binary compatible way.
>
> The only thing we could do is keep the return type as is, but return a
> WindowedOperation instance.
> Users could then manually cast the returned object to access the late
> stream.
>
> Downgrading to "source compatibility" only should fix the issue, but then
> users have to recompile their Flink jobs when upgrading the Flink version.
>
> On Tue, Feb 28, 2017 at 9:37 PM, Fabian Hueske <fhue...@gmail.com> wrote:
>
> > Hi Chen and Aljoscha,
> >
> > thanks for the great proposal and work.
> >
> > I prefer the WindowedOperator.getLateStream() variant without explicit
> > tags.
> > I think it is fine to start adding side output to ProcessFunction (keyed
> > and non-keyed) and window operators and see how it is picked up by users.
> >
> > Best, Fabian
> >
> >
> > 2017-02-28 15:42 GMT+01:00 Aljoscha Krettek <aljos...@apache.org>:
> >
> > > Quick update: I created a branch where I make the result type of
> > > WindowedStream operations more specific:
> > > https://github.com/aljoscha/flink/blob/windowed-stream-
> > > result-specific/flink-streaming-java/src/main/java/
> > > org/apache/flink/streaming/api/datastream/WindowedStream.java
> > >
> > > We would need this for the "lateStream()" API without the explicit
> > > OutputTag.
> > >
> > > It seems the backwards compatibility checker doesn't like this and
> > > complains about breaking binary backwards compatibility. +Robert
> Metzger
> > > <rmetz...@apache.org> Do you have an idea what we could do there?
> > >
> > > On Tue, 28 Feb 2017 at 12:39 Ufuk Celebi <u...@apache.org> wrote:
> > >
> > > > On Tue, Feb 28, 2017 at 11:38 AM, Aljoscha Krettek <
> > aljos...@apache.org>
> > > > wrote:
> > > > > I see the ProcessFunction as a bit of the generalised future of
> > > FlatMap,
> > > > so
> > > > > to me it makes sense to only allow side outputs on the
> > ProcessFunction
> > > > but
> > > > > I'm open for anything. If we decide for this I'm happy with an
> > > additional
> > > > > method on Collector.
> > > >
> > > > I think it's best to restrict this to ProcessFunction after all
> (given
> > > > that we allow it for non-keyed streams, etc.). ;-)
> > > >
> > >
> >
>

Reply via email to