Edit on GitHub

sqlmesh.schedulers.airflow.integration

  1from __future__ import annotations
  2
  3import logging
  4import typing as t
  5from datetime import datetime, timedelta
  6
  7from airflow import DAG
  8from airflow.models import BaseOperator, TaskInstance, Variable
  9from airflow.operators.python import PythonOperator
 10from airflow.utils.session import provide_session
 11from sqlalchemy.orm import Session
 12
 13from sqlmesh.engines import commands
 14from sqlmesh.schedulers.airflow import common, util
 15from sqlmesh.schedulers.airflow.dag_generator import SnapshotDagGenerator
 16from sqlmesh.schedulers.airflow.operators import targets
 17
 18logger = logging.getLogger(__name__)
 19
 20
 21class SQLMeshAirflow:
 22    """The entry point for the SQLMesh integration with Airflow.
 23
 24    The instance of this class should be created in a module that is part of the
 25    Airflow DAGs folder. Its primary purpose is to create DAG objects for the operational
 26    needs of the platform, as well as for model evaluation and backfills.
 27
 28    Please note that the user must pass created DAGs into the
 29    Airflow scheduler. See the example below:
 30
 31    Example:
 32        Create a new python module in the Airflow DAGs folder called "sqlmesh_integration.py"
 33        with the following content:
 34
 35            from sqlmesh.schedulers.airflow.integration import SQLMeshAirflow
 36
 37            for dag in SQLMeshAirflow("spark").dags:
 38                globals()[dag.dag_id] = dag
 39
 40    Args:
 41        engine_operator: The type of the Airflow operator that will be used for model evaluation.
 42            If a string value is passed, an automatic operator discovery is attempted based
 43            on the engine name specified in the string.
 44        engine_operator_args: The dictionary of arguments that will be passed into the evaluate engine
 45            operator during its construction.
 46            This can be used to customize parameters such as connection ID.
 47        ddl_engine_operator: The type of the Airflow operator that will be used for environment management.
 48            These operations are SQL only.
 49            If a string value is passed, an automatic operator discovery is attempted based
 50            on the engine name specified in the string.
 51        ddl_engine_operator_args: Args to be passed into just the environment management operator.
 52            This can be used to customize parameters such as connection ID.
 53            If not specified, and the operator is the same as `engine_operator`, falls back to using `engine_operator_args`.
 54        janitor_interval: Defines how often the janitor DAG runs.
 55            The janitor DAG removes platform-managed DAG instances that are pending
 56            deletion from Airflow. Default: 1 hour.
 57        plan_application_dag_ttl: Determines the time-to-live period for finished plan application DAGs.
 58            Once this period is exceeded, finished plan application DAGs are deleted by the janitor. Default: 2 days.
 59    """
 60
 61    def __init__(
 62        self,
 63        engine_operator: t.Union[str, t.Type[BaseOperator]],
 64        engine_operator_args: t.Optional[t.Dict[str, t.Any]] = None,
 65        ddl_engine_operator: t.Optional[t.Union[str, t.Type[BaseOperator]]] = None,
 66        ddl_engine_operator_args: t.Optional[t.Dict[str, t.Any]] = None,
 67        janitor_interval: timedelta = timedelta(hours=1),
 68        plan_application_dag_ttl: timedelta = timedelta(days=2),
 69    ):
 70        if isinstance(engine_operator, str):
 71            if not ddl_engine_operator:
 72                ddl_engine_operator = util.discover_engine_operator(engine_operator, sql_only=True)
 73            engine_operator = util.discover_engine_operator(engine_operator, sql_only=False)
 74
 75        if isinstance(ddl_engine_operator, str):
 76            ddl_engine_operator = util.discover_engine_operator(ddl_engine_operator, sql_only=True)
 77
 78        engine_operator_args = engine_operator_args or {}
 79        ddl_engine_operator_args = ddl_engine_operator_args or {}
 80
 81        self._engine_operator = engine_operator
 82        self._engine_operator_args = engine_operator_args
 83        self._ddl_engine_operator = ddl_engine_operator or engine_operator
 84        if self._engine_operator == self._ddl_engine_operator:
 85            self._ddl_engine_operator_args = {**engine_operator_args, **ddl_engine_operator_args}
 86        else:
 87            self._ddl_engine_operator_args = ddl_engine_operator_args or {}
 88        self._janitor_interval = janitor_interval
 89        self._plan_application_dag_ttl = plan_application_dag_ttl
 90
 91    @property
 92    def dags(self) -> t.List[DAG]:
 93        """Returns all DAG instances that must be registered with the Airflow scheduler
 94        for the integration to work.
 95
 96        Returns:
 97            The list of DAG instances managed by the platform.
 98        """
 99        with util.scoped_state_sync() as state_sync:
