Hi Xintong,

After experimenting a bit, I came to roughly the same conclusion: cleanup
is what's more or less incompatible if Kubernetes manages the pods. Then it
might be better to just allow using a more stable pod naming scheme that
doesn't depend on the attempt number and thus produces more stable task
manager metrics. I'll explore that.

Regards,
Alexis.

On Mon, 3 Jun 2024, 03:35 Xintong Song, <tonysong...@gmail.com> wrote:

> I think the reason we didn't choose StatefulSet when introducing the Native
> K8s Deployment is that, IIRC, we want Flink's ResourceManager to have full
> control of the individual pod lifecycles.
>
> E.g.,
> - Pods in a StatefulSet are always terminated in the reverse order as they
> are created. This prevents us from releasing a specific idle TM that is not
> necessarily created lastly.
> - If a pod is unexpectedly terminated, Flink's ResourceManager should
> decide whether to restart it or not according to the job status.
> (Technically, the same issue as above, that we may want pods to be
> terminated / deleted in a different order.)
>
> There might be some other reasons. I just cannot recall all the details.
>
> As for determining whether a pod is OOM killed, I think Flink does print
> diagnostics for terminated pods in JM logs, i.e. the `exitCode`, `reason`
> and `message` of the `Terminated` container state. In our production, it
> shows "(exitCode=137, reason=OOMKilled, message=null)". However, since the
> diagnostics are from K8s, I'm not 100% sure whether this behavior is same
> for all K8s versions,.
>
> Best,
>
> Xintong
>
>
>
> On Sun, Jun 2, 2024 at 7:35 PM Alexis Sarda-Espinosa <
> sarda.espin...@gmail.com> wrote:
>
> > Hi devs,
> >
> > Some time ago I asked about the way Task Manager pods are handled by the
> > native Kubernetes driver [1]. I have now looked a bit through the source
> > code and I think it could be possible to deploy TMs with a stateful set,
> > which could allow tracking OOM kills as I mentioned in my original email,
> > and could also make it easier to track metrics and create alerts, since
> the
> > labels wouldn't change as much.
> >
> > One challenge is probably the new elastic scaling features [2], since the
> > driver would have to differentiate between new pod requests due to a TM
> > terminating, and a request due to scaling. I'm also not sure where
> > downscaling requests are currently handled.
> >
> > I would be interested in taking a look at this and seeing if I can get
> > something working. I think it would be possible to make it configurable
> in
> > a way that maintains backwards compatibility. Would it be ok if I enter a
> > Jira ticket and try it out?
> >
> > Regards,
> > Alexis.
> >
> > [1] https://lists.apache.org/thread/jysgdldv8swgf4fhqwqochgf6hq0qs52
> > [2]
> >
> >
> https://nightlies.apache.org/flink/flink-docs-release-1.19/docs/deployment/elastic_scaling/
> >
>

Reply via email to