Hi, Jiangjie

Thanks for the clarification.

My key point is the meaning of the "submission" in
"client.attached.after.submission".
At first glance, I thought only job submissions were taken into account.
After your clarification, this option also works for cluster submissions.

It's fine for me.

Best,
Weihua


On Wed, Aug 23, 2023 at 8:35 AM Becket Qin <becket....@gmail.com> wrote:

> Hi Weihua,
>
> Thanks for the explanation. From the doc, it looks like the current
> behaviors of "execution.attached=true" between Yarn and K8S session
> cluster are exactly the opposite. For YARN it basically means the cluster
> will shutdown if the client disconnects. For K8S, it means the cluster will
> not shutdown until a client explicitly stops it. This sounds like a bad
> situation to me and needs to be fixed.
>
> My guess is that the YARN behavior here is the original intended behavior,
> while K8S reused the configuration for a different purpose. If we deprecate
> the execution.attached config here. The behavior would be:
>
> For YARN session clusters:
> 1. Current "execution.attached=true" would be equivalent to
> "execution.shutdown-on-attached-exit=true" +
> "client.attached.after.submission=true".
> 2. Current "execution.attached=false" would be equivalent to
> "execution.shutdown-on-attached-exit=false", i.e. the cluster will keep
> running until explicitly stopped.
>
> I am not sure what the current behavior of "execution.attached=true" +
> "execution.shutdown-on-attached-exit=false" is. Supposedly, it should be
> equivalent to "execution.shutdown-on-attached-exit=false", which means
> "execution.attached" only controls the client side behavior, while the
> cluster side behavior is controlled by
> "execution.shutdown-on-attached-exit".
>
> For K8S session clusters:
> 1. Current "execution.attached=true" would be equivalent to
> "execution.shutdown-on-attached-exit=false".
> 2. Current "execution.attached=false" would be equivalent to
> "execution.shutdown-on-attached-exit=true" +
> "client.attached.after.submission=true".
>
> This will make the same config behave the same for YARN and K8S.
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
> On Tue, Aug 22, 2023 at 11:04 PM Weihua Hu <huweihua....@gmail.com> wrote:
>
> > Hi, Jiangjie
> >
> > 'execution.attached' can be used to attach an existing cluster and stop
> it
> > [1][2],
> > which is not related to job submission. So does YARN session mode[3].
> > IMO, this behavior should not be controlled by the new option
> > 'client.attached.after.submission'.
> >
> > [1]
> >
> >
> https://nightlies.apache.org/flink/flink-docs-master/docs/deployment/resource-providers/native_kubernetes/#session-mode
> > [2]
> >
> >
> https://github.com/apache/flink/blob/a85ffc491874ecf3410f747df3ed09f61df52ac6/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/cli/KubernetesSessionCli.java#L126
> > [3]
> >
> >
> https://nightlies.apache.org/flink/flink-docs-master/docs/deployment/resource-providers/yarn/#session-mode
> >
> > Best,
> > Weihua
> >
> >
> > On Tue, Aug 22, 2023 at 5:16 PM Becket Qin <becket....@gmail.com> wrote:
> >
> > > Hi Weihua,
> > >
> > > Just want to clarify a little bit, what is the impact of
> > > `execution.attached` on a cluster startup before a client submits a job
> > to
> > > that cluster? Does this config only become effective after a job
> > > submission?
> > >
> > > Currently, the cluster behavior has an independent config of
> > > 'execution.shutdown-on-attached-exit'. So if a client submitted a job
> in
> > > attached mode, and this `execution.shutdown-on-attached-exit` is set to
> > > true, the cluster will shutdown if the client detaches from the
> cluster.
> > Is
> > > this sufficient? Or do you mean we need another independent
> > configuration?
> > >
> > > Thanks,
> > >
> > > Jiangjie (Becket) Qin
> > >
> > > On Tue, Aug 22, 2023 at 2:20 PM Weihua Hu <huweihua....@gmail.com>
> > wrote:
> > >
> > > > Hi Jiangjie
> > > >
> > > > Sorry for the late reply, I fully agree with the three user sensible
> > > > behaviors you described.
> > > >
> > > > I would like to bring up a point.
> > > >
> > > > Currently, 'execution.attached' is not only used for submitting jobs,
> > > > But also for starting a new cluster (YARN and Kubernetes). If it's
> > true,
> > > > the starting cluster script will
> > > > wait for the user to input the next command (quit or stop).
> > > >
> > > > In my opinion, this behavior should have an independent option
> besides
> > > > "client.attached.after.submission" for control.
> > > >
> > > >
> > > > Best,
> > > > Weihua
> > > >
> > > >
> > > > On Thu, Aug 17, 2023 at 10:07 AM liu ron <ron9....@gmail.com> wrote:
> > > >
> > > > > Hi, Jiangjie
> > > > >
> > > > > Thanks for your detailed explanation, I got your point. If the
> > > > > execution.attached is only used for client currently, removing it
> > also
> > > > make
> > > > > sense to me.
> > > > >
> > > > > Best,
> > > > > Ron
> > > > >
> > > > > Becket Qin <becket....@gmail.com> 于2023年8月17日周四 07:37写道:
> > > > >
> > > > > > Hi Ron,
> > > > > >
> > > > > > Isn't the cluster (session or per job) only using the
> > > > execution.attached
> > > > > to
> > > > > > determine whether the client is attached? If so, the client can
> > > always
> > > > > > include the information of whether it's an attached client or not
> > in
> > > > the
> > > > > > JobSubmissoinRequestBody, right? For a shared session cluster,
> > there
> > > > > could
> > > > > > be multiple clients submitting jobs to it. These clients may or
> may
> > > not
> > > > > be
> > > > > > attached. A static execution.attached configuration for the
> session
> > > > > cluster
> > > > > > does not work in this case, right?
> > > > > >
> > > > > > The current problem of execution.attached is that it is not
> always
> > > > > honored.
> > > > > > For example, if a session cluster was started with
> > execution.attached
> > > > set
> > > > > > to false. And a client submits a job later to that session
> cluster
> > > with
> > > > > > execution.attached set to true. In this case, the cluster won't
> > (and
> > > > > > shouldn't) shutdown after the job finishes or the attached client
> > > loses
> > > > > > connection. So, in fact, the execution.attached configuration is
> > only
> > > > > > honored by the client, but not the cluster. Therefore, I think
> > > removing
> > > > > it
> > > > > > makes sense.
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Jiangjie (Becket) Qin
> > > > > >
> > > > > > On Thu, Aug 17, 2023 at 12:31 AM liu ron <ron9....@gmail.com>
> > wrote:
> > > > > >
> > > > > > > Hi, Jiangjie
> > > > > > >
> > > > > > > Sorry for late reply. Thank you for such a detailed response.
> As
> > > you
> > > > > say,
> > > > > > > there are three behaviours here for users and I agree with you.
> > The
> > > > > goal
> > > > > > of
> > > > > > > this FLIP is to clarify the behaviour of the client side,
> which I
> > > > also
> > > > > > > agree with. However, as weihua said, the config
> > execution.attached
> > > is
> > > > > not
> > > > > > > only for per-job mode, but also for session mode, but the FLIP
> > says
> > > > > that
> > > > > > > this is only for per-job mode, and this config will be removed
> in
> > > the
> > > > > > > future because the per-job mode has been deprecated. I don't
> > think
> > > > this
> > > > > > is
> > > > > > > correct and we should change the description in the
> corresponding
> > > > > section
> > > > > > > of the FLIP. Since execution.attached is used in session mode,
> > > there
> > > > > is a
> > > > > > > compatibility issue here if we change it directly to
> > > > > > > client.attached.after.submission, and I think we should make
> this
> > > > clear
> > > > > > in
> > > > > > > the FLIP.
> > > > > > >
> > > > > > > Best,
> > > > > > > Ron
> > > > > > >
> > > > > > > Becket Qin <becket....@gmail.com> 于2023年8月14日周一 20:33写道:
> > > > > > >
> > > > > > > > Hi Ron and Weihua,
> > > > > > > >
> > > > > > > > Thanks for the feedback.
> > > > > > > >
> > > > > > > > There seem three user sensible behaviors that we are talking
> > > about:
> > > > > > > >
> > > > > > > > 1. The behavior on the client side, i.e. whether blocking
> until
> > > the
> > > > > job
> > > > > > > > finishes or not.
> > > > > > > >
> > > > > > > > 2. The behavior of the submitted job, whether stop the job
> > > > execution
> > > > > if
> > > > > > > the
> > > > > > > > client is detached from the Flink cluster, i.e. whether bind
> > the
> > > > > > > lifecycle
> > > > > > > > of the job with the connection status of the attached client.
> > For
> > > > > > > example,
> > > > > > > > one might want to keep a batch job running until finish even
> > > after
> > > > > the
> > > > > > > > client connection is lost. But it makes sense to stop the job
> > > upon
> > > > > > client
> > > > > > > > connection lost if the job invokes collect() on a streaming
> > job.
> > > > > > > >
> > > > > > > > 3. The behavior of the Flink cluster (JM and TMs), whether
> > > shutdown
> > > > > the
> > > > > > > > Flink cluster if the client is detached from the Flink
> cluster,
> > > > i.e.
> > > > > > > > whether bind the cluster lifecycle with the job lifecycle.
> For
> > > > > > dedicated
> > > > > > > > clusters (application cluster or dedicated session clusters),
> > the
> > > > > > > lifecycle
> > > > > > > > of the cluster should be bound with the job lifecycle. But
> for
> > > > shared
> > > > > > > > session clusters, the lifecycle of the Flink cluster should
> be
> > > > > > > independent
> > > > > > > > of the jobs running in it.
> > > > > > > >
> > > > > > > > As we can see, these three behaviors are sort of independent,
> > the
> > > > > > current
> > > > > > > > configurations fail to support all the combination of wanted
> > > > > behaviors.
> > > > > > > > Ideally there should be three separate configurations, for
> > > example:
> > > > > > > > - client.attached.after.submission and
> client.heartbeat.timeout
> > > > > control
> > > > > > > the
> > > > > > > > behavior on the client side.
> > > > > > > > - jobmanager.cancel-on-attached-client-exit controls the
> > behavior
> > > > of
> > > > > > the
> > > > > > > > job when an attached client lost connection. The client
> > heartbeat
> > > > > > timeout
> > > > > > > > and attach-ness will be also passed to the JM upon job
> > > submission.
> > > > > > > > - cluster.shutdown-on-first-job-finishes *(*or
> > > > > > > > jobmanager.shutdown-cluster-after-job-finishes) controls the
> > > > cluster
> > > > > > > > behavior after the job finishes normally / abnormally. This
> is
> > a
> > > > > > cluster
> > > > > > > > level setting instead of a job level setting. Therefore it
> can
> > > only
> > > > > be
> > > > > > > set
> > > > > > > > when launching the cluster.
> > > > > > > >
> > > > > > > > The current code sort of combines config 2 and 3 into
> > > > > > > > execution.shutdown-on-attach-exit.
> > > > > > > > This assumes the the life cycle of the cluster is the same as
> > the
> > > > job
> > > > > > > when
> > > > > > > > the client is attached. This FLIP does not intend to change
> > that.
> > > > but
> > > > > > > using
> > > > > > > > the execution.attached config for the client behavior control
> > > looks
> > > > > > > > misleading. So this FLIP proposes to replace it with a more
> > > > intuitive
> > > > > > > > config of client.attached.after.submission. This makes it
> clear
> > > > that
> > > > > it
> > > > > > > is
> > > > > > > > a configuration controlling the client side behavior, instead
> > of
> > > > the
> > > > > > > > execution of the job.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > >
> > > > > > > > Jiangjie (Becket) Qin
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > On Thu, Aug 10, 2023 at 10:34 PM Weihua Hu <
> > > huweihua....@gmail.com
> > > > >
> > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Hi Allison
> > > > > > > > >
> > > > > > > > > Thanks for driving this FLIP. It's a valuable feature for
> > batch
> > > > > jobs.
> > > > > > > > > This helps keep "Drop Per-Job Mode [1]" going.
> > > > > > > > >
> > > > > > > > > +1 for this proposal.
> > > > > > > > >
> > > > > > > > > However, it seems that the change in this FLIP is not
> > detailed
> > > > > > enough.
> > > > > > > > > I have a few questions.
> > > > > > > > >
> > > > > > > > > 1. The config 'execution.attached' is not only used in
> > per-job
> > > > > mode,
> > > > > > > > > but also in session mode to shutdown the cluster. IMHO,
> it's
> > > > better
> > > > > > to
> > > > > > > > > keep this option name.
> > > > > > > > >
> > > > > > > > > 2. This FLIP only mentions YARN mode. I believe this
> feature
> > > > should
> > > > > > > > > work in both YARN and Kubernetes mode.
> > > > > > > > >
> > > > > > > > > 3. Within the attach mode, we support two features:
> > > > > > > > > execution.shutdown-on-attached-exit
> > > > > > > > > and client.heartbeat.timeout. These should also be taken
> into
> > > > > > account.
> > > > > > > > >
> > > > > > > > > 4. The Application Mode will shut down once the job has
> been
> > > > > > completed.
> > > > > > > > > So, if we use the flink client to poll job status via REST
> > API
> > > > for
> > > > > > > attach
> > > > > > > > > mode,
> > > > > > > > > there is a chance that the client will not be able to
> > retrieve
> > > > the
> > > > > > job
> > > > > > > > > finish status.
> > > > > > > > > Perhaps FLINK-24113[3] will help with this.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > [1]https://issues.apache.org/jira/browse/FLINK-26000
> > > > > > > > > [2]
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://nightlies.apache.org/flink/flink-docs-master/docs/deployment/resource-providers/native_kubernetes/#session-mode
> > > > > > > > > [2]https://issues.apache.org/jira/browse/FLINK-24113
> > > > > > > > >
> > > > > > > > > Best,
> > > > > > > > > Weihua
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Thu, Aug 10, 2023 at 10:47 AM liu ron <
> ron9....@gmail.com
> > >
> > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hi, Allison
> > > > > > > > > >
> > > > > > > > > > Thanks for driving this proposal, it looks cool for batch
> > > jobs
> > > > > > under
> > > > > > > > > > application mode. But after reading your FLIP document
> and
> > > > [1], I
> > > > > > > have
> > > > > > > > a
> > > > > > > > > > question. Why do you want to rename the
> execution.attached
> > > > > > > > configuration
> > > > > > > > > to
> > > > > > > > > > client.attached.after.submission and at the same time
> > > deprecate
> > > > > > > > > > execution.attached? Based on your design, I understand
> the
> > > role
> > > > > of
> > > > > > > > these
> > > > > > > > > > two options are the same. Introducing a new option would
> > > > increase
> > > > > > the
> > > > > > > > > cost
> > > > > > > > > > of understanding and use for the user, so why not follow
> > the
> > > > idea
> > > > > > > > > discussed
> > > > > > > > > > in FLINK-25495 and make Application mode support
> > > > > > attached.execution.
> > > > > > > > > >
> > > > > > > > > > [1] https://issues.apache.org/jira/browse/FLINK-25495
> > > > > > > > > >
> > > > > > > > > > Best,
> > > > > > > > > > Ron
> > > > > > > > > >
> > > > > > > > > > Venkatakrishnan Sowrirajan <vsowr...@asu.edu>
> 于2023年8月9日周三
> > > > > > 02:07写道:
> > > > > > > > > >
> > > > > > > > > > > This is definitely a useful feature especially for the
> > > flink
> > > > > > batch
> > > > > > > > > > > execution workloads using flow orchestrators like
> > Airflow,
> > > > > > Azkaban,
> > > > > > > > > Oozie
> > > > > > > > > > > etc. Thanks for reviving this issue and starting a
> FLIP.
> > > > > > > > > > >
> > > > > > > > > > > Regards
> > > > > > > > > > > Venkata krishnan
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Mon, Aug 7, 2023 at 4:09 PM Allison Chang
> > > > > > > > > > <alch...@linkedin.com.invalid
> > > > > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Hi all,
> > > > > > > > > > > >
> > > > > > > > > > > > I am opening this thread to discuss this proposal to
> > > > support
> > > > > > > > attached
> > > > > > > > > > > > execution on Flink Application Completion for Batch
> > Jobs.
> > > > The
> > > > > > > link
> > > > > > > > to
> > > > > > > > > > the
> > > > > > > > > > > > FLIP proposal is here:
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://urldefense.com/v3/__https://cwiki.apache.org/confluence/display/FLINK/FLIP-323*3A*Support*Attached*Execution*on*Flink*Application*Completion*for*Batch*Jobs__;JSsrKysrKysrKys!!IKRxdwAv5BmarQ!friFO6bJub5FKSLhPIzA6kv-7uffv-zXlv9ZLMKqj_xMcmZl62HhsgvwDXSCS5hfSeyHZgoAVSFg3fk7ChaAFNKi$
> > > > > > > > > > > >
> > > > > > > > > > > > This FLIP proposes adding back attached execution for
> > > > > > Application
> > > > > > > > > Mode.
> > > > > > > > > > > In
> > > > > > > > > > > > the past attached execution was supported for the
> > per-job
> > > > > mode,
> > > > > > > > which
> > > > > > > > > > > will
> > > > > > > > > > > > be deprecated and we want to include this feature
> back
> > > into
> > > > > > > > > Application
> > > > > > > > > > > > mode.
> > > > > > > > > > > >
> > > > > > > > > > > > Please reply to this email thread and share your
> > > > > > > thoughts/opinions.
> > > > > > > > > > > >
> > > > > > > > > > > > Thank you!
> > > > > > > > > > > >
> > > > > > > > > > > > Allison Chang
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Reply via email to