Why side-outputs are only supported by Process functions?

classic Classic list List threaded Threaded
2 messages Options
Reply | Threaded
Open this post in threaded view
|

Why side-outputs are only supported by Process functions?

ivneet kaur
Hi folks,
I want to split my stream for some invalid message handling, and need help understanding a few things.
Question 1: Why is split operator deprecated?
Question 2: Why side-outputs are only supported for ProcessFunction, KeyedProcessFunction etc.

The doc on side-outputs says: "You can use the Context parameter, which is exposed to users in the above functions, to emit data to a side output identified by an OutputTag",
my question really is why is Context parameter only available to these functions? 

My understanding is that the process functions are meant to allow dealing with timers, timestamps and watermarks. Is there an inherent connection between side outputs and timers that I am missing? I don't think I need any such handling.

For use instead of a RichAsyncFunction etc., there does not seem to exist a dedicated async flavor of process functions.

Side-outputs from RichAsyncFunction thread suggests using Either<X, CustomError>. For Java developers, there is no standard way of achieving this, any other suggestions?

Best,
Ivneet


Reply | Threaded
Open this post in threaded view
|

Re: Why side-outputs are only supported by Process functions?

Arvid Heise-3
Hi Ivneet,

Q1) you can read about the deprecation of split in FLINK-11084 [1]. In general side-outputs subsume the functionality and allow some advanced cases (like emitting the same record into two outputs).

Q2) It's simply a matter of API design. The basic idea is to keep most interfaces as sleek as possible (MapFunction) to not overload new users completely.

Now for your actual problem. You can use the same receipt as on stackoverflow, but use Tuple2 instead of Either.


On Mon, Jun 22, 2020 at 3:21 AM ivneet kaur <[hidden email]> wrote:
Hi folks,
I want to split my stream for some invalid message handling, and need help understanding a few things.
Question 1: Why is split operator deprecated?
Question 2: Why side-outputs are only supported for ProcessFunction, KeyedProcessFunction etc.

The doc on side-outputs says: "You can use the Context parameter, which is exposed to users in the above functions, to emit data to a side output identified by an OutputTag",
my question really is why is Context parameter only available to these functions? 

My understanding is that the process functions are meant to allow dealing with timers, timestamps and watermarks. Is there an inherent connection between side outputs and timers that I am missing? I don't think I need any such handling.

For use instead of a RichAsyncFunction etc., there does not seem to exist a dedicated async flavor of process functions.

Side-outputs from RichAsyncFunction thread suggests using Either<X, CustomError>. For Java developers, there is no standard way of achieving this, any other suggestions?

Best,
Ivneet




--

Arvid Heise | Senior Java Developer


Follow us @VervericaData

--

Join Flink Forward - The Apache Flink Conference

Stream Processing | Event Driven | Real Time

--

Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany

--

Ververica GmbH
Registered at Amtsgericht Charlottenburg: HRB 158244 B
Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji (Toni) Cheng