diff --git a/.buildkite/dagster-buildkite/dagster_buildkite/pipelines/dagster_oss_nightly_pipeline.py b/.buildkite/dagster-buildkite/dagster_buildkite/pipelines/dagster_oss_nightly_pipeline.py index d0e307db08400..93ae9e9abf491 100644 --- a/.buildkite/dagster-buildkite/dagster_buildkite/pipelines/dagster_oss_nightly_pipeline.py +++ b/.buildkite/dagster-buildkite/dagster_buildkite/pipelines/dagster_oss_nightly_pipeline.py @@ -54,7 +54,7 @@ def build_dagster_oss_nightly_steps() -> List[BuildkiteStep]: ], ), PackageSpec( - "examples/experimental/dagster-airlift/examples/dbt-example", + "examples/starlift-demo", name="airlift-demo-live-tests", env_vars=[ "KS_DBT_CLOUD_ACCOUNT_ID", diff --git a/.buildkite/dagster-buildkite/dagster_buildkite/steps/packages.py b/.buildkite/dagster-buildkite/dagster_buildkite/steps/packages.py index ddc6291ff301f..6f4c7b9d48ed6 100644 --- a/.buildkite/dagster-buildkite/dagster_buildkite/steps/packages.py +++ b/.buildkite/dagster-buildkite/dagster_buildkite/steps/packages.py @@ -373,7 +373,7 @@ def k8s_extra_cmds(version: AvailablePythonVersion, _) -> List[str]: # Runs against live dbt cloud instance, we only want to run on commits and on the # nightly build PackageSpec( - "examples/experimental/dagster-airlift/examples/dbt-example", + "examples/starlift-demo", skip_if=skip_if_not_airlift_or_dlift_commit, env_vars=[ "KS_DBT_CLOUD_ACCOUNT_ID", @@ -386,15 +386,15 @@ def k8s_extra_cmds(version: AvailablePythonVersion, _) -> List[str]: queue=BuildkiteQueue.DOCKER, ), PackageSpec( - "examples/experimental/dagster-airlift/examples/perf-harness", + "examples/experimental/dagster-airlift/perf-harness", always_run_if=has_dagster_airlift_changes, ), PackageSpec( - "examples/experimental/dagster-airlift/examples/tutorial-example", + "examples/airlift-migration-tutorial", always_run_if=has_dagster_airlift_changes, ), PackageSpec( - "examples/experimental/dagster-airlift/examples/kitchen-sink", + "examples/experimental/dagster-airlift/kitchen-sink", always_run_if=has_dagster_airlift_changes, ), PackageSpec( diff --git a/docs/content/integrations/airlift/full_dag.mdx b/docs/content/integrations/airlift/full_dag.mdx index e38d7f375c16a..4b533e51858bd 100644 --- a/docs/content/integrations/airlift/full_dag.mdx +++ b/docs/content/integrations/airlift/full_dag.mdx @@ -17,7 +17,7 @@ When migrating an entire DAG at once, we'll want to create assets which map to t 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 +```python file=../../airlift-migration-tutorial/tutorial_example/dagster_defs/stages/observe_dag_level.py import os from pathlib import Path @@ -79,7 +79,7 @@ Now, instead of getting a materialization when a particular task completes, each 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 +```python file=../../airlift-migration-tutorial/tutorial_example/dagster_defs/stages/migrate_dag_level.py import os from pathlib import Path @@ -192,13 +192,13 @@ if __name__ == "__main__": 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 +```yaml file=../../airlift-migration-tutorial/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 +```python file=../../airlift-migration-tutorial/tutorial_example/snippets/dags_truncated.py # Dags file can be found at tutorial_example/airflow_dags/dags.py from pathlib import Path diff --git a/docs/content/integrations/airlift/reference.mdx b/docs/content/integrations/airlift/reference.mdx index 62932d6b67516..f1adda58eb2ca 100644 --- a/docs/content/integrations/airlift/reference.mdx +++ b/docs/content/integrations/airlift/reference.mdx @@ -16,7 +16,7 @@ description: "dagster-airlift is a toolkit for observing and migrating Airflow D 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 +```python file=../../airlift-migration-tutorial/tutorial_example/snippets/custom_operator_examples/custom_proxy.py from pathlib import Path import requests @@ -55,7 +55,7 @@ proxying_to_dagster( You can use a custom 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 +```python file=../../airlift-migration-tutorial/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 @@ -130,7 +130,7 @@ Similar to how we can customize the operator we construct on a per-dag basis, we 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 +```python file=../../airlift-migration-tutorial/tutorial_example/snippets/custom_operator_examples/custom_dag_level_proxy.py from pathlib import Path import requests diff --git a/docs/content/integrations/airlift/tutorial.mdx b/docs/content/integrations/airlift/tutorial.mdx new file mode 100644 index 0000000000000..cd5e973461923 --- /dev/null +++ b/docs/content/integrations/airlift/tutorial.mdx @@ -0,0 +1,1295 @@ +## 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-migration-tutorial +cd airlift-migration-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 `airlift-migration-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`. + +## 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=../../airlift-migration-tutorial/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 airlift-migration-tutorial 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=../../airlift-migration-tutorial/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`), 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=../../airlift-migration-tutorial/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. + +

+ + + +

+ +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=../../airlift-migration-tutorial/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=../../airlift-migration-tutorial/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=../../airlift-migration-tutorial/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=../../airlift-migration-tutorial/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=../../airlift-migration-tutorial/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=../../airlift-migration-tutorial/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())}, +) +``` + +## 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=../../airlift-migration-tutorial/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 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=../../airlift-migration-tutorial/tutorial_example/dagster_defs/stages/migrate_with_check.py +import os +from pathlib import Path + +from dagster import ( + AssetCheckResult, + AssetCheckSeverity, + AssetExecutionContext, + 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, + 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", + ), + ) + ], + }, +) + + +@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=../../airlift-migration-tutorial/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=../../airlift-migration-tutorial/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=../../airlift-migration-tutorial/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=../../airlift-migration-tutorial/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=../../airlift-migration-tutorial/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/decomission.mdx b/docs/content/integrations/airlift/tutorial/decomission.mdx index 4c60b4194ec5c..f4b87f020a369 100644 --- a/docs/content/integrations/airlift/tutorial/decomission.mdx +++ b/docs/content/integrations/airlift/tutorial/decomission.mdx @@ -6,7 +6,7 @@ Once we are confident in our migrated versions of the tasks, we can decommission 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 +```python file=../../airlift-migration-tutorial/tutorial_example/dagster_defs/stages/standalone.py import os from pathlib import Path diff --git a/docs/content/integrations/airlift/tutorial/migrate.mdx b/docs/content/integrations/airlift/tutorial/migrate.mdx index 723824910fda0..889770f53414a 100644 --- a/docs/content/integrations/airlift/tutorial/migrate.mdx +++ b/docs/content/integrations/airlift/tutorial/migrate.mdx @@ -10,7 +10,7 @@ To begin proxying tasks in a DAG, first you will need a file to track proxying s 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 +```yaml file=../../airlift-migration-tutorial/tutorial_example/airflow_dags/proxied_state/rebuild_customers_list.yaml tasks: - id: load_raw_customers proxied: False @@ -22,7 +22,7 @@ tasks: 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 +```python file=../../airlift-migration-tutorial/tutorial_example/snippets/dags_truncated.py # Dags file can be found at tutorial_example/airflow_dags/dags.py from pathlib import Path @@ -76,7 +76,7 @@ A task which has been proxied can be easily toggled back to run in Airflow (for 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 +```yaml file=../../airlift-migration-tutorial/tutorial_example/snippets/dbt_proxied.yaml tasks: - id: load_raw_customers proxied: False @@ -109,7 +109,7 @@ For all other operator types, we will need to build our own asset definitions. W 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 +```python file=../../airlift-migration-tutorial/tutorial_example/dagster_defs/stages/migrate.py import os from pathlib import Path @@ -232,7 +232,7 @@ if __name__ == "__main__": 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 +```yaml file=../../airlift-migration-tutorial/tutorial_example/snippets/all_proxied.yaml tasks: - id: load_raw_customers proxied: True diff --git a/docs/content/integrations/airlift/tutorial/observe.mdx b/docs/content/integrations/airlift/tutorial/observe.mdx index c6250d8f8a74b..90d8e25ad97b4 100644 --- a/docs/content/integrations/airlift/tutorial/observe.mdx +++ b/docs/content/integrations/airlift/tutorial/observe.mdx @@ -26,7 +26,7 @@ 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 +```python file=../../airlift-migration-tutorial/tutorial_example/dagster_defs/stages/observe.py import os from pathlib import Path @@ -136,7 +136,7 @@ Now that we've introduced an asset explicitly for the `customers.csv` file outpu 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 +```python file=../../airlift-migration-tutorial/tutorial_example/dagster_defs/stages/observe_check_on_asset.py import os from pathlib import Path @@ -225,7 +225,7 @@ defs = build_defs_from_airflow_instance( 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 +```python file=../../airlift-migration-tutorial/tutorial_example/dagster_defs/stages/observe_with_partitions.py import os from pathlib import Path diff --git a/docs/content/integrations/airlift/tutorial/peer.mdx b/docs/content/integrations/airlift/tutorial/peer.mdx index 23b1a22777376..633a14db18d62 100644 --- a/docs/content/integrations/airlift/tutorial/peer.mdx +++ b/docs/content/integrations/airlift/tutorial/peer.mdx @@ -13,7 +13,7 @@ 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 +```python file=../../airlift-migration-tutorial/tutorial_example/dagster_defs/stages/peer.py from dagster_airlift.core import ( AirflowBasicAuthBackend, AirflowInstance, @@ -41,7 +41,7 @@ This function creates: 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 +# Set up environment variables to point to the airlift-migration-tutorial 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" @@ -94,7 +94,7 @@ Asset checks can both act as useful _user acceptance tests_ to ensure that any m 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 +```python file=../../airlift-migration-tutorial/tutorial_example/dagster_defs/stages/peer_with_check.py import os from pathlib import Path diff --git a/docs/content/integrations/airlift/tutorial/setup.mdx b/docs/content/integrations/airlift/tutorial/setup.mdx index 8f42c1c09239a..f2291e839e7cd 100644 --- a/docs/content/integrations/airlift/tutorial/setup.mdx +++ b/docs/content/integrations/airlift/tutorial/setup.mdx @@ -11,8 +11,8 @@ In this step, we'll 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 +git clone git@github.com:dagster-io/airlift-migration-tutorial.git +cd airlift-migration-tutorial ``` Next, we'll create a fresh virtual environment using `uv`. @@ -25,7 +25,7 @@ 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. +The tutorial example involves running a local Airflow instance. This can be done by running the following commands from the root of the `airlift-migration-tutorial` directory. First, install the required python packages: diff --git a/docs/tox.ini b/docs/tox.ini index efa59cdc8b008..246cc13f0df99 100644 --- a/docs/tox.ini +++ b/docs/tox.ini @@ -49,7 +49,7 @@ deps = -e ../python_modules/libraries/dagster-tableau -e ../python_modules/libraries/dagster-powerbi -e ../examples/experimental/dagster-airlift[tutorial,core,in-airflow,dbt] - -e ../examples/experimental/dagster-airlift/examples/tutorial-example + -e ../examples/airlift-migration-tutorial sling diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/.gitignore b/examples/airlift-migration-tutorial/.gitignore similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/.gitignore rename to examples/airlift-migration-tutorial/.gitignore diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/Makefile b/examples/airlift-migration-tutorial/Makefile similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/Makefile rename to examples/airlift-migration-tutorial/Makefile diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/README.md b/examples/airlift-migration-tutorial/README.md similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/README.md rename to examples/airlift-migration-tutorial/README.md diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/conftest.py b/examples/airlift-migration-tutorial/conftest.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/conftest.py rename to examples/airlift-migration-tutorial/conftest.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/images/after_dag_override.png b/examples/airlift-migration-tutorial/images/after_dag_override.png similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/images/after_dag_override.png rename to examples/airlift-migration-tutorial/images/after_dag_override.png diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/images/before_dag_override.png b/examples/airlift-migration-tutorial/images/before_dag_override.png similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/images/before_dag_override.png rename to examples/airlift-migration-tutorial/images/before_dag_override.png diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/images/observe.svg b/examples/airlift-migration-tutorial/images/observe.svg similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/images/observe.svg rename to examples/airlift-migration-tutorial/images/observe.svg diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/images/partitioned_mat.png b/examples/airlift-migration-tutorial/images/partitioned_mat.png similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/images/partitioned_mat.png rename to examples/airlift-migration-tutorial/images/partitioned_mat.png diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/images/peer.svg b/examples/airlift-migration-tutorial/images/peer.svg similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/images/peer.svg rename to examples/airlift-migration-tutorial/images/peer.svg diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/images/peer_materialize.svg b/examples/airlift-migration-tutorial/images/peer_materialize.svg similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/images/peer_materialize.svg rename to examples/airlift-migration-tutorial/images/peer_materialize.svg diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/images/proxied_dag.png b/examples/airlift-migration-tutorial/images/proxied_dag.png similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/images/proxied_dag.png rename to examples/airlift-migration-tutorial/images/proxied_dag.png diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/images/state_in_airflow.png b/examples/airlift-migration-tutorial/images/state_in_airflow.png similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/images/state_in_airflow.png rename to examples/airlift-migration-tutorial/images/state_in_airflow.png diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/pyproject.toml b/examples/airlift-migration-tutorial/pyproject.toml similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/pyproject.toml rename to examples/airlift-migration-tutorial/pyproject.toml diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/scripts/airflow_setup.sh b/examples/airlift-migration-tutorial/scripts/airflow_setup.sh similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/scripts/airflow_setup.sh rename to examples/airlift-migration-tutorial/scripts/airflow_setup.sh diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/setup.py b/examples/airlift-migration-tutorial/setup.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/setup.py rename to examples/airlift-migration-tutorial/setup.py diff --git a/examples/airlift-migration-tutorial/tox.ini b/examples/airlift-migration-tutorial/tox.ini new file mode 100644 index 0000000000000..d6b8ca8fb3479 --- /dev/null +++ b/examples/airlift-migration-tutorial/tox.ini @@ -0,0 +1,29 @@ +[tox] +skipsdist = true + +[testenv] +download = True +passenv = + CI_* + COVERALLS_REPO_TOKEN + BUILDKITE* +install_command = uv pip install {opts} {packages} +deps = + -e ../../python_modules/dagster[test] + -e ../../python_modules/dagster-webserver + -e ../../python_modules/dagster-test + -e ../../python_modules/dagster-pipes + -e ../../python_modules/dagster-graphql + -e ../../python_modules/libraries/dagster-dbt + -e ../experimental/dagster-airlift[core,dbt,test,in-airflow] + -e . + pandas +allowlist_externals = + /bin/bash + uv + make +commands = + # We need to rebuild the UI to ensure that the dagster-webserver can run + make -C ../.. rebuild_ui + !windows: /bin/bash -c '! pip list --exclude-editable | grep -e dagster' + pytest -c ../../pyproject.toml ./tutorial_example_tests --snapshot-warn-unused -vv {posargs} diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/__init__.py b/examples/airlift-migration-tutorial/tutorial_example/__init__.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/__init__.py rename to examples/airlift-migration-tutorial/tutorial_example/__init__.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/airflow_dags/dags.py b/examples/airlift-migration-tutorial/tutorial_example/airflow_dags/dags.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/airflow_dags/dags.py rename to examples/airlift-migration-tutorial/tutorial_example/airflow_dags/dags.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/airflow_dags/proxied_state/rebuild_customers_list.yaml b/examples/airlift-migration-tutorial/tutorial_example/airflow_dags/proxied_state/rebuild_customers_list.yaml similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/airflow_dags/proxied_state/rebuild_customers_list.yaml rename to examples/airlift-migration-tutorial/tutorial_example/airflow_dags/proxied_state/rebuild_customers_list.yaml diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/airflow_dags/raw_customers.csv b/examples/airlift-migration-tutorial/tutorial_example/airflow_dags/raw_customers.csv similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/airflow_dags/raw_customers.csv rename to examples/airlift-migration-tutorial/tutorial_example/airflow_dags/raw_customers.csv diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/__init__.py b/examples/airlift-migration-tutorial/tutorial_example/dagster_defs/__init__.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/__init__.py rename to examples/airlift-migration-tutorial/tutorial_example/dagster_defs/__init__.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/definitions.py b/examples/airlift-migration-tutorial/tutorial_example/dagster_defs/definitions.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/definitions.py rename to examples/airlift-migration-tutorial/tutorial_example/dagster_defs/definitions.py diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/__init__.py b/examples/airlift-migration-tutorial/tutorial_example/dagster_defs/stages/__init__.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/__init__.py rename to examples/airlift-migration-tutorial/tutorial_example/dagster_defs/stages/__init__.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/migrate.py b/examples/airlift-migration-tutorial/tutorial_example/dagster_defs/stages/migrate.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/migrate.py rename to examples/airlift-migration-tutorial/tutorial_example/dagster_defs/stages/migrate.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/migrate_dag_level.py b/examples/airlift-migration-tutorial/tutorial_example/dagster_defs/stages/migrate_dag_level.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/migrate_dag_level.py rename to examples/airlift-migration-tutorial/tutorial_example/dagster_defs/stages/migrate_dag_level.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/migrate_with_check.py b/examples/airlift-migration-tutorial/tutorial_example/dagster_defs/stages/migrate_with_check.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/migrate_with_check.py rename to examples/airlift-migration-tutorial/tutorial_example/dagster_defs/stages/migrate_with_check.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/observe.py b/examples/airlift-migration-tutorial/tutorial_example/dagster_defs/stages/observe.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/observe.py rename to examples/airlift-migration-tutorial/tutorial_example/dagster_defs/stages/observe.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/observe_check_on_asset.py b/examples/airlift-migration-tutorial/tutorial_example/dagster_defs/stages/observe_check_on_asset.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/observe_check_on_asset.py rename to examples/airlift-migration-tutorial/tutorial_example/dagster_defs/stages/observe_check_on_asset.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/observe_dag_level.py b/examples/airlift-migration-tutorial/tutorial_example/dagster_defs/stages/observe_dag_level.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/observe_dag_level.py rename to examples/airlift-migration-tutorial/tutorial_example/dagster_defs/stages/observe_dag_level.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/observe_with_partitions.py b/examples/airlift-migration-tutorial/tutorial_example/dagster_defs/stages/observe_with_partitions.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/observe_with_partitions.py rename to examples/airlift-migration-tutorial/tutorial_example/dagster_defs/stages/observe_with_partitions.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/peer.py b/examples/airlift-migration-tutorial/tutorial_example/dagster_defs/stages/peer.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/peer.py rename to examples/airlift-migration-tutorial/tutorial_example/dagster_defs/stages/peer.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/peer_with_check.py b/examples/airlift-migration-tutorial/tutorial_example/dagster_defs/stages/peer_with_check.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/peer_with_check.py rename to examples/airlift-migration-tutorial/tutorial_example/dagster_defs/stages/peer_with_check.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/standalone.py b/examples/airlift-migration-tutorial/tutorial_example/dagster_defs/stages/standalone.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/standalone.py rename to examples/airlift-migration-tutorial/tutorial_example/dagster_defs/stages/standalone.py diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example_tests/__init__.py b/examples/airlift-migration-tutorial/tutorial_example/shared/__init__.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example_tests/__init__.py rename to examples/airlift-migration-tutorial/tutorial_example/shared/__init__.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/dbt/dbt_project.yml b/examples/airlift-migration-tutorial/tutorial_example/shared/dbt/dbt_project.yml similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/dbt/dbt_project.yml rename to examples/airlift-migration-tutorial/tutorial_example/shared/dbt/dbt_project.yml diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/models/customers.sql b/examples/airlift-migration-tutorial/tutorial_example/shared/dbt/models/customers.sql similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/models/customers.sql rename to examples/airlift-migration-tutorial/tutorial_example/shared/dbt/models/customers.sql diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/models/docs.md b/examples/airlift-migration-tutorial/tutorial_example/shared/dbt/models/docs.md similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/models/docs.md rename to examples/airlift-migration-tutorial/tutorial_example/shared/dbt/models/docs.md diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/models/orders.sql b/examples/airlift-migration-tutorial/tutorial_example/shared/dbt/models/orders.sql similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/models/orders.sql rename to examples/airlift-migration-tutorial/tutorial_example/shared/dbt/models/orders.sql diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/models/overview.md b/examples/airlift-migration-tutorial/tutorial_example/shared/dbt/models/overview.md similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/models/overview.md rename to examples/airlift-migration-tutorial/tutorial_example/shared/dbt/models/overview.md diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/dbt/models/schema.yml b/examples/airlift-migration-tutorial/tutorial_example/shared/dbt/models/schema.yml similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/dbt/models/schema.yml rename to examples/airlift-migration-tutorial/tutorial_example/shared/dbt/models/schema.yml diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/dbt/models/sources.yml b/examples/airlift-migration-tutorial/tutorial_example/shared/dbt/models/sources.yml similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/dbt/models/sources.yml rename to examples/airlift-migration-tutorial/tutorial_example/shared/dbt/models/sources.yml diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/models/staging/schema.yml b/examples/airlift-migration-tutorial/tutorial_example/shared/dbt/models/staging/schema.yml similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/models/staging/schema.yml rename to examples/airlift-migration-tutorial/tutorial_example/shared/dbt/models/staging/schema.yml diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/dbt/models/staging/stg_customers.sql b/examples/airlift-migration-tutorial/tutorial_example/shared/dbt/models/staging/stg_customers.sql similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/dbt/models/staging/stg_customers.sql rename to examples/airlift-migration-tutorial/tutorial_example/shared/dbt/models/staging/stg_customers.sql diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/models/staging/stg_orders.sql b/examples/airlift-migration-tutorial/tutorial_example/shared/dbt/models/staging/stg_orders.sql similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/models/staging/stg_orders.sql rename to examples/airlift-migration-tutorial/tutorial_example/shared/dbt/models/staging/stg_orders.sql diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/dbt/models/staging/stg_payments.sql b/examples/airlift-migration-tutorial/tutorial_example/shared/dbt/models/staging/stg_payments.sql similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/dbt/models/staging/stg_payments.sql rename to examples/airlift-migration-tutorial/tutorial_example/shared/dbt/models/staging/stg_payments.sql diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/dbt/profiles.yml b/examples/airlift-migration-tutorial/tutorial_example/shared/dbt/profiles.yml similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/dbt/profiles.yml rename to examples/airlift-migration-tutorial/tutorial_example/shared/dbt/profiles.yml diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/requirements.in b/examples/airlift-migration-tutorial/tutorial_example/shared/dbt/requirements.in similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/requirements.in rename to examples/airlift-migration-tutorial/tutorial_example/shared/dbt/requirements.in diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/requirements.txt b/examples/airlift-migration-tutorial/tutorial_example/shared/dbt/requirements.txt similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/requirements.txt rename to examples/airlift-migration-tutorial/tutorial_example/shared/dbt/requirements.txt diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/seeds/.gitkeep b/examples/airlift-migration-tutorial/tutorial_example/shared/dbt/seeds/.gitkeep similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/seeds/.gitkeep rename to examples/airlift-migration-tutorial/tutorial_example/shared/dbt/seeds/.gitkeep diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/seeds/raw_orders.csv b/examples/airlift-migration-tutorial/tutorial_example/shared/dbt/seeds/raw_orders.csv similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/seeds/raw_orders.csv rename to examples/airlift-migration-tutorial/tutorial_example/shared/dbt/seeds/raw_orders.csv diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/seeds/raw_payments.csv b/examples/airlift-migration-tutorial/tutorial_example/shared/dbt/seeds/raw_payments.csv similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/seeds/raw_payments.csv rename to examples/airlift-migration-tutorial/tutorial_example/shared/dbt/seeds/raw_payments.csv diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/export_duckdb_to_csv.py b/examples/airlift-migration-tutorial/tutorial_example/shared/export_duckdb_to_csv.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/export_duckdb_to_csv.py rename to examples/airlift-migration-tutorial/tutorial_example/shared/export_duckdb_to_csv.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/load_csv_to_duckdb.py b/examples/airlift-migration-tutorial/tutorial_example/shared/load_csv_to_duckdb.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/load_csv_to_duckdb.py rename to examples/airlift-migration-tutorial/tutorial_example/shared/load_csv_to_duckdb.py diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example_tests/integration_tests/__init__.py b/examples/airlift-migration-tutorial/tutorial_example/snippets/__init__.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example_tests/integration_tests/__init__.py rename to examples/airlift-migration-tutorial/tutorial_example/snippets/__init__.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/all_proxied.yaml b/examples/airlift-migration-tutorial/tutorial_example/snippets/all_proxied.yaml similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/all_proxied.yaml rename to examples/airlift-migration-tutorial/tutorial_example/snippets/all_proxied.yaml diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example_tests/unit_tests/__init__.py b/examples/airlift-migration-tutorial/tutorial_example/snippets/custom_operator_examples/__init__.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example_tests/unit_tests/__init__.py rename to examples/airlift-migration-tutorial/tutorial_example/snippets/custom_operator_examples/__init__.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/custom_operator_examples/custom_dag_level_proxy.py b/examples/airlift-migration-tutorial/tutorial_example/snippets/custom_operator_examples/custom_dag_level_proxy.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/custom_operator_examples/custom_dag_level_proxy.py rename to examples/airlift-migration-tutorial/tutorial_example/snippets/custom_operator_examples/custom_dag_level_proxy.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/custom_operator_examples/custom_proxy.py b/examples/airlift-migration-tutorial/tutorial_example/snippets/custom_operator_examples/custom_proxy.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/custom_operator_examples/custom_proxy.py rename to examples/airlift-migration-tutorial/tutorial_example/snippets/custom_operator_examples/custom_proxy.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/custom_operator_examples/plus_proxy_operator.py b/examples/airlift-migration-tutorial/tutorial_example/snippets/custom_operator_examples/plus_proxy_operator.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/custom_operator_examples/plus_proxy_operator.py rename to examples/airlift-migration-tutorial/tutorial_example/snippets/custom_operator_examples/plus_proxy_operator.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/custom_operator_examples/proxied_state/custom_dag_level_proxy_example.yaml b/examples/airlift-migration-tutorial/tutorial_example/snippets/custom_operator_examples/proxied_state/custom_dag_level_proxy_example.yaml similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/custom_operator_examples/proxied_state/custom_dag_level_proxy_example.yaml rename to examples/airlift-migration-tutorial/tutorial_example/snippets/custom_operator_examples/proxied_state/custom_dag_level_proxy_example.yaml diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/custom_operator_examples/proxied_state/custom_proxy_example.yaml b/examples/airlift-migration-tutorial/tutorial_example/snippets/custom_operator_examples/proxied_state/custom_proxy_example.yaml similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/custom_operator_examples/proxied_state/custom_proxy_example.yaml rename to examples/airlift-migration-tutorial/tutorial_example/snippets/custom_operator_examples/proxied_state/custom_proxy_example.yaml diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/dags_truncated.py b/examples/airlift-migration-tutorial/tutorial_example/snippets/dags_truncated.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/dags_truncated.py rename to examples/airlift-migration-tutorial/tutorial_example/snippets/dags_truncated.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/dbt_proxied.yaml b/examples/airlift-migration-tutorial/tutorial_example/snippets/dbt_proxied.yaml similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/dbt_proxied.yaml rename to examples/airlift-migration-tutorial/tutorial_example/snippets/dbt_proxied.yaml diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/rebuild_customers_list.yaml b/examples/airlift-migration-tutorial/tutorial_example/snippets/rebuild_customers_list.yaml similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/rebuild_customers_list.yaml rename to examples/airlift-migration-tutorial/tutorial_example/snippets/rebuild_customers_list.yaml diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/__init__.py b/examples/airlift-migration-tutorial/tutorial_example_tests/__init__.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/__init__.py rename to examples/airlift-migration-tutorial/tutorial_example_tests/__init__.py diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/dagster_defs/__init__.py b/examples/airlift-migration-tutorial/tutorial_example_tests/integration_tests/__init__.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/dagster_defs/__init__.py rename to examples/airlift-migration-tutorial/tutorial_example_tests/integration_tests/__init__.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example_tests/integration_tests/conftest.py b/examples/airlift-migration-tutorial/tutorial_example_tests/integration_tests/conftest.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example_tests/integration_tests/conftest.py rename to examples/airlift-migration-tutorial/tutorial_example_tests/integration_tests/conftest.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example_tests/integration_tests/test_load_dagster.py b/examples/airlift-migration-tutorial/tutorial_example_tests/integration_tests/test_load_dagster.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example_tests/integration_tests/test_load_dagster.py rename to examples/airlift-migration-tutorial/tutorial_example_tests/integration_tests/test_load_dagster.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example_tests/integration_tests/test_migrate_with_check_e2e.py b/examples/airlift-migration-tutorial/tutorial_example_tests/integration_tests/test_migrate_with_check_e2e.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example_tests/integration_tests/test_migrate_with_check_e2e.py rename to examples/airlift-migration-tutorial/tutorial_example_tests/integration_tests/test_migrate_with_check_e2e.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example_tests/integration_tests/test_migrating_e2e.py b/examples/airlift-migration-tutorial/tutorial_example_tests/integration_tests/test_migrating_e2e.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example_tests/integration_tests/test_migrating_e2e.py rename to examples/airlift-migration-tutorial/tutorial_example_tests/integration_tests/test_migrating_e2e.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example_tests/integration_tests/test_observing_e2e.py b/examples/airlift-migration-tutorial/tutorial_example_tests/integration_tests/test_observing_e2e.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example_tests/integration_tests/test_observing_e2e.py rename to examples/airlift-migration-tutorial/tutorial_example_tests/integration_tests/test_observing_e2e.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example_tests/integration_tests/test_peer_with_check_e2e.py b/examples/airlift-migration-tutorial/tutorial_example_tests/integration_tests/test_peer_with_check_e2e.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example_tests/integration_tests/test_peer_with_check_e2e.py rename to examples/airlift-migration-tutorial/tutorial_example_tests/integration_tests/test_peer_with_check_e2e.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example_tests/integration_tests/test_peering_e2e.py b/examples/airlift-migration-tutorial/tutorial_example_tests/integration_tests/test_peering_e2e.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example_tests/integration_tests/test_peering_e2e.py rename to examples/airlift-migration-tutorial/tutorial_example_tests/integration_tests/test_peering_e2e.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example_tests/integration_tests/utils.py b/examples/airlift-migration-tutorial/tutorial_example_tests/integration_tests/utils.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example_tests/integration_tests/utils.py rename to examples/airlift-migration-tutorial/tutorial_example_tests/integration_tests/utils.py diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/dagster_multi_code_locations/__init__.py b/examples/airlift-migration-tutorial/tutorial_example_tests/unit_tests/__init__.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/dagster_multi_code_locations/__init__.py rename to examples/airlift-migration-tutorial/tutorial_example_tests/unit_tests/__init__.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example_tests/unit_tests/test_proxy_operators.py b/examples/airlift-migration-tutorial/tutorial_example_tests/unit_tests/test_proxy_operators.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example_tests/unit_tests/test_proxy_operators.py rename to examples/airlift-migration-tutorial/tutorial_example_tests/unit_tests/test_proxy_operators.py diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/.gitignore b/examples/airlift-mwaa-example/.gitignore similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/.gitignore rename to examples/airlift-mwaa-example/.gitignore diff --git a/examples/experimental/dagster-airlift/examples/mwaa-example/Makefile b/examples/airlift-mwaa-example/Makefile similarity index 100% rename from examples/experimental/dagster-airlift/examples/mwaa-example/Makefile rename to examples/airlift-mwaa-example/Makefile diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/conftest.py b/examples/airlift-mwaa-example/conftest.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/conftest.py rename to examples/airlift-mwaa-example/conftest.py diff --git a/examples/experimental/dagster-airlift/examples/mwaa-example/dags/requirements.txt b/examples/airlift-mwaa-example/dags/requirements.txt similarity index 100% rename from examples/experimental/dagster-airlift/examples/mwaa-example/dags/requirements.txt rename to examples/airlift-mwaa-example/dags/requirements.txt diff --git a/examples/experimental/dagster-airlift/examples/mwaa-example/dags/simple_dag.py b/examples/airlift-mwaa-example/dags/simple_dag.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/mwaa-example/dags/simple_dag.py rename to examples/airlift-mwaa-example/dags/simple_dag.py diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink_tests/__init__.py b/examples/airlift-mwaa-example/mwaa_example/__init__.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink_tests/__init__.py rename to examples/airlift-mwaa-example/mwaa_example/__init__.py diff --git a/examples/experimental/dagster-airlift/examples/mwaa-example/mwaa_example/definitions.py b/examples/airlift-mwaa-example/mwaa_example/definitions.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/mwaa-example/mwaa_example/definitions.py rename to examples/airlift-mwaa-example/mwaa_example/definitions.py diff --git a/examples/experimental/dagster-airlift/examples/mwaa-example/scripts/setup_mwaa.py b/examples/airlift-mwaa-example/scripts/setup_mwaa.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/mwaa-example/scripts/setup_mwaa.py rename to examples/airlift-mwaa-example/scripts/setup_mwaa.py diff --git a/examples/experimental/dagster-airlift/examples/mwaa-example/setup.py b/examples/airlift-mwaa-example/setup.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/mwaa-example/setup.py rename to examples/airlift-mwaa-example/setup.py diff --git a/examples/experimental/dagster-airlift/Makefile b/examples/experimental/dagster-airlift/Makefile index f5082e48b46f4..14fb1b1795df6 100644 --- a/examples/experimental/dagster-airlift/Makefile +++ b/examples/experimental/dagster-airlift/Makefile @@ -5,7 +5,7 @@ adhoc_pypi: @chmod +x scripts/increment_pypi_version.sh @./scripts/increment_pypi_version.sh -# Mirrors the tutorial to the dagster-airlift-tutorial repo: https://github.com/dagster-io/airlift-tutorial +# Mirrors the tutorial to the dagster-airlift-migration-tutorial repo: https://github.com/dagster-io/airlift-migration-tutorial mirror_tutorial: @chmod +x scripts/mirror_tutorial.sh @./scripts/mirror_tutorial.sh @@ -14,7 +14,7 @@ mirror_tutorial: # - Enforces that we're on master # - Bumps the version in setup.py # - Pushes the changes to PyPI -# - Mirrors the tutorial to the dagster-airlift-tutorial repo +# - Mirrors the tutorial to the dagster-airlift-migration-tutorial repo # - Creates a release branch # - Pushes the release branch to origin adhoc_release: diff --git a/examples/experimental/dagster-airlift/README.md b/examples/experimental/dagster-airlift/README.md index 940526d520ff6..bea3e74cc91ad 100644 --- a/examples/experimental/dagster-airlift/README.md +++ b/examples/experimental/dagster-airlift/README.md @@ -36,4 +36,4 @@ Airlift depends on the the availability of Airflow’s REST API. Airflow’s RES # Tutorial -We've provided a tutorial to help get you started with airlift tooling and process, which can be found [here](https://github.com/dagster-io/airlift-tutorial). +We've provided a tutorial to help get you started with airlift tooling and process, which can be found [here](https://github.com/dagster-io/airlift-migration-tutorial). diff --git a/examples/experimental/dagster-airlift/examples/bi-demo/.airflow_home/airflow.db b/examples/experimental/dagster-airlift/examples/bi-demo/.airflow_home/airflow.db deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/setup.py b/examples/experimental/dagster-airlift/examples/dbt-example/setup.py deleted file mode 100644 index f50369549c451..0000000000000 --- a/examples/experimental/dagster-airlift/examples/dbt-example/setup.py +++ /dev/null @@ -1,33 +0,0 @@ -from pathlib import Path -from typing import Dict - -from setuptools import find_packages, setup - - -def get_version() -> str: - version: Dict[str, str] = {} - with open( - Path(__file__).parent / ".." / ".." / "dagster_airlift/version.py", encoding="utf8" - ) as fp: - exec(fp.read(), version) - - return version["__version__"] - - -ver = get_version() -# dont pin dev installs to avoid pip dep resolver issues -pin = "" if ver == "1!0+dev" else f"=={ver}" - -setup( - name="dbt-example", - packages=find_packages(), - install_requires=[ - f"dagster{pin}", - f"dagster-webserver{pin}", - f"dagster-airlift[dbt,core,in-airflow]{pin}", - "dagster-dlift", - "dbt-duckdb", - "pandas", - ], - extras_require={"test": ["pytest"]}, -) diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/tox.ini b/examples/experimental/dagster-airlift/examples/dbt-example/tox.ini deleted file mode 100644 index 0e1aeac90810e..0000000000000 --- a/examples/experimental/dagster-airlift/examples/dbt-example/tox.ini +++ /dev/null @@ -1,30 +0,0 @@ -[tox] -skipsdist = true - -[testenv] -download = True -passenv = - CI_* - COVERALLS_REPO_TOKEN - BUILDKITE* - KS_DBT_CLOUD* -install_command = uv pip install {opts} {packages} -deps = - -e ../../../../../python_modules/dagster[test] - -e ../../../../../python_modules/dagster-webserver - -e ../../../../../python_modules/dagster-test - -e ../../../../../python_modules/dagster-pipes - -e ../../../../../python_modules/dagster-graphql - -e ../../../../../python_modules/libraries/dagster-dbt - -e ../../../dagster-airlift[core,dbt,test,in-airflow] - -e ../../../dagster-dlift - -e . - pandas -allowlist_externals = - /bin/bash - uv -commands = - # We need to rebuild the UI to ensure that the dagster-webserver can run - make -C ../../../../.. rebuild_ui - !windows: /bin/bash -c '! pip list --exclude-editable | grep -e dagster' - pytest -c ../../../../../pyproject.toml ./dbt_example_tests --snapshot-warn-unused -vv -s {posargs} \ No newline at end of file diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/setup.py b/examples/experimental/dagster-airlift/examples/kitchen-sink/setup.py deleted file mode 100644 index 9150966595bc2..0000000000000 --- a/examples/experimental/dagster-airlift/examples/kitchen-sink/setup.py +++ /dev/null @@ -1,30 +0,0 @@ -from pathlib import Path -from typing import Dict - -from setuptools import find_packages, setup - - -def get_version() -> str: - version: Dict[str, str] = {} - with open( - Path(__file__).parent / ".." / ".." / "dagster_airlift/version.py", encoding="utf8" - ) as fp: - exec(fp.read(), version) - - return version["__version__"] - - -ver = get_version() -# dont pin dev installs to avoid pip dep resolver issues -pin = "" if ver == "1!0+dev" else f"=={ver}" - -setup( - name="kitchen-sink", - packages=find_packages(), - install_requires=[ - f"dagster{pin}", - f"dagster-webserver{pin}", - f"dagster-airlift[core,in-airflow]{pin}", - ], - extras_require={"test": ["pytest"]}, -) diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/tox.ini b/examples/experimental/dagster-airlift/examples/kitchen-sink/tox.ini deleted file mode 100644 index 8793bfd3fe2ec..0000000000000 --- a/examples/experimental/dagster-airlift/examples/kitchen-sink/tox.ini +++ /dev/null @@ -1,28 +0,0 @@ -[tox] -skipsdist = true - -[testenv] -download = True -passenv = - CI_* - COVERALLS_REPO_TOKEN - BUILDKITE* -install_command = uv pip install {opts} {packages} -deps = - -e ../../../../../python_modules/dagster[test] - -e ../../../../../python_modules/dagster-webserver - -e ../../../../../python_modules/dagster-test - -e ../../../../../python_modules/dagster-pipes - -e ../../../../../python_modules/dagster-graphql - -e ../../../../../python_modules/libraries/dagster-dbt - -e ../../../../../python_modules/libraries/dagster-pandas - -e ../../../dagster-airlift[core,test,in-airflow] - -e . -allowlist_externals = - /bin/bash - uv -commands = - # We need to rebuild the UI to ensure that the dagster-webserver can run - make -C ../../../../.. rebuild_ui - !windows: /bin/bash -c '! pip list --exclude-editable | grep -e dagster' - pytest -c ../../../../../pyproject.toml ./kitchen_sink_tests --snapshot-warn-unused -vv {posargs} \ No newline at end of file diff --git a/examples/experimental/dagster-airlift/examples/mwaa-example/tox.ini b/examples/experimental/dagster-airlift/examples/mwaa-example/tox.ini deleted file mode 100644 index 108b9353f9b89..0000000000000 --- a/examples/experimental/dagster-airlift/examples/mwaa-example/tox.ini +++ /dev/null @@ -1,26 +0,0 @@ -[tox] -skipsdist = true - -[testenv] -download = True -passenv = - CI_* - COVERALLS_REPO_TOKEN - BUILDKITE* -install_command = uv pip install {opts} {packages} -deps = - -e ../../../../../python_modules/dagster[test] - -e ../../../../../python_modules/dagster-webserver - -e ../../../../../python_modules/dagster-test - -e ../../../../../python_modules/dagster-pipes - -e ../../../../../python_modules/dagster-graphql - -e ../../../../../python_modules/libraries/dagster-dbt - -e ../../../dagster-airlift[core,test,mwaa] - -e . - pandas -allowlist_externals = - /bin/bash - uv -commands = - !windows: /bin/bash -c '! pip list --exclude-editable | grep -e dagster' - pytest -c ../../../../../pyproject.toml ./simple_migration_tests --snapshot-warn-unused -vv {posargs} \ No newline at end of file diff --git a/examples/experimental/dagster-airlift/examples/perf-harness/tox.ini b/examples/experimental/dagster-airlift/examples/perf-harness/tox.ini deleted file mode 100644 index 7a9f981e5c434..0000000000000 --- a/examples/experimental/dagster-airlift/examples/perf-harness/tox.ini +++ /dev/null @@ -1,29 +0,0 @@ -[tox] -skipsdist = true - -[testenv] -download = True -passenv = - CI_* - COVERALLS_REPO_TOKEN - BUILDKITE* -install_command = uv pip install {opts} {packages} -deps = - -e ../../../../../python_modules/dagster[test] - -e ../../../../../python_modules/dagster-webserver - -e ../../../../../python_modules/dagster-test - -e ../../../../../python_modules/dagster-pipes - -e ../../../../../python_modules/dagster-graphql - -e ../../../../../python_modules/libraries/dagster-dbt - -e ../../../dagster-airlift[core,dbt,test,in-airflow] - -e . - pandas -allowlist_externals = - /bin/bash - uv - make -commands = - # We need to rebuild the UI to ensure that the dagster-webserver can run - make -C ../../../../.. rebuild_ui - !windows: /bin/bash -c '! pip list --exclude-editable | grep -e dagster' - pytest -c ../../../../../pyproject.toml ./perf_harness_tests --snapshot-warn-unused -vv {posargs} diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/airflow_requirements.txt b/examples/experimental/dagster-airlift/examples/tutorial-example/airflow_requirements.txt deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tox.ini b/examples/experimental/dagster-airlift/examples/tutorial-example/tox.ini deleted file mode 100644 index 357ceacb2cc4a..0000000000000 --- a/examples/experimental/dagster-airlift/examples/tutorial-example/tox.ini +++ /dev/null @@ -1,29 +0,0 @@ -[tox] -skipsdist = true - -[testenv] -download = True -passenv = - CI_* - COVERALLS_REPO_TOKEN - BUILDKITE* -install_command = uv pip install {opts} {packages} -deps = - -e ../../../../../python_modules/dagster[test] - -e ../../../../../python_modules/dagster-webserver - -e ../../../../../python_modules/dagster-test - -e ../../../../../python_modules/dagster-pipes - -e ../../../../../python_modules/dagster-graphql - -e ../../../../../python_modules/libraries/dagster-dbt - -e ../../../dagster-airlift[core,dbt,test,in-airflow] - -e . - pandas -allowlist_externals = - /bin/bash - uv - make -commands = - # We need to rebuild the UI to ensure that the dagster-webserver can run - make -C ../../../../.. rebuild_ui - !windows: /bin/bash -c '! pip list --exclude-editable | grep -e dagster' - pytest -c ../../../../../pyproject.toml ./tutorial_example_tests --snapshot-warn-unused -vv {posargs} diff --git a/examples/experimental/dagster-airlift/examples/mwaa-example/.gitignore b/examples/experimental/dagster-airlift/kitchen-sink/.gitignore similarity index 100% rename from examples/experimental/dagster-airlift/examples/mwaa-example/.gitignore rename to examples/experimental/dagster-airlift/kitchen-sink/.gitignore diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/Makefile b/examples/experimental/dagster-airlift/kitchen-sink/Makefile similarity index 81% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/Makefile rename to examples/experimental/dagster-airlift/kitchen-sink/Makefile index e77c2c4fb78ed..b029af65e5cf0 100644 --- a/examples/experimental/dagster-airlift/examples/kitchen-sink/Makefile +++ b/examples/experimental/dagster-airlift/kitchen-sink/Makefile @@ -8,21 +8,22 @@ export MAKEFILE_DIR := $(GET_MAKEFILE_DIR) export DAGSTER_HOME := $(MAKEFILE_DIR)/.dagster_home export AIRFLOW_HOME := $(MAKEFILE_DIR)/.airflow_home export DAGSTER_URL := http://localhost:3333 +export AIRLIFT_MODULE_DIR := $(MAKEFILE_DIR)/.. help: @egrep -h '\s##\s' $(MAKEFILE_LIST) | sort | awk 'BEGIN {FS = ":.*?## "}; {printf "\033[36m%-20s\033[0m %s\n", $$1, $$2}' dev_install: - pip install uv && \ - uv pip install -e ../../../dagster-airlift + pip install uv + uv pip install -e $(AIRLIFT_MODULE_DIR) uv pip install -e . setup_local_env: $(MAKE) wipe mkdir -p $(AIRFLOW_HOME) mkdir -p $(DAGSTER_HOME) - chmod +x ../../scripts/airflow_setup.sh - ../../scripts/airflow_setup.sh $(MAKEFILE_DIR)/kitchen_sink/airflow_dags $(AIRFLOW_HOME) + chmod +x $(AIRLIFT_MODULE_DIR)/scripts/airflow_setup.sh + $(AIRLIFT_MODULE_DIR)/scripts/airflow_setup.sh $(MAKEFILE_DIR)/kitchen_sink/airflow_dags $(AIRFLOW_HOME) run_airflow: airflow standalone diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/README.md b/examples/experimental/dagster-airlift/kitchen-sink/README.md similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/README.md rename to examples/experimental/dagster-airlift/kitchen-sink/README.md diff --git a/examples/experimental/dagster-airlift/examples/mwaa-example/conftest.py b/examples/experimental/dagster-airlift/kitchen-sink/conftest.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/mwaa-example/conftest.py rename to examples/experimental/dagster-airlift/kitchen-sink/conftest.py diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink_tests/integration_tests/__init__.py b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/__init__.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink_tests/integration_tests/__init__.py rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/__init__.py diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/custom_callback.py b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/custom_callback.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/custom_callback.py rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/custom_callback.py diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/dag_level_custom_callback.py b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/dag_level_custom_callback.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/dag_level_custom_callback.py rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/dag_level_custom_callback.py diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/dag_level_proxied.py b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/dag_level_proxied.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/dag_level_proxied.py rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/dag_level_proxied.py diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/interval_dag.py b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/interval_dag.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/interval_dag.py rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/interval_dag.py diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/migrated_partitioned.py b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/migrated_partitioned.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/migrated_partitioned.py rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/migrated_partitioned.py diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/multi_job_assets_dag.py b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/multi_job_assets_dag.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/multi_job_assets_dag.py rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/multi_job_assets_dag.py diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/multi_location_dags.py b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/multi_location_dags.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/multi_location_dags.py rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/multi_location_dags.py diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/print_dag.py b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/print_dag.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/print_dag.py rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/print_dag.py diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/affected_dag.yaml b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/affected_dag.yaml similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/affected_dag.yaml rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/affected_dag.yaml diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/dag_first_code_location.yaml b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/dag_first_code_location.yaml similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/dag_first_code_location.yaml rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/dag_first_code_location.yaml diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/dag_second_code_location.yaml b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/dag_second_code_location.yaml similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/dag_second_code_location.yaml rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/dag_second_code_location.yaml diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/daily_dag.yaml b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/daily_dag.yaml similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/daily_dag.yaml rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/daily_dag.yaml diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/migrated_daily_interval_dag.yaml b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/migrated_daily_interval_dag.yaml similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/migrated_daily_interval_dag.yaml rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/migrated_daily_interval_dag.yaml diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/multi_job_assets_dag.yaml b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/multi_job_assets_dag.yaml similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/multi_job_assets_dag.yaml rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/multi_job_assets_dag.yaml diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/overridden_dag.yaml b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/overridden_dag.yaml similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/overridden_dag.yaml rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/overridden_dag.yaml diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/overridden_dag_custom_callback.yaml b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/overridden_dag_custom_callback.yaml similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/overridden_dag_custom_callback.yaml rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/overridden_dag_custom_callback.yaml diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/print_dag.yaml b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/print_dag.yaml similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/print_dag.yaml rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/print_dag.yaml diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/unaffected_dag.yaml b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/unaffected_dag.yaml similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/unaffected_dag.yaml rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/unaffected_dag.yaml diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/weekly_dag.yaml b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/weekly_dag.yaml similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/weekly_dag.yaml rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/weekly_dag.yaml diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/simple_unproxied.py b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/simple_unproxied.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/simple_unproxied.py rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_dags/simple_unproxied.py diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_instance.py b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_instance.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_instance.py rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/airflow_instance.py diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/constants.py b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/constants.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/constants.py rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/constants.py diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink_tests/unit_tests/__init__.py b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/dagster_defs/__init__.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink_tests/unit_tests/__init__.py rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/dagster_defs/__init__.py diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/dagster_defs/automapped_defs.py b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/dagster_defs/automapped_defs.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/dagster_defs/automapped_defs.py rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/dagster_defs/automapped_defs.py diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/dagster_defs/mapped_defs.py b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/dagster_defs/mapped_defs.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/dagster_defs/mapped_defs.py rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/dagster_defs/mapped_defs.py diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/dagster_defs/observation_defs.py b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/dagster_defs/observation_defs.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/dagster_defs/observation_defs.py rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/dagster_defs/observation_defs.py diff --git a/examples/experimental/dagster-airlift/examples/mwaa-example/mwaa_example/__init__.py b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/dagster_multi_code_locations/__init__.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/mwaa-example/mwaa_example/__init__.py rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/dagster_multi_code_locations/__init__.py diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/dagster_multi_code_locations/first_dag_defs.py b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/dagster_multi_code_locations/first_dag_defs.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/dagster_multi_code_locations/first_dag_defs.py rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/dagster_multi_code_locations/first_dag_defs.py diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/dagster_multi_code_locations/second_dag_defs.py b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/dagster_multi_code_locations/second_dag_defs.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/dagster_multi_code_locations/second_dag_defs.py rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/dagster_multi_code_locations/second_dag_defs.py diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/dagster_multi_code_locations/workspace.yaml b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/dagster_multi_code_locations/workspace.yaml similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink/dagster_multi_code_locations/workspace.yaml rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink/dagster_multi_code_locations/workspace.yaml diff --git a/examples/experimental/dagster-airlift/examples/perf-harness/perf_harness/__init__.py b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink_tests/__init__.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/perf-harness/perf_harness/__init__.py rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink_tests/__init__.py diff --git a/examples/experimental/dagster-airlift/examples/perf-harness/perf_harness/airflow_dags/__init__.py b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink_tests/integration_tests/__init__.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/perf-harness/perf_harness/airflow_dags/__init__.py rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink_tests/integration_tests/__init__.py diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink_tests/integration_tests/conftest.py b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink_tests/integration_tests/conftest.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink_tests/integration_tests/conftest.py rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink_tests/integration_tests/conftest.py diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink_tests/integration_tests/test_e2e_automapped.py b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink_tests/integration_tests/test_e2e_automapped.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink_tests/integration_tests/test_e2e_automapped.py rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink_tests/integration_tests/test_e2e_automapped.py diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink_tests/integration_tests/test_e2e_mapped.py b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink_tests/integration_tests/test_e2e_mapped.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink_tests/integration_tests/test_e2e_mapped.py rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink_tests/integration_tests/test_e2e_mapped.py diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink_tests/integration_tests/test_e2e_multi_code_location.py b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink_tests/integration_tests/test_e2e_multi_code_location.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink_tests/integration_tests/test_e2e_multi_code_location.py rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink_tests/integration_tests/test_e2e_multi_code_location.py diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink_tests/integration_tests/test_e2e_observation.py b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink_tests/integration_tests/test_e2e_observation.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/kitchen_sink_tests/integration_tests/test_e2e_observation.py rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink_tests/integration_tests/test_e2e_observation.py diff --git a/examples/experimental/dagster-airlift/examples/perf-harness/perf_harness/dagster_defs/__init__.py b/examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink_tests/unit_tests/__init__.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/perf-harness/perf_harness/dagster_defs/__init__.py rename to examples/experimental/dagster-airlift/kitchen-sink/kitchen_sink_tests/unit_tests/__init__.py diff --git a/examples/experimental/dagster-airlift/examples/kitchen-sink/pyproject.toml b/examples/experimental/dagster-airlift/kitchen-sink/pyproject.toml similarity index 100% rename from examples/experimental/dagster-airlift/examples/kitchen-sink/pyproject.toml rename to examples/experimental/dagster-airlift/kitchen-sink/pyproject.toml diff --git a/examples/experimental/dagster-airlift/kitchen-sink/setup.py b/examples/experimental/dagster-airlift/kitchen-sink/setup.py new file mode 100644 index 0000000000000..a71c8a12b0404 --- /dev/null +++ b/examples/experimental/dagster-airlift/kitchen-sink/setup.py @@ -0,0 +1,12 @@ +from setuptools import find_packages, setup + +setup( + name="kitchen-sink", + packages=find_packages(), + install_requires=[ + "dagster", + "dagster-webserver", + "dagster-airlift[core,in-airflow]", + ], + extras_require={"test": ["pytest"]}, +) diff --git a/examples/experimental/dagster-airlift/kitchen-sink/tox.ini b/examples/experimental/dagster-airlift/kitchen-sink/tox.ini new file mode 100644 index 0000000000000..e7cc5f956ba72 --- /dev/null +++ b/examples/experimental/dagster-airlift/kitchen-sink/tox.ini @@ -0,0 +1,28 @@ +[tox] +skipsdist = true + +[testenv] +download = True +passenv = + CI_* + COVERALLS_REPO_TOKEN + BUILDKITE* +install_command = uv pip install {opts} {packages} +deps = + -e ../../../../python_modules/dagster[test] + -e ../../../../python_modules/dagster-webserver + -e ../../../../python_modules/dagster-test + -e ../../../../python_modules/dagster-pipes + -e ../../../../python_modules/dagster-graphql + -e ../../../../python_modules/libraries/dagster-dbt + -e ../../../../python_modules/libraries/dagster-pandas + -e ../../dagster-airlift[core,test,in-airflow] + -e . +allowlist_externals = + /bin/bash + uv +commands = + # We need to rebuild the UI to ensure that the dagster-webserver can run + make -C ../../../.. rebuild_ui + !windows: /bin/bash -c '! pip list --exclude-editable | grep -e dagster' + pytest -c ./pyproject.toml ./kitchen_sink_tests --snapshot-warn-unused -vv {posargs} \ No newline at end of file diff --git a/examples/experimental/dagster-airlift/examples/perf-harness/.gitignore b/examples/experimental/dagster-airlift/perf-harness/.gitignore similarity index 100% rename from examples/experimental/dagster-airlift/examples/perf-harness/.gitignore rename to examples/experimental/dagster-airlift/perf-harness/.gitignore diff --git a/examples/experimental/dagster-airlift/examples/perf-harness/Makefile b/examples/experimental/dagster-airlift/perf-harness/Makefile similarity index 88% rename from examples/experimental/dagster-airlift/examples/perf-harness/Makefile rename to examples/experimental/dagster-airlift/perf-harness/Makefile index 1cd76b977704b..11646a5db5558 100644 --- a/examples/experimental/dagster-airlift/examples/perf-harness/Makefile +++ b/examples/experimental/dagster-airlift/perf-harness/Makefile @@ -5,16 +5,18 @@ $(shell dirname $(realpath $(lastword $(MAKEFILE_LIST))) | sed 's:/*$$::') endef MAKEFILE_DIR := $(GET_MAKEFILE_DIR) +AIRLIFT_MODULE_DIR := $(MAKEFILE_DIR)/../../dagster-airlift export DAGSTER_HOME := $(MAKEFILE_DIR)/.dagster_home export AIRFLOW_HOME := $(MAKEFILE_DIR)/.airflow_home export DAGSTER_URL := http://localhost:3333 + help: @egrep -h '\s##\s' $(MAKEFILE_LIST) | sort | awk 'BEGIN {FS = ":.*?## "}; {printf "\033[36m%-20s\033[0m %s\n", $$1, $$2}' dev_install: pip install uv - uv pip install -e ../../../dagster-airlift + uv pip install -e $(AIRLIFT_MODULE_DIR) uv pip install -e . run_airflow: @@ -50,8 +52,8 @@ setup_local_env: scaffold_observe $(MAKE) wipe mkdir -p $$AIRFLOW_HOME mkdir -p $$DAGSTER_HOME - chmod +x ../../scripts/airflow_setup.sh - ../../scripts/airflow_setup.sh $(MAKEFILE_DIR)/perf_harness/airflow_dags $(AIRFLOW_HOME) + chmod +x $(AIRLIFT_MODULE_DIR)/scripts/airflow_setup.sh + $(AIRLIFT_MODULE_DIR)/scripts/airflow_setup.sh $(MAKEFILE_DIR)/perf_harness/airflow_dags $(AIRFLOW_HOME) run_perf_scenarios_test: perf-harness 1 1 1 diff --git a/examples/experimental/dagster-airlift/examples/perf-harness/README.md b/examples/experimental/dagster-airlift/perf-harness/README.md similarity index 69% rename from examples/experimental/dagster-airlift/examples/perf-harness/README.md rename to examples/experimental/dagster-airlift/perf-harness/README.md index 490da599f56ac..5f49021032904 100644 --- a/examples/experimental/dagster-airlift/examples/perf-harness/README.md +++ b/examples/experimental/dagster-airlift/perf-harness/README.md @@ -7,7 +7,7 @@ This is used to time various airlift components. From the Dagster directory ```bash -cd experimental/dagster-airlift/examples/perf-harness +cd experimental/dagster-airlift/perf-harness pip install uv uv pip install -e . ``` @@ -18,7 +18,7 @@ Sanity check run perf-harness 1 1 ``` -Then check `experimental/dagster-airlift/examples/perf-harness/perf_harness/shared`. You should see a file +Then check `experimental/dagster-airlift/perf-harness/perf_harness/shared`. You should see a file `1_dags_1_tasks_perf_output.txt` with airlift timings. The first argument to `perf-harness` is the number of dags, the second is the number of tasks. diff --git a/examples/experimental/dagster-airlift/examples/perf-harness/conftest.py b/examples/experimental/dagster-airlift/perf-harness/conftest.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/perf-harness/conftest.py rename to examples/experimental/dagster-airlift/perf-harness/conftest.py diff --git a/examples/experimental/dagster-airlift/examples/perf-harness/perf_harness_tests/__init__.py b/examples/experimental/dagster-airlift/perf-harness/perf_harness/__init__.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/perf-harness/perf_harness_tests/__init__.py rename to examples/experimental/dagster-airlift/perf-harness/perf_harness/__init__.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/__init__.py b/examples/experimental/dagster-airlift/perf-harness/perf_harness/airflow_dags/__init__.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/__init__.py rename to examples/experimental/dagster-airlift/perf-harness/perf_harness/airflow_dags/__init__.py diff --git a/examples/experimental/dagster-airlift/examples/perf-harness/perf_harness/airflow_dags/dags.py b/examples/experimental/dagster-airlift/perf-harness/perf_harness/airflow_dags/dags.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/perf-harness/perf_harness/airflow_dags/dags.py rename to examples/experimental/dagster-airlift/perf-harness/perf_harness/airflow_dags/dags.py diff --git a/examples/experimental/dagster-airlift/examples/perf-harness/perf_harness/airflow_dags/migration_state/dag_0.yaml b/examples/experimental/dagster-airlift/perf-harness/perf_harness/airflow_dags/migration_state/dag_0.yaml similarity index 100% rename from examples/experimental/dagster-airlift/examples/perf-harness/perf_harness/airflow_dags/migration_state/dag_0.yaml rename to examples/experimental/dagster-airlift/perf-harness/perf_harness/airflow_dags/migration_state/dag_0.yaml diff --git a/examples/experimental/dagster-airlift/examples/perf-harness/perf_harness/cli.py b/examples/experimental/dagster-airlift/perf-harness/perf_harness/cli.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/perf-harness/perf_harness/cli.py rename to examples/experimental/dagster-airlift/perf-harness/perf_harness/cli.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/__init__.py b/examples/experimental/dagster-airlift/perf-harness/perf_harness/dagster_defs/__init__.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/__init__.py rename to examples/experimental/dagster-airlift/perf-harness/perf_harness/dagster_defs/__init__.py diff --git a/examples/experimental/dagster-airlift/examples/perf-harness/perf_harness/dagster_defs/constants.py b/examples/experimental/dagster-airlift/perf-harness/perf_harness/dagster_defs/constants.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/perf-harness/perf_harness/dagster_defs/constants.py rename to examples/experimental/dagster-airlift/perf-harness/perf_harness/dagster_defs/constants.py diff --git a/examples/experimental/dagster-airlift/examples/perf-harness/perf_harness/dagster_defs/migrate.py b/examples/experimental/dagster-airlift/perf-harness/perf_harness/dagster_defs/migrate.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/perf-harness/perf_harness/dagster_defs/migrate.py rename to examples/experimental/dagster-airlift/perf-harness/perf_harness/dagster_defs/migrate.py diff --git a/examples/experimental/dagster-airlift/examples/perf-harness/perf_harness/dagster_defs/observe.py b/examples/experimental/dagster-airlift/perf-harness/perf_harness/dagster_defs/observe.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/perf-harness/perf_harness/dagster_defs/observe.py rename to examples/experimental/dagster-airlift/perf-harness/perf_harness/dagster_defs/observe.py diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/peer.py b/examples/experimental/dagster-airlift/perf-harness/perf_harness/dagster_defs/peer.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/peer.py rename to examples/experimental/dagster-airlift/perf-harness/perf_harness/dagster_defs/peer.py diff --git a/examples/experimental/dagster-airlift/examples/perf-harness/perf_harness/shared/constants.py b/examples/experimental/dagster-airlift/perf-harness/perf_harness/shared/constants.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/perf-harness/perf_harness/shared/constants.py rename to examples/experimental/dagster-airlift/perf-harness/perf_harness/shared/constants.py diff --git a/examples/experimental/dagster-airlift/examples/perf-harness/perf_harness/shared/constants.txt b/examples/experimental/dagster-airlift/perf-harness/perf_harness/shared/constants.txt similarity index 100% rename from examples/experimental/dagster-airlift/examples/perf-harness/perf_harness/shared/constants.txt rename to examples/experimental/dagster-airlift/perf-harness/perf_harness/shared/constants.txt diff --git a/examples/experimental/dagster-airlift/examples/perf-harness/perf_harness/shared/utils.py b/examples/experimental/dagster-airlift/perf-harness/perf_harness/shared/utils.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/perf-harness/perf_harness/shared/utils.py rename to examples/experimental/dagster-airlift/perf-harness/perf_harness/shared/utils.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/__init__.py b/examples/experimental/dagster-airlift/perf-harness/perf_harness_tests/__init__.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/dagster_defs/stages/__init__.py rename to examples/experimental/dagster-airlift/perf-harness/perf_harness_tests/__init__.py diff --git a/examples/experimental/dagster-airlift/examples/perf-harness/perf_harness_tests/conftest.py b/examples/experimental/dagster-airlift/perf-harness/perf_harness_tests/conftest.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/perf-harness/perf_harness_tests/conftest.py rename to examples/experimental/dagster-airlift/perf-harness/perf_harness_tests/conftest.py diff --git a/examples/experimental/dagster-airlift/examples/perf-harness/perf_harness_tests/test_cli.py b/examples/experimental/dagster-airlift/perf-harness/perf_harness_tests/test_cli.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/perf-harness/perf_harness_tests/test_cli.py rename to examples/experimental/dagster-airlift/perf-harness/perf_harness_tests/test_cli.py diff --git a/examples/experimental/dagster-airlift/examples/perf-harness/perf_harness_tests/test_e2e.py b/examples/experimental/dagster-airlift/perf-harness/perf_harness_tests/test_e2e.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/perf-harness/perf_harness_tests/test_e2e.py rename to examples/experimental/dagster-airlift/perf-harness/perf_harness_tests/test_e2e.py diff --git a/examples/experimental/dagster-airlift/examples/perf-harness/pyproject.toml b/examples/experimental/dagster-airlift/perf-harness/pyproject.toml similarity index 100% rename from examples/experimental/dagster-airlift/examples/perf-harness/pyproject.toml rename to examples/experimental/dagster-airlift/perf-harness/pyproject.toml diff --git a/examples/experimental/dagster-airlift/examples/perf-harness/scripts/generate_yaml_files.py b/examples/experimental/dagster-airlift/perf-harness/scripts/generate_yaml_files.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/perf-harness/scripts/generate_yaml_files.py rename to examples/experimental/dagster-airlift/perf-harness/scripts/generate_yaml_files.py diff --git a/examples/experimental/dagster-airlift/examples/perf-harness/setup.py b/examples/experimental/dagster-airlift/perf-harness/setup.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/perf-harness/setup.py rename to examples/experimental/dagster-airlift/perf-harness/setup.py diff --git a/examples/experimental/dagster-airlift/perf-harness/tox.ini b/examples/experimental/dagster-airlift/perf-harness/tox.ini new file mode 100644 index 0000000000000..0136390b9638e --- /dev/null +++ b/examples/experimental/dagster-airlift/perf-harness/tox.ini @@ -0,0 +1,29 @@ +[tox] +skipsdist = true + +[testenv] +download = True +passenv = + CI_* + COVERALLS_REPO_TOKEN + BUILDKITE* +install_command = uv pip install {opts} {packages} +deps = + -e ../../../../python_modules/dagster[test] + -e ../../../../python_modules/dagster-webserver + -e ../../../../python_modules/dagster-test + -e ../../../../python_modules/dagster-pipes + -e ../../../../python_modules/dagster-graphql + -e ../../../../python_modules/libraries/dagster-dbt + -e ../../dagster-airlift[core,dbt,test,in-airflow] + -e . + pandas +allowlist_externals = + /bin/bash + uv + make +commands = + # We need to rebuild the UI to ensure that the dagster-webserver can run + make -C ../../../.. rebuild_ui + !windows: /bin/bash -c '! pip list --exclude-editable | grep -e dagster' + pytest -c ./pyproject.toml ./perf_harness_tests --snapshot-warn-unused -vv {posargs} diff --git a/examples/experimental/dagster-airlift/scripts/mirror_tutorial.sh b/examples/experimental/dagster-airlift/scripts/mirror_tutorial.sh index 2bcff30695cca..fa1f6913dc813 100755 --- a/examples/experimental/dagster-airlift/scripts/mirror_tutorial.sh +++ b/examples/experimental/dagster-airlift/scripts/mirror_tutorial.sh @@ -16,7 +16,7 @@ cd "$temp_dir" echo "Initializing git repository..." git init echo "Adding remote..." -git remote add origin git@github.com:dagster-io/airlift-tutorial.git +git remote add origin git@github.com:dagster-io/airlift-migration-tutorial.git echo "Fetching and resetting..." git fetch origin git reset --soft origin/main diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/.gitignore b/examples/starlift-demo/.gitignore similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/.gitignore rename to examples/starlift-demo/.gitignore diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/.vscode/settings.json b/examples/starlift-demo/.vscode/settings.json similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/.vscode/settings.json rename to examples/starlift-demo/.vscode/settings.json diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/Makefile b/examples/starlift-demo/Makefile similarity index 67% rename from examples/experimental/dagster-airlift/examples/dbt-example/Makefile rename to examples/starlift-demo/Makefile index 52a84f2b85747..0531040ecf1a7 100644 --- a/examples/experimental/dagster-airlift/examples/dbt-example/Makefile +++ b/examples/starlift-demo/Makefile @@ -12,6 +12,8 @@ export FEDERATED_AIRFLOW_HOME_2 := $(MAKEFILE_DIR)/.federated_airflow_home_2 export DBT_PROJECT_DIR := $(MAKEFILE_DIR)/dbt_example/shared/dbt export DBT_PROFILES_DIR := $(MAKEFILE_DIR)/dbt_example/shared/dbt export DAGSTER_URL := http://localhost:3333 +export AIRLIFT_MODULE_DIR := $(MAKEFILE_DIR)/../experimental/dagster-airlift +export DLIFT_MODULE_DIR := $(MAKEFILE_DIR)/../experimental/dagster-dlift help: @egrep -h '\s##\s' $(MAKEFILE_LIST) | sort | awk 'BEGIN {FS = ":.*?## "}; {printf "\033[36m%-20s\033[0m %s\n", $$1, $$2}' @@ -21,8 +23,8 @@ dbt_setup: ## Initialize dbt project dev_install: pip install uv && \ - uv pip install -e ../../../dagster-dlift && \ - uv pip install -e ../../../dagster-airlift[test] + uv pip install -e $(DLIFT_MODULE_DIR) && \ + uv pip install -e $(AIRLIFT_MODULE_DIR)[test] uv pip install -e . setup_local_env: @@ -31,19 +33,19 @@ setup_local_env: mkdir -p $(FEDERATED_AIRFLOW_HOME_1) mkdir -p $(FEDERATED_AIRFLOW_HOME_2) mkdir -p $(DAGSTER_HOME) - chmod +x ../../scripts/airflow_setup.sh - ../../scripts/airflow_setup.sh $(MAKEFILE_DIR)/dbt_example/migrating_airflow_dags $(MIGRATING_AIRFLOW_HOME) - ../../scripts/airflow_setup.sh $(MAKEFILE_DIR)/dbt_example/federated_airflow_dags_1 $(FEDERATED_AIRFLOW_HOME_1) 8081 - ../../scripts/airflow_setup.sh $(MAKEFILE_DIR)/dbt_example/federated_airflow_dags_2 $(FEDERATED_AIRFLOW_HOME_2) 8082 + chmod +x $(AIRLIFT_MODULE_DIR)/scripts/airflow_setup.sh + $(AIRLIFT_MODULE_DIR)/scripts/airflow_setup.sh $(MAKEFILE_DIR)/dbt_example/migrating_airflow_dags $(MIGRATING_AIRFLOW_HOME) + $(AIRLIFT_MODULE_DIR)/scripts/airflow_setup.sh $(MAKEFILE_DIR)/dbt_example/federated_airflow_dags_1 $(FEDERATED_AIRFLOW_HOME_1) 8081 + $(AIRLIFT_MODULE_DIR)/scripts/airflow_setup.sh $(MAKEFILE_DIR)/dbt_example/federated_airflow_dags_2 $(FEDERATED_AIRFLOW_HOME_2) 8082 $(MAKE) dbt_setup not_proxied: - chmod +x ../../scripts/find_and_replace_in_yaml_dir.sh - ../../scripts/find_and_replace_in_yaml_dir.sh $(MAKEFILE_DIR)/dbt_example/migrating_airflow_dags/proxied_state True False + chmod +x $(AIRLIFT_MODULE_DIR)/scripts/find_and_replace_in_yaml_dir.sh + $(AIRLIFT_MODULE_DIR)/scripts/find_and_replace_in_yaml_dir.sh $(MAKEFILE_DIR)/dbt_example/migrating_airflow_dags/proxied_state True False proxied: - chmod +x ../../scripts/find_and_replace_in_yaml_dir.sh - ../../scripts/find_and_replace_in_yaml_dir.sh $(MAKEFILE_DIR)/dbt_example/migrating_airflow_dags/proxied_state False True + chmod +x $(AIRLIFT_MODULE_DIR)/scripts/find_and_replace_in_yaml_dir.sh + $(AIRLIFT_MODULE_DIR)/scripts/find_and_replace_in_yaml_dir.sh $(MAKEFILE_DIR)/dbt_example/migrating_airflow_dags/proxied_state False True run_migrating_airflow: AIRFLOW_HOME=$(MIGRATING_AIRFLOW_HOME) airflow standalone diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/README.md b/examples/starlift-demo/README.md similarity index 90% rename from examples/experimental/dagster-airlift/examples/dbt-example/README.md rename to examples/starlift-demo/README.md index eb8800d4a3ab0..9bf3545b9522e 100644 --- a/examples/experimental/dagster-airlift/examples/dbt-example/README.md +++ b/examples/starlift-demo/README.md @@ -5,7 +5,7 @@ airflow dags into dagster as assets. ### Try it out -From the root of the `dbt-example` directory, run the `dev_install` make command to install python dependencies. +From the root of the `starlift-demo` directory, run the `dev_install` make command to install python dependencies. ```bash make dev_install diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/conftest.py b/examples/starlift-demo/conftest.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/conftest.py rename to examples/starlift-demo/conftest.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/__init__.py b/examples/starlift-demo/dbt_example/__init__.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/__init__.py rename to examples/starlift-demo/dbt_example/__init__.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/__init__.py b/examples/starlift-demo/dbt_example/dagster_defs/__init__.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/__init__.py rename to examples/starlift-demo/dbt_example/dagster_defs/__init__.py diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/complete.py b/examples/starlift-demo/dbt_example/dagster_defs/complete.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/complete.py rename to examples/starlift-demo/dbt_example/dagster_defs/complete.py diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/constants.py b/examples/starlift-demo/dbt_example/dagster_defs/constants.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/constants.py rename to examples/starlift-demo/dbt_example/dagster_defs/constants.py diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/dbt_cloud_airflow.py b/examples/starlift-demo/dbt_example/dagster_defs/dbt_cloud_airflow.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/dbt_cloud_airflow.py rename to examples/starlift-demo/dbt_example/dagster_defs/dbt_cloud_airflow.py diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/dbt_cloud_assets.py b/examples/starlift-demo/dbt_example/dagster_defs/dbt_cloud_assets.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/dbt_cloud_assets.py rename to examples/starlift-demo/dbt_example/dagster_defs/dbt_cloud_assets.py diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/dbt_cloud_utils.py b/examples/starlift-demo/dbt_example/dagster_defs/dbt_cloud_utils.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/dbt_cloud_utils.py rename to examples/starlift-demo/dbt_example/dagster_defs/dbt_cloud_utils.py diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/federated_airflow_1.py b/examples/starlift-demo/dbt_example/dagster_defs/federated_airflow_1.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/federated_airflow_1.py rename to examples/starlift-demo/dbt_example/dagster_defs/federated_airflow_1.py diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/federated_airflow_2.py b/examples/starlift-demo/dbt_example/dagster_defs/federated_airflow_2.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/federated_airflow_2.py rename to examples/starlift-demo/dbt_example/dagster_defs/federated_airflow_2.py diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/federated_airflow_defs_1.py b/examples/starlift-demo/dbt_example/dagster_defs/federated_airflow_defs_1.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/federated_airflow_defs_1.py rename to examples/starlift-demo/dbt_example/dagster_defs/federated_airflow_defs_1.py diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/federated_airflow_defs_2.py b/examples/starlift-demo/dbt_example/dagster_defs/federated_airflow_defs_2.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/federated_airflow_defs_2.py rename to examples/starlift-demo/dbt_example/dagster_defs/federated_airflow_defs_2.py diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/jaffle_shop.py b/examples/starlift-demo/dbt_example/dagster_defs/jaffle_shop.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/jaffle_shop.py rename to examples/starlift-demo/dbt_example/dagster_defs/jaffle_shop.py diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/lakehouse.py b/examples/starlift-demo/dbt_example/dagster_defs/lakehouse.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/lakehouse.py rename to examples/starlift-demo/dbt_example/dagster_defs/lakehouse.py diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/migrate.py b/examples/starlift-demo/dbt_example/dagster_defs/migrate.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/migrate.py rename to examples/starlift-demo/dbt_example/dagster_defs/migrate.py diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/observe.py b/examples/starlift-demo/dbt_example/dagster_defs/observe.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/observe.py rename to examples/starlift-demo/dbt_example/dagster_defs/observe.py diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/observe_with_check.py b/examples/starlift-demo/dbt_example/dagster_defs/observe_with_check.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/observe_with_check.py rename to examples/starlift-demo/dbt_example/dagster_defs/observe_with_check.py diff --git a/examples/experimental/dagster-airlift/examples/perf-harness/perf_harness/dagster_defs/peer.py b/examples/starlift-demo/dbt_example/dagster_defs/peer.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/perf-harness/perf_harness/dagster_defs/peer.py rename to examples/starlift-demo/dbt_example/dagster_defs/peer.py diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/table_existence_check.py b/examples/starlift-demo/dbt_example/dagster_defs/table_existence_check.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/table_existence_check.py rename to examples/starlift-demo/dbt_example/dagster_defs/table_existence_check.py diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/utils.py b/examples/starlift-demo/dbt_example/dagster_defs/utils.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/dagster_defs/utils.py rename to examples/starlift-demo/dbt_example/dagster_defs/utils.py diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/federated_airflow_dags_1/dags.py b/examples/starlift-demo/dbt_example/federated_airflow_dags_1/dags.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/federated_airflow_dags_1/dags.py rename to examples/starlift-demo/dbt_example/federated_airflow_dags_1/dags.py diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/federated_airflow_dags_2/dags.py b/examples/starlift-demo/dbt_example/federated_airflow_dags_2/dags.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/federated_airflow_dags_2/dags.py rename to examples/starlift-demo/dbt_example/federated_airflow_dags_2/dags.py diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/migrating_airflow_dags/dags.py b/examples/starlift-demo/dbt_example/migrating_airflow_dags/dags.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/migrating_airflow_dags/dags.py rename to examples/starlift-demo/dbt_example/migrating_airflow_dags/dags.py diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/migrating_airflow_dags/proxied_state/rebuild_iris_models.yaml b/examples/starlift-demo/dbt_example/migrating_airflow_dags/proxied_state/rebuild_iris_models.yaml similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/migrating_airflow_dags/proxied_state/rebuild_iris_models.yaml rename to examples/starlift-demo/dbt_example/migrating_airflow_dags/proxied_state/rebuild_iris_models.yaml diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/migrating_airflow_dags/proxied_state/rebuild_iris_models.yaml-e b/examples/starlift-demo/dbt_example/migrating_airflow_dags/proxied_state/rebuild_iris_models.yaml-e similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/migrating_airflow_dags/proxied_state/rebuild_iris_models.yaml-e rename to examples/starlift-demo/dbt_example/migrating_airflow_dags/proxied_state/rebuild_iris_models.yaml-e diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/custom_operator_examples/__init__.py b/examples/starlift-demo/dbt_example/shared/__init__.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/snippets/custom_operator_examples/__init__.py rename to examples/starlift-demo/dbt_example/shared/__init__.py diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/dbt_project.yml b/examples/starlift-demo/dbt_example/shared/dbt/dbt_project.yml similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/dbt_project.yml rename to examples/starlift-demo/dbt_example/shared/dbt/dbt_project.yml diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/dbt/models/customers.sql b/examples/starlift-demo/dbt_example/shared/dbt/models/customers.sql similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/dbt/models/customers.sql rename to examples/starlift-demo/dbt_example/shared/dbt/models/customers.sql diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/dbt/models/docs.md b/examples/starlift-demo/dbt_example/shared/dbt/models/docs.md similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/dbt/models/docs.md rename to examples/starlift-demo/dbt_example/shared/dbt/models/docs.md diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/models/iris_setosa.sql b/examples/starlift-demo/dbt_example/shared/dbt/models/iris_setosa.sql similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/models/iris_setosa.sql rename to examples/starlift-demo/dbt_example/shared/dbt/models/iris_setosa.sql diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/dbt/models/orders.sql b/examples/starlift-demo/dbt_example/shared/dbt/models/orders.sql similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/dbt/models/orders.sql rename to examples/starlift-demo/dbt_example/shared/dbt/models/orders.sql diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/dbt/models/overview.md b/examples/starlift-demo/dbt_example/shared/dbt/models/overview.md similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/dbt/models/overview.md rename to examples/starlift-demo/dbt_example/shared/dbt/models/overview.md diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/models/schema.yml b/examples/starlift-demo/dbt_example/shared/dbt/models/schema.yml similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/models/schema.yml rename to examples/starlift-demo/dbt_example/shared/dbt/models/schema.yml diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/models/sources.yml b/examples/starlift-demo/dbt_example/shared/dbt/models/sources.yml similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/models/sources.yml rename to examples/starlift-demo/dbt_example/shared/dbt/models/sources.yml diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/dbt/models/staging/schema.yml b/examples/starlift-demo/dbt_example/shared/dbt/models/staging/schema.yml similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/dbt/models/staging/schema.yml rename to examples/starlift-demo/dbt_example/shared/dbt/models/staging/schema.yml diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/models/staging/stg_customers.sql b/examples/starlift-demo/dbt_example/shared/dbt/models/staging/stg_customers.sql similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/models/staging/stg_customers.sql rename to examples/starlift-demo/dbt_example/shared/dbt/models/staging/stg_customers.sql diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/dbt/models/staging/stg_orders.sql b/examples/starlift-demo/dbt_example/shared/dbt/models/staging/stg_orders.sql similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/dbt/models/staging/stg_orders.sql rename to examples/starlift-demo/dbt_example/shared/dbt/models/staging/stg_orders.sql diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/models/staging/stg_payments.sql b/examples/starlift-demo/dbt_example/shared/dbt/models/staging/stg_payments.sql similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/models/staging/stg_payments.sql rename to examples/starlift-demo/dbt_example/shared/dbt/models/staging/stg_payments.sql diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/profiles.yml b/examples/starlift-demo/dbt_example/shared/dbt/profiles.yml similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/profiles.yml rename to examples/starlift-demo/dbt_example/shared/dbt/profiles.yml diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/dbt/requirements.in b/examples/starlift-demo/dbt_example/shared/dbt/requirements.in similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/dbt/requirements.in rename to examples/starlift-demo/dbt_example/shared/dbt/requirements.in diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/dbt/requirements.txt b/examples/starlift-demo/dbt_example/shared/dbt/requirements.txt similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/dbt/requirements.txt rename to examples/starlift-demo/dbt_example/shared/dbt/requirements.txt diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/dbt/seeds/.gitkeep b/examples/starlift-demo/dbt_example/shared/dbt/seeds/.gitkeep similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/dbt/seeds/.gitkeep rename to examples/starlift-demo/dbt_example/shared/dbt/seeds/.gitkeep diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/seeds/raw_customers.csv b/examples/starlift-demo/dbt_example/shared/dbt/seeds/raw_customers.csv similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/dbt/seeds/raw_customers.csv rename to examples/starlift-demo/dbt_example/shared/dbt/seeds/raw_customers.csv diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/dbt/seeds/raw_orders.csv b/examples/starlift-demo/dbt_example/shared/dbt/seeds/raw_orders.csv similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/dbt/seeds/raw_orders.csv rename to examples/starlift-demo/dbt_example/shared/dbt/seeds/raw_orders.csv diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/dbt/seeds/raw_payments.csv b/examples/starlift-demo/dbt_example/shared/dbt/seeds/raw_payments.csv similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example/shared/dbt/seeds/raw_payments.csv rename to examples/starlift-demo/dbt_example/shared/dbt/seeds/raw_payments.csv diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/iris.csv b/examples/starlift-demo/dbt_example/shared/iris.csv similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/iris.csv rename to examples/starlift-demo/dbt_example/shared/iris.csv diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/lakehouse_utils.py b/examples/starlift-demo/dbt_example/shared/lakehouse_utils.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/lakehouse_utils.py rename to examples/starlift-demo/dbt_example/shared/lakehouse_utils.py diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/load_iris.py b/examples/starlift-demo/dbt_example/shared/load_iris.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example/shared/load_iris.py rename to examples/starlift-demo/dbt_example/shared/load_iris.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example_tests/__init__.py b/examples/starlift-demo/dbt_example_tests/__init__.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example_tests/__init__.py rename to examples/starlift-demo/dbt_example_tests/__init__.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example_tests/integration_tests/__init__.py b/examples/starlift-demo/dbt_example_tests/integration_tests/__init__.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example_tests/integration_tests/__init__.py rename to examples/starlift-demo/dbt_example_tests/integration_tests/__init__.py diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example_tests/integration_tests/conftest.py b/examples/starlift-demo/dbt_example_tests/integration_tests/conftest.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example_tests/integration_tests/conftest.py rename to examples/starlift-demo/dbt_example_tests/integration_tests/conftest.py diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example_tests/integration_tests/test_completed_migration_stage.py b/examples/starlift-demo/dbt_example_tests/integration_tests/test_completed_migration_stage.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example_tests/integration_tests/test_completed_migration_stage.py rename to examples/starlift-demo/dbt_example_tests/integration_tests/test_completed_migration_stage.py diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example_tests/integration_tests/test_federated.py b/examples/starlift-demo/dbt_example_tests/integration_tests/test_federated.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example_tests/integration_tests/test_federated.py rename to examples/starlift-demo/dbt_example_tests/integration_tests/test_federated.py diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example_tests/integration_tests/test_migration_stages.py b/examples/starlift-demo/dbt_example_tests/integration_tests/test_migration_stages.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example_tests/integration_tests/test_migration_stages.py rename to examples/starlift-demo/dbt_example_tests/integration_tests/test_migration_stages.py diff --git a/examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example_tests/unit_tests/__init__.py b/examples/starlift-demo/dbt_example_tests/unit_tests/__init__.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/tutorial-example/tutorial_example_tests/unit_tests/__init__.py rename to examples/starlift-demo/dbt_example_tests/unit_tests/__init__.py diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/dbt_example_tests/unit_tests/test_all_variants_loadable.py b/examples/starlift-demo/dbt_example_tests/unit_tests/test_all_variants_loadable.py similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/dbt_example_tests/unit_tests/test_all_variants_loadable.py rename to examples/starlift-demo/dbt_example_tests/unit_tests/test_all_variants_loadable.py diff --git a/examples/experimental/dagster-airlift/examples/dbt-example/pyproject.toml b/examples/starlift-demo/pyproject.toml similarity index 100% rename from examples/experimental/dagster-airlift/examples/dbt-example/pyproject.toml rename to examples/starlift-demo/pyproject.toml diff --git a/examples/starlift-demo/setup.py b/examples/starlift-demo/setup.py new file mode 100644 index 0000000000000..142c4b80848a9 --- /dev/null +++ b/examples/starlift-demo/setup.py @@ -0,0 +1,15 @@ +from setuptools import find_packages, setup + +setup( + name="dbt-example", + packages=find_packages(), + install_requires=[ + "dagster", + "dagster-webserver", + "dagster-airlift[dbt,core,in-airflow]", + "dagster-dlift", + "dbt-duckdb", + "pandas", + ], + extras_require={"test": ["pytest"]}, +) diff --git a/examples/starlift-demo/tox.ini b/examples/starlift-demo/tox.ini new file mode 100644 index 0000000000000..c6e0589e343bb --- /dev/null +++ b/examples/starlift-demo/tox.ini @@ -0,0 +1,30 @@ +[tox] +skipsdist = true + +[testenv] +download = True +passenv = + CI_* + COVERALLS_REPO_TOKEN + BUILDKITE* + KS_DBT_CLOUD* +install_command = uv pip install {opts} {packages} +deps = + -e ../../python_modules/dagster[test] + -e ../../python_modules/dagster-webserver + -e ../../python_modules/dagster-test + -e ../../python_modules/dagster-pipes + -e ../../python_modules/dagster-graphql + -e ../../python_modules/libraries/dagster-dbt + -e ../experimental/dagster-airlift[core,dbt,test,in-airflow] + -e ../experimental/dagster-dlift + -e . + pandas +allowlist_externals = + /bin/bash + uv +commands = + # We need to rebuild the UI to ensure that the dagster-webserver can run + make -C ../.. rebuild_ui + !windows: /bin/bash -c '! pip list --exclude-editable | grep -e dagster' + pytest -c ./pyproject.toml ./dbt_example_tests --snapshot-warn-unused -vv -s {posargs} \ No newline at end of file diff --git a/pyright/master/requirements-pinned.txt b/pyright/master/requirements-pinned.txt index e3b90ca8f735a..56bfa0113863c 100644 --- a/pyright/master/requirements-pinned.txt +++ b/pyright/master/requirements-pinned.txt @@ -184,7 +184,7 @@ dbt-adapters==1.3.2 dbt-common==1.3.0 dbt-core==1.8.8 dbt-duckdb==1.9.0 --e examples/experimental/dagster-airlift/examples/dbt-example +-e examples/starlift-demo dbt-extractor==0.5.1 dbt-semantic-interfaces==0.5.1 debugpy==1.8.7 @@ -312,7 +312,7 @@ jupyterlab==4.2.5 jupyterlab-pygments==0.3.0 jupyterlab-server==2.27.3 jupyterlab-widgets==3.0.13 --e examples/experimental/dagster-airlift/examples/kitchen-sink +-e examples/experimental/dagster-airlift/kitchen-sink -e examples/experimental/dagster-dlift/kitchen-sink kiwisolver==1.4.7 kombu==5.4.2 @@ -420,7 +420,7 @@ pathable==0.4.3 pathspec==0.12.1 pathvalidate==3.2.1 pendulum==2.1.2 --e examples/experimental/dagster-airlift/examples/perf-harness +-e examples/experimental/dagster-airlift/perf-harness pexpect==4.9.0 pillow==11.0.0 pip==24.2 @@ -586,7 +586,7 @@ tqdm==4.66.5 traitlets==5.14.3 trio==0.27.0 trio-websocket==0.11.1 --e examples/experimental/dagster-airlift/examples/tutorial-example +-e examples/airlift-migration-tutorial -e examples/tutorial_notebook_assets twilio==9.3.6 twine==1.15.0 diff --git a/pyright/master/requirements.txt b/pyright/master/requirements.txt index 7b3c2e66f473a..1fdae6679b2c7 100644 --- a/pyright/master/requirements.txt +++ b/pyright/master/requirements.txt @@ -135,9 +135,9 @@ types-sqlalchemy==1.4.53.34 -e examples/experimental/dagster-blueprints -e examples/experimental/dagster-airlift[mwaa,dbt,test] # (includes airflow dependencies) -e examples/experimental/dagster-dlift --e examples/experimental/dagster-airlift/examples/dbt-example --e examples/experimental/dagster-airlift/examples/kitchen-sink +-e examples/starlift-demo +-e examples/experimental/dagster-airlift/kitchen-sink -e examples/experimental/dagster-dlift/kitchen-sink --e examples/experimental/dagster-airlift/examples/perf-harness --e examples/experimental/dagster-airlift/examples/tutorial-example +-e examples/experimental/dagster-airlift/perf-harness +-e examples/airlift-migration-tutorial -e examples/use_case_repository[dev]