Yeah. I think for sure "Docker" as a "common execution environment" is
convenient in certain situations. But for sure it should not be the default
as mentioned before (as much as I love containers I also know - from the
surveys we run for one but also from interacting with many users of Airflow
- for many of our users containers are not "default" way of doing things
(and we should embrace it).

I see some of the multi-tenancy deployment in the future could benefit from
having different sets of dependencies - both for parsing and execution (say
one environment per team). I think the current AIP-43 proposal
already handles a big part of it. You could have - dockerised or not -
different environments to parse your dags in per different "subfolder" - so
each DagProcessor for the sub-folder could have a different set of
dependencies (either coming from the virtualenv or Docker). This all
without putting the "requirement" on using Docker. I think we are well
aligned on the goal. I see that as the choice:

a) whether Airflow should be able to choose the run "environment" to parse
a Dag based on meta-data (which your proposal is about)
b) whether each DagProcessor (different per team) should be started in the
right "environment" to begin with (which is actually made possible by
AIP-43) - which is part of Deployment not Airflow code.

I think we should go with b) first and if b) will not be enough, future AIP
to implement a) is also possible.

When it comes to task execution - this is something we can definitely
discuss in the future - as the next AIP. We should think about how to
seamlessly map an execution of a task into different environments. We can
definitely make it a point for discussion in the Jan meeting I plan
to have.

As Ash mentioned - we already have some ways to do it (Celery Queues, K8S
executor, but also Airflow 2 @task.docker decorator). I am sure we can
discuss those and see how we can address it after AIP-43/44 are
discussed/approved and see if it makes sense to add another way.

J.


On Fri, Dec 17, 2021 at 11:53 AM Alexander Shorin <[email protected]> wrote:

> How should your idea work on systems without docker? Like FreeBSD? And why
> you made such leaky tasks which couldn't be isolated with common tools like
> system packages, venv, etc.
>
> --
> ,,,^..^,,,
>
>
> On Fri, Dec 17, 2021 at 2:53 AM Ping Zhang <[email protected]> wrote:
>
>> Hi Airflow Community,
>>
>> This is Ping Zhang from the Airbnb Airflow team.  We would like to open
>> source our internal feature: docker runtime isolation for airflow tasks. It
>> has been in our production for close to 1 year and it is very stable.
>>
>> I will create an AIP after the discussion.
>>
>> Thanks,
>>
>> Ping
>>
>>
>> Motivation
>>
>> Airflow worker host is a shared resource among all tasks running on it.
>> Thus, it requires hosts to provision dependencies for all tasks, including
>> system and python application level dependencies. It leads to a very fat
>> runtime, thus long host provision time and low elasticity in the worker
>> resource. This makes it challenging to prepare for unexpected burst load,
>> including a large backfill or a rerun of large DAGs.
>>
>> The lack of runtime isolation makes it challenging and risky to do
>> operations, including adding/upgrading system and python dependencies, and
>> it is almost impossible to remove any dependencies. It also incurs lots of
>> additional operating costs for the team as users do not have permission to
>> add/upgrade python dependencies, which requires us to coordinate with them.
>> When there are package version conflicts, it prevents installing them
>> directly on the host. Users have to use PythonVirtualenvOperator, which
>> slows down their development cycle.
>>
>> What change do you propose to make?
>>
>> To solve those problems, we propose introducing runtime isolation for
>> Airflow tasks. It leverages docker as the tasks runtime environment. There
>> are several benefits:
>>
>>    1.
>>
>>    Provide runtime isolation on task level
>>    2.
>>
>>    Customize runtime to parse dag files
>>    3.
>>
>>    Lean runtime on airflow host, which enables high worker resource
>>    elasticity
>>    4.
>>
>>    Immutable and portable task execution untime
>>    5.
>>
>>    Process isolation ensures that all subprocesses of a task are cleaned
>>    up after docker exits (we have seen some orphaned hive, spark subprocesses
>>    after the airflow run process exits)
>>
>> ChangesAirflow Worker
>>
>> In the new design, the `airflow run local` and `airflow run raw`
>> processes are running inside a docker container, which is launched by an
>> airflow worker. In this way, the airflow worker runtime only needs minimum
>> requirements to run airflow core and docker.
>> Airflow Scheduler
>>
>> Instead of processing the DAG file directly, the DagFileProcessor process
>>
>>    1.
>>
>>    launches a docker container required by that DAG file to process it
>>    and persists the serializable DAGs (SimpleDags) to a file so that the
>>    result can be read outside the docker container
>>    2.
>>
>>    reads the file persisted from the docker container, deserializes it
>>    and puts the result into the multiprocess queue
>>
>>
>> This ensures the DAG parsing runtime is exactly the same as DAG execution
>> runtime.
>>
>> This requires a DAG definition file to tell the DAG file processing loop
>> to use which docker image to process it. We can easily achieve this by
>> having a metadata file along with the DAG definition file to define the
>> docker runtime. To ease the burden of users, a default docker image is
>> provided when a DAG definition file does not require customized runtime.
>> As a Whole
>>
>>
>>
>>
>>
>>
>> Best wishes
>>
>> Ping Zhang
>>
>

Reply via email to