If there are no extra comments, I will start voting in three days, thank
you~

Best,
Rui

On Thu, Mar 28, 2024 at 4:46 PM Muhammet Orazov
<mor+fl...@morazow.com.invalid> wrote:

> Hey Rui,
>
> Thanks for the detailed explanation and updating the FLIP!
>
> It is much clearer definitely, thanks for the proposal.
>
> Best,
> Muhammet
>
> On 2024-03-28 07:37, Rui Fan wrote:
> > Hi Muhammet,
> >
> > Thanks for your reply!
> >
> >> The execution mode is also used for the DataStream API [1],
> >> would that also affect/hide the DataStream execution mode
> >> if we remove it from the WebUI?
> >
> > Sorry, I didn't describe it clearly in FLIP-441[2], I have updated it.
> > Let me clarify the Execution Mode here:
> >
> > 1. Flink 1.19 website[3] also mentions the Execution mode, but it
> > actually matches the JobType[4] in the Flink code. Both of them
> > have 2 types: STREAMING and BATCH.
> > 2. execution.runtime-mode can be set to 3 types: STREAMING,
> > BATCH and AUTOMATIC. But the jobType will be inferred as
> > STREAMING or BATCH when execution.runtime-mode is set
> > to AUTOMATIC.
> > 3. The ExecutionMode I describe is: code link[5] , as we can
> > see, ExecutionMode has 4 enums: PIPELINED,
> > PIPELINED_FORCED, BATCH and BATCH_FORCED.
> > And we can see a flink streaming job from Flink WebUI,
> > the Execution mode is PIPELINE instead of STREAMING.
> > I attached a screenshot to the FLIP doc[2], you can see it there.
> > 4. What this proposal wants to do is to remove the ExecutionMode
> > with four enumerations on Flink WebUI and introduce the
> > JobType with two enumerations (STREAMING or BATCH).
> > STREAMING or BATCH is clearer and more accurate for users.
> >
> > Please let me know if it's not clear or anything is wrong, thanks a
> > lot!
> >
> > [1]
> >
> https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/dev/datastream/execution_mode/
> > [2] https://cwiki.apache.org/confluence/x/agrPEQ
> > [3]
> >
> https://nightlies.apache.org/flink/flink-docs-release-1.19/docs/dev/datastream/execution_mode/
> > [4]
> >
> https://github.com/apache/flink/blob/f31c128bfc457b64dd7734f71123b74faa2958ba/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobType.java#L22
> > [5]
> >
> https://github.com/apache/flink/blob/f31c128bfc457b64dd7734f71123b74faa2958ba/flink-core/src/main/java/org/apache/flink/api/common/ExecutionMode.java#L54
> >
> > Best,
> > Rui
> >
> > On Thu, Mar 28, 2024 at 1:33 AM Venkatakrishnan Sowrirajan
> > <vsowr...@asu.edu>
> > wrote:
> >
> >> Rui,
> >>
> >> I assume the current proposal would also handle the case of mixed mode
> >> (BATCH + STREAMING within the same app) in the future, right?
> >>
> >> Regards
> >> Venkat
> >>
> >> On Wed, Mar 27, 2024 at 10:15 AM Venkatakrishnan Sowrirajan <
> >> vsowr...@asu.edu> wrote:
> >>
> >>> This will be a very useful addition to Flink UI. Thanks Rui for
> >>> starting
> >>> a FLIP for this improvement.
> >>>
> >>> Regards
> >>> Venkata krishnan
> >>>
> >>>
> >>> On Wed, Mar 27, 2024 at 4:49 AM Muhammet Orazov
> >>> <mor+fl...@morazow.com.invalid> wrote:
> >>>
> >>>> Hello Rui,
> >>>>
> >>>> Thanks for the proposal! It looks good!
> >>>>
> >>>> I have minor clarification from my side:
> >>>>
> >>>> The execution mode is also used for the DataStream API [1],
> >>>> would that also affect/hide the DataStream execution mode
> >>>> if we remove it from the WebUI?
> >>>>
> >>>> Best,
> >>>> Muhammet
> >>>>
> >>>> [1]:
> >>>>
> >>>>
> https://urldefense.com/v3/__https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/dev/datastream/execution_mode/__;!!IKRxdwAv5BmarQ!eFyqVJyje_8hu1vMSUwKGBsj8vqsFDisEvJ5AxPV0LduhhHWF3rPKYEEE-09biA0unFbfMy5AVQZMgBv1AOa5oTHmcYlkUE$
> >>>>
> >>>>
> >>>> On 2024-03-27 06:23, Rui Fan wrote:
> >>>> > Hi flink developers,
> >>>> >
> >>>> > I'd like to start a discussion to discuss FLIP-441:
> >>>> > Show the JobType and remove Execution Mode on Flink WebUI[1].
> >>>> >
> >>>> > Currently, the jobType has 2 types in Flink: STREAMING and BATCH.
> >>>> > They work on completely different principles, such as: scheduler,
> >>>> > shuffle, join, etc. These differences lead to different
> troubleshooting
> >>>> > processes, so when users are maintaining a job or troubleshooting,
> >>>> > it's needed to know whether the current job is a STREAMING or
> >>>> > BATCH job. Unfortunately, Flink WebUI doesn't expose it to the
> >>>> > users so far.
> >>>> >
> >>>> > Also, Execution Mode is related to DataSet api, it has been marked
> >>>> > as @Deprecated in FLINK-32258 (1.18), but it's still shown in Flink
> >>>> > WebUI.
> >>>> >
> >>>> > Looking forward to hearing more thoughts about it! Thank you~
> >>>> >
> >>>> > [1]
> >>>>
> https://urldefense.com/v3/__https://cwiki.apache.org/confluence/x/agrPEQ__;!!IKRxdwAv5BmarQ!eFyqVJyje_8hu1vMSUwKGBsj8vqsFDisEvJ5AxPV0LduhhHWF3rPKYEEE-09biA0unFbfMy5AVQZMgBv1AOa5oTHayPyFj8$
> >>>> > [2]
> >>>>
> https://urldefense.com/v3/__https://issues.apache.org/jira/browse/FLINK-32558__;!!IKRxdwAv5BmarQ!eFyqVJyje_8hu1vMSUwKGBsj8vqsFDisEvJ5AxPV0LduhhHWF3rPKYEEE-09biA0unFbfMy5AVQZMgBv1AOa5oTHftYeOLE$
> >>>> >
> >>>> > Best,
> >>>> > Rui
> >>>>
> >>>
>

Reply via email to