From 35a634f4bf3cff5da42d2c82f1a09b4dc50760c3 Mon Sep 17 00:00:00 2001 From: Christopher DeCarolis Date: Wed, 13 Nov 2024 10:27:09 -0800 Subject: [PATCH] [dagster-airlift] remove sections in reference (#25814) ## Summary & Motivation We moved some of the sections from the tutorial to the reference. Let's remove them. --- docs/content/_navigation.json | 28 +- docs/content/integrations/airlift.mdx | 52 +- .../content/integrations/airlift/full_dag.mdx | 243 +++ .../integrations/airlift/reference.mdx | 89 +- .../content/integrations/airlift/tutorial.mdx | 1298 ----------------- .../airlift/tutorial/decomission.mdx | 132 ++ .../integrations/airlift/tutorial/migrate.mdx | 247 ++++ .../integrations/airlift/tutorial/observe.mdx | 356 +++++ .../airlift/tutorial/overview.mdx | 50 + .../integrations/airlift/tutorial/peer.mdx | 179 +++ .../integrations/airlift/tutorial/setup.mdx | 61 + .../airlift/asset_check_peered_dag.png | Bin 0 -> 11779 bytes .../airlift/peer_check_success.png | Bin 0 -> 14536 bytes .../airlift/rebuild_customers_dag.png | Bin 0 -> 38840 bytes .../examples/tutorial-example/Makefile | 15 + .../examples/tutorial-example/README.md | 1251 +--------------- .../dagster_defs/stages/migrate.py | 9 +- .../dagster_defs/stages/migrate_with_check.py | 9 +- .../dagster_defs/stages/observe.py | 33 +- .../stages/observe_check_on_asset.py | 82 ++ .../stages/observe_with_partitions.py | 34 +- .../dagster_defs/stages/peer_with_check.py | 22 +- .../dagster_defs/stages/standalone.py | 9 +- 23 files changed, 1628 insertions(+), 2571 deletions(-) create mode 100644 docs/content/integrations/airlift/full_dag.mdx delete mode 100644 docs/content/integrations/airlift/tutorial.mdx create mode 100644 docs/content/integrations/airlift/tutorial/decomission.mdx create mode 100644 docs/content/integrations/airlift/tutorial/migrate.mdx create mode 100644 docs/content/integrations/airlift/tutorial/observe.mdx create mode 100644 docs/content/integrations/airlift/tutorial/overview.mdx create mode 100644 docs/content/integrations/airlift/tutorial/peer.mdx create mode 100644 docs/content/integrations/airlift/tutorial/setup.mdx create mode 100644 docs/next/public/images/integrations/airlift/asset_check_peered_dag.png create mode 100644 docs/next/public/images/integrations/airlift/peer_check_success.png create mode 100644 docs/next/public/images/integrations/airlift/rebuild_customers_dag.png create mode 100644 examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/observe_check_on_asset.py diff --git a/docs/content/_navigation.json b/docs/content/_navigation.json index 2d6ca9ca80d8c..e6b5afce9452a 100644 --- a/docs/content/_navigation.json +++ b/docs/content/_navigation.json @@ -918,15 +918,37 @@ "children": [ { "title": "Airflow Migration Tutorial", - "path": "/integrations/airlift/tutorial" + "path": "/integrations/airlift/tutorial/overview", + "children": [ + { + "title": "Part 1: Setup local Airflow", + "path": "/integrations/airlift/tutorial/setup" + }, + { + "title": "Part 2: Peering to the Airflow Instance", + "path": "/integrations/airlift/tutorial/peer" + }, + { + "title": "Part 3: Observing assets", + "path": "/integrations/airlift/tutorial/observe" + }, + { + "title": "Part 4: Migrating assets", + "path": "/integrations/airlift/tutorial/migrate" + }, + { + "title": "Part 5: Decomissioning the Airflow DAG", + "path": "/integrations/airlift/tutorial/decomission" + } + ] }, { "title": "Reference", "path": "/integrations/airlift/reference" }, { - "title": "Tutorial", - "path": "/integrations/airlift/tutorial" + "title": "DAG-level migration", + "path": "/integrations/airlift/full_dag" } ] }, diff --git a/docs/content/integrations/airlift.mdx b/docs/content/integrations/airlift.mdx index 2e7343fe33950..f46bad46d458e 100644 --- a/docs/content/integrations/airlift.mdx +++ b/docs/content/integrations/airlift.mdx @@ -1 +1,51 @@ -This is the main page. +# Airlift + +Airlift is a toolkit for integrating Dagster and Airflow. + +- Observe Airflow instances from within Dagster +- Accelerate the migration of Airflow DAGs to Dagster assets with opinionated tooling. + +--- + +## Compatibility + +### REST API Availability + +Airlift depends on the availability of Airflow’s REST API. Airflow’s REST API was made stable in its 2.0 release (Dec 2020) and was introduced experimentally in 1.10 in August 2018. Currently Airflow requires the availability of the REST API. + +- **OSS:** Stable as of 2.00 +- **MWAA** + - Note: only available in Airflow 2.4.3 or later on MWAA. +- **Cloud Composer:** No limitations as far as we know. +- **Astronomer:** No limitations as far as we know. + +--- + +## Airflow Migration Tutorial + +In this tutorial, we'll use `dagster-airlift` to migrate an Airflow DAG to Dagster assets. + +By the end of the tutorial, you'll understand how to use `dagster-airlift` to enable a migration process that + +- Can be done task-by-task in any order with minimal coordination +- Has task-by-task rollback to reduce risk +- Retains Airflow DAG structure and execution history during the migration + +[Click here to get started](/integrations/airlift/tutorial/overview). + +## References + + + + + + diff --git a/docs/content/integrations/airlift/full_dag.mdx b/docs/content/integrations/airlift/full_dag.mdx new file mode 100644 index 0000000000000..e38d7f375c16a --- /dev/null +++ b/docs/content/integrations/airlift/full_dag.mdx @@ -0,0 +1,243 @@ +# Migrating at the DAG-level. + +There may be DAGs for which you want to migrate the entire thing at once rather than on a per-task basis. Some reasons for taking this approach: + +- You're making use of "dynamic tasks" in Airflow, which don't conform neatly to the task mapping protocol we've laid out above. +- You want to make more substantial refactors to the dag structure that don't conform to the existing task structure + +For cases like this, we allow you to map assets to a full DAG. + +## Setup + +This guide utilizes the [Airflow Migration Tutorial](/integrations/airlift/tutorial/overview), and assumes you've completed the initial [setup](/integrations/airlift/tutorial/setup) and [peer](/integrations/airlift/tutorial/peer) stages. This guide will pick up from there.If you've already completed the migration tutorial, we advise downloading a fresh copy and following along with those steps. This guide will perform the observe and migrate steps at the DAG-level instead of on a task-by-task basis, for the `rebuild_customers_list` DAG. + +## Observing DAG-mapped + +When migrating an entire DAG at once, we'll want to create assets which map to the entire DAG. Whereas in the [task-by-task observation step](/integrations/airlift/tutorial/setup), we used the `assets_with_task_mappings` function, we'll instead use the `assets_with_dag_mappings` function. + +For our `rebuild_customers_list` DAG, let's take a look at what the new observation code looks like: + +```python file=../../experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/observe_dag_level.py +import os +from pathlib import Path + +from dagster import AssetExecutionContext, AssetSpec, Definitions +from dagster_airlift.core import ( + AirflowBasicAuthBackend, + AirflowInstance, + assets_with_dag_mappings, + build_defs_from_airflow_instance, +) +from dagster_dbt import DbtCliResource, DbtProject, dbt_assets + + +def dbt_project_path() -> Path: + env_val = os.getenv("TUTORIAL_DBT_PROJECT_DIR") + assert env_val, "TUTORIAL_DBT_PROJECT_DIR must be set" + return Path(env_val) + + +@dbt_assets( + manifest=dbt_project_path() / "target" / "manifest.json", + project=DbtProject(dbt_project_path()), +) +def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): + yield from dbt.cli(["build"], context=context).stream() + + +# Instead of mapping assets to individual tasks, we map them to the entire DAG. +mapped_assets = assets_with_dag_mappings( + dag_mappings={ + "rebuild_customers_list": [ + AssetSpec(key=["raw_data", "raw_customers"]), + dbt_project_assets, + AssetSpec(key="customers_csv", deps=["customers"]), + ], + }, +) + + +defs = build_defs_from_airflow_instance( + airflow_instance=AirflowInstance( + auth_backend=AirflowBasicAuthBackend( + webserver_url="http://localhost:8080", + username="admin", + password="admin", + ), + name="airflow_instance_one", + ), + defs=Definitions( + assets=mapped_assets, + resources={"dbt": DbtCliResource(project_dir=dbt_project_path())}, + ), +) +``` + +Now, instead of getting a materialization when a particular task completes, each mapped asset will receive a materialization when the entire DAG completes. + +## Migrating DAG-mapped assets + +Recall that in the [task-by-task migration step](/integrations/airlift/tutorial/migrate), we "proxy" execution on a task by task basis, which is controlled by a yaml document. For DAG-mapped assets, execution is proxied on a per-DAG basis. Proxying execution to Dagster will require all assets mapped to that DAG be _executable_ within Dagster. Let's take a look at some fully migrated code mapped to DAGs instead of tasks: + +```python file=../../experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/migrate_dag_level.py +import os +from pathlib import Path + +from dagster import ( + AssetExecutionContext, + AssetsDefinition, + AssetSpec, + Definitions, + materialize, + multi_asset, +) +from dagster_airlift.core import ( + AirflowBasicAuthBackend, + AirflowInstance, + assets_with_dag_mappings, + build_defs_from_airflow_instance, +) +from dagster_dbt import DbtCliResource, DbtProject, dbt_assets + +# Code also invoked from Airflow +from tutorial_example.shared.export_duckdb_to_csv import ExportDuckDbToCsvArgs, export_duckdb_to_csv +from tutorial_example.shared.load_csv_to_duckdb import LoadCsvToDuckDbArgs, load_csv_to_duckdb + + +def dbt_project_path() -> Path: + env_val = os.getenv("TUTORIAL_DBT_PROJECT_DIR") + assert env_val, "TUTORIAL_DBT_PROJECT_DIR must be set" + return Path(env_val) + + +def airflow_dags_path() -> Path: + return Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "tutorial_example" / "airflow_dags" + + +def load_csv_to_duckdb_asset(spec: AssetSpec, args: LoadCsvToDuckDbArgs) -> AssetsDefinition: + @multi_asset(name=f"load_{args.table_name}", specs=[spec]) + def _multi_asset() -> None: + load_csv_to_duckdb(args) + + return _multi_asset + + +def export_duckdb_to_csv_defs(spec: AssetSpec, args: ExportDuckDbToCsvArgs) -> AssetsDefinition: + @multi_asset(name=f"export_{args.table_name}", specs=[spec]) + def _multi_asset() -> None: + export_duckdb_to_csv(args) + + return _multi_asset + + +@dbt_assets( + manifest=dbt_project_path() / "target" / "manifest.json", + project=DbtProject(dbt_project_path()), +) +def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): + yield from dbt.cli(["build"], context=context).stream() + + +mapped_assets = assets_with_dag_mappings( + dag_mappings={ + "rebuild_customers_list": [ + load_csv_to_duckdb_asset( + AssetSpec(key=["raw_data", "raw_customers"]), + LoadCsvToDuckDbArgs( + table_name="raw_customers", + csv_path=airflow_dags_path() / "raw_customers.csv", + duckdb_path=Path(os.environ["AIRFLOW_HOME"]) / "jaffle_shop.duckdb", + names=["id", "first_name", "last_name"], + duckdb_schema="raw_data", + duckdb_database_name="jaffle_shop", + ), + ), + dbt_project_assets, + export_duckdb_to_csv_defs( + AssetSpec(key="customers_csv", deps=["customers"]), + ExportDuckDbToCsvArgs( + table_name="customers", + csv_path=Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "customers.csv", + duckdb_path=Path(os.environ["AIRFLOW_HOME"]) / "jaffle_shop.duckdb", + duckdb_database_name="jaffle_shop", + ), + ), + ], + }, +) + + +defs = build_defs_from_airflow_instance( + airflow_instance=AirflowInstance( + auth_backend=AirflowBasicAuthBackend( + webserver_url="http://localhost:8080", + username="admin", + password="admin", + ), + name="airflow_instance_one", + ), + defs=Definitions( + assets=mapped_assets, + resources={"dbt": DbtCliResource(project_dir=dbt_project_path())}, + ), +) + + +if __name__ == "__main__": + assert dbt_project_path().exists() + # print(dbt_project_path().absolute()) + Definitions.validate_loadable(defs) + materialize(defs.get_asset_graph().assets_defs) +``` + +Now that all of our assets are fully executable, we can create a simple yaml file to proxy execution for the whole dag: + +```yaml file=../../experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/rebuild_customers_list.yaml +proxied: True +``` + +We will similarly use `proxying_to_dagster` at the end of our DAG file (the code is exactly the same here as it was for the per-task migration step) + +```python file=../../experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/dags_truncated.py +# Dags file can be found at tutorial_example/airflow_dags/dags.py +from pathlib import Path + +from airflow import DAG +from dagster_airlift.in_airflow import proxying_to_dagster +from dagster_airlift.in_airflow.proxied_state import load_proxied_state_from_yaml + +dag = DAG("rebuild_customers_list", ...) + +... + +# Set this to True to begin the proxying process +PROXYING = False + +if PROXYING: + proxying_to_dagster( + global_vars=globals(), + proxied_state=load_proxied_state_from_yaml(Path(__file__).parent / "proxied_state"), + ) +``` + +Once the `proxied` bit is flipped to True, we can go to the Airflow UI, and we'll see that our tasks have been replaced with a single task. + +

+ + + +

+ +When performing dag-level mapping, we don't preserve task structure in the Airflow dags. This single task will materialize all mapped Dagster assets instead of executing the original Airflow task business logic. + +We can similarly mark `proxied` back to `False`, and the original task structure and business logic will return unchanged. diff --git a/docs/content/integrations/airlift/reference.mdx b/docs/content/integrations/airlift/reference.mdx index 40489ce946251..62932d6b67516 100644 --- a/docs/content/integrations/airlift/reference.mdx +++ b/docs/content/integrations/airlift/reference.mdx @@ -3,9 +3,9 @@ title: "dagster-airlift integration reference" description: "dagster-airlift is a toolkit for observing and migrating Airflow DAGs within Dagster." --- -# dagster-airlift integration reference +# `dagster-airlift` integration reference -`dagster-airlift` is a toolkit for observing and migrating Airflow DAGs within Dagster. This reference page provides additional information for working with `dagster-airlift` that is not provided within the tutorial. You should start by reading the [dagster-airlift tutorial](/integrations/airlift/tutorial) before using this reference page. +`dagster-airlift` is a toolkit for observing and migrating Airflow DAGs within Dagster. This reference page provides additional information for working with `dagster-airlift` that is not provided within the tutorial. You should start by reading the [dagster-airlift tutorial](/integrations/airlift/tutorial/overview) before using this reference page. - [Supporting custom authorization](#supporting-custom-authorization) - [Dagster Plus Authorization](#dagster-plus-authorization) @@ -90,3 +90,88 @@ In order to make spin-up more efficient, `dagster-airlift` caches the state of t ### Automating changes to code locations If changes to your Airflow instance are controlled via a ci/cd process, you can add a step to automatically induce a redeploy of the relevant code location. See the docs [here](https://docs.dagster.io/concepts/webserver/graphql-client#reloading-all-repositories-in-a-repository-location) on using the graphql client to do this. + +### Peering to multiple Airflow instances + +Airlift supports peering to multiple Airflow instances, as you can invoke `build_defs_from_airflow_instance` multiple times and combine them with `Definitions.merge`: + +```python +from dagster import Definitions + +from dagster_airlift.core import AirflowInstance, build_defs_from_airflow_instance + +defs = Definitions.merge( + build_defs_from_airflow_instance( + airflow_instance=AirflowInstance( + auth_backend=BasicAuthBackend( + webserver_url="http://yourcompany.com/instance_one", + username="admin", + password="admin", + ), + name="airflow_instance_one", + ) + ), + build_defs_from_airflow_instance( + airflow_instance=AirflowInstance( + auth_backend=BasicAuthBackend( + webserver_url="http://yourcompany.com/instance_two", + username="admin", + password="admin", + ), + name="airflow_instance_two", + ) + ), +) +``` + +### Customizing DAG proxying operator + +Similar to how we can customize the operator we construct on a per-dag basis, we can customize the operator we construct on a per-dag basis. We can use the `build_from_dag_fn` argument of `proxying_to_dagster` to provide a custom operator in place of the default. + +For example, in the following example we can see that the operator is customized to provide an authorization header which authenticates Dagster. + +```python file=../../experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/custom_operator_examples/custom_dag_level_proxy.py +from pathlib import Path + +import requests +from airflow import DAG +from airflow.utils.context import Context +from dagster_airlift.in_airflow import BaseProxyDAGToDagsterOperator, proxying_to_dagster +from dagster_airlift.in_airflow.proxied_state import load_proxied_state_from_yaml + + +class CustomProxyToDagsterOperator(BaseProxyDAGToDagsterOperator): + def get_dagster_session(self, context: Context) -> requests.Session: + if "var" not in context: + raise ValueError("No variables found in context") + api_key = context["var"]["value"].get("my_api_key") + session = requests.Session() + session.headers.update({"Authorization": f"Bearer {api_key}"}) + return session + + def get_dagster_url(self, context: Context) -> str: + return "https://dagster.example.com/" + + # This method controls how the operator is built from the dag. + @classmethod + def build_from_dag(cls, dag: DAG): + return CustomProxyToDagsterOperator(dag=dag, task_id="OVERRIDDEN") + + +dag = DAG( + dag_id="custom_dag_level_proxy_example", +) + +# At the end of your dag file +proxying_to_dagster( + global_vars=globals(), + proxied_state=load_proxied_state_from_yaml(Path(__file__).parent / "proxied_state"), + build_from_dag_fn=CustomProxyToDagsterOperator.build_from_dag, +) +``` + +`BaseProxyDAGToDagsterOperator` has three abstract methods which must be implemented: + +- `get_dagster_session`, which controls the creation of a valid session to access the Dagster graphql API. +- `get_dagster_url`, which retrieves the domain at which the dagster webserver lives. +- `build_from_dag`, which controls how the proxying task is constructed from the provided DAG. diff --git a/docs/content/integrations/airlift/tutorial.mdx b/docs/content/integrations/airlift/tutorial.mdx deleted file mode 100644 index 5ec16508df02f..0000000000000 --- a/docs/content/integrations/airlift/tutorial.mdx +++ /dev/null @@ -1,1298 +0,0 @@ -## Example: Migrating an Airflow DAG to Dagster - -This example demonstrates how to migrate an Airflow DAG to Dagster using the `dagster-airlift` package. It contains code examples of how to peer, observe, and migrate assets from an Airflow DAG to Dagster. The below guide will walk through each of these steps in detail. - -## Example Structure - -```plaintext -tutorial_example -├── shared: Contains shared Python & SQL code used Airflow and proxied Dagster code -│ -├── dagster_defs: Contains Dagster definitions -│ ├── stages: Contains reference implementations of each stage of the migration process -│ ├── definitions.py: Empty starter file for following along with the tutorial -│ -├── airflow_dags: Contains the Airflow DAG and associated files -│ ├── proxied_state: Contains migration state files for each DAG, see migration step below -│ ├── dags.py: The Airflow DAG definition -``` - -## Guide - -This tutorial will walk through the process of peering, observing, and migrating assets from an Airflow DAG to Dagster. - -First, clone the tutorial example repo locally, and enter the repo directory. - -```bash -gh repo clone dagster-io/airlift-tutorial -cd airlift-tutorial -``` - -First we strongly recommend that you setup a fresh virtual environment and that you use `uv`. - -```bash -pip install uv -uv venv -source .venv/bin/activate -``` - -## Running Airflow locally - -The tutorial example involves running a local Airflow instance. This can be done by running the following commands from the root of the `tutorial-example` directory. - -First, install the required python packages: - -```bash -make airflow_install -``` - -Next, scaffold the Airflow instance, and initialize the dbt project: - -```bash -make airflow_setup -``` - -Finally, run the Airflow instance with environment variables set: - -```bash -make airflow_run -``` - -This will run the Airflow Web UI in a shell. You should now be able to access the Airflow UI at `http://localhost:8080`, with the default username and password set to `admin`. - -You should be able to see the `rebuild_customers_list` DAG in the Airflow UI, made up of three tasks: `load_raw_customers`, `run_dbt_model`, and `export_customers`. - -## Peering Dagster to your Airflow instance - -The first step is to peer your Airflow instance with a Dagster code location, which will create an asset representation of each of your Airflow DAGs that you can view in Dagster. This process does not require any changes to your Airflow instance. - -First, you will want a new shell and navigate to the same directory. You will need to set up the `dagster-airlift` package in your Dagster environment: - -```bash -source .venv/bin/activate -uv pip install 'dagster-airlift[core]' dagster-webserver dagster -``` - -Next, create a `Definitions` object using `build_defs_from_airflow_instance`. You can use the empty `tutorial_example/dagster_defs/definitions.py` file as a starting point: - -```python file=../../experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/peer.py -from dagster_airlift.core import ( - AirflowBasicAuthBackend, - AirflowInstance, - build_defs_from_airflow_instance, -) - -defs = build_defs_from_airflow_instance( - airflow_instance=AirflowInstance( - # other backends available (e.g. MwaaSessionAuthBackend) - auth_backend=AirflowBasicAuthBackend( - webserver_url="http://localhost:8080", - username="admin", - password="admin", - ), - name="airflow_instance_one", - ) -) -``` - -This function creates: - -- An external asset representing each DAG. This asset is marked as materialized whenever a DAG run completes. -- A sensor that polls the Airflow instance for operational information. This sensor is responsible for creating materializations when a DAG executes. The sensor must remain on in order to properly update execution status. - -Let's set up some environment variables, and then point Dagster to see the asset created from our Airflow DAG: - -```bash -# Set up environment variables to point to the examples/tutorial-example directory on your machine -export TUTORIAL_EXAMPLE_DIR=$(pwd) -export TUTORIAL_DBT_PROJECT_DIR="$TUTORIAL_EXAMPLE_DIR/tutorial_example/shared/dbt" -export AIRFLOW_HOME="$TUTORIAL_EXAMPLE_DIR/.airflow_home" -dagster dev -f tutorial_example/dagster_defs/definitions.py -``` - -

- - - -

- -If we kick off a run of the `rebuild_customers_list` DAG in Airflow, we should see the corresponding asset materialize in Dagster. - -

- - - -

- -_Note: When the code location loads, Dagster will query the Airflow REST API in order to build a representation of your DAGs. In order for Dagster to reflect changes to your DAGs, you will need to reload your code location._ - -
- -Peering to multiple instances - - -Airlift supports peering to multiple Airflow instances, as you can invoke `create_airflow_instance_defs` multiple times and combine them with `Definitions.merge`: - -```python -from dagster import Definitions - -from dagster_airlift.core import AirflowInstance, build_defs_from_airflow_instance - -defs = Definitions.merge( - build_defs_from_airflow_instance( - airflow_instance=AirflowInstance( - auth_backend=BasicAuthBackend( - webserver_url="http://yourcompany.com/instance_one", - username="admin", - password="admin", - ), - name="airflow_instance_one", - ) - ), - build_defs_from_airflow_instance( - airflow_instance=AirflowInstance( - auth_backend=BasicAuthBackend( - webserver_url="http://yourcompany.com/instance_two", - username="admin", - password="admin", - ), - name="airflow_instance_two", - ) - ), -) -``` - -
- -## Observing Assets - -The next step is to represent our Airflow workflows more richly by observing the data assets that are produced by our tasks. In order to do this, we must define the relevant assets in the Dagster code location. - -In our example, we have three sequential tasks: - -1. `load_raw_customers` loads a CSV file of raw customer data into duckdb. -2. `run_dbt_model` builds a series of dbt models (from [jaffle shop](https://github.com/dbt-labs/jaffle_shop_duckdb)) combining customer, order, and payment data. -3. `export_customers` exports a CSV representation of the final customer file from duckdb to disk. - -We will first create a set of asset specs that correspond to the assets produced by these tasks. We will then annotate these asset specs so that Dagster can associate them with the Airflow tasks that produce them. - -The first and third tasks involve a single table each. We can manually construct specs for these two tasks. Dagster provides the `assets_with_task_mappings` utility to annotate our asset specs with the tasks that produce them. Assets which are properly annotated will be materialized by the Airlift sensor once the corresponding task completes: These annotated specs are then provided to the `defs` argument to `build_defs_from_airflow_instance`. - -We will also create a set of dbt asset definitions for the `build_dbt_models` task. We can use the `dagster-dbt`-supplied decorator `@dbt_assets` to generate these definitions using Dagster's dbt integration. - -First, you need to install the extra that has the dbt factory: - -```bash -uv pip install 'dagster-airlift[dbt]' -``` - -Then, we will construct our assets: - -```python file=../../experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/observe.py -import os -from pathlib import Path - -from dagster import AssetExecutionContext, AssetSpec, Definitions -from dagster_airlift.core import ( - AirflowBasicAuthBackend, - AirflowInstance, - assets_with_task_mappings, - build_defs_from_airflow_instance, -) -from dagster_dbt import DbtCliResource, DbtProject, dbt_assets - - -def dbt_project_path() -> Path: - env_val = os.getenv("TUTORIAL_DBT_PROJECT_DIR") - assert env_val, "TUTORIAL_DBT_PROJECT_DIR must be set" - return Path(env_val) - - -@dbt_assets( - manifest=dbt_project_path() / "target" / "manifest.json", - project=DbtProject(dbt_project_path()), -) -def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): - yield from dbt.cli(["build"], context=context).stream() - - -mapped_assets = assets_with_task_mappings( - dag_id="rebuild_customers_list", - task_mappings={ - "load_raw_customers": [AssetSpec(key=["raw_data", "raw_customers"])], - "build_dbt_models": [dbt_project_assets], - "export_customers": [AssetSpec(key="customers_csv", deps=["customers"])], - }, -) - - -defs = build_defs_from_airflow_instance( - airflow_instance=AirflowInstance( - auth_backend=AirflowBasicAuthBackend( - webserver_url="http://localhost:8080", - username="admin", - password="admin", - ), - name="airflow_instance_one", - ), - defs=Definitions( - assets=mapped_assets, - resources={"dbt": DbtCliResource(project_dir=dbt_project_path())}, - ), -) -``` - -### Viewing observed assets - -Once your assets are set up, you should be able to reload your Dagster definitions and see a full representation of the dbt project and other data assets in your code. - -

- - - -

- -Kicking off a run of the DAG in Airflow, you should see the newly created assets materialize in Dagster as each task completes. - -_Note: There will be some delay between task completion and assets materializing in Dagster, managed by the sensor. This sensor runs every 30 seconds by default (you can reduce down to one second via the `minimum_interval_seconds` argument to `sensor`), so there will be some delay._ - -### Adding partitions - -If your assets represent a time-partitioned data source, Airlift can automatically associate your materializations to the relevant partitions. In the case of `rebuild_customers_list`, data is daily partitioned in each created table, and as a result we've added a `@daily` cron schedule to the DAG to make sure it runs every day. We can likewise add a `DailyPartitionsDefinition` to each of our assets. - -```python file=../../experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/observe_with_partitions.py -import os -from pathlib import Path - -from dagster import AssetExecutionContext, AssetSpec, DailyPartitionsDefinition, Definitions -from dagster._time import get_current_datetime_midnight -from dagster_airlift.core import ( - AirflowBasicAuthBackend, - AirflowInstance, - assets_with_task_mappings, - build_defs_from_airflow_instance, -) -from dagster_dbt import DbtCliResource, DbtProject, dbt_assets - -PARTITIONS_DEF = DailyPartitionsDefinition(start_date=get_current_datetime_midnight()) - - -def dbt_project_path() -> Path: - env_val = os.getenv("TUTORIAL_DBT_PROJECT_DIR") - assert env_val, "TUTORIAL_DBT_PROJECT_DIR must be set" - return Path(env_val) - - -@dbt_assets( - manifest=dbt_project_path() / "target" / "manifest.json", - project=DbtProject(dbt_project_path()), - partitions_def=PARTITIONS_DEF, -) -def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): - yield from dbt.cli(["build"], context=context).stream() - - -mapped_assets = assets_with_task_mappings( - dag_id="rebuild_customers_list", - task_mappings={ - "load_raw_customers": [ - AssetSpec(key=["raw_data", "raw_customers"], partitions_def=PARTITIONS_DEF) - ], - "build_dbt_models": [dbt_project_assets], - "export_customers": [ - AssetSpec(key="customers_csv", deps=["customers"], partitions_def=PARTITIONS_DEF) - ], - }, -) - - -defs = build_defs_from_airflow_instance( - airflow_instance=AirflowInstance( - auth_backend=AirflowBasicAuthBackend( - webserver_url="http://localhost:8080", - username="admin", - password="admin", - ), - name="airflow_instance_one", - ), - defs=Definitions( - assets=mapped_assets, - resources={"dbt": DbtCliResource(project_dir=dbt_project_path())}, - ), -) -``` - -Now, every time the sensor synthesizes a materialization for an asset, it will automatically have a partition associated with it. - -

- - - -

- -In order for partitioned assets to work out of the box with `dagster-airlift`, the following things need to be true: - -- The asset can only be time-window partitioned. This means static, dynamic, and multi partitioned definitions will require custom functionality. -- The partitioning scheme must match up with the [logical_date / execution_date](https://airflow.apache.org/docs/apache-airflow/stable/faq.html#what-does-execution-date-mean) of corresponding Airflow runs. That is, each logical_date should correspond \_exactly\_ to a partition in Dagster. - -## Migrating Assets - -Once you have created corresponding definitions in Dagster to your Airflow tasks, you can proxy execution to Dagster on a per-task basis while Airflow is still controlling scheduling and orchestration. Once a task has been proxied, Airflow will kick off materializations of corresponding Dagster assets in place of executing the business logic of that task. - -To begin proxying tasks in a DAG, first you will need a file to track proxying state. In your Airflow DAG directory, create a `proxied_state` folder, and in it create a yaml file with the same name as your DAG. The included example at `airflow_dags/proxied_state` is used by `make airflow_run`, and can be used as a template for your own proxied state files. - -Given our example DAG `rebuild_customers_list` with three tasks, `load_raw_customers`, `run_dbt_model`, and `export_customers`, `proxied_state/rebuild_customers_list.yaml` should look like the following: - -```yaml file=../../experimental/dagster-airlift/examples/tutorial-example/tutorial_example/airflow_dags/proxied_state/rebuild_customers_list.yaml -tasks: - - id: load_raw_customers - proxied: False - - id: build_dbt_models - proxied: False - - id: export_customers - proxied: False -``` - -Next, you will need to modify your Airflow DAG to make it aware of the proxied state. This is already done in the example DAG: - -```python file=../../experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/dags_truncated.py -# Dags file can be found at tutorial_example/airflow_dags/dags.py -from pathlib import Path - -from airflow import DAG -from dagster_airlift.in_airflow import proxying_to_dagster -from dagster_airlift.in_airflow.proxied_state import load_proxied_state_from_yaml - -dag = DAG("rebuild_customers_list", ...) - -... - -# Set this to True to begin the proxying process -PROXYING = False - -if PROXYING: - proxying_to_dagster( - global_vars=globals(), - proxied_state=load_proxied_state_from_yaml(Path(__file__).parent / "proxied_state"), - ) -``` - -Set `PROXYING` to `True` or eliminate the `if` statement. - -The DAG will now display its proxied state in the Airflow UI. (There is some latency as Airflow evaluates the Python file periodically.) - -

- - - -

- -### Migrating individual tasks - -In order to proxy a task, you must do two things: - -1. First, ensure all associated assets are executable in Dagster by providing asset definitions in place of bare asset specs. -2. The `proxied: False` status in the `proxied_state` YAML folder must be adjusted to `proxied: True`. - -Any task marked as proxied will use the `DefaultProxyTaskToDagsterOperator` when executed as part of the DAG. This operator will use the Dagster GraphQL API to initiate a Dagster run of the assets corresponding to the task. - -The proxied file acts as the source of truth for proxied state. The information is attached to the DAG and then accessed by Dagster via the REST API. - -A task which has been proxied can be easily toggled back to run in Airflow (for example, if a bug in implementation was encountered) simply by editing the file to `proxied: False`. - -#### Supporting custom authorization - -If your Dagster deployment lives behind a custom auth backend, you can customize the Airflow-to-Dagster proxying behavior to authenticate to your backend. `proxying_to_dagster` can take a parameter `dagster_operator_klass`, which allows you to define a custom `BaseProxyTasktoDagsterOperator` class. This allows you to override how a session is created. Let's say for example, your Dagster installation requires an access key to be set whenever a request is made, and that access key is set in an Airflow `Variable` called `my_api_key`. We can create a custom `BaseProxyTasktoDagsterOperator` subclass which will retrieve that variable value and set it on the session, so that any requests to Dagster's graphql API will be made using that api key. - -```python file=../../experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/custom_operator_examples/custom_proxy.py -from pathlib import Path - -import requests -from airflow import DAG -from airflow.utils.context import Context -from dagster_airlift.in_airflow import BaseProxyTaskToDagsterOperator, proxying_to_dagster -from dagster_airlift.in_airflow.proxied_state import load_proxied_state_from_yaml - - -class CustomProxyToDagsterOperator(BaseProxyTaskToDagsterOperator): - def get_dagster_session(self, context: Context) -> requests.Session: - if "var" not in context: - raise ValueError("No variables found in context") - api_key = context["var"]["value"].get("my_api_key") - session = requests.Session() - session.headers.update({"Authorization": f"Bearer {api_key}"}) - return session - - def get_dagster_url(self, context: Context) -> str: - return "https://dagster.example.com/" - - -dag = DAG( - dag_id="custom_proxy_example", -) - -# At the end of your dag file -proxying_to_dagster( - global_vars=globals(), - proxied_state=load_proxied_state_from_yaml(Path(__file__).parent / "proxied_state"), - build_from_task_fn=CustomProxyToDagsterOperator.build_from_task, -) -``` - -#### Dagster Plus Authorization - -You can use a customer proxy operator to establish a connection to a Dagster plus deployment. The below example proxies to Dagster Plus using organization name, deployment name, and user token set as Airflow Variables. To set a Dagster+ user token, follow [this](https://docs.dagster.io/dagster-plus/account/managing-user-agent-tokens#managing-user-tokens) guide. - -```python file=../../experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/custom_operator_examples/plus_proxy_operator.py -import requests -from airflow.utils.context import Context -from dagster_airlift.in_airflow import BaseProxyTaskToDagsterOperator - - -class DagsterCloudProxyOperator(BaseProxyTaskToDagsterOperator): - def get_variable(self, context: Context, var_name: str) -> str: - if "var" not in context: - raise ValueError("No variables found in context") - return context["var"]["value"][var_name] - - def get_dagster_session(self, context: Context) -> requests.Session: - dagster_cloud_user_token = self.get_variable(context, "dagster_cloud_user_token") - session = requests.Session() - session.headers.update({"Dagster-Cloud-Api-Token": dagster_cloud_user_token}) - return session - - def get_dagster_url(self, context: Context) -> str: - org_name = self.get_variable(context, "dagster_plus_organization_name") - deployment_name = self.get_variable(context, "dagster_plus_deployment_name") - return f"https://{org_name}.dagster.plus/{deployment_name}" -``` - -#### Migrating common operators - -For some common operator patterns, like our dbt operator, Dagster supplies factories to build software defined assets for our tasks. In fact, the `@dbt_assets` decorator used earlier already backs its assets with definitions, so we can toggle the proxied state of the `build_dbt_models` task to `proxied: True` in the proxied state file: - -```yaml file=../../experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/dbt_proxied.yaml -tasks: - - id: load_raw_customers - proxied: False - - id: build_dbt_models - proxied: True - - id: export_customers - proxied: False -``` - -**Important**: It may take up to 30 seconds for the proxied state in the Airflow UI to reflect this change. You must subsequently reload the definitions in Dagster via the UI or by restarting `dagster dev`. - -You can now run the `rebuild_customers_list` DAG in Airflow, and the `build_dbt_models` task will be executed in a Dagster run: - -

- - - -

- -You'll note that we proxied a task in the _middle_ of the Airflow DAG. The Airflow DAG structure and execution history is stable in the Airflow UI, but execution of `build_dbt_models` has moved to Dagster. - -#### Migrating the remaining custom operators - -For all other operator types, we will need to build our own asset definitions. We recommend creating a factory function whose arguments match the inputs to your Airflow operator. Then, you can use this factory to build definitions for each Airflow task. - -For example, our `load_raw_customers` task uses a custom `LoadCSVToDuckDB` operator. We'll define a function `load_csv_to_duckdb_defs` factory to build corresponding software-defined assets. Similarly for `export_customers` we'll define a function `export_duckdb_to_csv_defs` to build SDAs: - -```python file=../../experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/migrate.py -import os -from pathlib import Path - -from dagster import ( - AssetExecutionContext, - AssetsDefinition, - AssetSpec, - Definitions, - materialize, - multi_asset, -) -from dagster_airlift.core import ( - AirflowBasicAuthBackend, - AirflowInstance, - assets_with_task_mappings, - build_defs_from_airflow_instance, -) -from dagster_dbt import DbtCliResource, DbtProject, dbt_assets - -# Code also invoked from Airflow -from tutorial_example.shared.export_duckdb_to_csv import ExportDuckDbToCsvArgs, export_duckdb_to_csv -from tutorial_example.shared.load_csv_to_duckdb import LoadCsvToDuckDbArgs, load_csv_to_duckdb - - -def dbt_project_path() -> Path: - env_val = os.getenv("TUTORIAL_DBT_PROJECT_DIR") - assert env_val, "TUTORIAL_DBT_PROJECT_DIR must be set" - return Path(env_val) - - -def airflow_dags_path() -> Path: - return Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "tutorial_example" / "airflow_dags" - - -def load_csv_to_duckdb_asset(spec: AssetSpec, args: LoadCsvToDuckDbArgs) -> AssetsDefinition: - @multi_asset(name=f"load_{args.table_name}", specs=[spec]) - def _multi_asset() -> None: - load_csv_to_duckdb(args) - - return _multi_asset - - -def export_duckdb_to_csv_defs(spec: AssetSpec, args: ExportDuckDbToCsvArgs) -> AssetsDefinition: - @multi_asset(name=f"export_{args.table_name}", specs=[spec]) - def _multi_asset() -> None: - export_duckdb_to_csv(args) - - return _multi_asset - - -@dbt_assets( - manifest=dbt_project_path() / "target" / "manifest.json", - project=DbtProject(dbt_project_path()), -) -def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): - yield from dbt.cli(["build"], context=context).stream() - - -mapped_assets = assets_with_task_mappings( - dag_id="rebuild_customers_list", - task_mappings={ - "load_raw_customers": [ - load_csv_to_duckdb_asset( - AssetSpec(key=["raw_data", "raw_customers"]), - LoadCsvToDuckDbArgs( - table_name="raw_customers", - csv_path=airflow_dags_path() / "raw_customers.csv", - duckdb_path=Path(os.environ["AIRFLOW_HOME"]) / "jaffle_shop.duckdb", - names=["id", "first_name", "last_name"], - duckdb_schema="raw_data", - duckdb_database_name="jaffle_shop", - ), - ) - ], - "build_dbt_models": - # load rich set of assets from dbt project - [dbt_project_assets], - "export_customers": [ - export_duckdb_to_csv_defs( - AssetSpec(key="customers_csv", deps=["customers"]), - ExportDuckDbToCsvArgs( - table_name="customers", - csv_path=Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "customers.csv", - duckdb_path=Path(os.environ["AIRFLOW_HOME"]) / "jaffle_shop.duckdb", - duckdb_database_name="jaffle_shop", - ), - ) - ], - }, -) - - -defs = build_defs_from_airflow_instance( - airflow_instance=AirflowInstance( - auth_backend=AirflowBasicAuthBackend( - webserver_url="http://localhost:8080", - username="admin", - password="admin", - ), - name="airflow_instance_one", - ), - defs=Definitions( - assets=mapped_assets, - resources={"dbt": DbtCliResource(project_dir=dbt_project_path())}, - ), -) - - -if __name__ == "__main__": - assert dbt_project_path().exists() - # print(dbt_project_path().absolute()) - Definitions.validate_loadable(defs) - materialize(defs.get_asset_graph().assets_defs) -``` - -We can then toggle the proxied state of the remaining tasks in the `proxied_state` file: - -```yaml file=../../experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/all_proxied.yaml -tasks: - - id: load_raw_customers - proxied: True - - id: build_dbt_models - proxied: True - - id: export_customers - proxied: True -``` - -## Decomissioning an Airflow DAG - -Once we are confident in our migrated versions of the tasks, we can decommission the Airflow DAG. First, we can remove the DAG from our Airflow DAG directory. - -Next, we can strip the task associations from our Dagster definitions. This can be done by removing the `assets_with_task_mappings` call. We can use this opportunity to attach our assets to a `ScheduleDefinition` so that Dagster's scheduler can manage their execution: - -```python file=../../experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/standalone.py -import os -from pathlib import Path - -from dagster import ( - AssetCheckResult, - AssetCheckSeverity, - AssetExecutionContext, - AssetKey, - AssetsDefinition, - AssetSelection, - AssetSpec, - Definitions, - ScheduleDefinition, - asset_check, - multi_asset, -) -from dagster_dbt import DbtCliResource, DbtProject, dbt_assets - -# Code also invoked from Airflow -from tutorial_example.shared.export_duckdb_to_csv import ExportDuckDbToCsvArgs, export_duckdb_to_csv -from tutorial_example.shared.load_csv_to_duckdb import LoadCsvToDuckDbArgs, load_csv_to_duckdb - - -def dbt_project_path() -> Path: - env_val = os.getenv("TUTORIAL_DBT_PROJECT_DIR") - assert env_val, "TUTORIAL_DBT_PROJECT_DIR must be set" - return Path(env_val) - - -def airflow_dags_path() -> Path: - return Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "tutorial_example" / "airflow_dags" - - -def load_csv_to_duckdb_asset(spec: AssetSpec, args: LoadCsvToDuckDbArgs) -> AssetsDefinition: - @multi_asset(name=f"load_{args.table_name}", specs=[spec]) - def _multi_asset() -> None: - load_csv_to_duckdb(args) - - return _multi_asset - - -def export_duckdb_to_csv_defs(spec: AssetSpec, args: ExportDuckDbToCsvArgs) -> AssetsDefinition: - @multi_asset(name=f"export_{args.table_name}", specs=[spec]) - def _multi_asset() -> None: - export_duckdb_to_csv(args) - - return _multi_asset - - -@dbt_assets( - manifest=dbt_project_path() / "target" / "manifest.json", - project=DbtProject(dbt_project_path()), -) -def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): - yield from dbt.cli(["build"], context=context).stream() - - -assets = [ - load_csv_to_duckdb_asset( - AssetSpec(key=["raw_data", "raw_customers"]), - LoadCsvToDuckDbArgs( - table_name="raw_customers", - csv_path=airflow_dags_path() / "raw_customers.csv", - duckdb_path=Path(os.environ["AIRFLOW_HOME"]) / "jaffle_shop.duckdb", - names=["id", "first_name", "last_name"], - duckdb_schema="raw_data", - duckdb_database_name="jaffle_shop", - ), - ), - dbt_project_assets, - export_duckdb_to_csv_defs( - AssetSpec(key="customers_csv", deps=["customers"]), - ExportDuckDbToCsvArgs( - table_name="customers", - csv_path=Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "customers.csv", - duckdb_path=Path(os.environ["AIRFLOW_HOME"]) / "jaffle_shop.duckdb", - duckdb_database_name="jaffle_shop", - ), - ), -] - - -@asset_check(asset=AssetKey(["customers_csv"])) -def validate_exported_csv() -> AssetCheckResult: - csv_path = Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "customers.csv" - - if not csv_path.exists(): - return AssetCheckResult(passed=False, description=f"Export CSV {csv_path} does not exist") - - rows = len(csv_path.read_text().split("\n")) - if rows < 2: - return AssetCheckResult( - passed=False, - description=f"Export CSV {csv_path} is empty", - severity=AssetCheckSeverity.WARN, - ) - - return AssetCheckResult( - passed=True, - description=f"Export CSV {csv_path} exists", - metadata={"rows": rows}, - ) - - -rebuild_customer_list_schedule = rebuild_customers_list_schedule = ScheduleDefinition( - name="rebuild_customers_list_schedule", - target=AssetSelection.assets(*assets), - cron_schedule="0 0 * * *", -) - - -defs = Definitions( - assets=assets, - schedules=[rebuild_customer_list_schedule], - asset_checks=[validate_exported_csv], - resources={"dbt": DbtCliResource(project_dir=dbt_project_path())}, -) -``` - -## Addendum: Adding asset checks - -Once you have peered your Airflow DAGs in Dagster, regardless of migration progress, you can begin to add asset checks to your Dagster code. Asset checks can be used to validate the quality of your data assets, and can provide additional observability and value on top of your Airflow DAG even before migration starts. - -For example, given a peered version of our DAG, we can add an asset check to ensure that the final `customers` CSV output exists and has a non-zero number of rows: - -```python file=../../experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/peer_with_check.py -import os -from pathlib import Path - -from dagster import AssetCheckResult, AssetCheckSeverity, AssetKey, Definitions, asset_check -from dagster_airlift.core import ( - AirflowBasicAuthBackend, - AirflowInstance, - build_defs_from_airflow_instance, -) - - -# Attach a check to the DAG representation asset, which will be executed by Dagster -# any time the DAG is run in Airflow -@asset_check(asset=AssetKey(["airflow_instance_one", "dag", "rebuild_customers_list"])) -def validate_exported_csv() -> AssetCheckResult: - csv_path = Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "customers.csv" - - if not csv_path.exists(): - return AssetCheckResult(passed=False, description=f"Export CSV {csv_path} does not exist") - - rows = len(csv_path.read_text().split("\n")) - if rows < 2: - return AssetCheckResult( - passed=False, - description=f"Export CSV {csv_path} is empty", - severity=AssetCheckSeverity.WARN, - ) - - return AssetCheckResult( - passed=True, - description=f"Export CSV {csv_path} exists", - metadata={"rows": rows}, - ) - - -defs = Definitions.merge( - build_defs_from_airflow_instance( - airflow_instance=AirflowInstance( - # other backends available (e.g. MwaaSessionAuthBackend) - auth_backend=AirflowBasicAuthBackend( - webserver_url="http://localhost:8080", - username="admin", - password="admin", - ), - name="airflow_instance_one", - ) - ), - Definitions(asset_checks=[validate_exported_csv]), -) -``` - -Once we have introduced representations of the assets produced by our Airflow tasks, we can directly attach asset checks to these assets. These checks will run once the corresponding task completes, regardless of whether the task is executed in Airflow or Dagster. - -
- -Asset checks on an observed or migrated DAG - - -```python file=../../experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/migrate_with_check.py -import os -from pathlib import Path - -from dagster import ( - AssetCheckResult, - AssetCheckSeverity, - AssetExecutionContext, - AssetKey, - AssetsDefinition, - AssetSpec, - Definitions, - asset_check, - materialize, - multi_asset, -) -from dagster_airlift.core import ( - AirflowBasicAuthBackend, - AirflowInstance, - assets_with_task_mappings, - build_defs_from_airflow_instance, -) -from dagster_dbt import DbtCliResource, DbtProject, dbt_assets - -# Code also invoked from Airflow -from tutorial_example.shared.export_duckdb_to_csv import ExportDuckDbToCsvArgs, export_duckdb_to_csv -from tutorial_example.shared.load_csv_to_duckdb import LoadCsvToDuckDbArgs, load_csv_to_duckdb - - -def dbt_project_path() -> Path: - env_val = os.getenv("TUTORIAL_DBT_PROJECT_DIR") - assert env_val, "TUTORIAL_DBT_PROJECT_DIR must be set" - return Path(env_val) - - -def airflow_dags_path() -> Path: - return Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "tutorial_example" / "airflow_dags" - - -def load_csv_to_duckdb_asset(spec: AssetSpec, args: LoadCsvToDuckDbArgs) -> AssetsDefinition: - @multi_asset(name=f"load_{args.table_name}", specs=[spec]) - def _multi_asset() -> None: - load_csv_to_duckdb(args) - - return _multi_asset - - -def export_duckdb_to_csv_defs(spec: AssetSpec, args: ExportDuckDbToCsvArgs) -> AssetsDefinition: - @multi_asset(name=f"export_{args.table_name}", specs=[spec]) - def _multi_asset() -> None: - export_duckdb_to_csv(args) - - return _multi_asset - - -@dbt_assets( - manifest=dbt_project_path() / "target" / "manifest.json", - project=DbtProject(dbt_project_path()), -) -def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): - yield from dbt.cli(["build"], context=context).stream() - - -mapped_assets = assets_with_task_mappings( - dag_id="rebuild_customers_list", - task_mappings={ - "load_raw_customers": [ - load_csv_to_duckdb_asset( - AssetSpec(key=["raw_data", "raw_customers"]), - LoadCsvToDuckDbArgs( - table_name="raw_customers", - csv_path=airflow_dags_path() / "raw_customers.csv", - duckdb_path=Path(os.environ["AIRFLOW_HOME"]) / "jaffle_shop.duckdb", - names=["id", "first_name", "last_name"], - duckdb_schema="raw_data", - duckdb_database_name="jaffle_shop", - ), - ) - ], - "build_dbt_models": - # load rich set of assets from dbt project - [dbt_project_assets], - "export_customers": [ - export_duckdb_to_csv_defs( - AssetSpec(key="customers_csv", deps=["customers"]), - ExportDuckDbToCsvArgs( - table_name="customers", - csv_path=Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "customers.csv", - duckdb_path=Path(os.environ["AIRFLOW_HOME"]) / "jaffle_shop.duckdb", - duckdb_database_name="jaffle_shop", - ), - ) - ], - }, -) - - -@asset_check(asset=AssetKey(["customers_csv"])) -def validate_exported_csv() -> AssetCheckResult: - csv_path = Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "customers.csv" - - if not csv_path.exists(): - return AssetCheckResult(passed=False, description=f"Export CSV {csv_path} does not exist") - - rows = len(csv_path.read_text().split("\n")) - if rows < 2: - return AssetCheckResult( - passed=False, - description=f"Export CSV {csv_path} is empty", - severity=AssetCheckSeverity.WARN, - ) - - return AssetCheckResult( - passed=True, - description=f"Export CSV {csv_path} exists", - metadata={"rows": rows}, - ) - - -defs = build_defs_from_airflow_instance( - airflow_instance=AirflowInstance( - auth_backend=AirflowBasicAuthBackend( - webserver_url="http://localhost:8080", - username="admin", - password="admin", - ), - name="airflow_instance_one", - ), - defs=Definitions( - assets=mapped_assets, - asset_checks=[validate_exported_csv], - resources={"dbt": DbtCliResource(project_dir=dbt_project_path())}, - ), -) - - -if __name__ == "__main__": - assert dbt_project_path().exists() - # print(dbt_project_path().absolute()) - Definitions.validate_loadable(defs) - materialize(defs.get_asset_graph().assets_defs) -``` - -
- -## Migrating an entire DAG at once - -There may be DAGs for which you want to migrate the entire thing at once rather than on a per-task basis. Some reasons for taking this approach: - -- You're making use of "dynamic tasks" in Airflow, which don't conform neatly to the task mapping protocol we've laid out above. -- You want to make more substantial refactors to the dag structure that don't conform to the existing task structure - -For cases like this, we allow you to map assets to a full DAG. - -### Observing DAG-mapped assets - -At the observation stage, you'll call `assets_with_dag_mappings` instead of `assets_with_task_mappings`. - -For our `rebuild_customers_list` DAG, let's take a look at what the new observation code looks like: - -```python file=../../experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/observe_dag_level.py -import os -from pathlib import Path - -from dagster import AssetExecutionContext, AssetSpec, Definitions -from dagster_airlift.core import ( - AirflowBasicAuthBackend, - AirflowInstance, - assets_with_dag_mappings, - build_defs_from_airflow_instance, -) -from dagster_dbt import DbtCliResource, DbtProject, dbt_assets - - -def dbt_project_path() -> Path: - env_val = os.getenv("TUTORIAL_DBT_PROJECT_DIR") - assert env_val, "TUTORIAL_DBT_PROJECT_DIR must be set" - return Path(env_val) - - -@dbt_assets( - manifest=dbt_project_path() / "target" / "manifest.json", - project=DbtProject(dbt_project_path()), -) -def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): - yield from dbt.cli(["build"], context=context).stream() - - -# Instead of mapping assets to individual tasks, we map them to the entire DAG. -mapped_assets = assets_with_dag_mappings( - dag_mappings={ - "rebuild_customers_list": [ - AssetSpec(key=["raw_data", "raw_customers"]), - dbt_project_assets, - AssetSpec(key="customers_csv", deps=["customers"]), - ], - }, -) - - -defs = build_defs_from_airflow_instance( - airflow_instance=AirflowInstance( - auth_backend=AirflowBasicAuthBackend( - webserver_url="http://localhost:8080", - username="admin", - password="admin", - ), - name="airflow_instance_one", - ), - defs=Definitions( - assets=mapped_assets, - resources={"dbt": DbtCliResource(project_dir=dbt_project_path())}, - ), -) -``` - -Now, instead of getting a materialization when a particular task completes, each mapped asset will receive a materialization when the entire DAG completes. - -### Migrating DAG-mapped assets - -Recall that in the task-by-task migration step, we "proxy" execution on a task by task basis, which is controlled by a yaml document. For DAG-mapped assets, execution is proxied on a per-DAG basis. Proxying execution to Dagster will require all assets mapped to that DAG be _executable_ within Dagster. Let's take a look at some fully migrated code mapped to DAGs instead of tasks: - -```python file=../../experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/migrate_dag_level.py -import os -from pathlib import Path - -from dagster import ( - AssetExecutionContext, - AssetsDefinition, - AssetSpec, - Definitions, - materialize, - multi_asset, -) -from dagster_airlift.core import ( - AirflowBasicAuthBackend, - AirflowInstance, - assets_with_dag_mappings, - build_defs_from_airflow_instance, -) -from dagster_dbt import DbtCliResource, DbtProject, dbt_assets - -# Code also invoked from Airflow -from tutorial_example.shared.export_duckdb_to_csv import ExportDuckDbToCsvArgs, export_duckdb_to_csv -from tutorial_example.shared.load_csv_to_duckdb import LoadCsvToDuckDbArgs, load_csv_to_duckdb - - -def dbt_project_path() -> Path: - env_val = os.getenv("TUTORIAL_DBT_PROJECT_DIR") - assert env_val, "TUTORIAL_DBT_PROJECT_DIR must be set" - return Path(env_val) - - -def airflow_dags_path() -> Path: - return Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "tutorial_example" / "airflow_dags" - - -def load_csv_to_duckdb_asset(spec: AssetSpec, args: LoadCsvToDuckDbArgs) -> AssetsDefinition: - @multi_asset(name=f"load_{args.table_name}", specs=[spec]) - def _multi_asset() -> None: - load_csv_to_duckdb(args) - - return _multi_asset - - -def export_duckdb_to_csv_defs(spec: AssetSpec, args: ExportDuckDbToCsvArgs) -> AssetsDefinition: - @multi_asset(name=f"export_{args.table_name}", specs=[spec]) - def _multi_asset() -> None: - export_duckdb_to_csv(args) - - return _multi_asset - - -@dbt_assets( - manifest=dbt_project_path() / "target" / "manifest.json", - project=DbtProject(dbt_project_path()), -) -def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): - yield from dbt.cli(["build"], context=context).stream() - - -mapped_assets = assets_with_dag_mappings( - dag_mappings={ - "rebuild_customers_list": [ - load_csv_to_duckdb_asset( - AssetSpec(key=["raw_data", "raw_customers"]), - LoadCsvToDuckDbArgs( - table_name="raw_customers", - csv_path=airflow_dags_path() / "raw_customers.csv", - duckdb_path=Path(os.environ["AIRFLOW_HOME"]) / "jaffle_shop.duckdb", - names=["id", "first_name", "last_name"], - duckdb_schema="raw_data", - duckdb_database_name="jaffle_shop", - ), - ), - dbt_project_assets, - export_duckdb_to_csv_defs( - AssetSpec(key="customers_csv", deps=["customers"]), - ExportDuckDbToCsvArgs( - table_name="customers", - csv_path=Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "customers.csv", - duckdb_path=Path(os.environ["AIRFLOW_HOME"]) / "jaffle_shop.duckdb", - duckdb_database_name="jaffle_shop", - ), - ), - ], - }, -) - - -defs = build_defs_from_airflow_instance( - airflow_instance=AirflowInstance( - auth_backend=AirflowBasicAuthBackend( - webserver_url="http://localhost:8080", - username="admin", - password="admin", - ), - name="airflow_instance_one", - ), - defs=Definitions( - assets=mapped_assets, - resources={"dbt": DbtCliResource(project_dir=dbt_project_path())}, - ), -) - - -if __name__ == "__main__": - assert dbt_project_path().exists() - # print(dbt_project_path().absolute()) - Definitions.validate_loadable(defs) - materialize(defs.get_asset_graph().assets_defs) -``` - -Now that all of our assets are fully executable, we can create a simple yaml file to proxy execution for the whole dag: - -```yaml file=../../experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/rebuild_customers_list.yaml -proxied: True -``` - -We will similarly use `proxying_to_dagster` at the end of our DAG file (the code is exactly the same here as it was for the per-task migration step) - -```python file=../../experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/dags_truncated.py -# Dags file can be found at tutorial_example/airflow_dags/dags.py -from pathlib import Path - -from airflow import DAG -from dagster_airlift.in_airflow import proxying_to_dagster -from dagster_airlift.in_airflow.proxied_state import load_proxied_state_from_yaml - -dag = DAG("rebuild_customers_list", ...) - -... - -# Set this to True to begin the proxying process -PROXYING = False - -if PROXYING: - proxying_to_dagster( - global_vars=globals(), - proxied_state=load_proxied_state_from_yaml(Path(__file__).parent / "proxied_state"), - ) -``` - -Once the `proxied` bit is flipped to True, we can go to the Airflow UI, and we'll see that our tasks have been replaced with a single task. - -

- - - -

- -When performing dag-level mapping, we don't preserve task structure in the Airflow dags. This single task will materialize all mapped Dagster assets instead of executing the original Airflow task business logic. - -We can similarly mark `proxied` back to `False`, and the original task structure and business logic will return unchanged. - -### Customizing DAG proxying operator - -Similar to how we can customize the operator we construct on a per-dag basis, we can customize the operator we construct on a per-dag basis. We can use the `build_from_dag_fn` argument of `proxying_to_dagster` to provide a custom operator in place of the default. - -For example, let's take a look at the following custom operator which expects an API key to be provided as a variable: - -```python file=../../experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/custom_operator_examples/custom_dag_level_proxy.py -from pathlib import Path - -import requests -from airflow import DAG -from airflow.utils.context import Context -from dagster_airlift.in_airflow import BaseProxyDAGToDagsterOperator, proxying_to_dagster -from dagster_airlift.in_airflow.proxied_state import load_proxied_state_from_yaml - - -class CustomProxyToDagsterOperator(BaseProxyDAGToDagsterOperator): - def get_dagster_session(self, context: Context) -> requests.Session: - if "var" not in context: - raise ValueError("No variables found in context") - api_key = context["var"]["value"].get("my_api_key") - session = requests.Session() - session.headers.update({"Authorization": f"Bearer {api_key}"}) - return session - - def get_dagster_url(self, context: Context) -> str: - return "https://dagster.example.com/" - - # This method controls how the operator is built from the dag. - @classmethod - def build_from_dag(cls, dag: DAG): - return CustomProxyToDagsterOperator(dag=dag, task_id="OVERRIDDEN") - - -dag = DAG( - dag_id="custom_dag_level_proxy_example", -) - -# At the end of your dag file -proxying_to_dagster( - global_vars=globals(), - proxied_state=load_proxied_state_from_yaml(Path(__file__).parent / "proxied_state"), - build_from_dag_fn=CustomProxyToDagsterOperator.build_from_dag, -) -``` - -`BaseProxyDAGToDagsterOperator` has three abstract methods which must be implemented: - -- `get_dagster_session`, which controls the creation of a valid session to access the Dagster graphql API. -- `get_dagster_url`, which retrieves the domain at which the dagster webserver lives. -- `build_from_dag`, which controls how the proxying task is constructed from the provided DAG. - -## Addendum: Dealing with changing Airflow - -In order to make spin-up more efficient, `dagster-airlift` caches the state of the Airflow instance in the dagster database, so that repeat fetches of the code location don't require additional calls to Airflow's rest API. However, this means that the Dagster definitions can potentially fall out of sync with Airflow. Here are a few different ways this can manifest: - -- A new Airflow dag is added. The lineage information does not show up for this dag, and materializations are not recorded. -- A dag is removed. The polling sensor begins failing, because there exist assets which expect that dag to exist. -- The task dependency structure within a dag changes. This may result in `unsynced` statuses in Dagster, or missing materializations. This is not an exhaustive list of problems, but most of the time the tell is that materializations are missing, or assets are missing. When you find yourself in this state, you can force `dagster-airlift` to reload Airflow state by reloading the code location. To do this, go to the `Deployment` tab on the top nav, and click `Redeploy` on the code location relevant to your asset. After some time, the code location should be reloaded with refreshed state from Airflow. - -### Automating changes to code locations - -If changes to your Airflow instance are controlled via a ci/cd process, you can add a step to automatically induce a redeploy of the relevant code location. See the docs [here](https://docs.dagster.io/concepts/webserver/graphql-client#reloading-all-repositories-in-a-repository-location) on using the graphql client to do this. diff --git a/docs/content/integrations/airlift/tutorial/decomission.mdx b/docs/content/integrations/airlift/tutorial/decomission.mdx new file mode 100644 index 0000000000000..4c60b4194ec5c --- /dev/null +++ b/docs/content/integrations/airlift/tutorial/decomission.mdx @@ -0,0 +1,132 @@ +# Decomissioning an Airflow DAG + +Previously, we completed migration of our Airflow DAG to Dagster assets. If you haven't finished that stage yet, please follow along [here](/integrations/airlift/tutorial/migrate). + +Once we are confident in our migrated versions of the tasks, we can decommission the Airflow DAG. First, we can remove the DAG from our Airflow DAG directory. + +Next, we can strip the task associations from our Dagster definitions. This can be done by removing the `assets_with_task_mappings` call. We can use this opportunity to attach our assets to a `ScheduleDefinition` so that Dagster's scheduler can manage their execution: + +```python file=../../experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/standalone.py +import os +from pathlib import Path + +from dagster import ( + AssetCheckResult, + AssetCheckSeverity, + AssetExecutionContext, + AssetKey, + AssetsDefinition, + AssetSelection, + AssetSpec, + DailyPartitionsDefinition, + Definitions, + ScheduleDefinition, + asset_check, + multi_asset, +) +from dagster._time import get_current_datetime_midnight +from dagster_dbt import DbtCliResource, DbtProject, dbt_assets + +# Code also invoked from Airflow +from tutorial_example.shared.export_duckdb_to_csv import ExportDuckDbToCsvArgs, export_duckdb_to_csv +from tutorial_example.shared.load_csv_to_duckdb import LoadCsvToDuckDbArgs, load_csv_to_duckdb + +PARTITIONS_DEF = DailyPartitionsDefinition(start_date=get_current_datetime_midnight()) + + +def dbt_project_path() -> Path: + env_val = os.getenv("TUTORIAL_DBT_PROJECT_DIR") + assert env_val, "TUTORIAL_DBT_PROJECT_DIR must be set" + return Path(env_val) + + +def airflow_dags_path() -> Path: + return Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "tutorial_example" / "airflow_dags" + + +def load_csv_to_duckdb_asset(spec: AssetSpec, args: LoadCsvToDuckDbArgs) -> AssetsDefinition: + @multi_asset(name=f"load_{args.table_name}", specs=[spec]) + def _multi_asset() -> None: + load_csv_to_duckdb(args) + + return _multi_asset + + +def export_duckdb_to_csv_defs(spec: AssetSpec, args: ExportDuckDbToCsvArgs) -> AssetsDefinition: + @multi_asset(name=f"export_{args.table_name}", specs=[spec]) + def _multi_asset() -> None: + export_duckdb_to_csv(args) + + return _multi_asset + + +@dbt_assets( + manifest=dbt_project_path() / "target" / "manifest.json", + project=DbtProject(dbt_project_path()), + partitions_def=PARTITIONS_DEF, +) +def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): + yield from dbt.cli(["build"], context=context).stream() + + +assets = [ + load_csv_to_duckdb_asset( + AssetSpec(key=["raw_data", "raw_customers"], partitions_def=PARTITIONS_DEF), + LoadCsvToDuckDbArgs( + table_name="raw_customers", + csv_path=airflow_dags_path() / "raw_customers.csv", + duckdb_path=Path(os.environ["AIRFLOW_HOME"]) / "jaffle_shop.duckdb", + names=["id", "first_name", "last_name"], + duckdb_schema="raw_data", + duckdb_database_name="jaffle_shop", + ), + ), + dbt_project_assets, + export_duckdb_to_csv_defs( + AssetSpec(key="customers_csv", deps=["customers"], partitions_def=PARTITIONS_DEF), + ExportDuckDbToCsvArgs( + table_name="customers", + csv_path=Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "customers.csv", + duckdb_path=Path(os.environ["AIRFLOW_HOME"]) / "jaffle_shop.duckdb", + duckdb_database_name="jaffle_shop", + ), + ), +] + + +@asset_check(asset=AssetKey(["customers_csv"])) +def validate_exported_csv() -> AssetCheckResult: + csv_path = Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "customers.csv" + + if not csv_path.exists(): + return AssetCheckResult(passed=False, description=f"Export CSV {csv_path} does not exist") + + rows = len(csv_path.read_text().split("\n")) + if rows < 2: + return AssetCheckResult( + passed=False, + description=f"Export CSV {csv_path} is empty", + severity=AssetCheckSeverity.WARN, + ) + + return AssetCheckResult( + passed=True, + description=f"Export CSV {csv_path} exists", + metadata={"rows": rows}, + ) + + +rebuild_customer_list_schedule = rebuild_customers_list_schedule = ScheduleDefinition( + name="rebuild_customers_list_schedule", + target=AssetSelection.assets(*assets), + cron_schedule="0 0 * * *", +) + + +defs = Definitions( + assets=assets, + schedules=[rebuild_customer_list_schedule], + asset_checks=[validate_exported_csv], + resources={"dbt": DbtCliResource(project_dir=dbt_project_path())}, +) +``` diff --git a/docs/content/integrations/airlift/tutorial/migrate.mdx b/docs/content/integrations/airlift/tutorial/migrate.mdx new file mode 100644 index 0000000000000..723824910fda0 --- /dev/null +++ b/docs/content/integrations/airlift/tutorial/migrate.mdx @@ -0,0 +1,247 @@ +# Airflow Migration Tutorial: Migrating Assets + +Previously, we completed the ["Observe" stage](/integrations/airlift/tutorial/observe) of the Airflow migration process by encoding the assets that are produced by each task. We also introduced partitioning to those assets. + +Up until now, we have not touched the Airflow code base at all. Now it's time to begin the actual migration process. + +Once you have created corresponding definitions in Dagster to your Airflow tasks, you can proxy execution to Dagster on a per-task basis while Airflow is still controlling scheduling and orchestration. Once a task has been proxied, Airflow will kick off materializations of corresponding Dagster assets in place of executing the business logic of that task. + +To begin proxying tasks in a DAG, first you will need a file to track proxying state. In your Airflow DAG directory, create a `proxied_state` folder, and in it create a yaml file with the same name as your DAG. The included example at `airflow_dags/proxied_state` is used by `make airflow_run`, and can be used as a template for your own proxied state files. + +Given our example DAG `rebuild_customers_list` with three tasks, `load_raw_customers`, `run_dbt_model`, and `export_customers`, `proxied_state/rebuild_customers_list.yaml` should look like the following: + +```yaml file=../../experimental/dagster-airlift/examples/tutorial-example/tutorial_example/airflow_dags/proxied_state/rebuild_customers_list.yaml +tasks: + - id: load_raw_customers + proxied: False + - id: build_dbt_models + proxied: False + - id: export_customers + proxied: False +``` + +Next, you will need to modify your Airflow DAG to make it aware of the proxied state. This is already done in the example DAG: + +```python file=../../experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/dags_truncated.py +# Dags file can be found at tutorial_example/airflow_dags/dags.py +from pathlib import Path + +from airflow import DAG +from dagster_airlift.in_airflow import proxying_to_dagster +from dagster_airlift.in_airflow.proxied_state import load_proxied_state_from_yaml + +dag = DAG("rebuild_customers_list", ...) + +... + +# Set this to True to begin the proxying process +PROXYING = False + +if PROXYING: + proxying_to_dagster( + global_vars=globals(), + proxied_state=load_proxied_state_from_yaml(Path(__file__).parent / "proxied_state"), + ) +``` + +Set `PROXYING` to `True` or eliminate the `if` statement. + +The DAG will now display its proxied state in the Airflow UI. (There is some latency as Airflow evaluates the Python file periodically.) + +

+ + + +

+ +### Migrating individual tasks + +In order to proxy a task, you must do two things: + +1. First, ensure all associated assets are executable in Dagster by providing asset definitions in place of bare asset specs. +2. The `proxied: False` status in the `proxied_state` YAML folder must be adjusted to `proxied: True`. + +Any task marked as proxied will use the `DefaultProxyTaskToDagsterOperator` when executed as part of the DAG. This operator will use the Dagster GraphQL API to initiate a Dagster run of the assets corresponding to the task. + +The proxied file acts as the source of truth for proxied state. The information is attached to the DAG and then accessed by Dagster via the REST API. + +A task which has been proxied can be easily toggled back to run in Airflow (for example, if a bug in implementation was encountered) simply by editing the file to `proxied: False`. + +#### Migrating common operators + +For some common operator patterns, like our dbt operator, Dagster supplies factories to build software defined assets for our tasks. In fact, the `@dbt_assets` decorator used earlier already backs its assets with definitions, so we can toggle the proxied state of the `build_dbt_models` task to `proxied: True` in the proxied state file: + +```yaml file=../../experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/dbt_proxied.yaml +tasks: + - id: load_raw_customers + proxied: False + - id: build_dbt_models + proxied: True + - id: export_customers + proxied: False +``` + +**Important**: It may take up to 30 seconds for the proxied state in the Airflow UI to reflect this change. You must subsequently reload the definitions in Dagster via the UI or by restarting `dagster dev`. + +You can now run the `rebuild_customers_list` DAG in Airflow, and the `build_dbt_models` task will be executed in a Dagster run: + +

+ + + +

+ +You'll note that we proxied a task in the _middle_ of the Airflow DAG. The Airflow DAG structure and execution history is stable in the Airflow UI, but execution of `build_dbt_models` has moved to Dagster. + +#### Migrating the remaining custom operators + +For all other operator types, we will need to build our own asset definitions. We recommend creating a factory function whose arguments match the inputs to your Airflow operator. Then, you can use this factory to build definitions for each Airflow task. + +For example, our `load_raw_customers` task uses a custom `LoadCSVToDuckDB` operator. We'll define a function `load_csv_to_duckdb_defs` factory to build corresponding software-defined assets. Similarly for `export_customers` we'll define a function `export_duckdb_to_csv_defs` to build SDAs: + +```python file=../../experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/migrate.py +import os +from pathlib import Path + +from dagster import ( + AssetExecutionContext, + AssetsDefinition, + AssetSpec, + DailyPartitionsDefinition, + Definitions, + materialize, + multi_asset, +) +from dagster._time import get_current_datetime_midnight +from dagster_airlift.core import ( + AirflowBasicAuthBackend, + AirflowInstance, + assets_with_task_mappings, + build_defs_from_airflow_instance, +) +from dagster_dbt import DbtCliResource, DbtProject, dbt_assets + +# Code also invoked from Airflow +from tutorial_example.shared.export_duckdb_to_csv import ExportDuckDbToCsvArgs, export_duckdb_to_csv +from tutorial_example.shared.load_csv_to_duckdb import LoadCsvToDuckDbArgs, load_csv_to_duckdb + +PARTITIONS_DEF = DailyPartitionsDefinition(start_date=get_current_datetime_midnight()) + + +def dbt_project_path() -> Path: + env_val = os.getenv("TUTORIAL_DBT_PROJECT_DIR") + assert env_val, "TUTORIAL_DBT_PROJECT_DIR must be set" + return Path(env_val) + + +def airflow_dags_path() -> Path: + return Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "tutorial_example" / "airflow_dags" + + +def load_csv_to_duckdb_asset(spec: AssetSpec, args: LoadCsvToDuckDbArgs) -> AssetsDefinition: + @multi_asset(name=f"load_{args.table_name}", specs=[spec]) + def _multi_asset() -> None: + load_csv_to_duckdb(args) + + return _multi_asset + + +def export_duckdb_to_csv_defs(spec: AssetSpec, args: ExportDuckDbToCsvArgs) -> AssetsDefinition: + @multi_asset(name=f"export_{args.table_name}", specs=[spec]) + def _multi_asset() -> None: + export_duckdb_to_csv(args) + + return _multi_asset + + +@dbt_assets( + manifest=dbt_project_path() / "target" / "manifest.json", + project=DbtProject(dbt_project_path()), + partitions_def=PARTITIONS_DEF, +) +def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): + yield from dbt.cli(["build"], context=context).stream() + + +mapped_assets = assets_with_task_mappings( + dag_id="rebuild_customers_list", + task_mappings={ + "load_raw_customers": [ + load_csv_to_duckdb_asset( + AssetSpec(key=["raw_data", "raw_customers"], partitions_def=PARTITIONS_DEF), + LoadCsvToDuckDbArgs( + table_name="raw_customers", + csv_path=airflow_dags_path() / "raw_customers.csv", + duckdb_path=Path(os.environ["AIRFLOW_HOME"]) / "jaffle_shop.duckdb", + names=["id", "first_name", "last_name"], + duckdb_schema="raw_data", + duckdb_database_name="jaffle_shop", + ), + ) + ], + "build_dbt_models": + # load rich set of assets from dbt project + [dbt_project_assets], + "export_customers": [ + export_duckdb_to_csv_defs( + AssetSpec(key="customers_csv", deps=["customers"], partitions_def=PARTITIONS_DEF), + ExportDuckDbToCsvArgs( + table_name="customers", + csv_path=Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "customers.csv", + duckdb_path=Path(os.environ["AIRFLOW_HOME"]) / "jaffle_shop.duckdb", + duckdb_database_name="jaffle_shop", + ), + ) + ], + }, +) + + +defs = build_defs_from_airflow_instance( + airflow_instance=AirflowInstance( + auth_backend=AirflowBasicAuthBackend( + webserver_url="http://localhost:8080", + username="admin", + password="admin", + ), + name="airflow_instance_one", + ), + defs=Definitions( + assets=mapped_assets, + resources={"dbt": DbtCliResource(project_dir=dbt_project_path())}, + ), +) + + +if __name__ == "__main__": + assert dbt_project_path().exists() + # print(dbt_project_path().absolute()) + Definitions.validate_loadable(defs) + materialize(defs.get_asset_graph().assets_defs) +``` + +We can then toggle the proxied state of the remaining tasks in the `proxied_state` file: + +```yaml file=../../experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/all_proxied.yaml +tasks: + - id: load_raw_customers + proxied: True + - id: build_dbt_models + proxied: True + - id: export_customers + proxied: True +``` + +## Next steps + +Now that we've completed migration of the Airflow DAG, we can decomission it. Follow along [here](/integrations/airlift/tutorial/decomission) diff --git a/docs/content/integrations/airlift/tutorial/observe.mdx b/docs/content/integrations/airlift/tutorial/observe.mdx new file mode 100644 index 0000000000000..c6250d8f8a74b --- /dev/null +++ b/docs/content/integrations/airlift/tutorial/observe.mdx @@ -0,0 +1,356 @@ +# Airflow Migration Tutorial: Observing Assets + +Previously, we completed the ["Peering" stage](/integrations/airlift/tutorial/peer) of the Airflow migration process by peering the Airflow instance with a Dagster code location. + +The next step is to represent our Airflow workflows more richly by observing the data assets that are produced by our tasks. Similar to the peering step, this stage does not require _any changes_ to Airflow code. + +In order to do this, we must define the relevant assets in the Dagster code location. + +In our example, we have three sequential tasks: + +1. `load_raw_customers` loads a CSV file of raw customer data into duckdb. +2. `run_dbt_model` builds a series of dbt models (from [jaffle shop](https://github.com/dbt-labs/jaffle_shop_duckdb)) combining customer, order, and payment data. +3. `export_customers` exports a CSV representation of the final customer file from duckdb to disk. + +We will first create a set of asset specs that correspond to the assets produced by these tasks. We will then annotate these asset specs so that Dagster can associate them with the Airflow tasks that produce them. + +The first and third tasks involve a single table each. We can manually construct specs for these two tasks. Dagster provides the `assets_with_task_mappings` utility to annotate our asset specs with the tasks that produce them. Assets which are properly annotated will be materialized by the Airlift sensor once the corresponding task completes: These annotated specs are then provided to the `defs` argument to `build_defs_from_airflow_instance`. + +We will also create a set of dbt asset definitions for the `build_dbt_models` task. We can use the `dagster-dbt`-supplied decorator `@dbt_assets` to generate these definitions using Dagster's dbt integration. + +First, you need to install the extra that has the dbt factory: + +```bash +uv pip install 'dagster-airlift[dbt]' +``` + +Then, we will construct our assets: + +```python file=../../experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/observe.py +import os +from pathlib import Path + +from dagster import ( + AssetCheckResult, + AssetCheckSeverity, + AssetExecutionContext, + AssetKey, + AssetSpec, + Definitions, + asset_check, +) +from dagster_airlift.core import ( + AirflowBasicAuthBackend, + AirflowInstance, + assets_with_task_mappings, + build_defs_from_airflow_instance, +) +from dagster_dbt import DbtCliResource, DbtProject, dbt_assets + + +@asset_check(asset=AssetKey(["airflow_instance_one", "dag", "rebuild_customers_list"])) +def validate_exported_csv() -> AssetCheckResult: + csv_path = Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "customers.csv" + + if not csv_path.exists(): + return AssetCheckResult(passed=False, description=f"Export CSV {csv_path} does not exist") + + rows = len(csv_path.read_text().split("\n")) + if rows < 2: + return AssetCheckResult( + passed=False, + description=f"Export CSV {csv_path} is empty", + severity=AssetCheckSeverity.WARN, + ) + + return AssetCheckResult( + passed=True, + description=f"Export CSV {csv_path} exists", + metadata={"rows": rows}, + ) + + +def dbt_project_path() -> Path: + env_val = os.getenv("TUTORIAL_DBT_PROJECT_DIR") + assert env_val, "TUTORIAL_DBT_PROJECT_DIR must be set" + return Path(env_val) + + +@dbt_assets( + manifest=dbt_project_path() / "target" / "manifest.json", + project=DbtProject(dbt_project_path()), +) +def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): + yield from dbt.cli(["build"], context=context).stream() + + +mapped_assets = assets_with_task_mappings( + dag_id="rebuild_customers_list", + task_mappings={ + "load_raw_customers": [AssetSpec(key=["raw_data", "raw_customers"])], + "build_dbt_models": [dbt_project_assets], + "export_customers": [AssetSpec(key="customers_csv", deps=["customers"])], + }, +) + + +defs = build_defs_from_airflow_instance( + airflow_instance=AirflowInstance( + auth_backend=AirflowBasicAuthBackend( + webserver_url="http://localhost:8080", + username="admin", + password="admin", + ), + name="airflow_instance_one", + ), + defs=Definitions( + assets=mapped_assets, + resources={"dbt": DbtCliResource(project_dir=dbt_project_path())}, + asset_checks=[validate_exported_csv], + ), +) +``` + +### Viewing observed assets + +Once your assets are set up, you should be able to reload your Dagster definitions and see a full representation of the dbt project and other data assets in your code. + +

+ + + +

+ +Kicking off a run of the DAG in Airflow, you should see the newly created assets materialize in Dagster as each task completes. + +_Note: There will be some delay between task completion and assets materializing in Dagster, managed by the sensor. This sensor runs every 30 seconds by default (you can reduce down to one second via the `minimum_interval_seconds` argument to `sensor`)._ + +### Moving the asset check + +Now that we've introduced an asset explicitly for the `customers.csv` file output by the DAG, we should move the asset check constructed during the Peering step to instead be on the `customers_csv` asset. Simply change the `asset` targeted by the `@asset_check` decorator to be `AssetKey(["customers_csv"])`. Doing this ensures that even when we delete the DAG, the asset check will live on. + +When done, our code will look like this. + +```python file=../../experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/observe_check_on_asset.py +import os +from pathlib import Path + +from dagster import ( + AssetCheckResult, + AssetCheckSeverity, + AssetExecutionContext, + AssetKey, + AssetSpec, + Definitions, + asset_check, +) +from dagster_airlift.core import ( + AirflowBasicAuthBackend, + AirflowInstance, + assets_with_task_mappings, + build_defs_from_airflow_instance, +) +from dagster_dbt import DbtCliResource, DbtProject, dbt_assets + + +@asset_check(asset=AssetKey(["customers_csv"])) +def validate_exported_csv() -> AssetCheckResult: + csv_path = Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "customers.csv" + + if not csv_path.exists(): + return AssetCheckResult(passed=False, description=f"Export CSV {csv_path} does not exist") + + rows = len(csv_path.read_text().split("\n")) + if rows < 2: + return AssetCheckResult( + passed=False, + description=f"Export CSV {csv_path} is empty", + severity=AssetCheckSeverity.WARN, + ) + + return AssetCheckResult( + passed=True, + description=f"Export CSV {csv_path} exists", + metadata={"rows": rows}, + ) + + +def dbt_project_path() -> Path: + env_val = os.getenv("TUTORIAL_DBT_PROJECT_DIR") + assert env_val, "TUTORIAL_DBT_PROJECT_DIR must be set" + return Path(env_val) + + +@dbt_assets( + manifest=dbt_project_path() / "target" / "manifest.json", + project=DbtProject(dbt_project_path()), +) +def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): + yield from dbt.cli(["build"], context=context).stream() + + +mapped_assets = assets_with_task_mappings( + dag_id="rebuild_customers_list", + task_mappings={ + "load_raw_customers": [AssetSpec(key=["raw_data", "raw_customers"])], + "build_dbt_models": [dbt_project_assets], + "export_customers": [AssetSpec(key="customers_csv", deps=["customers"])], + }, +) + + +defs = build_defs_from_airflow_instance( + airflow_instance=AirflowInstance( + auth_backend=AirflowBasicAuthBackend( + webserver_url="http://localhost:8080", + username="admin", + password="admin", + ), + name="airflow_instance_one", + ), + defs=Definitions( + assets=mapped_assets, + resources={"dbt": DbtCliResource(project_dir=dbt_project_path())}, + asset_checks=[validate_exported_csv], + ), +) +``` + +### Adding partitions + +If your Airflow tasks produce time-partitioned assets, Airlift can automatically associate your materializations to the relevant partitions. In the case of `rebuild_customers_list`, data is daily partitioned in each created table, and and the Airflow DAG runs on a `@daily` cron schedule. We can likewise add a `DailyPartitionsDefinition` to each of our assets. + +```python file=../../experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/observe_with_partitions.py +import os +from pathlib import Path + +from dagster import ( + AssetCheckResult, + AssetCheckSeverity, + AssetExecutionContext, + AssetKey, + AssetSpec, + DailyPartitionsDefinition, + Definitions, + asset_check, +) +from dagster._time import get_current_datetime_midnight +from dagster_airlift.core import ( + AirflowBasicAuthBackend, + AirflowInstance, + assets_with_task_mappings, + build_defs_from_airflow_instance, +) +from dagster_dbt import DbtCliResource, DbtProject, dbt_assets + +PARTITIONS_DEF = DailyPartitionsDefinition(start_date=get_current_datetime_midnight()) + + +@asset_check(asset=AssetKey(["customers_csv"])) +def validate_exported_csv() -> AssetCheckResult: + csv_path = Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "customers.csv" + + if not csv_path.exists(): + return AssetCheckResult(passed=False, description=f"Export CSV {csv_path} does not exist") + + rows = len(csv_path.read_text().split("\n")) + if rows < 2: + return AssetCheckResult( + passed=False, + description=f"Export CSV {csv_path} is empty", + severity=AssetCheckSeverity.WARN, + ) + + return AssetCheckResult( + passed=True, + description=f"Export CSV {csv_path} exists", + metadata={"rows": rows}, + ) + + +def dbt_project_path() -> Path: + env_val = os.getenv("TUTORIAL_DBT_PROJECT_DIR") + assert env_val, "TUTORIAL_DBT_PROJECT_DIR must be set" + return Path(env_val) + + +@dbt_assets( + manifest=dbt_project_path() / "target" / "manifest.json", + project=DbtProject(dbt_project_path()), + partitions_def=PARTITIONS_DEF, +) +def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): + yield from dbt.cli(["build"], context=context).stream() + + +mapped_assets = assets_with_task_mappings( + dag_id="rebuild_customers_list", + task_mappings={ + "load_raw_customers": [ + AssetSpec(key=["raw_data", "raw_customers"], partitions_def=PARTITIONS_DEF) + ], + "build_dbt_models": [dbt_project_assets], + "export_customers": [ + AssetSpec(key="customers_csv", deps=["customers"], partitions_def=PARTITIONS_DEF) + ], + }, +) + + +defs = build_defs_from_airflow_instance( + airflow_instance=AirflowInstance( + auth_backend=AirflowBasicAuthBackend( + webserver_url="http://localhost:8080", + username="admin", + password="admin", + ), + name="airflow_instance_one", + ), + defs=Definitions( + assets=mapped_assets, + resources={"dbt": DbtCliResource(project_dir=dbt_project_path())}, + asset_checks=[validate_exported_csv], + ), +) +``` + +Now, every time the sensor synthesizes a materialization for an asset, it will automatically have a partition associated with it. + +Let's try this out by kicking off an airflow backfill for today: + +```bash +airflow dags backfill rebuild_customers_list --start-date $(date +"%Y-%m-%d") +``` + +After this dag run completes, you should see a partitioned materialization appear in Dagster. + +

+ + + +

+ +Let's clear our Airflow runs so that we can kick off this backfill again for testing in the future. + +```bash +airflow db clean +``` + +In order for partitioned assets to work out of the box with `dagster-airlift`, the following things need to be true: + +- The asset can only be time-window partitioned. This means static, dynamic, and multi partitioned definitions will require custom functionality. +- The partitioning scheme must match up with the [logical_date / execution_date](https://airflow.apache.org/docs/apache-airflow/stable/faq.html#what-does-execution-date-mean) of corresponding Airflow runs. That is, each logical_date should correspond \_exactly\_ to a partition in Dagster. + +## Next steps + +Next, it's time to begin migrating our Airflow DAG code to Dagster. Follow along with the Migrate step [here](/integrations/airlift/tutorial/migrate) diff --git a/docs/content/integrations/airlift/tutorial/overview.mdx b/docs/content/integrations/airlift/tutorial/overview.mdx new file mode 100644 index 0000000000000..f2383ca81eaf6 --- /dev/null +++ b/docs/content/integrations/airlift/tutorial/overview.mdx @@ -0,0 +1,50 @@ +# Airflow Migration Tutorial + +This tutorial demonstrates using `dagster-airlift` to migrate an Airflow DAG to Dagster. + +Using `dagster-airlift` we can + +- Observe Airflow DAGs and their execution history with no changes to Airflow code +- Model and observe assets orchestrated by Airflow with no changes to Airflow code +- Enable a migration process that + - Can be done task-by-task in any order with minimal coordination + - Has task-by-task rollback to reduce risk + - That retains Airflow DAG structure and execution history during the migration + +## Process + +This is a high level overview of the steps to migrate an Airflow DAG to Dagster: + +- **Peer** + - Observe an Airflow instance from within a Dagster Deployment via the Airflow REST API. + - This loads every Airflow DAG as an asset definition and creates a sensor that polls Airflow for execution history. +- **Observe** + - Add a mapping that maps the Airflow DAG and task id to a basket of definitions that you want to observe. (e.g. render the full lineage the dbt models an Airflow task orchestrates) + - The sensor used for peering also polls for task execution history, and adds materializations to an observed asset when its corresponding task successfully executes +- **Migrate** + - Selectively move execution of Airflow tasks to Dagster Software Defined Assets + +## Pages + + + + + + + + diff --git a/docs/content/integrations/airlift/tutorial/peer.mdx b/docs/content/integrations/airlift/tutorial/peer.mdx new file mode 100644 index 0000000000000..23b1a22777376 --- /dev/null +++ b/docs/content/integrations/airlift/tutorial/peer.mdx @@ -0,0 +1,179 @@ +# Airflow Migration Tutorial: Peering + +At this point, we should have finished the [setup](/integrations/airlift/tutorial/setup) step, and now we have the example code setup with a fresh virtual environment, and Airflow running locally. Now, we can start writing Dagster code. + +We call the first stage of migration from Airflow to Dagster the "Peering" stage, at which we will "peer" the Airflow instance with a Dagster code location, which will create an asset representation of each Airflow DAG that you can view in Dagster. This process does not require any changes to your Airflow instance. + +First, you will want a new shell and navigate to the same directory. You will need to set up the `dagster-airlift` package in your Dagster environment: + +```bash +source .venv/bin/activate +uv pip install 'dagster-airlift[core]' dagster-webserver dagster +``` + +Next, create a `Definitions` object using `build_defs_from_airflow_instance`. You can use the empty `tutorial_example/dagster_defs/definitions.py` file as a starting point: + +```python file=../../experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/peer.py +from dagster_airlift.core import ( + AirflowBasicAuthBackend, + AirflowInstance, + build_defs_from_airflow_instance, +) + +defs = build_defs_from_airflow_instance( + airflow_instance=AirflowInstance( + # other backends available (e.g. MwaaSessionAuthBackend) + auth_backend=AirflowBasicAuthBackend( + webserver_url="http://localhost:8080", + username="admin", + password="admin", + ), + name="airflow_instance_one", + ) +) +``` + +This function creates: + +- An external asset representing each DAG. This asset is marked as materialized whenever a DAG run completes. +- A sensor that polls the Airflow instance for operational information. This sensor is responsible for creating materializations when a DAG executes. The sensor must remain on in order to properly update execution status. + +Let's set up some environment variables, and then point Dagster to see the asset created from our Airflow DAG: + +```bash +# Set up environment variables to point to the examples/tutorial-example directory on your machine +export TUTORIAL_EXAMPLE_DIR=$(pwd) +export TUTORIAL_DBT_PROJECT_DIR="$TUTORIAL_EXAMPLE_DIR/tutorial_example/shared/dbt" +export AIRFLOW_HOME="$TUTORIAL_EXAMPLE_DIR/.airflow_home" +dagster dev -f tutorial_example/dagster_defs/definitions.py +``` + +

+ + + +

+ +Let's kick off a run of the `reubild_customers_list` DAG in Airflow. + +```bash +airflow dags backfill rebuild_customers_list --start-date $(shell date +"%Y-%m-%d") +``` + +When this run has completed in Airflow, we should be able to navigate to the Dagster UI, and see that the Dagster has registered a materialization corresponding to that successful run. + +

+ + + +

+ +Run the following command to clean the Airflow and Dagster run history (we just do this so we can run the same example backfill in the future). Under the hood, this just deletes runs from Airflow and asset materializations from Dagster. + +```bash +make clean +``` + +_Note: When the code location loads, Dagster will query the Airflow REST API in order to build a representation of your DAGs. In order for Dagster to reflect changes to your DAGs, you will need to reload your code location._ + +## Asset checks as User Acceptance Tests + +Once you have peered your Airflow DAGs in Dagster, regardless of migration progress, you can begin to add asset checks to your Dagster code. In Dagster, Asset checks can be used to validate the quality of your data assets, and can provide additional observability and value on top of your Airflow DAG even before migration starts. + +Asset checks can both act as useful _user acceptance tests_ to ensure that any migration steps taken are successful, as well as _outlive_ the migration itself. + +For example, we're going to add an asset check to ensure that the final `customers` CSV output exists, and has a nonzero number of rows. + +```python file=../../experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/peer_with_check.py +import os +from pathlib import Path + +from dagster import AssetCheckResult, AssetCheckSeverity, AssetKey, Definitions, asset_check +from dagster_airlift.core import ( + AirflowBasicAuthBackend, + AirflowInstance, + build_defs_from_airflow_instance, +) + + +# Attach a check to the DAG representation asset, which will be executed by Dagster +# any time the DAG is run in Airflow +@asset_check(asset=AssetKey(["airflow_instance_one", "dag", "rebuild_customers_list"])) +def validate_exported_csv() -> AssetCheckResult: + csv_path = Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "customers.csv" + + if not csv_path.exists(): + return AssetCheckResult(passed=False, description=f"Export CSV {csv_path} does not exist") + + rows = len(csv_path.read_text().split("\n")) + if rows < 2: + return AssetCheckResult( + passed=False, + description=f"Export CSV {csv_path} is empty", + severity=AssetCheckSeverity.WARN, + ) + + return AssetCheckResult( + passed=True, + description=f"Export CSV {csv_path} exists", + metadata={"rows": rows}, + ) + + +defs = build_defs_from_airflow_instance( + airflow_instance=AirflowInstance( + # other backends available (e.g. MwaaSessionAuthBackend) + auth_backend=AirflowBasicAuthBackend( + webserver_url="http://localhost:8080", + username="admin", + password="admin", + ), + name="airflow_instance_one", + ), + defs=Definitions(asset_checks=[validate_exported_csv]), +) +``` + +Once we reload the code location, we'll see a tab `checks` indicating the presence of an asset check on our `rebuild_customers_list` asset. + + + +Let's run the backfill again: + +```bash +airflow dags backfill rebuild_customers_list --start-date $(shell date +"%Y-%m-%d") +``` + +And we'll see that the asset check executed successfully in Dagster (indicated by the green check mark). + + + +Let's again wipe materializations and runs for tutorial purposes. + +```bash +make clean +``` + +## Next Steps + +The next step is to start observing the asset dependencies within your DAG. Follow along at the Observe stage of the tutorial [here](/integrations/airlift/tutorial/observe) diff --git a/docs/content/integrations/airlift/tutorial/setup.mdx b/docs/content/integrations/airlift/tutorial/setup.mdx new file mode 100644 index 0000000000000..8f42c1c09239a --- /dev/null +++ b/docs/content/integrations/airlift/tutorial/setup.mdx @@ -0,0 +1,61 @@ +# Airflow Migration Tutorial: Setup + +In this step, we'll + +- Install the example code +- Set up a local environment +- Ensure we can run Airflow locally. + +## Installation & Project Structure + +First, clone the tutorial example repo locally, and enter the repo directory. + +```bash +git clone git@github.com:dagster-io/airlift-tutorial.git +cd airlift-tutorial +``` + +Next, we'll create a fresh virtual environment using `uv`. + +```bash +pip install uv +uv venv +source .venv/bin/activate +``` + +## Running Airflow locally + +The tutorial example involves running a local Airflow instance. This can be done by running the following commands from the root of the `airlift-tutorial` directory. + +First, install the required python packages: + +```bash +make airflow_install +``` + +Next, scaffold the Airflow instance, and initialize the dbt project: + +```bash +make airflow_setup +``` + +Finally, run the Airflow instance with environment variables set: + +```bash +make airflow_run +``` + +This will run the Airflow Web UI in a shell. You should now be able to access the Airflow UI at `http://localhost:8080`, with the default username and password set to `admin`. + +You should be able to see the `rebuild_customers_list` DAG in the Airflow UI, made up of three tasks: `load_raw_customers`, `run_dbt_model`, and `export_customers`. + + + +## Next Steps + +The next step is to peer a Dagster installation with the Airflow Instance. [Click here](/integrations/airlift/tutorial/peer) to follow along for part 2. diff --git a/docs/next/public/images/integrations/airlift/asset_check_peered_dag.png b/docs/next/public/images/integrations/airlift/asset_check_peered_dag.png new file mode 100644 index 0000000000000000000000000000000000000000..a50f4e5e21a85809e55da10d7eea09a6e3b78427 GIT binary patch literal 11779 zcmZX41ymf(wk{A{g1Zmy8XQ7!26uONg1ZKHcXtmOEV#Q%aCZpq@P_}Kd(V0A&00NO z-Mh9{b=9|bcbL4aI1)S_JQx@llB9%)A{ZDr0MvGag$DhruD&OLfx%Z;2n)+g3Ja6S zJJ^|8Set->NrWY(z$hmyqxoIDmvCwd6{35f7orOSVHs}LMJUL@h2Th0lut*t;VY>D zfL!{+^3VV)@V;m$gops0X($bhIpz)VWzWb{eIb&IAo{ksX*PjIQ zzV~lCse}6=Dnyfy4F;m3;}3Qrfx~tQ!fpB!rlc^K2K|5p6EPj=91Up;P)psF;pnPE2grL=7mGy@0V z=fI#XC+tE{2enJ?hr70ax$?T4K2Pa->g$`KX!I2 zHn-6B6A1ewd^y_TtR@*-%y+=)zLG`#TG_bll{8COt|VnDX6j<*>*RE1Z#r3O{dd`&%xrEW+GE;tT8Jc}B+4XS+8h*Y<%ANG^4I(~`A-YX^RTn()wNm$N=C{` zQnTVpl`X>80sC|et;ubw<*I25xK#!=-3m3IiitbzR!X3ETrtYS{=PF~K1 zJdHd$JUgvot#PdTw*x`XX@$u7x9 z$)5DY8Z;V@8ddd{^&VC>mgDu+bsaXswUP_rVSWLiVhV0{y zV~=gH`F4pe2AwFL%p8o^%h1efXFw7Zjw~=_Ad|LLlq=chlb^? z%_YoL4(pjKeqtIBZq5De--7A+D&i)RD8d)c9-S!GBt9(OFFr3GCF&`bB6inhN)`Qgsr{@y#mm(P*H?WLK#j>K=O9Tdg9mhxTubeWMGXQI%A&4^&LHL|9dxRfcPUK69ckzAA3N$I-@ zlS7OLzY9+ebx`-*ht?;3DY2of zqOZev(e%`GdcRF;wds^eg?^lla0#!uu_aPLRsGIqW!pw=)MnJG3H_JUuX`0p<%aXA zjav7?l##>{UKJu0nR&c<{E9)BE7!YIhmCpl0u3eQ>qoP#$$Kkr%ZwAmlMz=V*MoDP zCMrw0L8dXiF0FcMTw@-HExwNM{c#)HRKNXxeA#1K>aKAuAqAAuv$D3jhwDT?vzf3N zl+(;ybp;+M&s*%xQkaFBT}?`7j`Q+*$aObqua~aL#e4Kjufk9s$fCXnTe2HE8bx8hTysDkbcejttHuL z=1;u(;a$GlM;)HPQ}9x~Pl8ZlUN4ku1Zd!4;+W#G zt?}E22K{3e{CtO;h~IVA-UyW8!2$;$~xQ>&WBA5BR4C z52*cdn+ZVjPZuXEet^1+Jc+QKg9!--BQqm2KmeYEgoMw**px?6MC{+}peufWxs#JU z4-=ECt1F``8>5|r850XPH#ZYAD-$a#1E>dsqr0t>fg6LZBiX-%{Er+F6GtNl3wtLE zJ6nqhgO&o;nY(OfV1pYHJ z|5p3o#D6pLF@1RcUy=A%nE$y8in9PbAJhMinE-rA$UPAl7|Eoh$Twv-@G~74FAP!a zcVUv>Q=YKDDQ7&9B*Jb!0~5+gawU}HMIfP)0A6Siq&EN=52PRKa5{18o(P375BoAp zI}RcW>*wUv+IrLXI+Mw?$_sR>=r1(qMpm~>-1ku!fis;q zl#M^);I*`P3#C#%ttz7FN8Q>I)11*w#PIf{p&#Cj%=+>SV~|#yR7-i_NdZ!j4J`6K;u0&=kz|1( z>w;Cs2A8Q$bIvfehy(*&_5r%WLoXo3{7~I4V1EKUE>iq>yI`3iu;hF(=@k*v>W%v^ zdtqXEb9mF6j!K1{7v}U#I_`7xvSPM_})l^%N7Y=NaHbVH+7Ege7{BqjsXIQ7DM z&ReM|poSpDr|HSd%h%zHn#I!;t@Mk|zABZFF9>sVKUW%elFNBOb|Qy2bRwsLP4#wS zl*h$92!3z0{n{lVzk7ciZ|Clojmw=Lb|d;>#2CGx6Q9;X8ZEVyTCthr>F&rhKRKe2 z3VX}!M?nfpbxRaDYN3FGh(D`=t$@FQQO?>3m z6h$IOfUwnv1MJ$nBrOk8I)hV8U9VFS8=ut-6&JON4v%9Op;G5J3TaGDu2&24<^?B| zl`4baI$sO9Ohvd6tX`aC+JNNZs6{Y3R8p?B~D+}XJPKn%(cX|YTP8i?X?(zc;Qw+*g>S^CJw_|KF z)ciUc8w)En*mi!w8SSPfGh|1VtI!@V!Nu4GBhO^TyqGd~yp`vi^C?ksN@qMFd_G;H zuli=fC&KJ|p5q{W!|*=-m&rtzSPF(^oX7OW9ulg$J3h$AeC}sv*4}xx+9HRB7kP|1tRn>~Qqt^~7u(%M9M$2A6)%d0RaZ@T2Qlw(<$ zhMmmg3;#3St`b=mj_f{p7Ka(EsM%w0xn-x4~aHyv7?(KvUFauk(uB z3mZ`8`F=&;WW7dH@t%6cX1mrAq0@FxM$ad-&Q2GUU#eZ!{z7^4*M2a49wC;_>~mIg zIOa(H_W}>~-=<{cDpfxyB3=ZM35qnolmqlS_;~*CG*+BotQOHN{Y5R(G#LC6PbC{4iw(`dm<6IXk{fpI)#uJXPMh)EEK*dUP zgJGpsCrLN=0gT-WNOlpykvo|Nf{_Fxw8z5ewAVf@{4JxW^1 z_0-R$D#e$6;lpIk8CU69t zP!{Z#Xsge_Umkj4z225f>SRy8_FC0CYsIz)J8j?;D^f&A?s?t*L88}b!jnm7@1x%s zjz4|6&8V9UpO@g8KG-d*4>o+@gF~6xO<(Ku#Qz*E3+E!d@r6vZu-U@%-ga@XY*qPB zs|{5X%z4k`j@WR9$x}|fbKy;t43>ORIg}>Mna&?{sS<1hb$caqn2mRd$k_z6voIrr3aMQk7@tqJF zOaBe_O~^uAOa$2K=PtEd>pID(_k$)w?yOJFIeKr#8QO?njBu`^ z>g)|c7o7)8se89K^fVSjYszRE9CO4Jaf&gpF_1F-is2~m(cAXML5XOp8f{GsyUw{* z(--wbLs@iGb!IVbp=ax_Fy9kVhSRvMI)1KaZ_hjy%v2R{#1Qa*Q;|#$2)HxoFL&<2 z)QJ#&{*^1-hQ;OdpdQ~e+%r#_I}8{}qpQ&<+jWV%Eg&CBK+8vNu>j7fNdwbbcD@)0 z%m%8qE32&f{*{kc;tT9hNlK)1m8FOP*li2{x>!vxnP!=`cXW<(H#4K!@>xDIXBb|^ zvK7=tZ3AF4YkSon3}wq$msoj(1^Q^QQ--I-LODH+eXXO&!ih1>u2@L_hz&ZLu4_Mv<3?{-qcyx2KGza({ssJ{+8$ycuj^+hv> zHSqKNA9Sx zL|D>klj!uXlb2HC1v+bAOi|^m7K>XVaik`6BiM}Ef>goP z_$}4vk5`7`Q$*UWUYNVUGD)W_jMLj1sqEF3U1!B=Qk}03Uy~g1*!s%@VopV_i=&G2 zOL93iTrpCVX1>~Ot2gZW{~>M3jQ$l|QxWx3GYm z5M*o7lVq9QN1Ls3@0p?+Fa0C2H|0KZXk=VWs$2Znf3LVCH`8XEUi!V!z+1R1 zp{9}Cl!rwgE55 z968cTF_wNqD5P2CHhVNy++ zOdOVJ^|aj4d=eUd%Hu2{hn90NIU1a0FDN0`6hqx4_FWrJ2^0r(w3XRg@WxXlQAknZ zrY;N&&uOCMnlbUwD`Q4biq7R)Ae^q?rsnD^iVei)M@G&U-LX2LBkm|*fC=1B`j&Yg zrVok(r+3CrlLfA$GJfq{LB6t1P}xDoOQSlU&m&hPm$I!TaEh$*f)X1E2go0ya~)bW zkb(0+V)p(}3`zl32vGK;F9XR=1CR=sUpQqS1?2$#b3uN^IB_ld&c^}4;=Q0OD48Ue zLacBYP-kq?Nb$)~_e0tmY`e~IkzuC$8p4pM5v@ep4^(}~86F5ji^i6ud3cd@3{(~i zZ<){;6k=RZnqWgJXmvM7d7CJ-z5j)7e#C?aBt#!lnJkw^y-Jf79b}1My0{@jkEWeR#SKJEdCN&|5+&+PC6Iei1S9+vuaC!uGAk z%``^kzKx*bACvtzR)`B^@O9|jSA$6gHjp;}l&OYjiE#uh^l2DVka_;fp?+5ZqF;n( z3q$8%jq(0*tO1ZKiTeGgCSAMqBq-39D4Y6?c!{aOAY0hUy1FgLV_80dGPZU(NcI*} zf=Zv9l<*Z(NJT6N&4BrIEcW!kt==c+sZxJeg1Vo7?IVfUNM&(zKrVuuj{$|um{Hvu z1d!Uj9u@Tj3=IvpGq2%5;+P?VVeNnZK|vKT1BI=E1f-~OOW2V*k|( zTgU?009gTo%Iv{p$SEL;)=@#>16jCcAoea;6bOSnCS~wE8ZqVf0@Lyztf>EZqKPt9 z7|Nwy1*KVR0y)D0VHq$FF9b5Pr%(K!2v>%I;vmv(Df^EQb$oCcx{7(oim+m}$q923 z4oL)rn66gmj9QmhQ&56l;8JBdl+8e=TCHn0%~eS;m@muBMiv{4O0C(4ZdBg$kSjK*73V-bi!lTRlH2GAN{mB9cVSIFpD+I3}FH8 zKc-PXV&IWn1%}PZnIJroI*nzcc`H;S*ZD1*kBIW9LDk%0dh1;yCC%Yg8Ob|?#9|~;$z$-i9}j02 zB!uoB-By9>*`;L7q(ZTw3A^1rOL)>9EAaJro^G|(8S0m<9*bs;UMYool|_a6t5_tc zu5;VtA1IEBid$CU^LGD zs8+i77XO$1{kiwpRn`xKjmT~g0t5(BXX58;OMbq)Q*SlL!eY?%A5E#r1L0_UN5_5; zrWtt|vg_zs6z)_TdK>~xE@)We@$63&d^>bmz29y+7pD356Zw(`TDV^ z+x43Ig5FsmicyBL771@WnT%*$iCndxE{Kg(RGDV~NoshzKWB0|L+k1mZBH#NE$NRG zexI?)hxZo6#KMY3%wO+c#%(b1ODHIKIYs;vAsiCZJ>L%%L;Q7m*z@KPj!3|Ze$w;y zkiN3ypy#Y2C=%D`_9(L@g2m>EzQJ-C@*D4h;gp{MkM6|B($*%r*=TNi9$8k72U^3% zswxE>_lb}2oKL5K9I|ZYzdfD(v#J<+`mKVbi+aTa0 z?k8ha%KezSgbFDFueA=-d~D-`stGFS3=5xw#6?Q;sQ~F31xMe6Pr^pdMD?HgBji=; z64fzVJTpOSViz9-yLS$gFL>r>0m_n#oUfjy*~5sUJb7V%oZc}0Rm=fxpbm|vr-ZT5 znwx!~SVa2aLV9U7VGfLHvnk_YqDt0dTRG2^#OR2Jg%MiNCT->9m?GV486X(h%J zq-tdauXp7h8)O)j9>tqg5oA87bzeZ~$BPfj^r#3v1&(vsVS~D4E zRUOqrH0?-*Y?C}u9#m;nnWa+ed3HSy-Hp=q|(MUB}mj}nC4^&!qnTF17iy3l>8AD4~%Mw6fpevq4gA(qr93KMAn!IEifiSf@ z0eDv9u~N%tWk(}Mt7kNmQ{7?G%t+}NbcHg{z;BJss3g{?zhiiLks2iZEvhVz^ypZQ zpK}BUca!HL2MLg9vRQ?2ot>k5E;PtcQh71fwDY4r&pWH1W>`e37Uv2%rcw(VL`ZxK zh*1^xGwnBbk+#ZTRLJ6f-VgFQoBtm3>XYiVhG8S+aWcVwy6h^^VWUp?%Y)W&M}c?0 zJLZ%{lA2UeB2j|ocETtk{a~Zx&_7Xr01fWqhi0u1uG4cN|6Le%B%@A;7-uZY22yuI z$t1JMA9|fMpJIo82lMp?QX)RDALClC+YyrW?@q?dM--mD2?yR=yR+#0eV(|&F_ z^Mm-O#W=u=QWt_}Cq_@OAxzSV*lavm82x0b*w+vk<bDkeY54|6P%rBQR#ZZoJ*<}hgo z7N}462rWkrWdDX?F?&<=;T+f?4cZ$HPW;WcmbYPSfd3Zfgg$MX+amaujG?O|6VL87lKB(5N zZh%1_tCy`tuLsTF>WmWps`q$&N2gXZdNrz6y|w-uhKGoP&0|-;!S)h$pFum?^PX6L zA*a)SHz+8$pda64R-5|dScNWc45s#Bp-KlSg-K6E#9ZLm$q#EHHn!yI8$5!_eZH_` zU4jr%Rd}NqFuWrU4LGr+QSWP6<6mMX>10j3h&h|{tzff}(Y8>Cx?g}gZr*-t+a$5# z4+=#DHm0heDW(q-<4^u{xt>>}5MT4o|7K(&oNl1rDJi@~?> z(9nnbw>nHKRA%3_#Ivtf{2gt z`F6vEyjFLUD3d=XB&X?huzZW9^++lDYV;0t+JgCt?7YJX$M4@iArtYTeOn`QW5kFn z2;?`{+bcaZFRKh)nwU^Ms5=@}%%EB|{K5ByW~Rc3ppN+-N#CZO`It(cT!ubG5Nl&jmOzBowgmw4XMNtEtv?@9p zHs%8=DSPaO-<~eOO$ucj?Vmy&(ycC+0H{02%D3$@2}*pKNwkSjEpQm4Bx<`|c}A?1iEQF7 z?ACwa+#L=?a&)tbW=N}%GMWm6+xBx|Za>v5H6itBz9v@Yr4s1!w1}2RiU&&@!2a&Q zzLQOgn-*;^em!LQG4^+U8po76S~OX1usSE=mtNmQdlVr*B16U_j*3X|%hSQMT_P3D zFJ9M+t)b}z^W*EhhB|W>2FHDAE~njQuleUZvToM9Cd+X#d1>3LJ{C8Iw~hyg!j2)4 zgD?G|Xlw9ntROIq(`vk%O#_r1PAC#F39ioYr*5#Bhr~#=nXW_)a5)pd7tjM%gsd`# zI}?2U3B@Mb<-C3IF5HWJ0$-GfU^E~TP_yBMf%eVDqZtrHA%r#C28>qF($ArSXECTv zRz94Ubn|f&Ac}qx=w_7XGuPU*Ewg@qyoWffKfZ`6RjMdgCV*qbS61{onmck-=ev;8 zU^`u@g+)JKzl$1Wc6bnc;pkVRB`in8ILB>i<8?g~{wT~t<;3$nBNN4mOTAy(jnj+9 zKJJhTesjMM2-*gHy%{?%3@@g^FLO}nL``{fKO*X|5LrnEbgKTMTDXxM5m7HYJT=KY z6pzDc`lBJ=Jh{qCgQ>xCEAZGk17UHuMWBLFj_+5s&U_ZW>0BiDOa(fCj~CGUI%^M4 z05$leu{lSH4`DWwbEKNgdnK6(cugGgE3Kv)T zvunxEIhf}&&ol>=jlscL_)r#UYn^MB`{j0(VMBwBV>8W_1}crj-|2GLDIWn#P$f!G z)7K>uBhe>Ig_r7wg0=ba&2oAFjN2H+7o#TL-|W#iWsxG0X^C>d5H6uAB8>6--n0c= zSq+;aNVeqJI&27nrmC~Bu?UGP{GFop3dKd8D|qRsZQf)qNU}l=i6DLkZ-aW v1+FTh>K_;Mfut!uIzjU9oBUI&`0Lwo)u&w zX7}#u>guZAv%8+BJ4{YS6afwg4h#$oL0n8o0SpYB3{-Z3{RH|gPviy%1B0tD6BLva z7ZfCtv$ruevorz&6AMd9fl*3WM)ke?DEY1~Q25pTYvETu30T^@4Iy$eaDmUn$Vz8p zJ8+ehBqX^sN9CdZ7T^QX(C`udTC>nVwBJs3L{wClFo?bDE*l<)lRgJ)mpKzzX&w%j z?6;77d3J+4j%wh(U*x07CWZr0zTyt|Ab`X6@O|F$!%s<}H4X}d0uwSG>K+ei5B*Nk zGC^VR_Hy~5jP~SjNdP7X+ZXm~sr{*M7};y`tE4>`-#P!8yQKXcBN~Y>aM^mqK4U)NiiKTq%oF3r!;=g-r^oae83bqy{o(>P(KIrM6#Bu? z%i7z_G^1Y<_9fzLfy@(069-Af!eeIuh-eybZ3&^fFNbgi{>DEzY1e!fLTDQ#3-#)s zp-Bj1mOD1;n@kLSTC|Bp*}hYd_o+5JUs%3DpjfaOJ4aTsb@lwRSfEfipr5*q-A646 z8F__1=3(-1w*F~pjc89~au_^v&yaMA$<*KT)zuGdPtO-w7qT*L7_QC_uXk>g8Ez?U z#|Tx=UIH;7XpIRxfOH2qVWg&^z6!zMW>OL}(Eg(6f|9>$V#OW;W=lpYk^|J>y79?6 z1NUWNv0!!$?Kp+7J;s%#8qI2!)~4_U7a~G3AeEAh3JX0%LO@)uLN2J(UJ1V(Wj^?XZ(ZO|o^~EjUY#jb zx*wY;XMxvjoGAeZd?%b%WVO(x==TEpe4D(NS<)jLEI1KJgJ8QpTL#8d!wFO=@M63M zOb0xN&kiump-;Yp`p0Udm5>XVxDnEWbi)`1F8Ud@hqdcV;AfPGA&P@(+nTn$Kj`RS zRDD&w->j^uVJoCuDBl$AMN}hdVRNqV82~`9)~3l z05N{W)dL=`Jr1#kTisA=1h+f~X{(iX6*^sLI8E1KonrCuuU;e(k12E|O280eTp z99bMaoeDXcIJ7u++eF&p+D;ua90^ZR{-nB#yBoQmyOTbzJo`OU-{Ia7;vj!X{RH^5 zioo$@=u6a>0)+2a%~*)oWmtBRH<37z?oogt>mlHf(@=|a?Ki0emV~Tt-=*~wIe(%3 zqWXnvT$|RJHe*==e9)ZG9B$2Tb#k~&OPB;6v>qTFykR&-fJHcA&|=ga9#2PPv>0g* zci1F5Cm$z!&=3Lvz$ai;gL#9yg_ZeaLv?+Zm0;aKdPO%_8&(_Hvq%Op}%yd4yMee zU@}7va3?V*Hko~TgcpNO6;9_4$84o--m3tc|F~cra@Mag%()eEdv?WecXk7~3A-!1b6+p~k-4|NBDk)*1ztEzVeXz@whulK zh$i&!{&o;~#d}qG*1Ds333*X{=0#vYh=po|?(_5Z(}45{_~I|bPuokB`n7}g(JoY8 zOl4$L&eBxORQafZp`sRD4}bgj)!;TvAFq(BP@)i5I7@V*NVDju=%DDlXq2#rNQ%hA z;6wxv+ksn!Rt0MnCnO>wd=>Y5TaQI=GjzbU^ELFf3nmV&X4};XNt5t#gbaPM&V-gH z3wQ!@mMzgX{P$FiVSMR5G8KyGtEGRnK#ZiX1BM z^rghcvWkH&!$spW;zw$%REom}@Bm%0IO#Uv#pbu26`9zJ(} zJ-3>&8kSYhTyv2)RGK&2|3z*RYH~9zl{v}D=`Q1WG5@r83wa)~N~jguYWt#pG*~V9 zo>HxO-{qMXFmHDsKAxngnpK_sbEsp%e*N9up+xKOdh^VibHiwJ-m`yOP|sq<48{|m>>s1A=0?bpt(m)93Tq&dP4x6~Kg*V%M!DVuGBpy7R4 zmh9he);AuCIbNH`-qY_>CEK!u2v?u9-{anfQ+i)b5>57}n{pPBfJj+5_XJ%}`n?go zDLW~KE-xFaT{eUW*j_ipyZw)9+oJO^ zFT9MrPM*>4ub!Txm}@EFHQ#hryPkYbvqn-=ksZj_%2dFDC^HkGdRD!_b8|+qnbW*>W26D>>PJ_NSuL}>Ok`S$OpdwGea%bP)F?}4i)(cxilnF%=~t#RKo*+ zgNdV$$F#(49~t(GS@884aU}fETK~ZFqdD0e@&SLxf;PqcK4=fhUR8|L#Eqq;!KgrG zSTOJqGcYJn36#)+;eftiV36^_U@)L7D(EYe3-LcvaI##;|0#pH{B6juC@3xtx+)sj z8yQ(UnA$i-cqAi$x>_((Qgc+3mf|q5v7*y6w9z-BbG5Sls{+R5$^j}`89C|^xmsCT zJ8-yill;?y162N7O;1AfPZLKAZW1+VIU+$Ddm|z?ItDrh5*|1rA|ft(Lt_pFA(4Np zgYLLVOdTCXr{l}8E!@u1EIY|FkLeEIYK>xpe zgH*Zx)^f<1xf)plh0Lr#;Q`s;VPs_I`ltQ>Px2p&|D~z+A5A89rvKLbFUkK`Q`y1D zUeLx0WYUr6KQr@h<^LA`TakGSi!)k6UBx2 zm0ZEkwO};SL@_@!%pjbVglcs~&v5EoXqF_;>QYMUprn77GNzm6*OBY1o^=lRqr+pI zb?U3qsL96`d*2?cIk88#YyEgdKD_31c0QQQNc&jt^0=H_lPYI9Zt(Uc^7G~6BZ~ck zESB}Ia0GLP;G#0W>`4OJr?RgXly%R)@!fjHGfvhdA*iBoEnY7rD#X z#Bz7#GG6?j!+}f#>G0{g*5{PW7ZC~mW!^~)h37jAu_P+=xngMxxn9#rDLA*$R+~|L zPKW#H%=-8phnuFmOUXW&hM>ZRNPT1ijAG>ntUXycHx!#q97{d-^0;a|>lfe;pzsWHBlzmUc_=c@$z*&>d*>L1a9T5a51% z_56I4g@(5(q0z8VP!PsxRO1@CL`+6g{))rVL?+f0EatN+BF-0$yl!p7J%qy9JzzTy zzGxx6FAtZDK0}7@o}8_F{hh1d_f51cNM`xZ=g_D-M%GN&;CB z*!bbfN;SkYe|W~SCQ2w8_ya)>KXM=wQ3HbAVxhZ6(Ufy7QHEOvq7*jnw*@$n)x@EDIcw3tDlnDt;rgE{2)rM(m< zvuKF1p0XOw-aS^tG3Be4DO`x?pvJ63m+!wAyi!)`G>Fl*^Yci;!*ltd2HLE$#VB|A zOdGRhxFTAp()zZ0T&o!cTUTpygd?77U{gJxxmm5U2&T{#jFl@hYcc6_-;LmSD#>~wdIE>SYH;NN;{e^6h_bHk4>Xn z6%qJp^|3oB(+!;=-Z7K7$s~7Aw25@e5Vwa?gT;R&w^FHL=%>$9zbckQiuSx$r7~N< z!C~4*kyiV36Z|;(Agno6C(6<6a^2=NT`SG@?6zOl{9UCOu-Rg|5ZWS$fzh;F?`Dt| zgNjL~-8Ku1sP-T&+`|$N8xW+AG2$j|RxJT@z;Tl&V6T6B3R-Ha7n` zCMOsn@ly1a1xxVLpQKA>hR@W=Q~aH6tgUNOX-xJTGWaXH!_BOvchcQFj@WD;7tNiSMi@T1g206XbvEkqP%N6~Ew=+@gbR6~J8?r7a zr$vd8=x#O+YP2~0=7nu{A&xCxjSCbAo*Vc|n8oga!s+~cyK}7QGaRrby5zIP(6=da zg*44k%rKThJ=u+&m6cU%){nYpJ3<#htx-&NvRDy@kkP!I5x!rmzl=8%AgVN2Kw^J5 zMSiPNp)RiC_}zNF_a5KlF#?N*cOfnf{qu^XsJPqB-T{*16yEanW-|U-Qf+gyLj6F7 zOb!pe10-g?-LCu6>!x#aGc~P`E?nEep;I_IHAKsf@c#1!d3c`MtiTPoo>DO_K;gwIF7)oN?XZI?Ux*e^#yjOx{VBOZ^(Hb?UaZV#j@=M?(1s1qp+pfhW^ zs)$hxoagrGYwNG;t676YY7Mx_be63fF>L9N7Z=TL@2{fK1SSUW;QgI2SM}d*`dl?? zbh>*V-r52ZS%GXRbUOR{>+Q}xwS+MG;|p15o!tI9!(T*mHa4%Bsy^;EBeL4>uODK0 zzCSB9it$$$+HAIlBvr8G>h-s!p5A^pY4>>Gxoj{LMwPka@VLnD!m`T|f;Dws&gOy( zPHKvpy`a=$QxJ=P!9cFnkWuvZY;;KBb-rAV*&L49BTP^Gm`VU1<7U$2+oPWn9Szaa z=(HlHOyL`Ep0U=tj#5pDQV(hPtRm>?ZU1n3#%pPoNT!)0NdH=oCm6vYk-%IXt~)$-@*>`vd0E-*Zf+DW1sZ)3CKOhA{Q^+?xS2p?mmkBN+b zg$0jhySHEt_f#jmVsMFC_aVJncq>!!XmyXt6HVf20;8sEvl#TKBY9B-u*1;2UIDFb!dy@R&dPE6_PSbNc0#U3gHo_d6R5%dM9O{#f zLgZsUOOrKNR$OlkU*;@G&uWiwLuoSqdnlTKC-RL}wL?v>OX%%!R?8IyH87WYm>)yr zvoWifV*r3kK}1O53x7Qu8G(b2_;=h5BtxD_Q2-C2EpfR3 z&3xG6BT5-j6>)jvq9*g@z;2{F?Yr}JbasayL>kRY5!(d3pTmkG#iq_$yC%{-!f;h( zZCzqkxV(41ppyg6S6yH&7C(kHe^0`4bwFCFv+{4%C5W#SBUclEPh!LiB8n!=jpCD9 zh>=JnFy&*m+Moi^6BA8N+(Xjl?|RUw)KO#wvhAJ2U1nM;h1l&m#Kh4av3TLUA5#y7 zhlTn#rjkpT4QutR(}E^4QiT{Cknn((5t{uiXBj*m&6SoJn~y0Xl&krS@=`%G`L~#K zxB1|k4_jl2m2FtXU+N8~$W-%S-lO!X{!FcVlpe_wKHYE>$ef#}-akNX#6U+Rk2^Vr zEm?nSr4&NLOZM+Gq!oK=;|Tuv#nI^TK&VUa@Hxai=0J)i?aB|Jg3}Gi&%tjxT}q&C zQD-gJpXu4497>EY3xiO!mrl}92kO6GDqii@&Q6e5LD=I14n%V(>YAs}+*ihrzb+O{ z^A)I|o|j3lo&^VY1scPPOQ(!Hmua@L`G}h-@F=W_g#!jDBK#MF;EyMc1SYsiF()@R zq8kW#JipigX~NdnzuzOs5*pUWOZ@Z-ulr#*{VDyXXgOY<7s(jO21_P~efabjq>Ob= zgd_n$pmf4P!d$Mq;qlCPcvVw=NW$D*m8|D)q+{VfMJzy5G{U@7n_cgNw{#XeDbN0h z1X&flRyQ=!yhluEU%<@k-LXUK^xoVo7J#jo*Q?vH4heo^YMa-3LVH+g+TkqR^h8Hk%7@p5%Glpf zO}nQY015?(Vw!@_mb55?Orexw?q?%1EH*2#n=s6n)n3Eeuf0V`cP=!>jIKu`YJQ?{ zzKU0WdNw9C0kddbcec|8*Flv)CWkbZcKgc>NV=4kLF0NEsEY&ow#(HLYFSy`u^1Y( z6PQl*`jA+$HW$08{>k^c-S>eOtMuKiF4{GHW;CUJ^gr)$PK)JMs&bwNm(>WS4_5Hj zPGTzX(4*5jy4Q%`T6A)(*Y@SYk7iF{mX?>VfdyHsDh;j^W;E)B5%}Vs%pY0Pw9Sx~ z2XW8mnZsckP?H)i{OZM)E!Nt8ahCT}CQ||vde`WzEbeDQ1gE!YtZD69hCl=yCV$n* z)18S`v$>^&O~s7WmLq-7L`J}rN3V=vLQbJ5YY)schAVflO-w-$; znwEx|w^iN+khzFMXI?M*q3HSNYrO1c9}6cY&NGLKlRM>6hPpVKh8?Y@b{e!jZ`pM4 zNOWuZvwf1M(swSd%*k+JK!b7-C`mfB&RzMLqCg20bexkExFW+$@h+5R_C=k&i3qyg zt=4$byMi5X5-d_*sWt=}O=ZTx)Hcndt*ImDnp|4Brk+rxqbp>INZXG1#LRvY17MSg z1F)$ikKOioj)p5J&CwEErk6!KAq<~KwZ@OKT{Am<(qqu%#s$%$U4};I> zCsNOQ2uVj-f3eygECYO*)8U0kz{(hpX*4l_I-(I&PlU`s0Rb|m@4s?z_A{0ogoO&Z zpjd`urJ$P>354{8{(>R_lo=esIZI-4<05AP81j?PAZVlm1CgT=P=;4ikN~gaSsbrb z`%f)1gk@QCS<;GxD+@JQd1macbYfKaUqe?;M0^(Sjj?Rj-d6Wl z8zjVU)l>*1E<>kzgn|GK)>j1zWEti60$kk|CJ2@OCL!`)F^@y_{3|nt5|YzkHq7_` zJD?9C-{#uM^hv+-2&25W+!4I!R=wRjGoQncLq4I^zkV}``r>oWmD^Ditj18b)Jo4t zoEswg9cBO3h9>6ISelD2*w)iqp(X2t$BaI#sZ^V=lCD2m(P(eV+@!kVgx=@&WYGh#x3 zLW}@WMx&9g3Fnki{nH=HA6W8bJG>7O`ciL02z{%ywp?zxoL%XA zirx<>UX(#0E&;n%@l)dx<1#2-9~WL7T)}W{Z@+O5A*Xk^ImecfmyH_(R7;G1I^IAq z8I6Cbp_{b3esmM8Out#VKlRN>3E&eIgqL84ojG)pTfBZvIYA;=Plj12IA_T0bX599 zzqCI__%#_rU16OENU{ zi)#IqBX-M}>ES$P()fbj@Z>Pxi3{W6oauZn=ZUHX^y(LEH;>EB?g-_au~TI;KO2d8 zf$i4|OV6I&k_UFAq}Cs8JJ074btkU#7ryzGysFljQAssEij}dPQ<<>8r`cRKk!^FPQ{w;&BEn3h}3Cc5BmT%-X1Z(Zw#S5dlbcO9C zs_LD(*6o3brDhZTLYT@H`gcjc*cop#0{nuM$hPN2sd}|{=JQ~hK9|!q!z*ItU=|Gj z(V9LHRdV_0Q)#7g|Akn}{jb9{l1_9*`*peBO-gvcu}GvxT}OoY_>{TyMPhlmmo4#K zlkSl2&w>}8GPTq3C^gx~BxeNK$|#;Mzv||(&#Y!k4HVHR55gNnWegV6&ug2TdV6@i z_q-;OsX0A|WJEcfH#{N;tYUw$V@?t9L`_C4-rAJN2(}R)^|8k0Yg`Rm{#2!k>fsi3 zU7u4P%WG4W7V3%`xNzdM{c~nyop&v8AEOe8K?g)DLCoe$3Z9p*YYtHgOg+MSJ?BO- zn=9sz$hK;I_274@J;^c3wVMT0Sfwf_)`)p;*nCr~rbY$8`%xA{ITNk~7CUho3?2aO zKB-dbd}gVhM`TlJe)ZdQaingyiOG1V-r8wO$n3UGI&i(;`7EOi8+&Rp9S>j8GBwyA z>*4zJjNRfX#*bZp!E7=SY|%ou)y+Nb`WDshxvkvJEvxu~eO^y=70Ijq`vWmzK_;#$ z>8avf=%x6L!TW$OGII@*UNGmHqaPVV$bO=MK4D zl_qHSU3CAw`QjI%UY-qu!5-R$fO5GGG<{Cj)>zP^7;37sG~$g~@B& zwYeXl>g(&fdS`UuFU4&qr?TtAWpl&vPs@9`dAEaS>Gh*g{GoVt!pgik+?EFP96%xxA`Oly*#bY2sVXI9hi+!S!M z=84siD0rS8D5rJ!LPc7B`)UN6B74kYHkWTzp#8(=57atIN&vH_R`_+J;*{slUbJwb zmQ;m_^DY0~FKpl2Ryd=;cs2Y>nml#S@S&q(Cq@hH6Ihs_P zi6Tp~4AlhLOvtzYY2NkwxDsth#6*!h?z_sxeP5VEt;;T*wOK3);8;Gm8&IPYyz^&# z@m07kI}*smSa1m$o=h7({6LsT)({v>W8ptJdc>31zpJOEj z$NV)bP3UvP<{z7c;%;28PXO5r`%d(apCm9V29l&{zIA52x`pe`)ehFyH4jD(8n~8y z#C_B{?n-5UBNrGG&BwwJB!T{RZifGUk8r_sN};uK$3;MP4X6RlMo*<(2E%y@>sbVQ zIM8T(e}|dII(fQ`;mnHJFVQaAQl=iAk~xhyZ`nw(T_IyF6N#oLb9f$_AbWnR#hzM{ zb6o5GwHd{wfiCN3589+)Kpi=?1vN&d2MoL#cjQ zwGOm_s_MJs<2Wjo<|HzBy)gfaAh%eLxt`6vtw!>O=n<=g6>V+{_PJtN_;BI4{crMy zLV<~Bi)i6st+d(m8siJF2)|awKGpPVkCGNQHWeZ0ZwNQ2O%NAnQuVXi*x~&+ptzOu zCoqj+a=LmuI;dCPTR)Xeq|_wXlffp5sdt*q6XzumDrj8H zB9P6yV3;r>FUM_|!!M-dnK{->;lk z*(96B-zMoY(sP@C5Br|K05eNJdz`>=vg$P{p(1qg#E-RRJf=#?mI;c ziufCBn{5`5H?Kp>oYx}SlIxq(>6og1Ky8{+Z zqs0W|`gk$z!G2f5;qGv^Xf8&AI3s;M6m*d%lyz;{J;k&?zn>wA{V!)2%Yj4`>ZL;( zoJDJ$9z-@lVaFr@wJ1jS=Qalv8`3_v?7JAv5^~&PgRv3x2_yJV&_734exf7(ZEFoo+4L6Og;dm`17-NUQm_QF@ysPZn!Q&ZbNjOA}U|3Lm0WD%7Q6V0s~t0D%^N!(<+Z6L^91>P+?!W&LLd#I&@QY@ViO$%Oo zSXU7^B5jLr9Fn@?Ed2f}YJ7)<9QVU6QY z+DpN*5vhZmCGSnzCi!e8-xj0zZNn~#Ss_y=Xog!&#Nt+cmTEi47<{)G?moDkTW4SgIwY)fb}=h`S3cWB$+ z1+EUB58qnD4y4}!d`9L-W^kSlM;j<0zBE@}u-ON0Rf-lwOx_EwJd~>M<0jH7qNdbn zO8)jRq+<^|2`r^S76XShtkHpLT}|00{k&|Krsbqjd32-TmzY3oPb-J_qzOm*n0Jj& zA5F+(TH4mj_0L=A{<1qtsEI1+>Se$zmD>bmrO`TEo4f4_@yDT^z-)fd7iSU4;?_M= zGsP=7CF{hz_jmJ!(urm>vHllx8MaKR(c870a(wcuK2O6T#6pssjJX}nkjAB%uguwooEWaO?fa_-vOm9nJrE86P zz)HG-*md)3U9O|}cRY4?f2$o3PTP!)H8EYdNgyfHDB)0VIv=tC?FSHE7?qQ%%E0{H zka54q_Me%NE7mpMy6A*)y*v7R@GRx`AZuW^Z;p(?H0ThsFKK9G7M3)Apuc?6^waGY z(cSSB*29Cd@pLJo1hnY^E9m_ytdFVsce~TCXZKtc%_S#3zMg~E7BYrZ-cvlBVzxS7 zH0GQ?8c!yNDiu%!Bu;}fPiVS1Q7)amIA7~TQmuiu%RZ#n?i(aW0#Q&=9uEO=yxs8A z-A<>7=JS;={Db% zSoM{AKE%o!-Ni?Dit_z4RF^%}2tG3!VSFM=tEJEYDRg@R4$~yvYJ%u>4u@nGx5EX* z+M2$Nl{iJ9R2EZqyW&8BlG^qe87iFns_{sVi&JA=v+)a>M!$~^7(`4~U{)1vHm?u= z{10xAdkEPap2On{ElviD6IKV93L<>Y^JLTB{QXo$2fB1tt57eG?*_Ub*<3oWUqSzq z9TGlfv%Tn6X^Wv62_rI~kkWYUPB=BK^%P8Ulj%a^Pw4AC5VkG$U%{l)JfKrtWt((O zn2XGS=DfN-*XVCwM%k_VS`x?Yp<;Yo*_-LsRol=oc>3Mur$V_#$)}MU67@#iZ(UZa znI$bR=f-p!x_Ccap$bI4?7Q(I5e`U=o~=n+EaZRXaIx&a9pwFsrRgA+Ptdokv0AO& zx@+{AiqTa4-5fyFfFl37sq>9R1)J7vlO$&*&qYjvpNCwu^z8>*@1tF>$+ULQ4f?6= z$)h&Z*(Tf~#FOp81au~+&u4f{deN zKHV2T9CdLQsv>qw|Uef=SA(6rMEoiH2gL%_zi@jnah??O;7>Lr!x!KioIaZfhhM z052RV&R&6v1P2vBRi7>njMNw@}Gx1T=veg*LW7z_F@PX|K&~QN%_XTZqR8gQDHzUK)R16Zoi6|h^WXF ztDwBRyZ~cbpaughRF@1Cl}g!^@lZ^<A(IwKnTC+$~HsV0SF zF?i5=0DuI4uF`OHM6hl{0G!D=NNABC-9ZFf zjwbyg$)McNRNRfa97{80kwBGv8nVw99nf$)DzVl+1s9OPW7cbHWn!m4{ zm`D$bFev^#hr*y^1=A5NrqZCr=l>}=ptXz|K^2DZb4AJ3A~9W0PijJ#tDAgQD9X>W zkEfVTPp53AcwN0>=&=ZJSg zkqn?H5!7wxj6ZJE!@ieM8YDUKZUYft_x;oWO+g}GT7x%7%EXeZ!dz-pl%iFd`Wl%V zNiZ2Paka6iufB-+dgIPu_*RRK=+Lp?FO2iuy*FN_qeoX9q`}0Q6d#wMDEHZCLQ0$I|fe@Qq-E(Apo%KkL1 zs=oiJ)A7UzY}7gl2#K+~k|W~f25MRT(JUS+aGnUr0qVmkiWH`Xin0G0Ng_z9nFAPe zkQgOa+Y*DG=O#;R7f0(&oRB5{GU`Rkc(@2r7S%BP7WwxEo9d_7{oEV5YxaqNuQTeFqA?P655d6_W+(w_4~SjySJ z-X|vu{pG&2D}psuE2nwS;vvIw&1GajQ3IR{Ez=zW;-c*yjd>ypqqF5A>Yo#(e&4Qj z{nnw9rP?*n#*@Iv z>~#(zA0Cs9pdo}){q4*$wN6L2uG8Nl-md4%j9QA{qI>V9$7k9-L?_bUVM4zX3+3$6 z02?gIpA8Cp80QUqH7O*AzNo1+_*!BM>$Fm(98*|YR?3Iqv+_DD2zWD?2iX@HCZi5U zi&zIyM+15MZa;F&)NVn~9t7&lFr6h)eCskO1#@pkT0qDu(b8VQ>%uLk{>eG2T0 zZZxB(C&EmZJ5Xdj7AnGu;7y36ak8F?cxpuc?AzO7Nl%*+`&q>rG&R+O ze~{KEZI@OPPN2x0a=!{9qw7LdFJ{VI>IjMX!9HM-T8V-8rfqNg!AP>bz~Ycis#j-Q zbZ)cFE9G#r!wDvaFl5B3aNr0fV75aXl197jQ%p{v?nSqc-(FBir*J}y#Bc=QTmpT( zw=)oN^F+D3cQT`e93{PG>`>}9L6QP;22%KCGd&<)fS{`WAGhNZs)pQlbEh@ zG)EL?+7%GB;HfGQC+d?5847D6wS_65pXQ4Zs~V%D@gJRjFV(2P;mm(SD_1J_Wxq|) zAFlRV>U<#LXtF61Xw$=Z2GOpDgDWh;jLQ+b%1Yi`NS;%`mStw@!Vw2c(={5E`P7PQ z4KeA?lU*>yddJ)owr*~$5gChjqqUsS(9oAg%wnnuZ3WkrIZuwiKY z{DoRMo`7^rQV<(F=(thW&rAY=NXkKwLK`B;rbMF?{P6G`)L&nz&Pru)de42+nFx#I zLror~Z59?>gw7cqswX5=QAN5T(lfM@PR(mS+vfC+Q6r#ZaNj|GG`O~wLgU)yqQmF! zqb+!j>^yfU(;z?s4#Hvwe78AVTe@s`4*=2w6PW6xJPxg!!wFN?Y zh>a%!KHC#gF7VTh#q}g`8Slk%c;&5bk z+7=<0S(godLMHtc>13gLDp3(J5}y~E$H6l$2DNSBWEBdbA2#-d;(sT%>+6F>pFr5oFYr*Eq zRQAyNjrH2ZgE>=K-GTcNAN%ARyfK1593Wk6G=fn2KW){N@qnVYO>od0hkMrjf!x>E zXLobQu6=klyZdS$?X%NjdB7(>5*h#oeVKK05Rp(bK|~#j7S)K8aUuRBH(|QlX5^V1 zRY^|LwumFC&}?Lp!LP!0cZ*n0GZceARYjGaM6ET4Fq!U}ui!*~P~V}VlJraIus3V< zrqf$sGL>|_OIYZ-S?pzx%kn-l(5Y&o)+`P^PL_AjaC1FeV5#pXxXIgn{0)!bCtPdx zc#8}*rl1UkF5BgQuZdNZ`DufB#`2n;%w~m|;g~Nhx5=qf09wM)No-y_9Ct@6r1YWU zt?#XT*8o!Uj3I-_-_bw?5i)Ae7`oc5=JX^Q4(W}%+ETp`u@5=+O9zULj;BJRudB^~ zwZxks0fl_xIb&hp`>zTGv59^b`<6HWDx)+Y=Ww9>nd!=eTlwIet5jS%qei9hmu6nxasTnLQErI}x0}BEIf*>X;BnJWlISc{zfKm$!dAl^>_{&0Xl5D@T8P!Mq774-ePOtAlI zg&fWV|F8F;H18+kWDP)pfbfHe2?;2;fF7qod7ue@3WV_+7eEo>oJ=89;qP2UcR$k< ziZXgZ<5H=@JZ`=F`rR}HQdS_6N|93(E{*@knb;~rxqHHvotV{G*PGXK*YR0|mEG3@NapN3odvGWCWw zsg(FU@wkwynwXX4#;dmn&)ls&1PC8lfGPxdA>gTOT<5Si!|(|D{gAv92}Gwk32T(^}ya7 z&4s-=CnPmGZvLTMYOsd#B2?tFTpXS$QNG#)x}kejY`59i6b_A_Emmw*t1G^EJDn|6 z<+9&Rwc8n-+yr_rx`sg|vtFpqlS=Uf#6OraE+{Bqvs^6v>VFBGjX|#^exh8V73O_W z;ilFEG{;fOH`#7|06y~8UTl#Y8W@aEz~&xCxlk*WZZA+P7Gu2?Yd1Mye<3i{E>Q}* zSAGdV_;jG3Ko!@ULLfR*5>dum|2S`ZYEk(9?xsq9-^FsJsZ6En#7ES7^~rLfoge4* zc)7uPsXop)!n<4Mv}5mJfb$ihT`X0z@$h7X{(JU4o%&6SXQ!vLYKEt5Bh=H1>Z!+% z=HmX5;zv#S?bA!e_M`gxr3Pb1$LWbhLv?k<*5XySm5b+;nb{AE6M8{eyiVNa%RA^U zGx9$&d+T9lCI;oIf2&0aLJ0By(?Q4zcA=I1QJoT9;-8xNAwH)5h^2wi5}-CyBOcaR=k!v6p?j}bm{7=L3 zzc>BAGySgZ|Eu2r=ht-X{z~&*v0@3}95jj*ieh79?=hWcimx4yW)o=D$s(8M$~F7i zr>8$YZ@wL;uo{+XwHhteTarrT_hh`xmZ=v#{`xhNCjq5+Q!-bf4cOxY{^IlotC3_! zQgJjHDXB2;8Dj$o)teSO59d#8Mi7ecUS%U<`dv1BxZdSc)H^I_KqYZ z;MPml+C1HzW~^Zkn*}!k7e_}+n^?S7soQ-xV~)a3D)Hfb25=+(#bd+6!{v`I4S@Z{ zRIf9e0j&Q}EX=p<%ZZ7_$@$NYTl98pR?DKG{S5{K(SUOsikB=-$IMiqYZGh;LjM}n|J}=D-V(#F5_kaD0o}W6^iI7Dw!lcvq}GE&)YaiUpvUv9{uxf%0>+N zTb-pk$oYXC0`}!ge^k4H(T7N`+JWR^ktsCq%WZbk$>~ntKst@zSf*HSU^HsA+e_Ag zSC5c#Hz}-^*@q9;Ka1TU5Ea|4Rs!y0$oq=q3*>L?_ePO&4-@D#Z%-PCd&8cXkfWMi zui`%odaY~AD)PBqpMgUmRx!#vn|nWAtFC!4S==5s`Y60y?ob0DNJ$d^D7 ze9mhqHQaBW7vwVG-0X@G;q;NSPP3)$P)PW!`n|{e>sQkXFZ&bv;17pBd=C4f>p4d- zxrcpWDGVm#&Whue5KfoELam@n{#D`H%fw z2M$c4xGJ;>!Pe&bq;+G5reP=f3O_?NUJc!zsDSqsi zC0N30kq~(%;cvF z|5Gy|0|3k*0<^G0{|Q>5kpit~d5hCE{{Z=ZKtp^3U|YvHwf;W>uWTWpHCrLY4dWl1 zJD^1eK4Q5U()<%FlZ^#hlSI-nH6f4)5iHnPuZ+zVCeeR)-Y&tM!HLzHXX7HC2c-Le?DzLms@l&NLi<{!+ zx$-s<1(awKS@8)a&P?@(s|we*Qk684nJm4KcPvv5v46^DEo;BFdm|?i+s`7K$!0Y6 zW;}WAS~6RTREsvK@vbSbKR_1aF~ZF1%;XwCCaF?M@<+6oU3)$9k>SW)^Qhi$=7LzZ z+~OcU%#5W>o5@QR!$?|Kb>Q+TccGSNe5p*YvDI90#XdrOXDFU79X{rxzv`B_Iw0!+ zG$48KB3Rr-sjCE~o4=hQXRW11*`QVMEf!zIi}`K~X>76CPKcwq(n^h8Pj#Mz*e018 zOA$BFWfY9XVhiNb-h+hFuzGE3q@q*BxNUp>e7#UO`d%}33X?Yumt|)(`nJ0M*vgSY z#a9lA1ZfGd?iBVIIj8sWmjtojlS*Nwm(dPw>48p;o?R^YLWFr04?*XsAWU5?99Ofs{~+ zr^MTmS^OKb4Dkdrc&(GvNORaK^ zFY2tKh4q;o1WGW#o)P7L}VW}tJz0Zn&CVA)pE$1U>G?n^uN2qOEDb)XA$>GGCnv> z3p{Nz)#>D53QYV0gGYcnani7&zpr7 zfOM{8vMnwb#EUEbCa z5h4446u_~V%ld+(?xKEH!L_O)KAYS*NM%?lJk_%8msb6|Rd|M}JhIGjQ*D0AS`7C- zUB#`^-JtnA$~nVJqeWu}*jd1;&fE=uwC(4#$cwpa3`veQ9Jq9#%DL526wfEyM=2P1 zD*xMIuwbTDSm}r%CjHQ2Fi*!?wBZ6#gIgTPP-f}7cgw7VvGG7st%R}CHAds`q~=e< zaTh|&Z+;gwavH)ZX!(bGmU9wi_e z1B7@%oTQl8z&!4GIt`iK%#c*#-B(CJPbkp948@JZcEB+kA6)CR^9?BLc|yO5dqq-| zzjsZ7@qm^uRr^08fUn4+nssJ3=NrV- z3KZVkQGo!u5Ax1xdpt}5f?a9$qo%;-rI*inhvL20W<;Nn6gDKRbG$M4K}vE6qH|}w zX*aodsa5KzJnzp5z>oDyVBe+CnfqL+)0zD}$E)JIe#Py{9sPHr)qIjRUZ=x)OZB-@ z)jl6?^#-F4lY3eXR+?^)$-~36a@EJ2Yd_KzfcZcM!+rQAmjWJ)+~@=S%%857^w$p% zLeyKWU|0}vMHVRKzYha4v=6Ye0X8B^#R@1d-=5=WfF8_t2Aj2AHdKMw8vu(H2-!z@ zpO~AT3Sh=27`0vA=qN4Ca1}V)H+)&*^BU?QyYS zpjmA&aF5vm;7}BT6AnNb%v(@p)Glk9Cs?pxBq`A>*PJO;#*TMcd^dusNkjh^2{ITW4R^1UAVb zJ3|A7PHCLzR0=_Y;e%fxLa2Ofw|bNNX_d=0HYinFey&ie@;;smq83k3tM*0^Sm&+O zJF0)CRQ)RKX%SR!xx{LDsGyZHX(&Edv6flb#^?J10ry5CKH4h7Q>`c8K(*PaY{s-Z zAPfM!#b?3k<_Scy3VCa(3pMYXVntTV*-{tVGVb|2Ei)dA?}YDShkV65#D-1k*vs&~ zU1?a09vER;vm5V(!nCsz2Ua;h#HA!5F>#Lb8COui;YMD9)~LOQ@OwnUXPe~x+Qbpu z!z2Wbm$zJJUudIJ<8tIeJIW;h_^H`&db90H8Y#z3TJaqXxD??l#9%C~dbSYEr4(%u zE8XjTT+ZJ?cyRLhb54`Xpo0H(He zM}XV(F3Eo$KJ(E1(WD<6_}-plcy9B@5_zppHM(Ys)EjJbdZF+fWJ_kty1%J6+U?9~ zd*2)-G6Dc_vDOsz_7DiG>n-)7C4zU^Ab6 zo|cyxp?w)gv%5NYd%gmGU8>(WAWi^>M{%X~q4nW{kH^6N|p_x>~Ky%+*@>?Db(U zm zdK0D8YO&ulr&caMK9x>ky&0l!&y6fmuIR0?Sg3YCZQNt*!*~CKA|i)!-?8WaI<}R_ zs2|Ydws+DHi1aDo>DYes=5zsOqto}p_h3Jgx)tZORsr7+QpwCb8-Gy6W;YkLM-g1l zR-A3*Q7o5E?@rj(Boi6D24g9Yd-?_IEE>m53o_b(E|Av?GQ4n^jNqReg^2(&%)7U2 zzf@~_BsH=Om^1=bi+O^rUO>Pw%nE~n+ScMT$LEVa0{h{rqM|pi^{Illn}I$8)SYGf zG5v@=u^2MX>xn=n4l%OP#FWaGGe0w9QJnpP#pELNU@<+Zx z9s_YI0`WNh~0LgC_r>%FZqSGc?ARjlTvw#HB>C>4AjcSX*Q2YqDLQe>~+G&kUK@2D9 zbpHLR>khIHH%cP=#aI;IukDm-#2KsR6Q}*Gz;wGq(>N+cy94f9mG)m!DZk-QE;qy) z47QQ@)^&J{BJg=sySoZ_){iRAIbQux57LE+$6ze@j+zKdjE57$v9i{L`-;Bj^ornl zI*cTY;#$ASzM2_1Ch4fdHsmi!$b2HVwImS>~`CzGTf&K5$pU~&*?8*SHm@QE&D`g zn!QWTYmVe$epf;|S@P=yWkFZJMmig(bANdVS*^MajntyHIb_q&K7xg0;N3Zoa$LgR z1*4Lm{#HDW;CnrfPpZG_M&cFXN4;1yZ@#Y%-~EUt!*jEpIFX~Nqf-rv#m7u9g6DXB zquJb~-u}vIG?6W=G3nw*rHb`gg+qpCN1?#Lb;JL)KA}v)FFE^Lp+h*K>*0Yhl~SpR z;~v;LA?8q_=aX-0w14LZK^$wI?1TRDt2ON16)zEM9EODc5cHwO9>Xm%DU+TNOXoeK zxBZ>=^UY8b!4F9I+j`$W+BLqlJ&p35aVwCF55$GrV6P0DM8FFSpsis+wqy>C8oB=>Q2*z@!`Y5Q2Q2Oo8TQl;9f*m)`ZfTlJmn+p)StK=DCLAlvQFvBUAl!pW!D z8P4{HJE7FTmNiQHgco)cXzYktLk_0N5He!D-A~Z=V#JUp!+Cy+!AUF;6C8;7cz1^g zo4B_XxP3NNK;K-zG;CXy$ns~+>S1&)RBKy0?MqL!RUCHNc#??`b-_FQH*dBBF?TfT_`;<%f@>EPG6aNq7ImNqJBV7-Xkm&OV7x zu_j6k<#s$&s0e_Zm@W5wQYuqxH05bJhQpw-*+7wLb-T4e_}Ho|6l}gX&bE};=^wFR z`{sG)=3kVD>S@YRJ8ybAVd);q*lB-owE4zk9Dbu@K3({oV~MW&lWl<&iNzTHN?HJ9 zK-ecXGD=L_`O2)b)~E93;S%5SfT>FU^kluuoOkKY^}l@CD~3xe1Nh)#W3Hz1s(x&OSY6|0s|!zvy9OFA_P_AhX1Sccj8P=rM^ENK|B?Jr zRpQC+Ay2DT+o8}&>TlW$0J38`xEY^-X;YQ^@kH^R(3*<$h>Mg?->W2dFCov^AS@SI z9dc5N%3w+|aD6)Nm{c;-{qdbiGi-65^Efhj%;flF|E=znXQ?_ZBvQW;_Ath{uYe1Y`HaU7Q`)k#($UchU*5OR1Su?A(YJ7;s7QeW^y zHzL=vf5hqqu(rP>>1@z9AKT^EjY#?WU@XVdXLwvP_C90re9I_>dzgygOM3D{m2P=V zHc#ba{le;!nUuAOd`IAQjpCdZ+!w~SvQ5e#=<5iJkyalDqoo`H&nj!3lz+TnobI|a zGpFIbVi-7v*LvoFhm$P1W#Q=w`|W~;LCP!QbY&z1okyDZYWI^h1KDSIVN%My2-7kB z7kpyw9xMi(j)Q`X(=RTX4)|lHM)(RS2;gn}0);jryluhSyF0Mwv-)I$=eDmq=X9W_shc}~^41*28r6*Y zmb#yxsc$0I{(K2H6&IF(~YoFs&GFg z>2d!0wA^I`3&*R3)+cG7ei(Y|d@FW?EjnO(OUJb`rtY)p?s$ zw@yCv5wdFELS&o16PPcmtUj-3yDh#kj%-M)IO&QexGIPci8>C|kK)Q`f5M{uG3AKw zbiXkiHd7?u0PG^r@-uVSK2@|EU55Pxg*a*784Aj@UgK4{vVFK5qF*AyY({@s+uwmF zn$9&zFy~S?z8`5=by`HWGx(*mBD}oc0(9-vpp$W+tDtZ%cwaZC)q2sJ@>QZ=L1PUH z-=Ge$;memVZfmy9jO4I4?sF^7%}~Y@O=UAXp8PCEG5L(4%fKK;;U3Ux+idvX0PLDb zzgz~thL5Ya?Qe~q=;sCJtZ$vLb)ocaB-fPN#9+ZP{P8reoL@Q(fbD}b?hE(aiO~+^ zw_oKv{1EI+pyZFL3v_I4!g4Cq??H3|B?C1Q}F3kHXKNFF9c~{kS zS4s(4@GzV+TrMfCdGzl7Bol`UV>wyCNcmv&n5z7Ce|^CG%}`qbcrEfp)J2VEF6&ip z`*Tj+3Ez}&D5#zVx}AXmhqFsoH9FE+MhUl=pRhyv9_a%RW?3=OP_60gLir8AOg^Rs z_(#rqItwn$X?e2PT|cims-BZ(7u!_($5JM$d!X5Wd%(N_CDM~#0zApMQ~S%Z0uy+O z)k<^oz&Bhg#^dJ)4lt^K_}HlElTX4<7NEP#G#YX&JnpxL<5l`5nwf|-y(U0b-n#7p zLzzkvWsGBQIYx$~C%Y0#z(nIpR|49;G9}Jgwy!Y2v+Wq8d375@^OY#F5+V`jxLsdZ z?2HRA>2Q!nISo(*ZvB4vo?ne`WXm-;jD8&l8;z!R-eOB4H+_D=Yjv+ll9=BMlT)ec z2z(Ksfg~%}h`=Fuba7Bhqh%;b?>3rlzDom`37Hw)bIB=9B zaYqcNqh9|YT}_t!3*7El!|$cg9U6K)V~s0;L6?(@VbrRxa&*leoF?Vt*zQYNE%EU@ zZ<4;@Wd1L`rQGFx2c_Ru^oXuE10WTnoVN*fe1{WxSGFl54|bmPvcS&V1lejoKM%&! z?lphf$+KJ*UB}ekx2~z6(~C%NO?PkOd#hA-NN?ou8f1mppkrmRYQV%y>~kKvm-0Td z+R%yJT%!6}<->I2fFVZ*%fz}nx*Tm9zUKWjf?JU0q_(dfSReEm3e+ScvAiMjls@hA zoK3Q^0HdvH|7RfYbKX@GT1*cgeK_0F53;tBJH^xkil7JpUaj&lg=h{m*BX+ zo8QAM;w41aWLE*niT8afH`18;>RP7!8{kcWi>us6^&CacuVI zq!!F|Ql;!w-!0#|_3rYXwLj%Xi)u+o#D?Q>RU;q(bNdD$h8SB(tc*H_^m3bm7%m`Y z(0nak)x`Gnt-zbZAP@%rbIu#BM(sG*nRf?h;k+O*e*6pEO~fix4*sm`$8ae7AISb5g7X*`fNIv+rJx{dGjOB?YeeDa zCFK8lR-NzrAeo}H(>-7rEMoGf<&iy(5$|{9Q8aSg@t$P?+762?T!zed0z#r(2-L;4 z0y_2jUk}rz-=~~IKgW-e119bSpBBZh{m9ilMp$WTO9XxM#$d)*)9y#lwpdI{LyHv- ztddV1PwyN+i*dAk z1#5vgd54x*m^c>Ics+d8Xp)z?Jq!dS^>_R}27GQSsFWz}x7^<_a@DNtm3mGao8+2Z zVPX+2+54KjrJdFvuFe>$A&FwT+&JrBrgA!o(0Es@G~()p zM{}4prbbxvyAk1gHa#sQ*Y#}9uig=A!q2%A?zC<^m~D?#{ipiWU_1+}kr?IC+agnA8DACv^-OzYt3eg8+%6fK)3*d!!Lv)gj_0~ z8!!?6fYm!5&ACAU%GJ;!jBnR;^=Eo8F`AiGnSu3=^#?6`+SeoC`tMe0jg7$89 zrLu&{Sm#QZmP+GUgLIlV0)+g9j7FX`bQEs-7ke4?+Ays1)h_{&2Y##|;J#c0UZQASpD-jAyO%ZyU29^hXZ! z5;TUR7)xVqaWE9>OMf7sBacpx8{3|Y(3PSDvhl}SSwC%O13>7{*U5zzqGHnY@VKkl z*E-7-!&=k)6cgOxsd_72BIf#uC?-t!G!e(0u>$a8nk={HU~(Dm3z7`AGy%M&p=Q34pryY5quN2#r;8IDLZrw^yIKnX4_Bo=HAr7YVZ}AhQ*O_|D}Fa+^?a zYJh3?BSNF_So>T2h&oNw<_5gku~MPWxAe=AfHgS|``cnD} zjg}K0je)Jd2xk_4R(Zc+&i@Qc(ya(cL;prY^gHY5hIIs)Rn;|wzw2ZFJep;nuUFa` zrA4#G9`-{}`M!UgI;aAX90AuJ(_I`@!-P?Uy4 zv3155puKh0=F<6 zr`5}(aow`3W&M>Bh&M7*2cFs+UKupRjEM9K{Hpl1pK+#Br+o?#JvX0r8Wxqi_}0<{m{v!(l@h1-9GF@JzFrhllv({s{yT|m7)F7X`1W^t%26d6|tSCd#6i{(I+7zfw=EFi6)hsvlVNA~H&xpOz?a#3!|?8&=BqGh z>Je0f)Ju$LC3F|HUr*FVGO&AU#yDfVuiZw-NlEuuoa_sBt-aG=ny&t+#buaMoEt2V zQ>VCHA3yY_Saa0mz@WZ5Y}nhgt4ob0BYf(8IIB~>sRX^;8Ym_R9j*Q{Is^C8r|Aa& zjGF?=?5QRFcU7>x`Ifl>J*8dsH0C7T_vqw%1b4Qd7SNiq{_)usZk+LgOgs=0)d_pdPyvuQ)xKxh(n&<0Q{ zyopL}k)&!vDG;}2nQJ?a-*XNL^G=P#(CpNnu5{o}z~Y^HsvPE~X}!Kyw5sw5OXiT{ zwCMkNwdmXTFuxMw!-qLiXrY4aQ1|k)6 zxn$kGSl)0fZa5VXB)p`Io;Fdjz?H;Qx4&Y_B12j#-%##vJkJM&h{t~ud*7#*Gw?NR zDd*wkW54+ght;$V%t2?7ovpU210j^trYY;fpJDqfX%%(22U_gDY2#lKWq>IA@C6W7 zap9ua6@$58a~@`HXM2<3d-0=-qKAL#xI13h+D+jYIwLDUz4o_Fm|z-+$t#r5 zMnr)o?MP%TWp&*j|BQjf*p*6dg%Of+E^r>ucIXQg$U(rv+$>9AP^Z2+CW3|o@&84zD#*wd-e1URUc&5v>=2E7P;7QkXZ_u57sUS)wXh*aaOgVGl4G0t zd`gDT@C9Nie)QDq^}%@7lK=enaOP-Ig1*Lo=GqK@e~DZH8kO2jcP7_Xdd~2^Qg?K5 zPQ4h|3y;u6jkVnnuj)XamaEejTyeOZuaB1_M(+Vx-JwIb(|wQKI$bIX4#6SA8Bm`& z?IGQryw3Df+Y?jG6es23&P|D>h8wgB#e{9%}tMGJxA30GzJ(g6o_#71RJ^b0yB3M4w zHUVeF|8p{N)b-n6$7FmZ{cse*bEWCcVtO;Yf-DCJf%Y!>_Pq>G$#5AiKR`N@HBg&1 zJDpf|(nXL00gtE~^Lv0-21b3!+ ze<$D}>Z8)1%Et;(TmP}^lCOF(RD!lV56rubPI^eI@BSE{82C!SviF%sYLiF}r_+v&odBzq-DpQ_4sk#2~< z=Bp2J~vH=TH#LhBeyWzV4+3D#E_w8K0e_YBUf z8O=j5=(|>vBqA}VRs)~G#02==tNn`dL+KkK?9oYR;w>6h+^7p&stFnFhvE;I@~6UT zpy8)sjI1nRtEKY?R$t?dcK7we1-{tzFUat{205*DonzXI|AJqtvzAdlyQD5(mvrGa zttxLXhZ|Af`WE)4T5qvDzlLq}o5Fu1D63WpK3v-Agpt?E5aE>7aPQ?L}xMg!&&FHo`Q z1OCA!MGQ_-|5trK{&pwSqa2vW z4+xe(l3AMLX1b#7Il39`2V+N7AAaZ~y!+y3!GPbYIl%Hyy9mgW{o(BzY0jlR6pWQS zFs|2?ff~;K98~OnD%J*ME(7icFFAp%sm2>|!?M0kw7T3Srq4c1Ce*eB-}5=~IQW<3 zGWNP zI2Rv_yTIDz$8m1m_>kX?$k*apvzw&eAf0+&khC8d4}0J?DV7FgqdUs>fL!uE#mmAk zi_zl+(XTitp>~hh#(upl885eMZvfS%Ln_vs%b6G8RV>uC8kmWd>LgA%h0dupSjDNs z0=MW#YBSN3@Cz$yCM!>Q{2|#qoZ}V?(Y3QHp)Rt#Az0Jq=D04&p?3ezV$FD8&YZE; zEd|F;to#YLd%n#r;FHGn&vMX|M0<|Po6`}b#ISvLM&}%OFLF_V^;E|}yp^EsqDt!(sA;! zw&6A9nKpmFgf9VwtOHK<30h9rob*n;o$D*Tm_VrvZrO|g6ke)QF`7RvvrXg_nsejn zW17?3a*fbge|x{9YgKAkI`>gJ^8HG)IQZCMBDow|W3P0IUs!0U74=jmzu90Hu53R2CdbSDB;va;`78+-Y70sQg zKR|_wHaoRudB5#RjR~JEaE{YaUJC)-5uax6^A4xl@mYxKzDnm5T?Kr!C1(bqyHg&w z(Sy;``rIG&JtZ8~%wi{lCFOPpIeJWNNGA3)^NaINdYGCHjO2QWW&v3^eG-i-yE?W9 zkSNY6SN>yDqede#^S zC}{upgwfsbKa8JVPn)e_ez_yJ_6w+fobiIfbs8L}cL8ads&EJF7|3Jh<#n zD1Ga2-#*y3m`YX~pc&*esB|e_! zBNE?zM*<#glzQQ488?e0V}B9`ty=PIFk-rC#=*oy8R-!q#$FZ_K`O5&=c#O#2w#gt z!VwWc@Va{~x8|<=c2kSVPi)JT`I3TGn3mM6v%M%0yqPL}k^fiYnjh`5qD@J=XEDs& zlOut^^YBs#ag+xcIasPQ-;)T*54F51Z#p!SCbMq{DacLyv?BqeM1*)Y ztTy<+Og>kxr87;PuZFFrrD!r!{!vm3-7i(1H~3RR*te1`98Lztg_2J6m>qx+tl+u3 zuqT^;AN`YW3L4tb`ZV|4q}^*el#PZ9xSgclI3<)rXv=oo$G zC~|#>88uAM9u4ZjOq5{PwwKylZ9S`1CA0?d82_W`Il{6ka*ls^j9_spXPWnw$CKAI z?^WJN{Aqqm=2)}6jSVxKfxRV5y&=8T?IC;5DM^{EF^||AB`6wX>+NmDE3oEMgNjEq zRzlSl-c;hM1AF7u!e08F>*b_gy%UfOb)8}4IPgO~*2QdQEcV_UAm@iRhz*-LYH#>Gb4aU+5xD#KR9o2m4v_mqqX2+ud_-kIm*0i^k^coZBGyp)i zoVJF(C|_Tu2GN|1e5U`eX94KBJti4H{PwtnT?&JvZvs{k{%!hfs+!&2RVO6T3&Ym} z5$Cy;yG*7_UOX!|hwCfA#H}6ha9=Gl>?n@fC{cd*s562T+=C-kX?#%~CP~&Rux)GHs%^?M|FYhbNvFCO=^CY``dW8!E-m zzlu<-&WqKMk0h>TS;ho7-i093$TZBSINcsE2o-6!^V)-@}XBWSdG zx+x*iKJ8@$gyHz-|48Fj^0oj8D80a!K>M5n)`+fd3P*CZkGk<<(3BEGFc?RzU1Jp1 zZyzi@dB(c1f8n-p`~Ia40q1GK_|&YKEAmF#)Efzj^&6o-B~VB&XnuS7sKT=e*Lap zI;q)a!8WjVwPet$%CHlP-TL(zph+X(FxPT4K_3u?8IL3fOL6&d*fM?X$FhpLAI4^4 zAB?r1vY18@Btpsl^R7x1UqOGiqI*gl4hElRFd4I+n65P4c=$46_U`o$4-3Ihrk_@c zpg-=9u1{mqG)R1lNTnHy1}JtC3gsF!EH=CF-nHTzsuAuU(q7Y8L*E1u zuo$g`9~W!F%>lY(djQDKdkXj{+xzb!o1Dhj` zTFcj)fr5TV!!1fIQd;#kkE`8ZE6x=fb>isx{L7jgP6A*@Y(^fJBm0BRaeNO!-k5SE zm{0(tKC=y(=lUU=hN1*pA74i#NjUu{spH1gbuKIom@Szsb1aQ*KzznA)>U9SXO#NS zU*|c~aNF@;UtbpXJ{r2k9%LGZb4Lb!!GlEg$rjIqrci~gH~Sh#1DWFGB^Pr&gv8T> z?$ZH-1_~75U1j(vYkfptZ$4}D+`4&FG5SWq{b3A==ULPb#I`MN+CPGc+7Kw`kL-tN zTCoA=f)-Nkg2MMet~sFq$CRrq$Z(r%7E#>R7zJ{OSFjtnwf#wle+vlEK7fj~cV!kc z1FZ7w_d2? ze|igcaczKk91Lv(t^6@AKD%1tr)6xUMvs!|JGq@O1A^q`5=M3c_;4u(aF#v56de8s zpLo6wx>CK&60AQ9w0Sn>=@m%V);iIk%u&0~MNO!}4m?~+4)xpz^DN@V`UqcR!}RdCY77|YBl)Zo@5ci>oX0MjYkmhRRNC^0r+n+!5={- z`NQyEo&f8O#B;q4l#vT(G^ESE56VyCN<)d=iuUjxgY8UTL};P)W|6fP)0g}(^Lp#G zU5{-F9Ef_$q2<)h_8Zcv!B{%=A{X1bW8oaV8?-896@ZO>e-Q(AtluYR((_);36Rh$ zJsv`U;%FfB+NfIdttYfky-e5BqQA~MW2TQTJ*vtTD1ioQvX`4!n2^(fqU8)meJ7*c zcz!$Ozl9f-N60#gqp2KM_n4*qE5E&FdJzsLE749ct%r8Qj%I0MlH?}J4S6&Tz{My@0yhf-Xx9K0o8@)P(DQhq#{O}>2%RdHZ!;*oP+d;mSCwC&TcX7d1 z%s38IKEN^Db(LG>{VlTz76jn{o|JL}B5c*0?>JdWGWvuZ7=2!Csga@8HkoeQFQFR5 zpZ#9tk!HfLfd8(r6x?8jL-9q5B}U@2tk0DG2@+O7=wV*9B#uHv7AL;LeheYz`R_1@ zeuS1?*Pp92?M@~VQ$zx+2td02!df1N%-v52_kE+;XKuV0NqWe|R@~;Yj_y3s`AKxo!XN%Lt z-nHms1PE6anO2rUwhQw7Xp*2hYY#E`&gXyE6A7R~8`UgjX1I|*c;7A<1A86dZf1dm zZEH;CMe7StDdPS1a#woNv>h$+)gE}z&c%F{eqY$9yQ7NsSHQ6YRf?ALl^X-(8O!-n zDdQcxOhDlW67Rjv2a|bss{k$i=193rZN|CeoUZ+TGxYlU z8YuAq9?V1am9EBk44=a;m!2OeNa&DATKF*LJVfUW@Lj38oI{)FzGp*-9!(cbwYXk^ zuu-1OI@CC>5sM?ocG>e*C@BcI(^BEzRC8ZA;1i44G%#JL((Iobj0;9QpH3Rc zrBnMs0QNg=Y`pX(u)$c%6+=y6hmlTX@EOt%=lFm&y0!=G@CkN@lYEWgN$*JWBv2^$ z!6+@nm46ICrceJ*u+_)_><0zKCIBk_I|*!82$p!$evD_X?*C!$Eu-RWnzd0NXb2&= z1rP4-5N}%OqeRd(!2bw}IPWKoTK2dYaSGdB z2M*l3`7Gxvfs{;U%UgiVL?YMd*Z~Rm2t?+3-2rZGmy=SeI-ZU{+sAeQ!wby{`)tzb zzwCLl;(k~a%%Jb9Gr3KtR+;5|0)92wfZsYmav4oU3v)fhD~kh2JX~nBH_7aU?!h^d zv93C$*Z}shT>w4B;d3>c(S65)ADfIwes(z+ixq%@oN+->ER2_?Zwr2-@8sPn518yhmE;iM63~TA3h9~H(>HfnI;7x z?YG|?TSVYJEp;Lx#eU^q_2RHRn3Zm72-Mh)F*6!Wcv1Dt{0gmvjS>>nv7{3&EkpOs ztML5Pi4@f} zhTi+bq8Hu_6qf>yE&~_>m+1?olP62nYnG3L?pYpQ`;XcBZ7a;tt6lxfjXVeI4-1Lk z`@#{Nx^Z)$wF2ng1E}+KjakQw5hj4u~JBV(;}4k4QAla2i(7p3C`sLC8y;s>~`D-?Lq^st5AI7ZNyy zu0u#8)qa^q4D-^IQ*Z@xMM?dCvf~K`Az!VvY&|dhMz5R*)v-1JK zC=kV3dujyYw&w5nsBCs-dY_N4u+5`*y;8JHBR}*(wavp7Yui7Hzx7Xb$(|HfCIr4_y|w4<1jQ)# z4hfKbJc=gAS$=Sp&f~<{FUe*b!8#vzROcK4o4%t?D^D(ec6*68w?3YxW?l|{gs3wj7 zmKe)p_hVpTnL56Iu2$*yc)n?WsjBOLFsO}Q>5}d~hIy^* zMkcgIgZZiP6+@2+n=6oW)yOrPhOzeti}CO+khCll+cTK8871I#cfOS^+Vy+7M?WWs zB40M$Y&ceq9|u>IpX4hv{IZip1&()w^9$y1dY$1!DL^)u;`UI}lukx1tF_ObbZL7T zLgv3rGCJ#Qt{6*Y*=lvxKC)j;f75n+j<8~IFrL%cp-+N4aC;2Bp)f6wKlf1nysV6uvh6y>vBEJ#n4QpX&JlW>?oSs zJRiD}pO7lCt2Na+$5J`QmXih;g*sOnz8UnA0~yQ%z{)7z4^YvV0+^cdg7B0+=rMlP zpL+B0GMPdq#gOXuxk+7{gX|D7O9}-(6gb@b9LvV zBBRAXBpd(~8q@dFG?v!-)(>Hv?)Q6zBUXssU*FZ}!xN(80J%CfKwg?UO+ed64muUX z*#7Z`yEB0B1iNo9!Yd#JS0WVda2{*C-eVL(Mz?HV~S|u+Ng9GC_nA zz76du^XIgQ21?W)!oB*V^P&70|H>GF$)6~_KXPfA4nWbRV8Rgvhn#yHu8mY+gnOIU z(#q7DoSTnl5;JQQuG$XT0``CRCC68zz2aSH;;#kLI0@ytdY$G)!tc2P6quy2TqB$+ z$@;GIK+R6HCD{koD#XJ<3b*MAq2y)&%rS`>69T9R^*j-%nh=*NYFQt!S}}4Vrh*#s zAVxvx^xK=;`r;9|M*P^>4|VE#@SEgr=`+_bSX#R409C`t*MGnO584;rGxjE7ndAef zZ->e`-18q8H3(Y%asDh)al|3PlnCQjbfg54tzN2DE&#y$za5s+}y=HbYm@@bULIpbJFU1ZEDU6r*M+(b@GF!)w)q|8V} z5LyG7hJFwWo6^cV)|mDmzO!oAX6L%F()SZ7cw5<&7jG0xXJZ}**^0W&TiQXg0dx%9 z1-$!{hE}Qj`0rG^9Uv+>w_eziS#FfqU?PQN1w$?dW$?L+fyL%XCe0_87e^Zer961F zZ(4wJOn!vQ?9zN70(q;WaJa*)+ra^BQ)<;&ay910ZhwQoh45YrZyZhr}3~&UiTm1$fnjr2Mo{(UEb)f3D?+lKt8aU&+9InZ_STp zg>s)P*SrS~sgEe4{KujMo~m`%NkUtftmA<~mTr>>eL$2E+iX3g=T72;oi}`&>A1aT zy{WCBtGO!hl6JZC`wQSc3^Z3gH7fD9pS5Qufz1~&4}qA7&;@QdSI%#fae%9J9hmec z*hTX~Xsv&It;q0+7Lwn)x3^AmtF&hHyqsiR>J^$tONBu3gsuZr1|wK_O;A(kE(5kB zK*~-YAZ}8y^x0ow>EW|s zOibsRZ}(5+kdS`;@NRn2ErV7iI!6_!3AV+qo zInRYaZT;xas|4O+kj9n-xV;D6AE|=~)Dkzpe}1d74{mO3#PJ`WXX5S5Y;G>Yqu@LD7m@=rWGzMu)UO&`nLrT*o*7iUcNIGA8JfUp zZZTPxN+*5Zn4Tn!Ulm)*O0}cht>tpV1lByg)iK%N9JA%yb6S09{s-%K8`g?qNhkEy za@_jEhJ%$Z3&rW%4MrCQml%jbFHh{`Ch>$rfkG!6-OL{!vf6GBJ+X9q1nehNlG;yB zBKT}vXe=`-NmL<|oy>aW*n{J}oFcCrD(l4kPL zU|O!579aIja}?jb&mHf*1R=m5%J2p+v_ZReG#|InvHoFG1-02N@Fp5ek*mX6BVV@y z)M&ZJj+3^hw~)rqY{W{dhI;%iCrD77=kz&T1=5crlbI~`sL~$JjlBE)-Onla!8F}G zHm&-4@HT_^^i)ZRx{+5F2`86i@({0OSZq9>yWMqO&@B<#3HULin*pTdCB$kJRv@=E z`58%1hhs{06b%mVE?HnHI;H(eqoVa%qrN#4W{sgjbNWZX^$}insR`#N^7?XZ)NZo9rU5(_J;mNR>4WO>>e0Th!h7Za2rD+*+IC;Ru z{dTy&>6mYj6tQvYGaN`CQMjP|U9`ajeyzU2%SMh1K6pz=+k06)(n4z|)2o&N=7cP2 z7y{ZK_rl@fM1N-kX?yt{-|i$!;jYKLWlsjoR>}oyaPp*WqO0{P9f+LA7pS>Je9(sb z?0(K4LaTl_6meF;a~ynYO8*sS+-)JKTEaDfh+7KH0SPFFP-Ur>E}__|hC4z}t@ej@ z$m#3NB1q|6_S;ZH3hAZk8-NDL6M!gS^LgTGF@I@u)_tx~Q9&V1rH7ZwhiZEuL`F>> zO;=!pZ9$i$OSI271;{05@CJe(Dmn|1R&rPCvJV=DHgCbZdMiIOSN42%_gob;zJB5f zGbQl6lqaY8(I3Zax%^em2`0BS+l5~wo4EVF(lLd~Ujq6FsF;LZ_yad!feVH&Y^-1c zHGAPrx;i8CKnMTh1e)y*Di;Ru*&5HInA31_Q}hPE%Z=@>KnGF-L+h8OEo z7hU~IU^I+;1MmIM?kb!==)}(x2Yg527Zgu-B{#oGN@d}{_@&G8mMi3kW_mue)e%1) zOdQM-0vVO+)!qqoEV)r(^qdr>#L}|sp%}on2LhLViJyA#1#90({bg_Gia)Y-aE{0A zX|&aIsF`td+&I&U8fVyZLul>R7Xa9E1U#4eL6)kO!ODcucRjl66 zM!zt?^i+8GPxhnD0Z?oHXNW3{z{cOR!DA*+Xb?K{g1*7>ZnB-pl5021>0=q6+pvwx z>`(bt*&_{3N#jg`2+CvIU8X;#z=A1B7tQVQ8`<(8EM2xwI1=aTqLNzq*}gzCv7FEQ zfzM)(QUH}B_&HQCn%JjushaE}y}}Mt3+`Mikj~B%xrd*KcDZ9T@r_Q>mkrxDP`W;r z;;rL4`ild2J>);eNynY_Es)>k;24lf!+&8wltcU|jQk^>|5AXCTa0Etd_0W32L2vQ zTaK_Kcabo1Q1HnZwJXCJP~{4|Sd=XABI&9zwwHgjg{6~T2HxdidM^qvX70G36?M>4f`l3n49tF z#%j0KwQW(y60}fDv17Jd*Oa*n8!$BO;@qkbvKH=bN>MgXMh2oqizpC4A^?jT_ zk2lpGsQvL13J6JxZMnM-vwlMb-9J>(^8JNLobU|HQ;|rUJoN))H{sfs=>YrO{>+X^ z2fJ|@#!y2}j1Y7LJE*_>nBjTc19o1CGT)3kLu(1ev;KCsjmH2GpsA1BJdfMlIbb%OUIGSc*uXZ5E$9q*B~@36?X=+0$LGV%l<*B&x# z$gO~(oXZ$9HTu>+e!+w-V+4A@f^oBa1EKjMRIzFx$9Jh4`8=eb-T7H`=B1J1{hmu@ z9BpLYM^XQd-tZ9#f{Z{%dtZTx&`U=q!u7+qQ4uVBAD#CcPg@19=M8QdGfV<&BP{pE zCk;NMSU&OHM+~C}Ibq=(Gk@u|bCLQ%MUx7jcjLFtZpT?iK&0sZr=P?kVJugH_#64m zuFz%{-azQ5A^O2~S<-Hi#MH}Ba{55$R)_oU@>EHN^ji6!v6YRzAD3i_uE`~wqZK1g z^@s$KmrH0*VTVSJ*mKiaOd{uCmoL%jF`X9D$OgH|CH*LhnQWp~V1`Ceo*H`7)j}cV zt&0M$OhSCON3pvC2sr+)$Kq-%*Rq6NRtl18gVd&#>T56x8`|1J} zt5JEpd)LW(I1~G5dVUaDDw=ow(rNcvCPT)1B!y24f{~*o2Qq9TggVQTMV6E3V4JM~ z0voYeIrhs#DOx&8fCXKSN7AoftV*}L@6~q9wi$WhF+9P?v-<-?>x$TNd53gTfgxW}<1 zA}z|kqmSElR6G)t;G_PDqJPuGcxn1i4dX7%Ef#z9)3uGi+4tsJ!vTdF%EtY{A%LIU z{9s}u;%9JIjqwO1>6&%UmaVQm*?zUvo$`gX*X-BbE21D|LPDIt;nlJ`yDKPA|K)-} z{T&7ob7QPx(R-|&Rc>{ao9Df05>U`0wl`uKclU~V&^2!0&-_7L6PL zyE1l^hHL1)f~hp@?1RAQd`bb|)zZ>{Yu#mP2 z36z81kOs~i=Y5w(m3Q-27G)DL;qJ4{3Pd|$J9(MN9h^rxtAz_thP%dB^rMR#xr+K+ zQ{KqDRUrV^Y^3)PJ=okqom`^wAuc7_GBUz+W5#ty-7TcSlHjF7ZM@@=vvKbQ8_x%# z2KD?*5|4o~PM8%^T;yH&R*Bm$iEWYO$%PyZqW}2TuL*${&%M5FZ!$^`Z}gSOfAPN}<`w@Wtq* zw@-(rSTw4x>O)W{Gh$*A5}I7?AsuQjx7iMT8Per=HB|Au1AUh1}IDPGS$qE8cYhoiAy*G-4 z7!<}(;uk%Zz2B5>&vbCE?_t*;4r}D*;m#df)r$R%1GcE^%>60nXM#Y%LU82Mbxl%< z^mTr)iq>(oRaX$nE*IAl-IzRXa%g)2V+Wt|X?UeQ7P9dLa|0+!l3VK6U?DF)m#s~c zff;!mo1dUt-_q)5zrRq1rX$C9N7uW|$sXa=-}S0$G{OQ_&twxIJ9Z~Q1KL~c%j51c z^Q8~M_G8>^7mL&gvfK$5UMV&+2$lU7NLfMl{i2?u`xr9^ql5CcK$GOF53nuz9iM!O z7A5&fBuzb-gHBrDYxVs6{3DL40kt3H=P$kjP48@6?0GihxE=Qcz&-F&C5GREHybW? zgmaUS-5~A6kvGte0z2uny1zH5nj>4Ak7K8#D46n@&Ej|iq~6D6|&Y=8D`bR1ei zb_A32%AOGAH%dJIiN|-KT(Or?Y;m-(9!^Z!LpNCs2Cp`U<&4@}gXVtAAocQlKhZXk z9%D!$gdYVKty(15AvyZOHU1d%Qe)$hjhc3c)vr|!MF#9d9umxoxDfR=qVyksHA*EG z7A8<;yp#gm@Lx;Tds|Y$_#fsg@-(AT&rTMP3Ix1f8kLGz5o_$WiU9zO&$ssl`OswVLjfUlL$?xy=chDfA?(6i zp&;#?OD4HYaA#p(6(gM{D@z}6R5uh)XK_rC>6YQWef^Y=cLYivqb{U=St01@%B8_! zHspjRmr3c;rfw4<(W?F&zG5}`$-)4%TJa_JBNLCsY+5%m&+nC<*`-FH+C+{xgs)F2 zPqpU|1fkXREs!5=Mfn42yr|#m=EwMD)whn2ZcDXYhN^UPQ*4i}3}*iYcr;zuwaIlDj zbQRE3;0=(5*Y}1M6)(ZLTIuM3ja`92okByp{Bgd7)@!%q@UEEbwY8pB>fQm_Xm#t? zZj2CTxb#(3VXZ=xD$)af{QcnGajv~(NfIEE7L%z@Wzn)a6`FOxMmEtL6{S*>SoN8 zJ^pZ}b?rg_K!f1`DLWDsNPtV0@MoTz6rHu;j2PdfWoL+^UTVsNl*!ZoBjO1;WZE`= zy+fM7gLuj>N9pCV{`ab+{B_X=@}H&W#??^MM;t7>)BEK;oUP=sGqR6SKjtvWZQpRZ>5O>IKAtUas)i= zeVAkJa|dD^tkzlym>Fey9vUtt?>CW7cvZ_JSF2rjjG2FfTHPX8j7vey1srV4N!OPx zXWlmLv9)PaUbOPi^1uP>SB<07p{p05?AlR?GCk{$B^#ry$nHhjaJh7JTzP(yABWK6 zHHXTM^$?jkhJw_NP1U1NiDp7Uv(_17Ql$0AH0Qz3F?5dK%iz>}jA_y5lOrA+Fu^7W z2X(Cy9$`1#L6>fmFMX!CP?YfFo-Q29XRu*wz3?H;4ym4EVvVAmZT_`8#-?MB)^U;v zyM1h?(mySFIY)m{)W=l5$u09U5$sHH7UK)}v( zF^!?~BjX8yTRD(M!n|TJp?4r74lr{+LcuUTC2=C7?ZhNa1SkeS*kv-2T72R4!|tJ@ zhPYm>JLAPm$x6?M3kOK3jW$bGQfTW!5ITNZDYnkfveCou-q2X2wU7B$1yKsvP?e+^5`O0+v+9A?j-8;CI#h-MeNf)CbPTJN*M$p|Qk16jrW)^e@5FPk4zzT`|-lEzkLCA0B}O9w_g%*?&9nu>k8 zvDrkd)bEkZL{5~|l&=xHYI3yD^x<(|<#zA=UHS5y!M0E`F~8Y9yC>p(zPBUO-Sw6# z!wsBPgt#sHupfqO(^w>KdYsMg!zL%|{I-?H+bfb-d*ww#i+fAT6#ecs<%-Q0INH~w zn9cBuww+gwon3n8sj$@5?N$aXEC!4?@9Bc>DidnFU!J$YMPy+Itc$(cvkY5I>gA{@ zZCBWye43bU-oQ~Cu)UQMXp$~JPvB*0J_<);teHwU`?VkDf=g>H9XGWhWFpEUME}Xk zSo^LKT?l_y^3*UPQI@vi2a@1NZygsU32Qva(&tv^mC<)<%Zfl{e|V&@)Wj>;1WjmE zU*PPgYR<1HM_&0Bc0Dg7As|B5wz7iGI~x6VO;<4Tw&S>4-S)Qk+SCN0H)<k4L(ae z3A7_zP~EJE!od-hUEVCGH7sMCGevlnglES)lBM-vuNRh!{i_&>EZ?tMpG31B(iglW`bl$CDgxFT& z$ZW@9STh4H&%61+P-@TH&EUc$xTo@E)o{E!VwFu{9eCg@Kg~{oX>z*6zT+ZA3=4UJ z@$`D9Hs6XtNYW$|*h;mt-Rl!lBY9L_7B@hbH?|tVkoLKWL2%ayfa4q2`|&j`iERAY zUy_mgoKtGsM$W5^d}T13yu3_B>cUy&GJU!HvHYV@_}~1r-nxqDXvzdd^Vy zL#Lq%ohVYIprO!TRMn;E62_4qZdfnEnR3S0CI8el*f1sz@jFkgNdRft zSDwPbVd(LAH8y8$_T?+mHEM1PhOO4y=q&#kHEwNdZUiZOGLV2s0WbRn-Y>Ns+}2yn z(O;H(h-CdlRl6Un?#pzU%_vu&BIg8(qM9WFSxwjcSHXl^v@-Qvy%d%F&kXT~bscl^%Fu9;{n^xQk=Y2NEbPSU7L3r!ee^n&eu z8BXpSDHIPq?{HG`*-5^kxdFv^coRR)@VfVbDKYKt&M|hL+go{}A~tzXI7heWofCs4 zB+*4B<7hJ#(w4rB6FH(~QEE__O~RT&eR|3@i_MbI%#%rJs9kD8uq3jInKg|iF-^&B zTq3g4pO2)~thQ8lqE$>J3!DA&o%_rFnkRkZ(DP4KR~~Hz^u)Ae=f(>%c>M;Ri_w-$ z=j#=qa+S&U&C6*<`+akt*K_fwFxR<$dtwOp1|rKSG;wB^2<<)!{{$%X;6>5o>=x#h#RoEWI7CX&>sb%!rwkLyf{kC;>|WfntT} zx0Q+RQVAQg{vZWv<5a7{iVTT4?qX=8T*neCMZdd)vUHA~Tq!~}5@9o#^bFIz!Rk`Y9j!RZJtyWwxP?$7% zscddsm$_+!Cn}cg**U(BVTDDBBV;g4a%5?TT&9Qo%F6t$1I6b4 z&K}m%FQjxV_ljmnv}7jGG|PqaT-ee`TN!$H)XPCLc=g2NVfCc>hQ)aL`*!0^#mgk^ zbF!Syk-=5K=J5rE8$u#;lnx2)S52?^?Ys zXK&gckgS5(AF9N?a*)_74n$cB8$I^q_|LvIld$M9RNfCnwKHT&t>81Xo&KUW#hlZz zDhB<2m*)Dgv|3R7q~A9i2EuofLQF|^gqihzIThd;Bbb7<&LYdD4u{bXVS-G-(C+_2<)IZ$cr>#-` zQxgtcoJh4=K7l7g1r!5mduZ9r#RynBnh4<2Qoa1y985_tYbL>CEqXOcN`N`5-4xveH>B3RZw+H2;~bQ9T_ zB7Rtxrk=BdaoV>LsMCr4uQ1r16kJuoJX%<34~o*S>N zDxhNH;wb9e^w+IO)hkTxTubfs^?eF^!XD$+dp{8$+BTc6GDBS4&RDna-OAT|q@z?l zVrsg0DbsY@|H;h+RN=~0G0^sWRWZg?1D;fCYZ0{!TB6@wp>F+Yvzm;b_I3iR7$kz$ zB&DJ~;QFFXGeCm&5Wh_~E?weWw>9shyKQ-mk)HCkt(!jQd-9xZaGO#(GmQ^Poyp_7otkAVej*~#(Yr=w<5INLoG1m5%LYF}eS!wwf} zyA3&70^JL1YX`%Vx<+(T4XeJqJ$X#=TD$N*NPkIv`Q~^k3%0yhMfR<;2NXEpVi;kL z(HUC&6ILMbTosjMXL;OPdJv!JoIEG+R3yMsG5q+@NBBaz6+dkGNu6W~Mzm~$EWt4O zuJO7pXG+6d$DOjAabRiLiPASRcI1FOXgKVXl`r@9T(X2_*CS~MR}8IdcPmCj>Ujm8 z%-(X+XzM}*6IzWnOZky1dCzoLz;23qJ?9y@jwL=@Hs?6Gzp+fU={LMUbJ}qA+JtjV zW_t}Yd^GKpyPd|kL6l}|v;)wf>{7xN(2vKwQ5Ke5xUtMP*EEd@SefpMECtoB%@HY+ z=1UmK2&r!-{miRdobT%6pe2b0y0(N^+-sn;+32OyE6Pa{iMNs%2p!|0PIbcJk^Q)W54E4;2sYOK?aQFA(pn(QblGAD=K zbGb6o@T_VEr4xQ%Ocly(bKkfo0t zU3cc-)mDW2GRm1~CWH3g?_9f6+8kZ$fXLl|6p;fR5UU}=fiS=0PQNQXAjH@27Q zCP=Lp!%Ss{Y4qMRKS3*>T_YD<5Pdf=LVAo>>I_ADN-I*a;$vK1PTFpkRv%L$|4bFd zxt@*KxaIb$`~iKx$Z%eLT8^a4d|6&iaupL3XFF@VY11TSz6>z9$Qh+fK^Q5O9$E6* zD%)P2W*|hgdQQKaTLSHZ_=fKG3cbo7_C}>^cM-&O?l=4;kEdrwz#RO~^Dob%58zEu zpzCO~)mvRFf0^K2AW+>V^x8zdVS<@$m<43GUb7IJ>p zFpKC4k1ZPZ@|r~LeeFiK*(kAecavF?;({(&n-vr5Anl1<>S4dP4Dl{0js&egc_UHP zxQT%q(PE~fTbi0S%%-vEsNQjh6%liBBXS-HXZ;F^IK8+Ib|sjbkKezif?3DC^9nt4)>6P_4%*9_rETZ}rlU8IBSlf<&-q4g!* zFqYWqjNPw4@_sv=Pi_WRpUqy4gi)D54()FUM}Vg5`#;tvP4e!mPp|AIGhgqe;!+iQ zsoz{?ECi=8JN{Z-=B?QO$u?tD!YZ>1YoKx3gzP?b2OZ%%t+4pcs_~xfVLS(hh}IGX zDOjFd;-+CT(f0{gQ%#;FagsFIVJo`H072*Gr=Y(xYn*(cxCloLmoerj4Lm3M?Lxw= zZ}tT8(>3Tl;rgTW0Zrj3sO16Q##~!nfJ2l%5&!TY#XZ9hJkzlaPyqIh>zdd~kBRl0 z->*EVM71|QvEf+~mgrweXO@>+i`osT((2SSaWQbxo3^^L9Kh1RExk9BVz+Lv>JD0p z8Cb9vYw8>{IZl>^FuM$X-iI7^K}+y^z?kP(41vuD{Hc* z*rwC7^w4rv+S}U$d3STKD9p9xD<6EGxDTx~y)ILy=gQT=nm4H+Pf^h(7NeS zY9a?IJFhGxLHg!k+&*`Yk)xg=w>u|<8=5XX&QG~B+#q?a9oH+X(w)X)j~DF6rE;r* zFv~=OguVtUG-ZE3N4ZaSkMI@qv9EC3ZrBdVdk7_w?imYZ#CY7v zM{TL0o7lViIOkGQU?0H^F(GIdI1bfv{3csWT-*^NC;IcE@+J0Le1FrTNrOvNCO&W9 z^@+2Wg%*2J&yfeOr~My+cMO`J>|+e6VuEy8J_thic0nho#E3d7#)uXrbWy8Q+wYL6 zOXh39Oy?FGi;7lP`@ke&_>!Oz4KR!QyTsDk5BF-q6Zv#_{MXWYVP?Ah83GdegIcK| ziMm#yHkKw~0)q&V{O1F#jh+_ZQV1w`qDn&;iE7$~0)uW8ph+^aAJWPUMMp*ko@f#U zC5~2cHa+qmfXn{5_Umt`PwJr!-{x7NKL}<8KyRFd8a5RKP>U#UlYIz_FPs|MW}{&t z_UTZ-JOx|2Q_cBXi30*)()>`fgrtpnZO*6T5t_PWzd9D@Y1HO=GWo!O@afG{f~@D; zqAx~{o~69n{iwrX5KvOmy1?IeE2dQ71w1rWheQIABNZN?jo{i0)C?QV#`ixZI-Bh= z7+-?|A66194-}Yy&tBC{lJWU8WJ%o;^xEEsa35H<>1lR5?*v_ku%FZCE;()slDJJF zDFHJ7bPk~OSLg;UN_1ltzt=CUp^QTW8y1^;c(^dLaI>*(xdsoAF=Rzc4j9H;J2n5Y z5Q6!?&Abi}&~@n6y{P|mPZ;KSwo80DUa@w$=C40?3)RdDbYbj{T0R*>N(509KxMB@ zC*7<6>d7@FpHE?mUMi75lOo!J3z*hnxOc4(5dWouV3r7U1BKLg)=xA4sPwOgdSDQi z?`xd?``TBX|G9t?1u}+~Z1RWFoBw|2f4;Z;_J`rOgRw`W#|vK#B030PV#26-vK2XP_RFK+&?@lE-%-vDd56H6LUIR zRMl)Q&ChRaX(925X?52*J|S$l+rBDNqJFHc5d-8R;)aTiT>xsY%E-i*>9o@)f2zPw zPbe$^MoUoFW&T_EQPyZFDxjF{r}3qwxy7c6d?QCiMFWy(cvl+faF7Zn3zL&jQN)sU zGg-W3Q055}+Z@77bhP|vQBVtEB{%ekkSsx7?eg~mi9=jC_1K?ni^CzHN9oc+SQgB5ZudZrkCvjbwoN|hb_eXhFf3k=4ZOUqFjM#X!ip?o0abk;X=Czfg zzY0T^)$)w*-15zw8q3RXRXJXr;nlt*vzitGiHkQ%v4D%_ zr;lGr%GPx^S-eB+Rx5=?0}Mtm3rg}=V*I!4uUWn5uNI?)VT)%l_3rf}2WVEG9;?KI zh$@8I+?q{!#lMCT{;SU?DW3-Ph*#_WXBz+a!vqS^0BU+&0nm^C{nmf|_aO`7O9I3< zD%^h?3t&4rq=GSI+*UF&e_0ZdAYgrv8w#xd*$ljhI3R(;VoD0he>CGG=mXd?)bD)L zzoZU{1SHt~#l|N39|jBI3jxeJ=n$!izoo_pB$zO>Fc1ApRY<`85k4g5{4F&%AVJR5 z?DXr1{d;mip#$ARL@!GEm(&ixoDnH3E28>KRcK0}djz#Y#r~3d7ygIgcUeuv|7QdA zB^b^S0s`yp7m?3O|2tU!uMSq$bPVlx5iJe0*ae6Y%GVh}2?aSdL551-d?w=Ic7%rJ z&?s6dT!{cI8c7|7?5U|K%%6>QX&Svc3%bWXY-wkPa(xigML?96mNq{?JKNj5C%fNd z=0^6jRCmiC(mj0I#onT7}+y$@Q0?A@;A4DUtipp^c~;A9MJAG24=n>RO{1} zZ~opHltzz7X#P-o6zT=wUKOx*TkU9dG+FQs=_d#iCtN=mmZ(e@&D&$S`BA1ZzdCsm z6wW2Km7y~8q~o)X&q+vGLyq&%rGO?vA2{bg(xBt`XMq5Xnzz7&==d%e?a&xjmZ}0KuaBm0c68PC)D+w~N2$}tE&ASU-qexn8{~KI z-|x{FGQrTfEcp6%wzYW?&ON7SYW!k)(B4}crA4$x#}t9^)yL$V_Bz%d>loY`di!)Q zFIVy&s6FA-Sifr7v>e+*+gnErNPvhrKjma+NL%AIGKk^G27|HT9-dw*vFyb*m_J1K zlPU=Z5;`Hg16;TFd{kRsE<=U}V|DracYx6H`g*iq!Z~o$QS9cHcJI@(zD&341*}V% zwK?v$)FubBN5}##1IyroESL(mnl>04OEabSmVL*h zu9&UdZ%CfuqQA5~vPAyWY?IRl<2dp4X6qU28lTR=J&EwuknDU(DaPvYc@zo4=(2{C z8!E8f`y$i%d*_{(y>gFx9nTK~6W=?cbK$1~aTGi{bykMVlcuBc{Yd?)+HN%ngat5r zt4+qd2y>V3Z2Z4hVHl^@CLRL?VSIcrb&k!lLFCee@mvombsEsGr1tG&;BPc#m{|S!eFp*r>&xEKP5DWc#BjEZ5=3N*15}`TA!Jz9OuZcj}I}0MtTyd1p zK~j#V0@V~f*wZzim)65rehPbCp{-#YVv7`a81{KLPInvVLp=b_pUzT?K4hw4)3RDA zZ8YT}cP>n+#jI!xlhbhDpZii8`-hE;IyAVb45p>5@%AQW44SCMl~1#;9v}S^BUALu zRg`gb&^<1ax{aAezq8zD8c@e@6VmK);?fAdYpn~lx_q+ien})Dpt&vrLbwUJW6r4M z>3;8)jq8d`k86VLDq=Ho2B+h3&VmvWg>)hH;I3WbczJIDiBE^p!u)LjK6I;>Os<$Y zLy2r6g!Fd`^=YAP*dCK+`U4RQGm|(;UFW>F@I;IyKGhg9NRCZQ)AtFz>L!#KSdJSv zzxS;9rW@0Y7rHbYiw#_ojKzFing?onBPV1{q3NfQsl*w z4g>td^s#CtXo1im8cTwFqv>QI#$Y@!D|z@6=s3oLlO5Za64CS!q~K}o>bgfPLNlP? zb|en){{O=(XGuUyGTJD8s1 z0BDJ$-TGVVyjRdt+*m~WpUuESlL4TGQho4msU-jjzAM=)s{f@b{A>4ot&aO!>gZPq zina@?|3e$V9lmx?mLb(E?)`Vqk-SQvFd}30x2lvt_k12up#Mwi{~aK}$^8FsfCL{E zGw&|}Cpnsapw;eW<=W_-dDKF!_rw~<4=ry%t*Cs7K1Dm?`A@8zvm;2YzPtzFIXB9h z-!==_Ueip6)?jF7eB0CAz`#InZ_w!+cy3NETqgYJkK#X1 zl)e1Bti8v}O;-yf->c9TELBz^7c8yqnDL_ex})44?qM%^xVYry*ViO8GdnUe+N!F6 z8(|imwtM#Vv9z{6MMQMJ_3l3LtDOXc+Xe>*EuEO7ZO1PGOu@(3iiUk&c(CT?;L)S}_ z{MF25celafSsJv4uyFU{#R`nZrpCkgkCKAS$g0{o6azGa;N`2=CmsM?=~MNpvu^9u@k2Nq|rx%=hv6=Y!*DZvUE zqJ8s=KBpxWVpwhbZ%j*ZaAzKZdGP<~7Cs32d_sKqkE>r_uK7cJ!6f`Z1@*TC zz^M}de>MPo<(5b=rUyi9(*Z;SS>O=cZhC5Kc(fiKsq*xh<)RT}!G1Y#TgB})_n2h4{y8JAqRncsw_3&W32e-ogdc(@j z9?QU)LqleD(9x&fW2y}7W)Dl++}x4KNETp=8_FNk+gRXR&gOXbW^63lGrFoS zReCyB6fy#gjGrDyN9pnJ&>PnIhs#N=JolHDc(mABBQM_* z^7FH^Z$rOy1xnp7UtM2wcd8DT6$m(&v!ptXUjlGIrb6fF4gcYnK&)&75ZhCK|IgS0 zpn{iprrTCy8fz=Fg}Iwi!}Qz0Avz}n-viPLzSWufGV1bAVIR_1J>}#o_PzEdjQ~HS zh--d7RXE0F=61w=-%rS2THt(9q?#fuBf!e~CvVT){bBlY{~EwAo12?ms&0wc?o}YQ z^`TcU#ItuwN?>q6Tivvrk-<-K+a6xk97^09DRA2HW+-v4{ue6`j&wG^A_WIf=&cMb zRaUIp6kXW(xW_NaSrqIafloiYsf(H;VBebTp3;A^jfh41= z;%feH=5D~nN;>HS9IJKfb0C?DSdHT5M;hvn@AN0JLuq7I{}}G(aC4)X&dhLOOA|lR z_|D>b?s4ALl@%8&q~&IZ-ESazuS(^|?AxkTXy>1y11Ms1d-`(m;(JwhA4M;DualjN zXm}Zfrf&jCInd}^#`53uGVcv6J&&cs@u$8AYRUR@7NmJx?KiJl`nUsh%jbDjJlwweEfeaO?XE7D-KUl9 z_hv^{Rq$Vp^|qVl|4PcJ3jeX}-MVXaPlMlE9Gli~+kdZiP12hV^SySp>%P?_<%G|# zdfqr~`^)8%b#5utVX0wR#FQVT{W+1eZ3bqS5IBG-^26Rdw)AIB;Cz1uAn - -![Peered asset in Dagster UI](./images/peer.svg) - -

- -If we kick off a run of the `rebuild_customers_list` DAG in Airflow, we should see the corresponding asset materialize in Dagster. - -

- -![Materialized peer asset in Dagster UI](./images/peer_materialize.svg) - -

- -_Note: When the code location loads, Dagster will query the Airflow REST API in order to build a representation of your DAGs. In order for Dagster to reflect changes to your DAGs, you will need to reload your code location._ - -
- -Peering to multiple instances - - -Airlift supports peering to multiple Airflow instances, as you can invoke `create_airflow_instance_defs` multiple times and combine them with `Definitions.merge`: - -```python -from dagster import Definitions - -from dagster_airlift.core import AirflowInstance, build_defs_from_airflow_instance - -defs = Definitions.merge( - build_defs_from_airflow_instance( - airflow_instance=AirflowInstance( - auth_backend=AirflowBasicAuthBackend( - webserver_url="http://yourcompany.com/instance_one", - username="admin", - password="admin", - ), - name="airflow_instance_one", - ) - ), - build_defs_from_airflow_instance( - airflow_instance=AirflowInstance( - auth_backend=AirflowBasicAuthBackend( - webserver_url="http://yourcompany.com/instance_two", - username="admin", - password="admin", - ), - name="airflow_instance_two", - ) - ), -) -``` - -
- -## Observing Assets - -The next step is to represent our Airflow workflows more richly by observing the data assets that are produced by our tasks. In order to do this, we must define the relevant assets in the Dagster code location. - -In our example, we have three sequential tasks: - -1. `load_raw_customers` loads a CSV file of raw customer data into duckdb. -2. `run_dbt_model` builds a series of dbt models (from [jaffle shop](https://github.com/dbt-labs/jaffle_shop_duckdb)) combining customer, order, and payment data. -3. `export_customers` exports a CSV representation of the final customer file from duckdb to disk. - -We will first create a set of asset specs that correspond to the assets produced by these tasks. We will then annotate these asset specs so that Dagster can associate them with the Airflow tasks that produce them. - -The first and third tasks involve a single table each. We can manually construct specs for these two tasks. Dagster provides the `assets_with_task_mappings` utility to annotate our asset specs with the tasks that produce them. Assets which are properly annotated will be materialized by the Airlift sensor once the corresponding task completes: These annotated specs are then provided to the `defs` argument to `build_defs_from_airflow_instance`. - -We will also create a set of dbt asset definitions for the `build_dbt_models` task. -We can use the `dagster-dbt`-supplied decorator `@dbt_assets` to generate these definitions using Dagster's dbt integration. - -First, you need to install the extra that has the dbt factory: - -```bash -uv pip install 'dagster-airlift[dbt]' -``` - -Then, we will construct our assets: - -```python -# tutorial_example/dagster_defs/stages/observe.py -import os -from pathlib import Path - -from dagster import AssetExecutionContext, AssetSpec, Definitions -from dagster_airlift.core import ( - AirflowInstance, - AirflowBasicAuthBackend, - assets_with_task_mappings, - build_defs_from_airflow_instance, -) -from dagster_dbt import DbtCliResource, DbtProject, dbt_assets - - -def dbt_project_path() -> Path: - env_val = os.getenv("TUTORIAL_DBT_PROJECT_DIR") - assert env_val, "TUTORIAL_DBT_PROJECT_DIR must be set" - return Path(env_val) - - -@dbt_assets( - manifest=dbt_project_path() / "target" / "manifest.json", - project=DbtProject(dbt_project_path()), -) -def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): - yield from dbt.cli(["build"], context=context).stream() - - -mapped_assets = assets_with_task_mappings( - dag_id="rebuild_customers_list", - task_mappings={ - "load_raw_customers": [AssetSpec(key=["raw_data", "raw_customers"])], - "build_dbt_models": [dbt_project_assets], - "export_customers": [AssetSpec(key="customers_csv", deps=["customers"])], - }, -) - - -defs = build_defs_from_airflow_instance( - airflow_instance=AirflowInstance( - auth_backend=AirflowBasicAuthBackend( - webserver_url="http://localhost:8080", - username="admin", - password="admin", - ), - name="airflow_instance_one", - ), - defs=Definitions( - assets=mapped_assets, - resources={"dbt": DbtCliResource(project_dir=dbt_project_path())}, - ), -) -``` - -### Viewing observed assets - -Once your assets are set up, you should be able to reload your Dagster definitions and see a full representation of the dbt project and other data assets in your code. - -

- -![Observed asset graph in Dagster](./images/observe.svg) - -

- -Kicking off a run of the DAG in Airflow, you should see the newly created assets materialize in Dagster as each task completes. - -_Note: There will be some delay between task completion and assets materializing in Dagster, managed by the sensor. This sensor runs every 30 seconds by default (you can reduce down to one second via the `minimum_interval_seconds` argument to `sensor`), so there will be some delay._ - -### Adding partitions - -If your assets represent a time-partitioned data source, Airlift can automatically associate your materializations to the relevant partitions. -In the case of `rebuild_customers_list`, data is daily partitioned in each created table, and as a result we've added a `@daily` cron schedule to the DAG to make sure it runs every day. We can likewise add a `DailyPartitionsDefinition` to each of our assets. - -```python -# tutorial_example/dagster_defs/stages/observe_with_partitions.py -import os -from pathlib import Path - -from dagster import AssetExecutionContext, AssetSpec, Definitions, DailyPartitionsDefinition -from dagster_airlift.core import ( - AirflowInstance, - AirflowBasicAuthBackend, - assets_with_task_mappings, - build_defs_from_airflow_instance, -) -from dagster_dbt import DbtCliResource, DbtProject, dbt_assets -from dagster._time import get_current_datetime_midnight - -PARTITIONS_DEF = DailyPartitionsDefinition(start_date=get_current_datetime_midnight()) - - -def dbt_project_path() -> Path: - env_val = os.getenv("TUTORIAL_DBT_PROJECT_DIR") - assert env_val, "TUTORIAL_DBT_PROJECT_DIR must be set" - return Path(env_val) - - -@dbt_assets( - manifest=dbt_project_path() / "target" / "manifest.json", - project=DbtProject(dbt_project_path()), - partitions_def=PARTITIONS_DEF, -) -def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): - yield from dbt.cli(["build"], context=context).stream() - - -mapped_assets = assets_with_task_mappings( - dag_id="rebuild_customers_list", - task_mappings={ - "load_raw_customers": [AssetSpec(key=["raw_data", "raw_customers"], partitions_def=PARTITIONS_DEF)], - "build_dbt_models": [dbt_project_assets], - "export_customers": [AssetSpec(key="customers_csv", deps=["customers"], partitions_def=PARTITIONS_DEF)], - }, -) - - -defs = build_defs_from_airflow_instance( - airflow_instance=AirflowInstance( - auth_backend=AirflowBasicAuthBackend( - webserver_url="http://localhost:8080", - username="admin", - password="admin", - ), - name="airflow_instance_one", - ), - defs=Definitions( - assets=mapped_assets, - resources={"dbt": DbtCliResource(project_dir=dbt_project_path())}, - ), -) -``` - -Now, every time the sensor synthesizes a materialization for an asset, it will automatically have a partition associated with it. - -

- -![Partitioned Materialization in Dagster](./images/partitioned_mat.png) - -

- -In order for partitioned assets to work out of the box with `dagster-airlift`, the following things need to be true: - -- The asset can only be time-window partitioned. This means static, dynamic, and multi partitioned definitions will require custom functionality. -- The partitioning scheme must match up with the [logical_date / execution_date](https://airflow.apache.org/docs/apache-airflow/stable/faq.html#what-does-execution-date-mean) of corresponding Airflow runs. That is, each logical*date should correspond \_exactly* to a partition in Dagster. - -## Migrating Assets - -Once you have created corresponding definitions in Dagster to your Airflow tasks, you can proxy execution to Dagster on a per-task basis while Airflow is still controlling scheduling and orchestration. -Once a task has been proxied, Airflow will kick off materializations of corresponding Dagster assets in place of executing the business logic of that task. - -To begin proxying tasks in a DAG, first you will need a file to track proxying state. In your Airflow DAG directory, create a `proxied_state` folder, and in it create a yaml file with the same name as your DAG. The included example at [`airflow_dags/proxied_state`](./tutorial_example/airflow_dags/proxied_state) is used by `make airflow_run`, and can be used as a template for your own proxied state files. - -Given our example DAG `rebuild_customers_list` with three tasks, `load_raw_customers`, `run_dbt_model`, and `export_customers`, [`proxied_state/rebuild_customers_list.yaml`](./tutorial_example/airflow_dags/proxied_state/rebuild_customers_list.yaml) should look like the following: - -```yaml -# tutorial_example/airflow_dags/proxied_state/rebuild_customers_list.yaml -tasks: - - id: load_raw_customers - proxied: False - - id: build_dbt_models - proxied: False - - id: export_customers - proxied: False -``` - -Next, you will need to modify your Airflow DAG to make it aware of the proxied state. This is already done in the example DAG: - -```python -# tutorial_example/snippets/dags_truncated.py -# Dags file can be found at tutorial_example/airflow_dags/dags.py -from pathlib import Path - -from airflow import DAG -from dagster_airlift.in_airflow import proxying_to_dagster -from dagster_airlift.in_airflow.proxied_state import load_proxied_state_from_yaml - -dag = DAG("rebuild_customers_list", ...) - -... - -# Set this to True to begin the proxying process -PROXYING = False - -if PROXYING: - proxying_to_dagster( - global_vars=globals(), - proxied_state=load_proxied_state_from_yaml(Path(__file__).parent / "proxied_state"), - ) -``` - -Set `PROXYING` to `True` or eliminate the `if` statement. - -The DAG will now display its proxied state in the Airflow UI. (There is some latency as Airflow evaluates the Python file periodically.) - -

- -![Migration state rendering in Airflow UI](./images/state_in_airflow.png) - -

- -### Migrating individual tasks - -In order to proxy a task, you must do two things: - -1. First, ensure all associated assets are executable in Dagster by providing asset definitions in place of bare asset specs. -2. The `proxied: False` status in the `proxied_state` YAML folder must be adjusted to `proxied: True`. - -Any task marked as proxied will use the `DefaultProxyTaskToDagsterOperator` when executed as part of the DAG. This operator will use the Dagster GraphQL API to initiate a Dagster run of the assets corresponding to the task. - -The proxied file acts as the source of truth for proxied state. The information is attached to the DAG and then accessed by Dagster via the REST API. - -A task which has been proxied can be easily toggled back to run in Airflow (for example, if a bug in implementation was encountered) simply by editing the file to `proxied: False`. - -#### Supporting custom authorization - -If your Dagster deployment lives behind a custom auth backend, you can customize the Airflow-to-Dagster proxying behavior to authenticate to your backend. -`proxying_to_dagster` can take a parameter `dagster_operator_klass`, which allows you to define a custom `BaseProxyTasktoDagsterOperator` class. This allows you to -override how a session is created. Let's say for example, your Dagster installation requires an access key to be set whenever a request is made, and that access key is set in an Airflow `Variable` called `my_api_key`. -We can create a custom `BaseProxyTasktoDagsterOperator` subclass which will retrieve that variable value and set it on the session, so that any requests to Dagster's graphql API -will be made using that api key. - -```python -# tutorial_example/snippets/custom_operator_examples/custom_proxy.py -from pathlib import Path - -import requests -from airflow import DAG -from airflow.utils.context import Context -from dagster_airlift.in_airflow import BaseProxyTaskToDagsterOperator, proxying_to_dagster -from dagster_airlift.in_airflow.proxied_state import load_proxied_state_from_yaml - - -class CustomProxyTaskToDagsterOperator(BaseProxyTaskToDagsterOperator): - def get_dagster_session(self, context: Context) -> requests.Session: - if "var" not in context: - raise ValueError("No variables found in context") - api_key = context["var"]["value"].get("my_api_key") - session = requests.Session() - session.headers.update({"Authorization": f"Bearer {api_key}"}) - return session - - def get_dagster_url(self, context: Context) -> str: - return "https://dagster.example.com/" - - -dag = DAG( - dag_id="custom_proxy_example", -) - -# At the end of your dag file -proxying_to_dagster( - global_vars=globals(), - proxied_state=load_proxied_state_from_yaml(Path(__file__).parent / "proxied_state"), - build_from_task_fn=CustomProxyTaskToDagsterOperator.build_from_task, -) -``` - -#### Dagster Plus Authorization - -You can use a customer proxy operator to establish a connection to a Dagster plus deployment. The below example proxies to Dagster Plus using organization name, deployment name, and user token set as -Airflow Variables. To set a Dagster+ user token, follow this guide: https://docs.dagster.io/dagster-plus/account/managing-user-agent-tokens#managing-user-tokens. - -```python -# tutorial_example/snippets/custom_operator_examples/plus_proxy_operator.py -import requests -from airflow.utils.context import Context -from dagster_airlift.in_airflow import BaseProxyTaskToDagsterOperator - - -class DagsterCloudProxyOperator(BaseProxyTaskToDagsterOperator): - def get_variable(self, context: Context, var_name: str) -> str: - if "var" not in context: - raise ValueError("No variables found in context") - return context["var"]["value"][var_name] - - def get_dagster_session(self, context: Context) -> requests.Session: - dagster_cloud_user_token = self.get_variable(context, "dagster_cloud_user_token") - session = requests.Session() - session.headers.update({"Dagster-Cloud-Api-Token": dagster_cloud_user_token}) - return session - - def get_dagster_url(self, context: Context) -> str: - org_name = self.get_variable(context, "dagster_plus_organization_name") - deployment_name = self.get_variable(context, "dagster_plus_deployment_name") - return f"https://{org_name}.dagster.plus/{deployment_name}" -``` - -#### Migrating common operators - -For some common operator patterns, like our dbt operator, Dagster supplies factories to build software defined assets for our tasks. In fact, the `@dbt_assets` decorator used earlier already backs its assets with definitions, so we can toggle the proxied state of the `build_dbt_models` task to `proxied: True` in the proxied state file: - -```yaml -# tutorial_example/snippets/dbt_proxied.yaml -tasks: - - id: load_raw_customers - proxied: False - - id: build_dbt_models - proxied: True - - id: export_customers - proxied: False -``` - -**Important**: It may take up to 30 seconds for the proxied state in the Airflow UI to reflect this change. You must subsequently reload the definitions in Dagster via the UI or by restarting `dagster dev`. - -You can now run the `rebuild_customers_list` DAG in Airflow, and the `build_dbt_models` task will be executed in a Dagster run: - -

- -![dbt build executing in Dagster](./images/proxied_dag.png) - -

- -You'll note that we proxied a task in the _middle_ of the Airflow DAG. The Airflow DAG structure and execution history is stable in the Airflow UI, but execution of `build_dbt_models` has moved to Dagster. - -#### Migrating the remaining custom operators - -For all other operator types, we will need to build our own asset definitions. We recommend creating a factory function whose arguments match the inputs to your Airflow operator. Then, you can use this factory to build definitions for each Airflow task. - -For example, our `load_raw_customers` task uses a custom `LoadCSVToDuckDB` operator. We'll define a function `load_csv_to_duckdb_defs` factory to build corresponding software-defined assets. Similarly for `export_customers` we'll define a function `export_duckdb_to_csv_defs` to build SDAs: - -```python -# tutorial_example/dagster_defs/stages/migrate.py -import os -from pathlib import Path - -from dagster import ( - AssetExecutionContext, - AssetsDefinition, - AssetSpec, - Definitions, - materialize, - multi_asset, -) -from dagster_airlift.core import ( - AirflowInstance, - AirflowBasicAuthBackend, - assets_with_task_mappings, - build_defs_from_airflow_instance, -) -from dagster_dbt import DbtCliResource, DbtProject, dbt_assets - -# Code also invoked from Airflow -from tutorial_example.shared.export_duckdb_to_csv import ExportDuckDbToCsvArgs, export_duckdb_to_csv -from tutorial_example.shared.load_csv_to_duckdb import LoadCsvToDuckDbArgs, load_csv_to_duckdb - - -def dbt_project_path() -> Path: - env_val = os.getenv("TUTORIAL_DBT_PROJECT_DIR") - assert env_val, "TUTORIAL_DBT_PROJECT_DIR must be set" - return Path(env_val) - - -def airflow_dags_path() -> Path: - return Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "tutorial_example" / "airflow_dags" - - -def load_csv_to_duckdb_asset(spec: AssetSpec, args: LoadCsvToDuckDbArgs) -> AssetsDefinition: - @multi_asset(name=f"load_{args.table_name}", specs=[spec]) - def _multi_asset() -> None: - load_csv_to_duckdb(args) - - return _multi_asset - - -def export_duckdb_to_csv_defs(spec: AssetSpec, args: ExportDuckDbToCsvArgs) -> AssetsDefinition: - @multi_asset(name=f"export_{args.table_name}", specs=[spec]) - def _multi_asset() -> None: - export_duckdb_to_csv(args) - - return _multi_asset - - -@dbt_assets( - manifest=dbt_project_path() / "target" / "manifest.json", - project=DbtProject(dbt_project_path()), -) -def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): - yield from dbt.cli(["build"], context=context).stream() - - -mapped_assets = assets_with_task_mappings( - dag_id="rebuild_customers_list", - task_mappings={ - "load_raw_customers": [ - load_csv_to_duckdb_asset( - AssetSpec(key=["raw_data", "raw_customers"]), - LoadCsvToDuckDbArgs( - table_name="raw_customers", - csv_path=airflow_dags_path() / "raw_customers.csv", - duckdb_path=Path(os.environ["AIRFLOW_HOME"]) / "jaffle_shop.duckdb", - names=["id", "first_name", "last_name"], - duckdb_schema="raw_data", - duckdb_database_name="jaffle_shop", - ), - ) - ], - "build_dbt_models": - # load rich set of assets from dbt project - [dbt_project_assets], - "export_customers": [ - export_duckdb_to_csv_defs( - AssetSpec(key="customers_csv", deps=["customers"]), - ExportDuckDbToCsvArgs( - table_name="customers", - csv_path=Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "customers.csv", - duckdb_path=Path(os.environ["AIRFLOW_HOME"]) / "jaffle_shop.duckdb", - duckdb_database_name="jaffle_shop", - ), - ) - ], - }, -) - - -defs = build_defs_from_airflow_instance( - airflow_instance=AirflowInstance( - auth_backend=AirflowBasicAuthBackend( - webserver_url="http://localhost:8080", - username="admin", - password="admin", - ), - name="airflow_instance_one", - ), - defs=Definitions( - assets=mapped_assets, - resources={"dbt": DbtCliResource(project_dir=dbt_project_path())}, - ), -) -``` - -We can then toggle the proxied state of the remaining tasks in the `proxied_state` file: - -```yaml -# tutorial_example/snippets/all_proxied.yaml -tasks: - - id: load_raw_customers - proxied: True - - id: build_dbt_models - proxied: True - - id: export_customers - proxied: True -``` - -## Decomissioning an Airflow DAG - -Once we are confident in our migrated versions of the tasks, we can decommission the Airflow DAG. First, we can remove the DAG from our Airflow DAG directory. - -Next, we can strip the task associations from our Dagster definitions. This can be done by removing the `assets_with_task_mappings` call. We can use this opportunity to attach our assets to a `ScheduleDefinition` so that Dagster's scheduler can manage their execution: - -```python -# tutorial_example/dagster_defs/stages/standalone.py -import os -from pathlib import Path - -from dagster import ( - AssetCheckResult, - AssetCheckSeverity, - AssetExecutionContext, - AssetKey, - AssetsDefinition, - AssetSelection, - AssetSpec, - Definitions, - ScheduleDefinition, - asset_check, - multi_asset, -) -from dagster_dbt import DbtCliResource, DbtProject, dbt_assets - -# Code also invoked from Airflow -from tutorial_example.shared.export_duckdb_to_csv import ExportDuckDbToCsvArgs, export_duckdb_to_csv -from tutorial_example.shared.load_csv_to_duckdb import LoadCsvToDuckDbArgs, load_csv_to_duckdb - - -def dbt_project_path() -> Path: - env_val = os.getenv("TUTORIAL_DBT_PROJECT_DIR") - assert env_val, "TUTORIAL_DBT_PROJECT_DIR must be set" - return Path(env_val) - - -def airflow_dags_path() -> Path: - return Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "tutorial_example" / "airflow_dags" - - -def load_csv_to_duckdb_asset(spec: AssetSpec, args: LoadCsvToDuckDbArgs) -> AssetsDefinition: - @multi_asset(name=f"load_{args.table_name}", specs=[spec]) - def _multi_asset() -> None: - load_csv_to_duckdb(args) - - return _multi_asset - - -def export_duckdb_to_csv_defs(spec: AssetSpec, args: ExportDuckDbToCsvArgs) -> AssetsDefinition: - @multi_asset(name=f"export_{args.table_name}", specs=[spec]) - def _multi_asset() -> None: - export_duckdb_to_csv(args) - - return _multi_asset - - -@dbt_assets( - manifest=dbt_project_path() / "target" / "manifest.json", - project=DbtProject(dbt_project_path()), -) -def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): - yield from dbt.cli(["build"], context=context).stream() - - -assets = [ - load_csv_to_duckdb_asset( - AssetSpec(key=["raw_data", "raw_customers"]), - LoadCsvToDuckDbArgs( - table_name="raw_customers", - csv_path=airflow_dags_path() / "raw_customers.csv", - duckdb_path=Path(os.environ["AIRFLOW_HOME"]) / "jaffle_shop.duckdb", - names=["id", "first_name", "last_name"], - duckdb_schema="raw_data", - duckdb_database_name="jaffle_shop", - ), - ), - dbt_project_assets, - export_duckdb_to_csv_defs( - AssetSpec(key="customers_csv", deps=["customers"]), - ExportDuckDbToCsvArgs( - table_name="customers", - csv_path=Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "customers.csv", - duckdb_path=Path(os.environ["AIRFLOW_HOME"]) / "jaffle_shop.duckdb", - duckdb_database_name="jaffle_shop", - ), - ), -] - - -@asset_check(asset=AssetKey(["customers_csv"])) -def validate_exported_csv() -> AssetCheckResult: - csv_path = Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "customers.csv" - - if not csv_path.exists(): - return AssetCheckResult(passed=False, description=f"Export CSV {csv_path} does not exist") - - rows = len(csv_path.read_text().split("\n")) - if rows < 2: - return AssetCheckResult( - passed=False, - description=f"Export CSV {csv_path} is empty", - severity=AssetCheckSeverity.WARN, - ) - - return AssetCheckResult( - passed=True, - description=f"Export CSV {csv_path} exists", - metadata={"rows": rows}, - ) - - -rebuild_customer_list_schedule = rebuild_customers_list_schedule = ScheduleDefinition( - name="rebuild_customers_list_schedule", - target=AssetSelection.assets(*assets), - cron_schedule="0 0 * * *", -) - - -defs = Definitions( - assets=assets, - schedules=[rebuild_customer_list_schedule], - asset_checks=[validate_exported_csv], - resources={"dbt": DbtCliResource(project_dir=dbt_project_path())}, -) -``` - -## Addendum: Adding asset checks - -Once you have peered your Airflow DAGs in Dagster, regardless of migration progress, you can begin to add asset checks to your Dagster code. Asset checks can be used to validate the quality of your data assets, and can provide additional observability and value on top of your Airflow DAG even before migration starts. - -For example, given a peered version of our DAG, we can add an asset check to ensure that the final `customers` CSV output exists and has a non-zero number of rows: - -```python -# tutorial_example/dagster_defs/stages/peer_with_check.py -import os -from pathlib import Path - -from dagster import AssetCheckResult, AssetCheckSeverity, AssetKey, Definitions, asset_check -from dagster_airlift.core import AirflowInstance, AirflowBasicAuthBackend, build_defs_from_airflow_instance - - -# Attach a check to the DAG representation asset, which will be executed by Dagster -# any time the DAG is run in Airflow -@asset_check(asset=AssetKey(["airflow_instance_one", "dag", "rebuild_customers_list"])) -def validate_exported_csv() -> AssetCheckResult: - csv_path = Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "customers.csv" - - if not csv_path.exists(): - return AssetCheckResult(passed=False, description=f"Export CSV {csv_path} does not exist") - - rows = len(csv_path.read_text().split("\n")) - if rows < 2: - return AssetCheckResult( - passed=False, - description=f"Export CSV {csv_path} is empty", - severity=AssetCheckSeverity.WARN, - ) - - return AssetCheckResult( - passed=True, - description=f"Export CSV {csv_path} exists", - metadata={"rows": rows}, - ) - - -defs = Definitions.merge( - build_defs_from_airflow_instance( - airflow_instance=AirflowInstance( - # other backends available (e.g. MwaaSessionAuthBackend) - auth_backend=AirflowBasicAuthBackend( - webserver_url="http://localhost:8080", - username="admin", - password="admin", - ), - name="airflow_instance_one", - ) - ), - Definitions(asset_checks=[validate_exported_csv]), -) -``` - -Once we have introduced representations of the assets produced by our Airflow tasks, we can directly attach asset checks to these assets. These checks will run once the corresponding task completes, regardless of whether the task is executed in Airflow or Dagster. - -
- -Asset checks on an observed or migrated DAG - - -```python -# tutorial_example/dagster_defs/stages/migrate_with_check.py -import os -from pathlib import Path - -from dagster import ( - AssetCheckResult, - AssetCheckSeverity, - AssetExecutionContext, - AssetKey, - AssetsDefinition, - AssetSpec, - Definitions, - asset_check, - materialize, - multi_asset, -) -from dagster_airlift.core import ( - AirflowInstance, - AirflowBasicAuthBackend, - assets_with_task_mappings, - build_defs_from_airflow_instance, -) -from dagster_dbt import DbtCliResource, DbtProject, dbt_assets - -# Code also invoked from Airflow -from tutorial_example.shared.export_duckdb_to_csv import ExportDuckDbToCsvArgs, export_duckdb_to_csv -from tutorial_example.shared.load_csv_to_duckdb import LoadCsvToDuckDbArgs, load_csv_to_duckdb - - -def dbt_project_path() -> Path: - env_val = os.getenv("TUTORIAL_DBT_PROJECT_DIR") - assert env_val, "TUTORIAL_DBT_PROJECT_DIR must be set" - return Path(env_val) - - -def airflow_dags_path() -> Path: - return Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "tutorial_example" / "airflow_dags" - - -def load_csv_to_duckdb_asset(spec: AssetSpec, args: LoadCsvToDuckDbArgs) -> AssetsDefinition: - @multi_asset(name=f"load_{args.table_name}", specs=[spec]) - def _multi_asset() -> None: - load_csv_to_duckdb(args) - - return _multi_asset - - -def export_duckdb_to_csv_defs(spec: AssetSpec, args: ExportDuckDbToCsvArgs) -> AssetsDefinition: - @multi_asset(name=f"export_{args.table_name}", specs=[spec]) - def _multi_asset() -> None: - export_duckdb_to_csv(args) - - return _multi_asset - - -@dbt_assets( - manifest=dbt_project_path() / "target" / "manifest.json", - project=DbtProject(dbt_project_path()), -) -def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): - yield from dbt.cli(["build"], context=context).stream() - - -mapped_assets = assets_with_task_mappings( - dag_id="rebuild_customers_list", - task_mappings={ - "load_raw_customers": [ - load_csv_to_duckdb_asset( - AssetSpec(key=["raw_data", "raw_customers"]), - LoadCsvToDuckDbArgs( - table_name="raw_customers", - csv_path=airflow_dags_path() / "raw_customers.csv", - duckdb_path=Path(os.environ["AIRFLOW_HOME"]) / "jaffle_shop.duckdb", - names=["id", "first_name", "last_name"], - duckdb_schema="raw_data", - duckdb_database_name="jaffle_shop", - ), - ) - ], - "build_dbt_models": - # load rich set of assets from dbt project - [dbt_project_assets], - "export_customers": [ - export_duckdb_to_csv_defs( - AssetSpec(key="customers_csv", deps=["customers"]), - ExportDuckDbToCsvArgs( - table_name="customers", - csv_path=Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "customers.csv", - duckdb_path=Path(os.environ["AIRFLOW_HOME"]) / "jaffle_shop.duckdb", - duckdb_database_name="jaffle_shop", - ), - ) - ], - }, -) - - -@asset_check(asset=AssetKey(["customers_csv"])) -def validate_exported_csv() -> AssetCheckResult: - csv_path = Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "customers.csv" - - if not csv_path.exists(): - return AssetCheckResult(passed=False, description=f"Export CSV {csv_path} does not exist") - - rows = len(csv_path.read_text().split("\n")) - if rows < 2: - return AssetCheckResult( - passed=False, - description=f"Export CSV {csv_path} is empty", - severity=AssetCheckSeverity.WARN, - ) - - return AssetCheckResult( - passed=True, - description=f"Export CSV {csv_path} exists", - metadata={"rows": rows}, - ) - - -defs = build_defs_from_airflow_instance( - airflow_instance=AirflowInstance( - auth_backend=AirflowBasicAuthBackend( - webserver_url="http://localhost:8080", - username="admin", - password="admin", - ), - name="airflow_instance_one", - ), - defs=Definitions( - assets=mapped_assets, - asset_checks=[validate_exported_csv], - resources={"dbt": DbtCliResource(project_dir=dbt_project_path())}, - ), -) -``` - -
- -## Migrating an entire DAG at once - -There may be DAGs for which you want to migrate the entire thing at once rather than on a per-task basis. -Some reasons for taking this approach: - -- You're making use of "dynamic tasks" in Airflow, which don't conform neatly to the task mapping protocol we've laid out above. -- You want to make more substantial refactors to the dag structure that don't conform to the existing task structure - -For cases like this, we allow you to map assets to a full DAG. - -### Observing DAG-mapped assets - -At the observation stage, you'll call `assets_with_dag_mappings` instead of `assets_with_task_mappings`. - -For our `rebuild_customers_list` DAG, let's take a look at what the new observation code looks like: - -```python -# tutorial_example/dagster_defs/stages/observe_dag_level.py -import os -from pathlib import Path - -from dagster import AssetExecutionContext, AssetSpec, Definitions -from dagster_airlift.core import ( - AirflowInstance, - AirflowBasicAuthBackend, - assets_with_dag_mappings, - build_defs_from_airflow_instance, -) -from dagster_dbt import DbtCliResource, DbtProject, dbt_assets - - -def dbt_project_path() -> Path: - env_val = os.getenv("TUTORIAL_DBT_PROJECT_DIR") - assert env_val, "TUTORIAL_DBT_PROJECT_DIR must be set" - return Path(env_val) - - -@dbt_assets( - manifest=dbt_project_path() / "target" / "manifest.json", - project=DbtProject(dbt_project_path()), -) -def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): - yield from dbt.cli(["build"], context=context).stream() - - -# Instead of mapping assets to individual tasks, we map them to the entire DAG. -mapped_assets = assets_with_dag_mappings( - dag_mappings={ - "rebuild_customers_list": [ - AssetSpec(key=["raw_data", "raw_customers"]), - dbt_project_assets, - AssetSpec(key="customers_csv", deps=["customers"]), - ], - }, -) - - -defs = build_defs_from_airflow_instance( - airflow_instance=AirflowInstance( - auth_backend=AirflowBasicAuthBackend( - webserver_url="http://localhost:8080", - username="admin", - password="admin", - ), - name="airflow_instance_one", - ), - defs=Definitions( - assets=mapped_assets, - resources={"dbt": DbtCliResource(project_dir=dbt_project_path())}, - ), -) -``` - -Now, instead of getting a materialization when a particular task completes, each mapped asset will receive a materialization when the entire DAG completes. - -### Migrating DAG-mapped assets - -Recall that in the task-by-task migration step, we "proxy" execution on a task by task basis, which is controlled by a yaml document. -For DAG-mapped assets, execution is proxied on a per-DAG basis. -Proxying execution to Dagster will require all assets mapped to that DAG be _executable_ within Dagster. -Let's take a look at some fully migrated code mapped to DAGs instead of tasks: - -```python -# tutorial_example/dagster_defs/stages/migrate_dag_level.py -import os -from pathlib import Path - -from dagster import ( - AssetExecutionContext, - AssetsDefinition, - AssetSpec, - Definitions, - materialize, - multi_asset, -) -from dagster_airlift.core import ( - AirflowInstance, - AirflowBasicAuthBackend, - assets_with_dag_mappings, - build_defs_from_airflow_instance, -) -from dagster_dbt import DbtCliResource, DbtProject, dbt_assets - -# Code also invoked from Airflow -from tutorial_example.shared.export_duckdb_to_csv import ExportDuckDbToCsvArgs, export_duckdb_to_csv -from tutorial_example.shared.load_csv_to_duckdb import LoadCsvToDuckDbArgs, load_csv_to_duckdb - - -def dbt_project_path() -> Path: - env_val = os.getenv("TUTORIAL_DBT_PROJECT_DIR") - assert env_val, "TUTORIAL_DBT_PROJECT_DIR must be set" - return Path(env_val) - - -def airflow_dags_path() -> Path: - return Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "tutorial_example" / "airflow_dags" - - -def load_csv_to_duckdb_asset(spec: AssetSpec, args: LoadCsvToDuckDbArgs) -> AssetsDefinition: - @multi_asset(name=f"load_{args.table_name}", specs=[spec]) - def _multi_asset() -> None: - load_csv_to_duckdb(args) - - return _multi_asset - - -def export_duckdb_to_csv_defs(spec: AssetSpec, args: ExportDuckDbToCsvArgs) -> AssetsDefinition: - @multi_asset(name=f"export_{args.table_name}", specs=[spec]) - def _multi_asset() -> None: - export_duckdb_to_csv(args) - - return _multi_asset - - -@dbt_assets( - manifest=dbt_project_path() / "target" / "manifest.json", - project=DbtProject(dbt_project_path()), -) -def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): - yield from dbt.cli(["build"], context=context).stream() - - -mapped_assets = assets_with_dag_mappings( - dag_mappings={ - "rebuild_customers_list": [ - load_csv_to_duckdb_asset( - AssetSpec(key=["raw_data", "raw_customers"]), - LoadCsvToDuckDbArgs( - table_name="raw_customers", - csv_path=airflow_dags_path() / "raw_customers.csv", - duckdb_path=Path(os.environ["AIRFLOW_HOME"]) / "jaffle_shop.duckdb", - names=["id", "first_name", "last_name"], - duckdb_schema="raw_data", - duckdb_database_name="jaffle_shop", - ), - ), - dbt_project_assets, - export_duckdb_to_csv_defs( - AssetSpec(key="customers_csv", deps=["customers"]), - ExportDuckDbToCsvArgs( - table_name="customers", - csv_path=Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "customers.csv", - duckdb_path=Path(os.environ["AIRFLOW_HOME"]) / "jaffle_shop.duckdb", - duckdb_database_name="jaffle_shop", - ), - ), - ], - }, -) - - -defs = build_defs_from_airflow_instance( - airflow_instance=AirflowInstance( - auth_backend=AirflowBasicAuthBackend( - webserver_url="http://localhost:8080", - username="admin", - password="admin", - ), - name="airflow_instance_one", - ), - defs=Definitions( - assets=mapped_assets, - resources={"dbt": DbtCliResource(project_dir=dbt_project_path())}, - ), -) -``` - -Now that all of our assets are fully executable, we can create a simple yaml file to proxy execution for the whole dag: - -```yaml -# tutorial_example/snippets/rebuild_customers_list.yaml -proxied: True -``` - -We will similarly use `proxying_to_dagster` at the end of our DAG file (the code is exactly the same here as it was for the per-task migration step) - -```python -# tutorial_example/snippets/dags_truncated.py -# Dags file can be found at tutorial_example/airflow_dags/dags.py -from pathlib import Path - -from airflow import DAG -from dagster_airlift.in_airflow import proxying_to_dagster -from dagster_airlift.in_airflow.proxied_state import load_proxied_state_from_yaml - -dag = DAG("rebuild_customers_list", ...) - -... - -# Set this to True to begin the proxying process -PROXYING = False - -if PROXYING: - proxying_to_dagster( - global_vars=globals(), - proxied_state=load_proxied_state_from_yaml(Path(__file__).parent / "proxied_state"), - ) -``` - -Once the `proxied` bit is flipped to True, we can go to the Airflow UI, and we'll see that our tasks have been replaced with a single task. - -

- -![Before DAG proxying](./images/before_dag_override.svg) -![After DAG proxying](./images/after_dag_override.svg) - -

- -When performing dag-level mapping, we don't preserve task structure in the Airflow dags. This single task will materialize all mapped Dagster assets instead of executing the original Airflow task business logic. - -We can similarly mark `proxied` back to `False`, and the original task structure and business logic will return unchanged. - -### Customizing DAG proxying operator - -Similar to how we can customize the operator we construct on a per-dag basis, we can customize the operator we construct on a per-dag basis. We can use the `build_from_dag_fn` argument of `proxying_to_dagster` to provide a custom operator in place of the default. - -For example, let's take a look at the following custom operator which expects an API key to be provided as a variable: - -```python -# tutorial_example/snippets/custom_operator_examples/custom_dag_level_proxy.py -from pathlib import Path - -import requests -from airflow import DAG -from airflow.utils.context import Context -from dagster_airlift.in_airflow import BaseProxyDAGToDagsterOperator, proxying_to_dagster -from dagster_airlift.in_airflow.proxied_state import load_proxied_state_from_yaml - - -class CustomProxyTaskToDagsterOperator(BaseProxyDAGToDagsterOperator): - def get_dagster_session(self, context: Context) -> requests.Session: - if "var" not in context: - raise ValueError("No variables found in context") - api_key = context["var"]["value"].get("my_api_key") - session = requests.Session() - session.headers.update({"Authorization": f"Bearer {api_key}"}) - return session - - def get_dagster_url(self, context: Context) -> str: - return "https://dagster.example.com/" - - # This method controls how the operator is built from the dag. - @classmethod - def build_from_dag(cls, dag: DAG): - return CustomProxyTaskToDagsterOperator(dag=dag, task_id="OVERRIDDEN") - - -dag = DAG( - dag_id="custom_dag_level_proxy_example", -) - -# At the end of your dag file -proxying_to_dagster( - global_vars=globals(), - proxied_state=load_proxied_state_from_yaml(Path(__file__).parent / "proxied_state"), - build_from_dag_fn=CustomProxyTaskToDagsterOperator.build_from_dag, -) -``` - -`BaseProxyDAGToDagsterOperator` has three abstract methods which must be implemented: - -- `get_dagster_session`, which controls the creation of a valid session to access the Dagster graphql API. -- `get_dagster_url`, which retrieves the domain at which the dagster webserver lives. -- `build_from_dag`, which controls how the proxying task is constructed from the provided DAG. - -## Addendum: Dealing with changing Airflow - -In order to make spin-up more efficient, `dagster-airlift` caches the state of the Airflow instance in the dagster database, -so that repeat fetches of the code location don't require additional calls to Airflow's rest API. However, this means that -the Dagster definitions can potentially fall out of sync with Airflow. Here are a few different ways this can manifest: - -- A new Airflow dag is added. The lineage information does not show up for this dag, and materializations are not recorded. -- A dag is removed. The polling sensor begins failing, because there exist assets which expect that dag to exist. -- The task dependency structure within a dag changes. This may result in `unsynced` statuses in Dagster, or missing materializations. - This is not an exhaustive list of problems, but most of the time the tell is that materializations are missing, or assets are missing. - When you find yourself in this state, you can force `dagster-airlift` to reload Airflow state by reloading the code location. - To do this, go to the `Deployment` tab on the top nav, and click `Redeploy` on the code location relevant to your asset. After some time, - the code location should be reloaded with refreshed state from Airflow. - -### Automating changes to code locations - -If changes to your Airflow instance are controlled via a ci/cd process, you can add a step to automatically induce a redeploy of the relevant code location. -See the docs [here](https://docs.dagster.io/concepts/webserver/graphql-client#reloading-all-repositories-in-a-repository-location) on using the graphql client to do this. diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/migrate.py b/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/migrate.py index 567242f20bda4..4e77cbcac686d 100644 --- a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/migrate.py +++ b/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/migrate.py @@ -5,10 +5,12 @@ AssetExecutionContext, AssetsDefinition, AssetSpec, + DailyPartitionsDefinition, Definitions, materialize, multi_asset, ) +from dagster._time import get_current_datetime_midnight from dagster_airlift.core import ( AirflowBasicAuthBackend, AirflowInstance, @@ -21,6 +23,8 @@ from tutorial_example.shared.export_duckdb_to_csv import ExportDuckDbToCsvArgs, export_duckdb_to_csv from tutorial_example.shared.load_csv_to_duckdb import LoadCsvToDuckDbArgs, load_csv_to_duckdb +PARTITIONS_DEF = DailyPartitionsDefinition(start_date=get_current_datetime_midnight()) + def dbt_project_path() -> Path: env_val = os.getenv("TUTORIAL_DBT_PROJECT_DIR") @@ -51,6 +55,7 @@ def _multi_asset() -> None: @dbt_assets( manifest=dbt_project_path() / "target" / "manifest.json", project=DbtProject(dbt_project_path()), + partitions_def=PARTITIONS_DEF, ) def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): yield from dbt.cli(["build"], context=context).stream() @@ -61,7 +66,7 @@ def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): task_mappings={ "load_raw_customers": [ load_csv_to_duckdb_asset( - AssetSpec(key=["raw_data", "raw_customers"]), + AssetSpec(key=["raw_data", "raw_customers"], partitions_def=PARTITIONS_DEF), LoadCsvToDuckDbArgs( table_name="raw_customers", csv_path=airflow_dags_path() / "raw_customers.csv", @@ -77,7 +82,7 @@ def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): [dbt_project_assets], "export_customers": [ export_duckdb_to_csv_defs( - AssetSpec(key="customers_csv", deps=["customers"]), + AssetSpec(key="customers_csv", deps=["customers"], partitions_def=PARTITIONS_DEF), ExportDuckDbToCsvArgs( table_name="customers", csv_path=Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "customers.csv", diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/migrate_with_check.py b/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/migrate_with_check.py index f03bca084ae6c..314327c7f11d7 100644 --- a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/migrate_with_check.py +++ b/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/migrate_with_check.py @@ -8,11 +8,13 @@ AssetKey, AssetsDefinition, AssetSpec, + DailyPartitionsDefinition, Definitions, asset_check, materialize, multi_asset, ) +from dagster._time import get_current_datetime_midnight from dagster_airlift.core import ( AirflowBasicAuthBackend, AirflowInstance, @@ -25,6 +27,8 @@ from tutorial_example.shared.export_duckdb_to_csv import ExportDuckDbToCsvArgs, export_duckdb_to_csv from tutorial_example.shared.load_csv_to_duckdb import LoadCsvToDuckDbArgs, load_csv_to_duckdb +PARTITIONS_DEF = DailyPartitionsDefinition(start_date=get_current_datetime_midnight()) + def dbt_project_path() -> Path: env_val = os.getenv("TUTORIAL_DBT_PROJECT_DIR") @@ -55,6 +59,7 @@ def _multi_asset() -> None: @dbt_assets( manifest=dbt_project_path() / "target" / "manifest.json", project=DbtProject(dbt_project_path()), + partitions_def=PARTITIONS_DEF, ) def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): yield from dbt.cli(["build"], context=context).stream() @@ -65,7 +70,7 @@ def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): task_mappings={ "load_raw_customers": [ load_csv_to_duckdb_asset( - AssetSpec(key=["raw_data", "raw_customers"]), + AssetSpec(key=["raw_data", "raw_customers"], partitions_def=PARTITIONS_DEF), LoadCsvToDuckDbArgs( table_name="raw_customers", csv_path=airflow_dags_path() / "raw_customers.csv", @@ -81,7 +86,7 @@ def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): [dbt_project_assets], "export_customers": [ export_duckdb_to_csv_defs( - AssetSpec(key="customers_csv", deps=["customers"]), + AssetSpec(key="customers_csv", deps=["customers"], partitions_def=PARTITIONS_DEF), ExportDuckDbToCsvArgs( table_name="customers", csv_path=Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "customers.csv", diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/observe.py b/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/observe.py index ea1e8e65413ba..73af301732ee8 100644 --- a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/observe.py +++ b/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/observe.py @@ -1,7 +1,15 @@ import os from pathlib import Path -from dagster import AssetExecutionContext, AssetSpec, Definitions +from dagster import ( + AssetCheckResult, + AssetCheckSeverity, + AssetExecutionContext, + AssetKey, + AssetSpec, + Definitions, + asset_check, +) from dagster_airlift.core import ( AirflowBasicAuthBackend, AirflowInstance, @@ -11,6 +19,28 @@ from dagster_dbt import DbtCliResource, DbtProject, dbt_assets +@asset_check(asset=AssetKey(["airflow_instance_one", "dag", "rebuild_customers_list"])) +def validate_exported_csv() -> AssetCheckResult: + csv_path = Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "customers.csv" + + if not csv_path.exists(): + return AssetCheckResult(passed=False, description=f"Export CSV {csv_path} does not exist") + + rows = len(csv_path.read_text().split("\n")) + if rows < 2: + return AssetCheckResult( + passed=False, + description=f"Export CSV {csv_path} is empty", + severity=AssetCheckSeverity.WARN, + ) + + return AssetCheckResult( + passed=True, + description=f"Export CSV {csv_path} exists", + metadata={"rows": rows}, + ) + + def dbt_project_path() -> Path: env_val = os.getenv("TUTORIAL_DBT_PROJECT_DIR") assert env_val, "TUTORIAL_DBT_PROJECT_DIR must be set" @@ -47,5 +77,6 @@ def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): defs=Definitions( assets=mapped_assets, resources={"dbt": DbtCliResource(project_dir=dbt_project_path())}, + asset_checks=[validate_exported_csv], ), ) diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/observe_check_on_asset.py b/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/observe_check_on_asset.py new file mode 100644 index 0000000000000..360950591cf82 --- /dev/null +++ b/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/observe_check_on_asset.py @@ -0,0 +1,82 @@ +import os +from pathlib import Path + +from dagster import ( + AssetCheckResult, + AssetCheckSeverity, + AssetExecutionContext, + AssetKey, + AssetSpec, + Definitions, + asset_check, +) +from dagster_airlift.core import ( + AirflowBasicAuthBackend, + AirflowInstance, + assets_with_task_mappings, + build_defs_from_airflow_instance, +) +from dagster_dbt import DbtCliResource, DbtProject, dbt_assets + + +@asset_check(asset=AssetKey(["customers_csv"])) +def validate_exported_csv() -> AssetCheckResult: + csv_path = Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "customers.csv" + + if not csv_path.exists(): + return AssetCheckResult(passed=False, description=f"Export CSV {csv_path} does not exist") + + rows = len(csv_path.read_text().split("\n")) + if rows < 2: + return AssetCheckResult( + passed=False, + description=f"Export CSV {csv_path} is empty", + severity=AssetCheckSeverity.WARN, + ) + + return AssetCheckResult( + passed=True, + description=f"Export CSV {csv_path} exists", + metadata={"rows": rows}, + ) + + +def dbt_project_path() -> Path: + env_val = os.getenv("TUTORIAL_DBT_PROJECT_DIR") + assert env_val, "TUTORIAL_DBT_PROJECT_DIR must be set" + return Path(env_val) + + +@dbt_assets( + manifest=dbt_project_path() / "target" / "manifest.json", + project=DbtProject(dbt_project_path()), +) +def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): + yield from dbt.cli(["build"], context=context).stream() + + +mapped_assets = assets_with_task_mappings( + dag_id="rebuild_customers_list", + task_mappings={ + "load_raw_customers": [AssetSpec(key=["raw_data", "raw_customers"])], + "build_dbt_models": [dbt_project_assets], + "export_customers": [AssetSpec(key="customers_csv", deps=["customers"])], + }, +) + + +defs = build_defs_from_airflow_instance( + airflow_instance=AirflowInstance( + auth_backend=AirflowBasicAuthBackend( + webserver_url="http://localhost:8080", + username="admin", + password="admin", + ), + name="airflow_instance_one", + ), + defs=Definitions( + assets=mapped_assets, + resources={"dbt": DbtCliResource(project_dir=dbt_project_path())}, + asset_checks=[validate_exported_csv], + ), +) diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/observe_with_partitions.py b/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/observe_with_partitions.py index 264b9db1adefe..a9d88160b09a1 100644 --- a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/observe_with_partitions.py +++ b/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/observe_with_partitions.py @@ -1,7 +1,16 @@ import os from pathlib import Path -from dagster import AssetExecutionContext, AssetSpec, DailyPartitionsDefinition, Definitions +from dagster import ( + AssetCheckResult, + AssetCheckSeverity, + AssetExecutionContext, + AssetKey, + AssetSpec, + DailyPartitionsDefinition, + Definitions, + asset_check, +) from dagster._time import get_current_datetime_midnight from dagster_airlift.core import ( AirflowBasicAuthBackend, @@ -14,6 +23,28 @@ PARTITIONS_DEF = DailyPartitionsDefinition(start_date=get_current_datetime_midnight()) +@asset_check(asset=AssetKey(["customers_csv"])) +def validate_exported_csv() -> AssetCheckResult: + csv_path = Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "customers.csv" + + if not csv_path.exists(): + return AssetCheckResult(passed=False, description=f"Export CSV {csv_path} does not exist") + + rows = len(csv_path.read_text().split("\n")) + if rows < 2: + return AssetCheckResult( + passed=False, + description=f"Export CSV {csv_path} is empty", + severity=AssetCheckSeverity.WARN, + ) + + return AssetCheckResult( + passed=True, + description=f"Export CSV {csv_path} exists", + metadata={"rows": rows}, + ) + + def dbt_project_path() -> Path: env_val = os.getenv("TUTORIAL_DBT_PROJECT_DIR") assert env_val, "TUTORIAL_DBT_PROJECT_DIR must be set" @@ -55,5 +86,6 @@ def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): defs=Definitions( assets=mapped_assets, resources={"dbt": DbtCliResource(project_dir=dbt_project_path())}, + asset_checks=[validate_exported_csv], ), ) diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/peer_with_check.py b/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/peer_with_check.py index 60f5a53b60f70..6abcef989641d 100644 --- a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/peer_with_check.py +++ b/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/peer_with_check.py @@ -33,17 +33,15 @@ def validate_exported_csv() -> AssetCheckResult: ) -defs = Definitions.merge( - build_defs_from_airflow_instance( - airflow_instance=AirflowInstance( - # other backends available (e.g. MwaaSessionAuthBackend) - auth_backend=AirflowBasicAuthBackend( - webserver_url="http://localhost:8080", - username="admin", - password="admin", - ), - name="airflow_instance_one", - ) +defs = build_defs_from_airflow_instance( + airflow_instance=AirflowInstance( + # other backends available (e.g. MwaaSessionAuthBackend) + auth_backend=AirflowBasicAuthBackend( + webserver_url="http://localhost:8080", + username="admin", + password="admin", + ), + name="airflow_instance_one", ), - Definitions(asset_checks=[validate_exported_csv]), + defs=Definitions(asset_checks=[validate_exported_csv]), ) diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/standalone.py b/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/standalone.py index 99340a7040e62..2eedee5788956 100644 --- a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/standalone.py +++ b/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/standalone.py @@ -9,17 +9,21 @@ AssetsDefinition, AssetSelection, AssetSpec, + DailyPartitionsDefinition, Definitions, ScheduleDefinition, asset_check, multi_asset, ) +from dagster._time import get_current_datetime_midnight from dagster_dbt import DbtCliResource, DbtProject, dbt_assets # Code also invoked from Airflow from tutorial_example.shared.export_duckdb_to_csv import ExportDuckDbToCsvArgs, export_duckdb_to_csv from tutorial_example.shared.load_csv_to_duckdb import LoadCsvToDuckDbArgs, load_csv_to_duckdb +PARTITIONS_DEF = DailyPartitionsDefinition(start_date=get_current_datetime_midnight()) + def dbt_project_path() -> Path: env_val = os.getenv("TUTORIAL_DBT_PROJECT_DIR") @@ -50,6 +54,7 @@ def _multi_asset() -> None: @dbt_assets( manifest=dbt_project_path() / "target" / "manifest.json", project=DbtProject(dbt_project_path()), + partitions_def=PARTITIONS_DEF, ) def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): yield from dbt.cli(["build"], context=context).stream() @@ -57,7 +62,7 @@ def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): assets = [ load_csv_to_duckdb_asset( - AssetSpec(key=["raw_data", "raw_customers"]), + AssetSpec(key=["raw_data", "raw_customers"], partitions_def=PARTITIONS_DEF), LoadCsvToDuckDbArgs( table_name="raw_customers", csv_path=airflow_dags_path() / "raw_customers.csv", @@ -69,7 +74,7 @@ def dbt_project_assets(context: AssetExecutionContext, dbt: DbtCliResource): ), dbt_project_assets, export_duckdb_to_csv_defs( - AssetSpec(key="customers_csv", deps=["customers"]), + AssetSpec(key="customers_csv", deps=["customers"], partitions_def=PARTITIONS_DEF), ExportDuckDbToCsvArgs( table_name="customers", csv_path=Path(os.environ["TUTORIAL_EXAMPLE_DIR"]) / "customers.csv",