100            stored_snapshots = state_sync.get_snapshots(None)
101
102        dag_generator = SnapshotDagGenerator(
103            self._engine_operator,
104            self._engine_operator_args,
105            self._ddl_engine_operator,
106            self._ddl_engine_operator_args,
107            stored_snapshots,
108        )
109
110        cadence_dags = dag_generator.generate_cadence_dags()
111
112        plan_application_dags = [
113            dag_generator.generate_plan_application_dag(s) for s in _get_plan_dag_specs()
114        ]
115
116        system_dags = [
117            self._create_janitor_dag(),
118        ]
119
120        return system_dags + cadence_dags + plan_application_dags
121
122    def _create_janitor_dag(self) -> DAG:
123        dag = self._create_system_dag(common.JANITOR_DAG_ID, self._janitor_interval)
124        janitor_task_op = PythonOperator(
125            task_id=common.JANITOR_TASK_ID,
126            python_callable=_janitor_task,
127            op_kwargs={"plan_application_dag_ttl": self._plan_application_dag_ttl},
128            dag=dag,
129        )
130
131        table_cleanup_task_op = self._ddl_engine_operator(
132            **self._ddl_engine_operator_args,
133            target=targets.SnapshotCleanupTarget(),
134            task_id="snapshot_table_cleanup_task",
135            dag=dag,
136        )
137
138        janitor_task_op >> table_cleanup_task_op
139
140        return dag
141
142    def _create_system_dag(self, dag_id: str, schedule_interval: t.Optional[timedelta]) -> DAG:
143        return DAG(
144            dag_id=dag_id,
145            default_args=dict(
146                execution_timeout=timedelta(minutes=10),
147                retries=0,
148            ),
149            schedule_interval=schedule_interval,
150            start_date=datetime(2023, 1, 1),
151            max_active_runs=1,
152            catchup=False,
153            is_paused_upon_creation=False,
154            tags=[common.SQLMESH_AIRFLOW_TAG],
155        )
156
157
158@provide_session
159def _janitor_task(
160    plan_application_dag_ttl: timedelta,
161    ti: TaskInstance,
162    session: Session = util.PROVIDED_SESSION,
163) -> None:
164    with util.scoped_state_sync() as state_sync:
165        expired_environments = state_sync.delete_expired_environments()
166        expired_snapshots = state_sync.delete_expired_snapshots()
167        ti.xcom_push(
168            key=common.SNAPSHOT_CLEANUP_COMMAND_XCOM_KEY,
169            value=commands.CleanupCommandPayload(
170                environments=expired_environments,
171                snapshots=[s.table_info for s in expired_snapshots],
172            ).json(),
173            session=session,
174        )
175
176        all_snapshot_dag_ids = set(util.get_snapshot_dag_ids())
177        active_snapshot_dag_ids = {
178            common.dag_id_for_snapshot_info(s) for s in state_sync.get_snapshots(None).values()
179        }
180        expired_snapshot_dag_ids = all_snapshot_dag_ids - active_snapshot_dag_ids
181        logger.info("Deleting expired Snapshot DAGs: %s", expired_snapshot_dag_ids)
182        util.delete_dags(expired_snapshot_dag_ids, session=session)
183
184        plan_application_dag_ids = util.get_finished_plan_application_dag_ids(
185            ttl=plan_application_dag_ttl, session=session
186        )
187        logger.info("Deleting expired Plan Application DAGs: %s", plan_application_dag_ids)
188        util.delete_variables(
189            {common.plan_dag_spec_key_from_dag_id(dag_id) for dag_id in plan_application_dag_ids},
190            session=session,
191        )
192        util.delete_dags(plan_application_dag_ids, session=session)
193
194
195@provide_session
196def _get_plan_dag_specs(
197    session: Session = util.PROVIDED_SESSION,
198) -> t.List[common.PlanDagSpec]:
199    records = (
200        session.query(Variable)
201        .filter(Variable.key.like(f"{common.PLAN_DAG_SPEC_KEY_PREFIX}%"))
202        .all()
203    )
204    return [common.PlanDagSpec.parse_raw(r.val) for r in records]
class SQLMeshAirflow:
 22class SQLMeshAirflow:
 23    """The entry point for the SQLMesh integration with Airflow.
 24
 25    The instance of this class should be created in a module that is part of the
 26    Airflow DAGs folder. Its primary purpose is to create DAG objects for the operational
 27    needs of the platform, as well as for model evaluation and backfills.
 28
 29    Please note that the user must pass created DAGs into the
 30    Airflow scheduler. See the example below:
 31
 32    Example:
 33        Create a new python module in the Airflow DAGs folder called "sqlmesh_integration.py"
 34        with the following content:
 35
 36            from sqlmesh.schedulers.airflow.integration import SQLMeshAirflow
 37
 38            for dag in SQLMeshAirflow("spark").dags:
 39                globals()[dag.dag_id] = dag
 40
 41    Args:
 42        engine_operator: The type of the Airflow operator that will be used for model evaluation.
 43            If a string value is passed, an automatic operator discovery is attempted based
 44            on the engine name specified in the string.
 45        engine_operator_args: The dictionary of arguments that will be passed into the evaluate engine
 46            operator during its construction.
 47            This can be used to customize parameters such as connection ID.
 48        ddl_engine_operator: The type of the Airflow operator that will be used for environment management.
 49            These operations are SQL only.
 50            If a string value is passed, an automatic operator discovery is attempted based
 51            on the engine name specified in the string.
 52        ddl_engine_operator_args: Args to be passed into just the environment management operator.
 53            This can be used to customize parameters such as connection ID.
 54            If not specified, and the operator is the same as `engine_operator`, falls back to using `engine_operator_args`.
 55        janitor_interval: Defines how often the janitor DAG runs.
 56            The janitor DAG removes platform-managed DAG instances that are pending
 57            deletion from Airflow. Default: 1 hour.
 58        plan_application_dag_ttl: Determines the time-to-live period for finished plan application DAGs.
 59            Once this period is exceeded, finished plan application DAGs are deleted by the janitor. Default: 2 days.
 60    """
 61
 62    def __init__(
 63        self,
 64        engine_operator: t.Union[str, t.Type[BaseOperator]],
 65        engine_operator_args: t.Optional[t.Dict[str, t.Any]] = None,
 66        ddl_engine_operator: t.Optional[t.Union[str, t.Type[BaseOperator]]] = None,
 67        ddl_engine_operator_args: t.Optional[t.Dict[str, t.Any]] = None,
 68        janitor_interval: timedelta = timedelta(hours=1),
 69        plan_application_dag_ttl: timedelta = timedelta(days=2),
 70    ):
 71        if isinstance(engine_operator, str):
 72            if not ddl_engine_operator:
 73                ddl_engine_operator = util.discover_engine_operator(engine_operator, sql_only=True)
 74            engine_operator = util.discover_engine_operator(engine_operator, sql_only=False)
 75
 76        if isinstance(ddl_engine_operator, str):
 77            ddl_engine_operator = util.discover_engine_operator(ddl_engine_operator, sql_only=True)
 78
 79        engine_operator_args = engine_operator_args or {}
 80        ddl_engine_operator_args = ddl_engine_operator_args or {}
 81
 82        self._engine_operator = engine_operator
 83        self._engine_operator_args = engine_operator_args
 84        self._ddl_engine_operator = ddl_engine_operator or engine_operator
 85        if self._engine_operator == self._ddl_engine_operator:
 86            self._ddl_engine_operator_args = {**engine_operator_args, **ddl_engine_operator_args}
 87        else:
 88            self._ddl_engine_operator_args = ddl_engine_operator_args or {}
 89        self._janitor_interval = janitor_interval
 90        self._plan_application_dag_ttl = plan_application_dag_ttl
 91
 92    @property
 93    def dags(self) -> t.List[DAG]:
 94        """Returns all DAG instances that must be registered with the Airflow scheduler
 95        for the integration to work.
 96
 97        Returns:
 98            The list of DAG instances managed by the platform.
 99        """
