Data-aware scheduling¶
New in version 2.4.
Quickstart¶
In addition to scheduling DAGs based upon time, they can also be scheduled based upon a task updating a dataset.
from airflow import Dataset
with DAG(...):
MyOperator(
# this task updates example.csv
outlets=[Dataset("s3://dataset-bucket/example.csv")],
...,
)
with DAG(
# this DAG should be run when example.csv is updated (by dag1)
schedule=[Dataset("s3://dataset-bucket/example.csv")],
...,
):
...
What is a “dataset”?¶
An Airflow dataset is a stand-in for a logical grouping of data. Datasets may be updated by upstream “producer” tasks, and dataset updates contribute to scheduling downstream “consumer” DAGs.
A dataset is defined by a Uniform Resource Identifier (URI):
from airflow import Dataset
example_dataset = Dataset('s3://dataset-bucket/example.csv')
Airflow makes no assumptions about the content or location of the data represented by the URI. It is treated as a string, so any use of regular expressions (eg input_\d+.csv
) or file glob patterns (eg input_2022*.csv
) as an attempt to create multiple datasets from one declaration will not work.
There are two restrictions on the dataset URI:
It must be a valid URI, which means it must be composed of only ASCII characters.
The URI scheme cannot be
airflow
(this is reserved for future use).
If you try to use either of the examples below, your code will cause a ValueError to be raised, and Airflow will not import it.
# invalid datasets:
reserved = Dataset("airflow://example_dataset")
not_ascii = Dataset("èxample_datašet")
The identifier does not have to be an absolute URI, it can be a scheme-less, relative URI, or even just a simple path or string:
# valid datasets:
schemeless = Dataset("//example/dataset")
csv_file = Dataset("example_dataset")
If required, an extra dictionary can be included in a Dataset:
example_dataset = Dataset(
"s3://dataset/example.csv",
extra={'team': 'trainees'},
)
Note
Security Note: Dataset URI and extra fields are not encrypted, they are stored in cleartext, in Airflow’s metadata database. Do NOT store any sensitive values, especially credentials, in dataset URIs or extra key values!
The URI is also case sensitive throughout, so s3://example_dataset
and s3://Example_Dataset
are considered different, as is s3://example_dataset
and S3://example_dataset
.
How to use datasets in your DAGs¶
You can use datasets to specify data dependencies in your DAGs. Take the following example:
example_dataset = Dataset("s3://dataset/example.csv")
with DAG(dag_id='producer', ...):
BashOperator(task_id='producer', outlets=[example_dataset], ...)
with DAG(dag_id='consumer', schedule=[example_dataset], ...):
...
Once the producer
task in the producer
DAG has completed successfully, Airflow schedules the consumer
DAG. A dataset will be marked as updated only if the task completes successfully — if the task fails or if it is skipped, no update occurs, and the consumer
DAG will not be scheduled.
A listing of the relationships between datasets and DAGs can be found in the Datasets View
Multiple Datasets¶
As the schedule
parameter is a list, DAGs can require multiple datasets, and the DAG will be scheduled once all datasets it consumes have been updated at least once since the last time it was run:
with DAG(
dag_id='multiple_datasets_example',
schedule=[
example_dataset_1,
example_dataset_2,
example_dataset_3,
],
...,
):
...
If one dataset is updated multiple times before all consumed datasets have been updated, the downstream DAG will still only be run once, as shown in this illustration:
Notes on schedules¶
The schedule
parameter to your DAG can take either a list of datasets to consume or a timetable-based option. The two cannot currently be mixed.
When using datasets, in this first release (v2.4) waiting for all datasets in the list to be updated is the only option when multiple datasets are consumed by a DAG. A later release may introduce more fine-grained options allowing for greater flexibility.