You are viewing an old version of this page. View the current version.

Compare with Current View Page History

« Previous Version 9 Next »

Status

StateDraft
Discussion Thread
JIRA


Created

2020-02-21

Motivation

Airflow does not currently have an explicit way to declare messages passed between tasks in a DAG. XCom are available but are hidden in execution functions inside the operator. This can create confusion when working on ETL pipelines that need to send messages between tasks. This includes PythonOperator, which could be used for simple message processing but is not as it's difficult to accept messages from previous tasks as arguments and it needs to be pulled explicitly creating a lot of bootstrapping code. 

This AIP focuses on proposing a Functional API that would allow to explicitly declare message passing while implicitly declaring task dependencies. At the same time, it introduces a new way to easily declare PythonOperators using a simple decorator. 

Considerations

What change do you propose to make?

The proposal end result is to provide an explicit way to declare message passing dependencies in a DAG. That can be accomplished by extending the Operator API to have a __call__ method that can be used after initialization of the operator to declare run-time attributes needed on execution time. To specify this message dependency, we will add a new class (XComArg) that is a lazy reference to an XCom value from an existing operator. This class will be resolved and parsed into the class attribute on pre_execute/execute time.

In addition, in order to make it easier to use custom functions in DAGs, we can add a decorator to make it simpler to declare PythonOperators. This paired with the __call__ method declared above will allow for a more functional declaration of DAGs that rely heavily on messages passing. 

All the changes proposed are non-breaking, which would allow to be merged in 1.10 and allow a slow-paced adoption.

XComArg 

  • Explicit way to pass around results between operators. 
  • This object is a reference to an XCom value that has not been created and will need to be resolved in the future. 
  • It can generate new XComArgs that point to the same operator but has a different key by using the key accessor. E.g: output['train'].
  • To do so, it includes as class attributes both the originating operator instance and a key (both needed to pull the XCom).
  • Has a property op  that allows access to the origin operator.

Add __call__ in BaseOperator

Add a functional interface to replace class attributes on execution time. This function should also be in charge of :

  • Checking if any of the attributes are XComArgs and resolve them
  • Replacing class attributes with the resolved args when executing on a worker. 
  • Adding dependencies to operators which current operator needs an XCom from. 
  • Return an XComArg pointing to ‘return_value’ key by default 

Python function decorator

  • Enable easy transformation of python functions into PythonFunctionalOperators by making a decorator that takes a function and converts it into a PythonFunctionalOperator. 
  • Should also allow to pass args/kwargs to the PythonOperator from the decorator itself 
  • with_context=True  adds a the context dictionary to the first arg of the function. This arg is then removed from the signature to make it mypy compliant.
  • 2 ways to use it:
    • @dag.task: Uses dag object, does not need the DAG context, task automatically assigned to DAG.
    • @airflow.task: Lazy imported from main Airflow module (real location airflow.decorators.task). Makes function an operator, but does not automatically assign it to a DAG (unless declared inside a  DAG context)
  • PythonFunctionalOperator:
    • Make it easier to set op_arg and op_kwargs from __call__ on a PythonOperator, effectively enabling function like operations based on XCom values. 
    • Should allow copies with copy method for reuse in the same DAG or a different DAG

XComArg in templated fields

  • Re-use the already existing template_fields operator attribute to resolve XComArg if found. Add clause in the render_template method that resolves if isinstance(XComArg).
  • This should unlock setting XCom in operator __init__ args and kwargs. 

DAG example

See: https://github.com/casassg/corrent/tree/master/dags 

import json
from datetime import datetime, timedelta

from airflow.operators.email_operator import EmailOperator
from airflow.operators.http_operator import SimpleHttpOperator

from airflow import DAG, task

default_args = {
    'owner': 'airflow',
    'depends_on_past': False,
    'start_date': datetime(2019, 12, 27),
    'retries': 1,
    'retry_delay': timedelta(minutes=1),
}


with DAG(
    'send_server_ip', default_args=default_args, schedule_interval=None
) as dag:

  # Using default connection as it's set to httpbin.org by default
  get_ip = SimpleHttpOperator(
      task_id='get_ip', endpoint='get', method='GET', xcom_push=True
  )

  @dag.task(add_context=True)
  def email_subject_generator(context: dict, raw_json: str) -> str:
    external_ip = json.loads(raw_json)['origin']
    return f"Server connected from {external_ip}"

  @task
  def email_body_generator(raw_json: str) -> str:
    return """
    Seems like today your server executing Airflow is connected from the external IP {origin}<br>
    <br>
    Have a nice day!
    """.format(**json.loads(raw_json))

  send_email = EmailOperator(
      task_id='send_email',
      to="example@example.com",
      subject='',
      html_content=''
  )

  ip_info = get_ip()
  subject = email_subject_generator(ip_info)
  body = email_body_generator(ip_info)
  send_email(subject=subject, html_content=body)


What problem does it solve?

  • Explicit message passing in Airflow DAGs

Why is it needed?

  • Extends Airflow DAG API to allow explicit message passing between tasks
  • Improves developer experience for simple usages of PythonOperator
  • Incorporates a new abstraction for message passing preparing Airflow to better support on data pipelines use cases

Are there any downsides to this change?

  • Agreeing on the right abstraction is key, at the same time keeping in mind future features that could be added to Airflow to build on this API. Difficulties will mostly be in agreeing on a single implementation for this proposal.
  • Implementation-wise, it may require some extension of the webserver to load the resolved XCom specified as a dependency for observability. 
  • Potentially, this could lead to more users/customers using XCom to pass around large pieces of data 
  • XCom uses pickling for serialization/deserialization, which may be unstable for certain data passed around. 

Which users are affected by the change?

Most users will be able to explicitly declare message passing on their DAGs using a Python functional approach. No existing DAGs will need to change.

How are users affected by the change? (e.g. DB upgrade required?)

It's an incremental and optional update. Users will not be affected by the change.

Other considerations?

This proposal comes from inspecting APIs in new DSL frameworks like Dagster, Prefect, Metaflow and Databand.

What defines this AIP as "done"?

  • AIP has been voted to move forward with an agreement on the right API abstraction.
  • __call__ interface is added to BaseOperator and XComArg class is added
  • PythonCallableOperator and the python functional decorator are merged
  • - Stretch - Webserver loads resolved XComArgs into the UI when inspecting a Task instance (both inlets and outlets).
  • No labels