+1
On Sat, Mar 4, 2017 at 12:25 AM, Kostas Kloudas wrote:
> +1
>
> > On Mar 2, 2017, at 1:08 PM, Fabian Hueske wrote:
> >
> > +1
> >
> > 2017-03-02 12:11 GMT+01:00 Aljoscha Krettek :
> >
> >> Ok, so it seems we have to go with the OutputTag variant for windows as
> >> well, for now.
> >>
> >> F
+1
> On Mar 2, 2017, at 1:08 PM, Fabian Hueske wrote:
>
> +1
>
> 2017-03-02 12:11 GMT+01:00 Aljoscha Krettek :
>
>> 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,
+1
2017-03-02 12:11 GMT+01:00 Aljoscha Krettek :
> 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
> wrote:
>
> > Flink enforces bina
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 wrote:
> Flink enforces binary compatibility for all classes tagged with the @Public
> annotat
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
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, Fabi
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 t
On Tue, Feb 28, 2017 at 11:38 AM, Aljoscha Krettek 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
About 1: We can definitely go with Jamie's proposal for the late data side
output, for me this is just a name and anything that has "late" in it is
perfect!
Regarding 2: I agree, and I though about implementing split/select on top
of side outputs and it should be easily doable. I think side output
1. I like the variant without the explicit OutputTag for the WindowOperator:
WindowedOperator windowedResult = input
.keyBy(...)
.window(...)
.apply(...)
DataStream lateData = windowedResult.getLateDataSideOutput();
I like Jamie's proposal getLateStream() a little better though. On the
oth
Aljoscha,
Ahh, that is much better. As long as it's explicitly referring to late
data I think it's fine. I also like the second variant where a user
doesn't have to explicitly create the OutputTag.
On Mon, Feb 27, 2017 at 8:45 AM, Aljoscha Krettek
wrote:
> @Jamie I must have mistyped my las
I like this proposal!
- The side output API seems more powerful, because it allows different
output types.
- It would be nice to eventually have only one construct, because
multiple variants for the same thing tend to confuse users.
- One can probably implement split/select with side output
I'm curious to know what people think about the OutputTag API for the
general side-output implementation?
One thing that might easily go overlooked is that I changed ProcessFunction
from an interface to an abstract class. So that I could provide a default
onTime() method. This also would require a
@Jamie I must have mistyped my last API proposal. This piece of code:
WindowedOperator windowedResult = input
.keyBy(...)
.window(...)
.apply(...)
DataStream lateData = windowedResult.getSideOutput();
should actually have been:
WindowedOperator windowedResult = input
.keyBy(...)
.windo
Hi Jamie,
I think it does make consuming late arriving events more explicit! At cost of
fix a predefined OutputTag which user have no control nor definition
an extra UDF which essentially filter out all mainOutputs and only let
sideOutput pass (like filterFunction)
Thanks,
Chen
> On Feb 24, 20
I prefer the ProcessFunction and side outputs solution over split() and
select() which I've never liked primarily due to the lack of type safety
and it also doesn't really seem to fit with the rest of Flink's API.
On the late data question I strongly prefer the late data concept being
explicit in
Hi,
Thanks for the nice proposal, I like the idea of side outputs, and it would
make a lot of topologies much simpler.
Regarding the API I think we should come up with a way of making side
otuputs accessible from all sort of operators in a similar way. For
instance through the RichFunction interf
Hi Folks,
Chen and I have been working for a while now on making FLIP-13 (side
outputs) [1] a reality. We think we have a pretty good internal
implementation and also a proposal for an API but now we need to discuss
how we want to go forward with this, especially how we should deal with
split/selec
18 matches
Mail list logo