Great ideas.

Actually, when I evaluated using KubernetesExecutor in Airbnb, I created
*JobDispatcherExecutor*, which is similar to the idea 2.

We have tons of sensor tasks launched at around UTC 00:00 and we enabled
the SmartSensor, which means each sensor task is very short running. If we
choose to have one pod per task, the k8s cluster will not be able to handle
the bursty load.

Thus, I created a JobDispatcherExecutor which takes an instance of
`celery_executor` and `kubernetes_executor`.
[image: image.png]
And there is a router
<https://gist.github.com/pingzh/cc44c97336560b658d012c225a2242cc#file-jobdispatcherexecutor-py-L165-L168>
method to determine use which executor to queue_command,
[image: image.png]
It is currently based on whether a task is a smart sensor task, but it can
be easily changed to other attributes and/or settings.policy (so that infra
can have control).

Here is the gist:
https://gist.github.com/pingzh/cc44c97336560b658d012c225a2242cc#file-jobdispatcherexecutor-py

[image: image.png]


Would love to get some feedback.

Best wishes

Ping Zhang


On Thu, Aug 13, 2020 at 12:58 PM Jarek Potiuk <jarek.pot...@polidea.com>
wrote:

> +1 for pod mutation in config. It's not a "yamly" thing - it's a python
> code to run, so it's quite appropriate to have it in config.
>
> On Wed, Aug 12, 2020 at 9:43 PM Daniel Imberman <daniel.imber...@gmail.com
> >
> wrote:
>
> > I think pod_mutation is important as an “admin override”. E.g. there
> might
> > be a label that is required by security policy for all tasks launched by
> > airflow. I do agree that the executor_config should take in a V1Pod
> object
> >
> > One option is that the executor_config could take in a pod_mutation_hook
> > function as well? So users can manually override the pod functionally
> > rather than just giving an object and hoping it merges the way they
> expect
> > it to?
> > via Newton Mail [
> >
> https://cloudmagic.com/k/d/mailapp?ct=dx&cv=10.0.50&pv=10.15.5&source=email_footer_2
> > ]
> > On Wed, Aug 12, 2020 at 12:19 PM, David Lum <davidlu...@gmail.com>
> wrote:
> > I'll just leave some passing thoughts here. How the pod template file is
> > modified by the executor config might not be very well defined. Currently
> > when a user passes in a k8s.V1Pod object through executor_config we kind
> of
> > arbitrarily merge the two pod definitions to what we imagine the client
> > would want, but it could be often misinterpreted. Previously the
> > Airflow-defined Pod class was fairly flat. This made merging the two
> > objects fairly straight forward. If the user wrote the field in
> > executor_config, it overwrote that field in the Pod definition, but the
> > rest stayed the same. When attributes are arbitrarily nested like in the
> > k8s.V1Pod, some aspects can be mistakenly overwritten. For example,
> > specifying the volumeMounts of a container requires also specifying the
> > rest of the V1Pod. How do we handle this? Do we just overwrite
> > volumeMounts? The client must have specified a container, so do we
> > overwrite that too? The fact that there is a choice means the API could
> be
> > unintuitive to the user.
> >
> > What I suggest might be easier is that executor_config completely
> replaces
> > the pod_template_file if it is specified, but then also offer something
> > like pod_mutation_hook at the operator level. This makes it quite clear
> to
> > the user, you are either replacing the pod_template, or you are modifying
> > the pod_template with pod_mutation_hook and you are doing it by your own
> > rules. the argument to executor_config can be something like
> > ```
> > executor_config: { 'KubernetesExector': {'pod_mutation': some_func,
> > 'pod_template': k8s.V1Pod(...) }} # Pod template could also be a path to
> a
> > file
> > ```
> >
> > If both are specified then pod_mutation is a function applied to
> > pod_template. Thoughts?
> >
> > On Wed, Aug 12, 2020 at 11:24 AM Tomasz Urbaszek <turbas...@apache.org>
> > wrote:
> >
> > > +1 for the idea, thanks Daniel! I agree that multi-executor and pod
> > > templates should be 2.1.
> > >
> > > T.
> > >
> > > On Wed, Aug 12, 2020 at 5:13 PM Daniel Imberman <
> > daniel.imber...@gmail.com>
> > > wrote:
> > >
> > >> (Also funny enough we could use a lot of the existing infrastructure
> int
> > >> 1.10 to create that migration script. Would just need to take the
> python
> > >> dictionary and use the python yaml library)
> > >>
> > >> via Newton Mail [
> > >>
> >
> https://cloudmagic.com/k/d/mailapp?ct=dx&cv=10.0.50&pv=10.15.5&source=email_footer_2
> > >> ]
> > >> On Wed, Aug 12, 2020 at 8:09 AM, Daniel Imberman <
> > >> daniel.imber...@gmail.com> wrote:
> > >> 100% agreed on timing. I think 2.0 should be for the breaking aspect
> > >> (losing the configurations) and then 2.1/2.2 we can start adding on
> the
> > new
> > >> features. I also like the idea of a migration tool. We can make a
> script
> > >> that takes your airflow.cfg and converts that into a yaml file.
> > >>
> > >> via Newton Mail [
> > >>
> >
> https://cloudmagic.com/k/d/mailapp?ct=dx&cv=10.0.50&pv=10.15.5&source=email_footer_2
> > >> ]
> > >> On Wed, Aug 12, 2020 at 7:48 AM, Jarek Potiuk <
> jarek.pot...@polidea.com
> > >
> > >> wrote:
> > >> Big +1. All the arguments are very appealing to me and simplifying the
> > >> Kubernetes Executor down to YAML-configurable one seems like a
> > no-brainer
> > >> especially if we provide some migration tools. I've lost countless
> hours
> > >> on
> > >> debugging some configuration problems, simply because the relevant
> > >> Kubernetes-related configuration was in the least expected place -
> i.e.
> > >> airflow.cfg. YAML configuration.
> > >>
> > >> I am also a big fan of both 1. and 2. I've implemented a POC of
> > >> queue-based multi-scheduler once but having it embedded as part of
> core
> > >> Airflow rather than based it on queues (which are basically a Celery
> > >> Executor concept) is I think much better approach. Both 1. and 2. are
> > >> cool.
> > >>
> > >> Now - question about timing. If we decide to go that route - my view
> is
> > >> that simplifying Kubernetes should be an Airflow 2.0 task - alongside
> > more
> > >> comprehensive tests (which will be much easier to write in this case).
> > The
> > >> new features/ideas 1. 2. for KE I think should come after that - when
> we
> > >> release and stabilize 2.0. Sounds like great candidates for 2.1 to me.
> > >>
> > >> J.
> > >>
> > >>
> > >> On Wed, Aug 12, 2020 at 4:24 PM Daniel Imberman <
> > >> daniel.imber...@gmail.com>
> > >> wrote:
> > >>
> > >> > Hello, fellow Airflowers! I hope you are all well in these trying
> > times.
> > >> >
> > >> >
> > >> > With the recent launch of Airflow 2.0 preparation, it now seems
> like a
> > >> > good time to review the project's state and where we can fit in some
> > >> > breaking changes that will improve the project for the future.
> > >> >
> > >> >
> > >> > When we first created the KubernetesExecutor, we had two goals in
> > mind.
> > >> > The first goal was to improve the airflow Auto scaling story.
> > >> Previously,
> > >> > airflow users would have to manually provision celery workers, which
> > >> could
> > >> > lead to wasted resources or missed SLAs. The other goal was to
> > >> introduce a
> > >> > community that was not yet well versed in the Kubernetes API to the
> > >> > Kubernetes system.
> > >> >
> > >> >
> > >> > To ease the community's transition, we abstracted many of the
> > >> complexities
> > >> > of creating a Kubernetes object. We chose to offer a limited number
> of
> > >> > configurations and keep much of the pod creation process internal to
> > >> > airflow. In the short-term, this system lowered the barrier to
> entry.
> > >> Over
> > >> > time, however, this abstraction has become a nightmare of tech debt
> as
> > >> the
> > >> > Kubernetes API is expensive and constantly changing.
> > >> >
> > >> >
> > >> > With this in mind, I think it's time for us to consider a more
> > >> > straightforward approach that takes the complexity out of Airflow
> and
> > >> > offers the full Kubernetes API to the airflow user.
> > >> >
> > >> >
> > >> > What I'm proposing here is pretty straightforward. We remove all
> > >> > Kubernetes pod creation configurations from the airflow.cfg and
> > instead
> > >> > offer only one way to use the KubernetesExecutor: with a YAML file.
> > >> >
> > >> >
> > >> > We can easily supply all of the configurations to the
> > KubernetesExecutor
> > >> > by offering example YAMLs (git sync mode is just a sidecar and an
> init
> > >> > container, DAG volumes are just an example volume and volume mount,
> > >> etc.).
> > >> >
> > >> >
> > >> > This system would simplify a user's ability to predict what a pod
> will
> > >> > look like once it is launched by airflow. They will know it's a base
> > pod
> > >> > and will be able to simply modify the pod object using the executor
> > >> config
> > >> > and the pod mutation hook.
> > >> >
> > >> >
> > >> >
> > >> >
> > >> >
> > >> >
> > >> >
> > >> > This simplification could also lead to some pretty great new
> features
> > in
> > >> > the KubernetesExecutor
> > >> >
> > >> >
> > >> > Idea 1: Picking a pod_template_file per-task
> > >> > Along with the existing customization with the executor config,
> solely
> > >> > relying on pod files can allow users to pick the pod template file
> > that
> > >> > they want to use as their base pod on a per-task basis. An Airflow
> > >> engineer
> > >> > could supply several pre-made templates for their data scientists to
> > >> reduce
> > >> > the amount of customization an airflow user would need to use.
> > >> >
> > >> >
> > >> > Idea 2: Merging the KubernetesExecutor into the CeleryExecutor
> > >> > One idea that we've been excited about recently has been the idea of
> > >> > creating merged Celery and Kubernetes executor. This hybrid executor
> > >> would
> > >> > default to launching celery workers with KEDA and would have the
> > option
> > >> to
> > >> > launch individual tasks using the Kubernetes executor when a user
> > wants
> > >> > isolation or customization. Simplifying the Kubernetes executor
> > reduces
> > >> the
> > >> > number of fail-points that this merged executor would need to
> account
> > >> for.
> > >> >
> > >> >
> > >> >
> > >> > What would we need to do to implement this?
> > >> > The good news here is that the hard work has already been done! As
> of
> > >> > AIRFLOW-5413 [https://issues.apache.org/jira/browse/AIRFLOW-5413]
> by
> > >> > David Lum, airflow already has the ability to use base worker pods
> on
> > a
> > >> > template file. This would involve primarily code deletion and very
> > >> little
> > >> > new code.
> > >> >
> > >> >
> > >> > Thank you for your time and I look forward to the community’s
> > >> discussion.
> > >> >
> > >> > Daniel
> > >>
> > >>
> > >>
> > >> --
> > >>
> > >> Jarek Potiuk
> > >> Polidea <https://www.polidea.com/> | Principal Software Engineer
> > >>
> > >> M: +48 660 796 129 <+48660796129>
> > >> [image: Polidea] <https://www.polidea.com/>
> > >
> > >
>
>
>
> --
>
> Jarek Potiuk
> Polidea <https://www.polidea.com/> | Principal Software Engineer
>
> M: +48 660 796 129 <+48660796129>
> [image: Polidea] <https://www.polidea.com/>
>

Reply via email to