100        with util.scoped_state_sync() as state_sync:
101            stored_snapshots = state_sync.get_snapshots(None)
102
103        dag_generator = SnapshotDagGenerator(
104            self._engine_operator,
105            self._engine_operator_args,
106            self._ddl_engine_operator,
107            self._ddl_engine_operator_args,
108            stored_snapshots,
109        )
110
111        cadence_dags = dag_generator.generate_cadence_dags()
112
113        plan_application_dags = [
114            dag_generator.generate_plan_application_dag(s) for s in _get_plan_dag_specs()
115        ]
116
117        system_dags = [
118            self._create_janitor_dag(),
119        ]
120
121        return system_dags + cadence_dags + plan_application_dags
122
123    def _create_janitor_dag(self) -> DAG:
124        dag = self._create_system_dag(common.JANITOR_DAG_ID, self._janitor_interval)
125        janitor_task_op = PythonOperator(
126            task_id=common.JANITOR_TASK_ID,
127            python_callable=_janitor_task,
128            op_kwargs={"plan_application_dag_ttl": self._plan_application_dag_ttl},
129            dag=dag,
130        )
131
132        table_cleanup_task_op = self._ddl_engine_operator(
133            **self._ddl_engine_operator_args,
134            target=targets.SnapshotCleanupTarget(),
135            task_id="snapshot_table_cleanup_task",
136            dag=dag,
137        )
138
139        janitor_task_op >> table_cleanup_task_op
140
141        return dag
142
143    def _create_system_dag(self, dag_id: str, schedule_interval: t.Optional[timedelta]) -> DAG:
144        return DAG(
145            dag_id=dag_id,
146            default_args=dict(
147                execution_timeout=timedelta(minutes=10),
148                retries=0,
149            ),
150            schedule_interval=schedule_interval,
151            start_date=datetime(2023, 1, 1),
152            max_active_runs=1,
153            catchup=False,
154            is_paused_upon_creation=False,
155            tags=[common.SQLMESH_AIRFLOW_TAG],
156        )

The entry point for the SQLMesh integration with Airflow.

The instance of this class should be created in a module that is part of the Airflow DAGs folder. Its primary purpose is to create DAG objects for the operational needs of the platform, as well as for model evaluation and backfills.

Please note that the user must pass created DAGs into the Airflow scheduler. See the example below:

Example:

Create a new python module in the Airflow DAGs folder called "sqlmesh_integration.py" with the following content:

from sqlmesh.schedulers.airflow.integration import SQLMeshAirflow

for dag in SQLMeshAirflow("spark").dags:
    globals()[dag.dag_id] = dag
Arguments:
  • engine_operator: The type of the Airflow operator that will be used for model evaluation. If a string value is passed, an automatic operator discovery is attempted based on the engine name specified in the string.
  • engine_operator_args: The dictionary of arguments that will be passed into the evaluate engine operator during its construction. This can be used to customize parameters such as connection ID.
  • ddl_engine_operator: The type of the Airflow operator that will be used for environment management. These operations are SQL only. If a string value is passed, an automatic operator discovery is attempted based on the engine name specified in the string.
  • ddl_engine_operator_args: Args to be passed into just the environment management operator. This can be used to customize parameters such as connection ID. If not specified, and the operator is the same as engine_operator, falls back to using engine_operator_args.
  • janitor_interval: Defines how often the janitor DAG runs. The janitor DAG removes platform-managed DAG instances that are pending deletion from Airflow. Default: 1 hour.
  • plan_application_dag_ttl: Determines the time-to-live period for finished plan application DAGs. Once this period is exceeded, finished plan application DAGs are deleted by the janitor. Default: 2 days.
SQLMeshAirflow( engine_operator: Union[str, Type[airflow.models.baseoperator.BaseOperator]], engine_operator_args: Optional[Dict[str, Any]] = None, ddl_engine_operator: Union[str, Type[airflow.models.baseoperator.BaseOperator], NoneType] = None, ddl_engine_operator_args: Optional[Dict[str, Any]] = None, janitor_interval: datetime.timedelta = datetime.timedelta(seconds=3600), plan_application_dag_ttl: datetime.timedelta = datetime.timedelta(days=2))
62    def __init__(
63        self,
64        engine_operator: t.Union[str, t.Type[BaseOperator]],
65        engine_operator_args: t.Optional[t.Dict[str, t.Any]] = None,
66        ddl_engine_operator: t.Optional[t.Union[str, t.Type[BaseOperator]]] = None,
67        ddl_engine_operator_args: t.Optional[t.Dict[str, t.Any]] = None,
68        janitor_interval: timedelta = timedelta(hours=1),
69        plan_application_dag_ttl: timedelta = timedelta(days=2),
70    ):
71        if isinstance(engine_operator, str):
72            if not ddl_engine_operator:
73                ddl_engine_operator = util.discover_engine_operator(engine_operator, sql_only=True)
74            engine_operator = util.discover_engine_operator(engine_operator, sql_only=False)
75
76        if isinstance(ddl_engine_operator, str):
77            ddl_engine_operator = util.discover_engine_operator(ddl_engine_operator, sql_only=True)
78
79        engine_operator_args = engine_operator_args or {}
80        ddl_engine_operator_args = ddl_engine_operator_args or {}
81
82        self._engine_operator = engine_operator
83        self._engine_operator_args = engine_operator_args
84        self._ddl_engine_operator = ddl_engine_operator or engine_operator
85        if self._engine_operator == self._ddl_engine_operator:
86            self._ddl_engine_operator_args = {**engine_operator_args, **ddl_engine_operator_args}
87        else:
88            self._ddl_engine_operator_args = ddl_engine_operator_args or {}
89        self._janitor_interval = janitor_interval
90        self._plan_application_dag_ttl = plan_application_dag_ttl
dags: List[airflow.models.dag.DAG]

Returns all DAG instances that must be registered with the Airflow scheduler for the integration to work.

Returns:

The list of DAG instances managed by the platform.