-
Notifications
You must be signed in to change notification settings - Fork 1.5k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[dagster-airlift] operator to proxy dag
- Loading branch information
Showing
7 changed files
with
184 additions
and
14 deletions.
There are no files selected for viewing
53 changes: 53 additions & 0 deletions
53
examples/experimental/dagster-airlift/dagster_airlift/in_airflow/dag_proxy_operator.py
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,53 @@ | ||
import json | ||
import os | ||
from typing import Any, Iterable, Mapping, Sequence | ||
|
||
import requests | ||
from airflow import DAG | ||
from airflow.utils.context import Context | ||
|
||
from dagster_airlift.constants import DAG_MAPPING_METADATA_KEY | ||
from dagster_airlift.in_airflow.base_asset_operator import BaseDagsterAssetsOperator | ||
|
||
|
||
class BaseProxyDAGToDagsterOperator(BaseDagsterAssetsOperator): | ||
"""An operator that proxies task execution to Dagster assets with metadata that map to this task's dag ID and task ID.""" | ||
|
||
def filter_asset_nodes( | ||
self, context: Context, asset_nodes: Sequence[Mapping[str, Any]] | ||
) -> Iterable[Mapping[str, Any]]: | ||
for asset_node in asset_nodes: | ||
if matched_dag_id(asset_node, self.get_airflow_dag_id(context)): | ||
yield asset_node | ||
|
||
|
||
class DefaultProxyDAGToDagsterOperator(BaseProxyDAGToDagsterOperator): | ||
"""The default task proxying operator - which opens a blank session and expects the dagster URL to be set in the environment. | ||
The dagster url is expected to be set in the environment as DAGSTER_URL. | ||
""" | ||
|
||
def get_dagster_session(self, context: Context) -> requests.Session: | ||
return requests.Session() | ||
|
||
def get_dagster_url(self, context: Context) -> str: | ||
return os.environ["DAGSTER_URL"] | ||
|
||
|
||
def build_dag_level_proxied_task(dag: DAG) -> DefaultProxyDAGToDagsterOperator: | ||
return DefaultProxyDAGToDagsterOperator( | ||
task_id=f"DAGSTER_OVERRIDE_DAG_{dag.dag_id}", | ||
dag=dag, | ||
) | ||
|
||
|
||
def matched_dag_id(asset_node: Mapping[str, Any], dag_id: str) -> bool: | ||
json_metadata_entries = { | ||
entry["label"]: entry["jsonString"] | ||
for entry in asset_node["metadataEntries"] | ||
if entry["__typename"] == "JsonMetadataEntry" | ||
} | ||
|
||
if mapping_entry := json_metadata_entries.get(DAG_MAPPING_METADATA_KEY): | ||
mappings = json.loads(mapping_entry) | ||
return any(mapping["dag_id"] == dag_id for mapping in mappings) | ||
return False |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
36 changes: 36 additions & 0 deletions
36
...ntal/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/dag_level_proxied.py
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,36 @@ | ||
from datetime import datetime | ||
from pathlib import Path | ||
|
||
from airflow import DAG | ||
from airflow.operators.python import PythonOperator | ||
from dagster_airlift.in_airflow import proxying_to_dagster | ||
from dagster_airlift.in_airflow.proxied_state import load_proxied_state_from_yaml | ||
|
||
|
||
def print_hello() -> None: | ||
print("Hello") # noqa: T201 | ||
|
||
|
||
default_args = { | ||
"owner": "airflow", | ||
"depends_on_past": False, | ||
"start_date": datetime(2023, 1, 1), | ||
"retries": 1, | ||
} | ||
|
||
|
||
with DAG( | ||
"overridden_dag", | ||
default_args=default_args, | ||
schedule_interval=None, | ||
is_paused_upon_creation=False, | ||
) as dag: | ||
PythonOperator(task_id="print_task", python_callable=print_hello) << PythonOperator( | ||
task_id="downstream_print_task", python_callable=print_hello | ||
) # type: ignore | ||
|
||
|
||
proxying_to_dagster( | ||
proxied_state=load_proxied_state_from_yaml(Path(__file__).parent / "proxied_state"), | ||
global_vars=globals(), | ||
) |
1 change: 1 addition & 0 deletions
1
...airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/overridden_dag.yaml
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1 @@ | ||
proxied: True |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters