Edit on GitHub

PlanEvaluator

A plan evaluator is responsible for evaluating a plan when it is being applied.

Evaluation steps

At a high level, when a plan is evaluated, SQLMesh will:

  • Push new snapshots to the state sync.
  • Create snapshot tables.
  • Backfill data.
  • Promote the snapshots.

Refer to sqlmesh.core.plan.

  1"""
  2# PlanEvaluator
  3
  4A plan evaluator is responsible for evaluating a plan when it is being applied.
  5
  6# Evaluation steps
  7
  8At a high level, when a plan is evaluated, SQLMesh will:
  9- Push new snapshots to the state sync.
 10- Create snapshot tables.
 11- Backfill data.
 12- Promote the snapshots.
 13
 14Refer to `sqlmesh.core.plan`.
 15"""
 16import abc
 17import typing as t
 18
 19from sqlmesh.core._typing import NotificationTarget
 20from sqlmesh.core.console import Console, get_console
 21from sqlmesh.core.plan.definition import Plan
 22from sqlmesh.core.scheduler import Scheduler
 23from sqlmesh.core.snapshot import SnapshotEvaluator, SnapshotInfoLike
 24from sqlmesh.core.state_sync import StateSync
 25from sqlmesh.core.user import User
 26from sqlmesh.schedulers.airflow import common as airflow_common
 27from sqlmesh.schedulers.airflow.client import AirflowClient
 28from sqlmesh.utils import random_id
 29from sqlmesh.utils.date import now
 30from sqlmesh.utils.errors import SQLMeshError
 31
 32
 33class PlanEvaluator(abc.ABC):
 34    @abc.abstractmethod
 35    def evaluate(self, plan: Plan) -> None:
 36        """Evaluates a plan by pushing snapshots and backfilling data.
 37
 38        Given a plan, it pushes snapshots into the state and then kicks off
 39        the backfill process for all affected snapshots. Once backfill is done,
 40        snapshots that are part of the plan are promoted in the environment targeted
 41        by this plan.
 42
 43        Args:
 44            plan: The plan to evaluate.
 45        """
 46
 47
 48class BuiltInPlanEvaluator(PlanEvaluator):
 49    def __init__(
 50        self,
 51        state_sync: StateSync,
 52        snapshot_evaluator: SnapshotEvaluator,
 53        backfill_concurrent_tasks: int = 1,
 54        console: t.Optional[Console] = None,
 55    ):
 56        self.state_sync = state_sync
 57        self.snapshot_evaluator = snapshot_evaluator
 58        self.backfill_concurrent_tasks = backfill_concurrent_tasks
 59        self.console = console or get_console()
 60
 61    def evaluate(self, plan: Plan) -> None:
 62        self._push(plan)
 63
 64        if plan.restatements:
 65            self._restate(plan)
 66
 67        if plan.requires_backfill:
 68            snapshots = plan.snapshots
 69            scheduler = Scheduler(
 70                snapshots,
 71                self.snapshot_evaluator,
 72                self.state_sync,
 73                max_workers=self.backfill_concurrent_tasks,
 74                console=self.console,
 75            )
 76            is_run_successful = scheduler.run(plan.start, plan.end, is_dev=plan.is_dev)
 77            if not is_run_successful:
 78                raise SQLMeshError("Plan application failed.")
 79
 80        self._promote(plan)
 81
 82        if not plan.requires_backfill:
 83            self.console.log_success("Virtual Update executed successfully")
 84
 85    def _push(self, plan: Plan) -> None:
 86        """
 87        Push the snapshots to the state sync.
 88
 89        As a part of plan pushing, snapshot tables are created.
 90
 91        Args:
 92            plan: The plan to source snapshots from.
 93        """
 94        parent_snapshot_ids = {
 95            p_sid for snapshot in plan.new_snapshots for p_sid in snapshot.parents
 96        }
 97
 98        stored_snapshots_by_id = self.state_sync.get_snapshots(parent_snapshot_ids)
 99        new_snapshots_by_id = {snapshot.snapshot_id: snapshot for snapshot in plan.new_snapshots}
100        all_snapshots_by_id = {**stored_snapshots_by_id, **new_snapshots_by_id}
101
102        self.snapshot_evaluator.create(plan.new_snapshots, all_snapshots_by_id)
103        self.state_sync.push_snapshots(plan.new_snapshots)
104
105    def _promote(self, plan: Plan) -> None:
106        """Promote a plan.
107
108        Promotion creates views with a model's name + env pointing to a physical snapshot.
109
110        Args:
111            plan: The plan to promote.
112        """
113        environment = plan.environment
114
115        added, removed = self.state_sync.promote(environment, no_gaps=plan.no_gaps)
116
117        self.console.start_promotion_progress(environment.name, len(added) + len(removed))
118
119        if not environment.end_at:
120            if not plan.is_dev:
121                self.snapshot_evaluator.migrate(plan.environment.snapshots)
122            self.state_sync.unpause_snapshots(added, now())
123
124        def on_complete(snapshot: SnapshotInfoLike) -> None:
125            self.console.update_promotion_progress(1)
126
127        completed = False
128        try:
129            self.snapshot_evaluator.promote(
130                added,
131                environment=environment.name,
132                is_dev=plan.is_dev,
133                on_complete=on_complete,
134            )
135            self.snapshot_evaluator.demote(
136                removed,
137                environment=environment.name,
138                on_complete=on_complete,
139            )
140            completed = True
141        finally:
142            self.console.stop_promotion_progress(success=completed)
143
144    def _restate(self, plan: Plan) -> None:
145        all_snapshots = (
146            [s for s in plan.snapshots if s.name in plan.restatements]
147            if plan.is_dev
148            else self.state_sync.get_snapshots_by_models(*plan.restatements)
149        )
150        self.state_sync.remove_interval(
151            [],
152            start=plan.start,
153            end=plan.end,
154            all_snapshots=all_snapshots,
155        )
156
157
158class AirflowPlanEvaluator(PlanEvaluator):
159    def __init__(
160        self,
161        airflow_client: AirflowClient,
162        console: t.Optional[Console] = None,
163        blocking: bool = True,
164        dag_run_poll_interval_secs: int = 10,
165        dag_creation_poll_interval_secs: int = 30,
166        dag_creation_max_retry_attempts: int = 10,
167        notification_targets: t.Optional[t.List[NotificationTarget]] = None,
168        backfill_concurrent_tasks: int = 1,
169        ddl_concurrent_tasks: int = 1,
170        users: t.Optional[t.List[User]] = None,
171    ):
172        self.airflow_client = airflow_client
173        self.blocking = blocking
174        self.dag_run_poll_interval_secs = dag_run_poll_interval_secs
175        self.dag_creation_poll_interval_secs = dag_creation_poll_interval_secs
176        self.dag_creation_max_retry_attempts = dag_creation_max_retry_attempts
177        self.console = console or get_console()
178        self.notification_targets = notification_targets or []
179        self.backfill_concurrent_tasks = backfill_concurrent_tasks
180        self.ddl_concurrent_tasks = ddl_concurrent_tasks
181        self.users = users or []
182
183    def evaluate(self, plan: Plan) -> None:
184        environment = plan.environment
185
186        plan_request_id = random_id()
187
188        self.airflow_client.apply_plan(
189            plan.new_snapshots,
190            environment,
191            plan_request_id,
192            no_gaps=plan.no_gaps,
193            skip_backfill=plan.skip_backfill,
194            restatements=plan.restatements,
195            notification_targets=self.notification_targets,
196            backfill_concurrent_tasks=self.backfill_concurrent_tasks,
197            ddl_concurrent_tasks=self.ddl_concurrent_tasks,
198            users=self.users,
199            is_dev=plan.is_dev,
200        )
201
202        if self.blocking:
203            plan_application_dag_id = airflow_common.plan_application_dag_id(
204                environment.name, plan_request_id
205            )
206
207            self.console.log_status_update(
208                f"Waiting for the plan application DAG '{plan_application_dag_id}' to be provisioned on Airflow"
209            )
210
211            plan_application_dag_run_id = self.airflow_client.wait_for_first_dag_run(
212                plan_application_dag_id,
213                self.dag_creation_poll_interval_secs,
214                self.dag_creation_max_retry_attempts,
215            )
216
217            self.airflow_client.print_tracking_url(
218                plan_application_dag_id,
219                plan_application_dag_run_id,
220                "plan application",
221            )
222            plan_application_succeeded = self.airflow_client.wait_for_dag_run_completion(
223                plan_application_dag_id,
224                plan_application_dag_run_id,
225                self.dag_run_poll_interval_secs,
226            )
227            if not plan_application_succeeded:
228                raise SQLMeshError("Plan application failed.")
class PlanEvaluator(abc.ABC):
34class PlanEvaluator(abc.ABC):
35    @abc.abstractmethod
36    def evaluate(self, plan: Plan) -> None:
37        """Evaluates a plan by pushing snapshots and backfilling data.
38
39        Given a plan, it pushes snapshots into the state and then kicks off
40        the backfill process for all affected snapshots. Once backfill is done,
41        snapshots that are part of the plan are promoted in the environment targeted
42        by this plan.
43
44        Args:
45            plan: The plan to evaluate.
46        """

Helper class that provides a standard way to create an ABC using inheritance.

@abc.abstractmethod
def evaluate(self, plan: sqlmesh.core.plan.definition.Plan) -> None:
35    @abc.abstractmethod
36    def evaluate(self, plan: Plan) -> None:
37        """Evaluates a plan by pushing snapshots and backfilling data.
38
39        Given a plan, it pushes snapshots into the state and then kicks off
40        the backfill process for all affected snapshots. Once backfill is done,
41        snapshots that are part of the plan are promoted in the environment targeted
42        by this plan.
43
44        Args:
45            plan: The plan to evaluate.
46        """

Evaluates a plan by pushing snapshots and backfilling data.

Given a plan, it pushes snapshots into the state and then kicks off the backfill process for all affected snapshots. Once backfill is done, snapshots that are part of the plan are promoted in the environment targeted by this plan.

Arguments:
  • plan: The plan to evaluate.
class BuiltInPlanEvaluator(PlanEvaluator):
 49class BuiltInPlanEvaluator(PlanEvaluator):
 50    def __init__(
 51        self,
 52        state_sync: StateSync,
 53        snapshot_evaluator: SnapshotEvaluator,
 54        backfill_concurrent_tasks: int = 1,
 55        console: t.Optional[Console] = None,
 56    ):
 57        self.state_sync = state_sync
 58        self.snapshot_evaluator = snapshot_evaluator
 59        self.backfill_concurrent_tasks = backfill_concurrent_tasks
 60        self.console = console or get_console()
 61
 62    def evaluate(self, plan: Plan) -> None:
 63        self._push(plan)
 64
 65        if plan.restatements:
 66            self._restate(plan)
 67
 68        if plan.requires_backfill:
 69            snapshots = plan.snapshots
 70            scheduler = Scheduler(
 71                snapshots,
 72                self.snapshot_evaluator,
 73                self.state_sync,
 74                max_workers=self.backfill_concurrent_tasks,
 75                console=self.console,
 76            )
 77            is_run_successful = scheduler.run(plan.start, plan.end, is_dev=plan.is_dev)
 78            if not is_run_successful:
 79                raise SQLMeshError("Plan application failed.")
 80
 81        self._promote(plan)
 82
 83        if not plan.requires_backfill:
 84            self.console.log_success("Virtual Update executed successfully")
 85
 86    def _push(self, plan: Plan) -> None:
 87        """
 88        Push the snapshots to the state sync.
 89
 90        As a part of plan pushing, snapshot tables are created.
 91
 92        Args:
 93            plan: The plan to source snapshots from.
 94        """
 95        parent_snapshot_ids = {
 96            p_sid for snapshot in plan.new_snapshots for p_sid in snapshot.parents
 97        }
 98
 99        stored_snapshots_by_id = self.state_sync.get_snapshots(parent_snapshot_ids)
100        new_snapshots_by_id = {snapshot.snapshot_id: snapshot for snapshot in plan.new_snapshots}
101        all_snapshots_by_id = {**stored_snapshots_by_id, **new_snapshots_by_id}
102
103        self.snapshot_evaluator.create(plan.new_snapshots, all_snapshots_by_id)
104        self.state_sync.push_snapshots(plan.new_snapshots)
105
106    def _promote(self, plan: Plan) -> None:
107        """Promote a plan.
108
109        Promotion creates views with a model's name + env pointing to a physical snapshot.
110
111        Args:
112            plan: The plan to promote.
113        """
114        environment = plan.environment
115
116        added, removed = self.state_sync.promote(environment, no_gaps=plan.no_gaps)
117
118        self.console.start_promotion_progress(environment.name, len(added) + len(removed))
119
120        if not environment.end_at:
121            if not plan.is_dev:
122                self.snapshot_evaluator.migrate(plan.environment.snapshots)
123            self.state_sync.unpause_snapshots(added, now())
124
125        def on_complete(snapshot: SnapshotInfoLike) -> None:
126            self.console.update_promotion_progress(1)
127
128        completed = False
129        try:
130            self.snapshot_evaluator.promote(
131                added,
132                environment=environment.name,
133                is_dev=plan.is_dev,
134                on_complete=on_complete,
135            )
136            self.snapshot_evaluator.demote(
137                removed,
138                environment=environment.name,
139                on_complete=on_complete,
140            )
141            completed = True
142        finally:
143            self.console.stop_promotion_progress(success=completed)
144
145    def _restate(self, plan: Plan) -> None:
146        all_snapshots = (
147            [s for s in plan.snapshots if s.name in plan.restatements]
148            if plan.is_dev
149            else self.state_sync.get_snapshots_by_models(*plan.restatements)
150        )
151        self.state_sync.remove_interval(
152            [],
153            start=plan.start,
154            end=plan.end,
155            all_snapshots=all_snapshots,
156        )

Helper class that provides a standard way to create an ABC using inheritance.

BuiltInPlanEvaluator( state_sync: sqlmesh.core.state_sync.base.StateSync, snapshot_evaluator: sqlmesh.core.snapshot.evaluator.SnapshotEvaluator, backfill_concurrent_tasks: int = 1, console: Optional[sqlmesh.core.console.Console] = None)
50    def __init__(
51        self,
52        state_sync: StateSync,
53        snapshot_evaluator: SnapshotEvaluator,
54        backfill_concurrent_tasks: int = 1,
55        console: t.Optional[Console] = None,
56    ):
57        self.state_sync = state_sync
58        self.snapshot_evaluator = snapshot_evaluator
59        self.backfill_concurrent_tasks = backfill_concurrent_tasks
60        self.console = console or get_console()
def evaluate(self, plan: sqlmesh.core.plan.definition.Plan) -> None:
62    def evaluate(self, plan: Plan) -> None:
63        self._push(plan)
64
65        if plan.restatements:
66            self._restate(plan)
67
68        if plan.requires_backfill:
69            snapshots = plan.snapshots
70            scheduler = Scheduler(
71                snapshots,
72                self.snapshot_evaluator,
73                self.state_sync,
74                max_workers=self.backfill_concurrent_tasks,
75                console=self.console,
76            )
77            is_run_successful = scheduler.run(plan.start, plan.end, is_dev=plan.is_dev)
78            if not is_run_successful:
79                raise SQLMeshError("Plan application failed.")
80
81        self._promote(plan)
82
83        if not plan.requires_backfill:
84            self.console.log_success("Virtual Update executed successfully")

Evaluates a plan by pushing snapshots and backfilling data.

Given a plan, it pushes snapshots into the state and then kicks off the backfill process for all affected snapshots. Once backfill is done, snapshots that are part of the plan are promoted in the environment targeted by this plan.

Arguments:
  • plan: The plan to evaluate.
class AirflowPlanEvaluator(PlanEvaluator):
159class AirflowPlanEvaluator(PlanEvaluator):
160    def __init__(
161        self,
162        airflow_client: AirflowClient,
163        console: t.Optional[Console] = None,
164        blocking: bool = True,
165        dag_run_poll_interval_secs: int = 10,
166        dag_creation_poll_interval_secs: int = 30,
167        dag_creation_max_retry_attempts: int = 10,
168        notification_targets: t.Optional[t.List[NotificationTarget]] = None,
169        backfill_concurrent_tasks: int = 1,
170        ddl_concurrent_tasks: int = 1,
171        users: t.Optional[t.List[User]] = None,
172    ):
173        self.airflow_client = airflow_client
174        self.blocking = blocking
175        self.dag_run_poll_interval_secs = dag_run_poll_interval_secs
176        self.dag_creation_poll_interval_secs = dag_creation_poll_interval_secs
177        self.dag_creation_max_retry_attempts = dag_creation_max_retry_attempts
178        self.console = console or get_console()
179        self.notification_targets = notification_targets or []
180        self.backfill_concurrent_tasks = backfill_concurrent_tasks
181        self.ddl_concurrent_tasks = ddl_concurrent_tasks
182        self.users = users or []
183
184    def evaluate(self, plan: Plan) -> None:
185        environment = plan.environment
186
187        plan_request_id = random_id()
188
189        self.airflow_client.apply_plan(
190            plan.new_snapshots,
191            environment,
192            plan_request_id,
193            no_gaps=plan.no_gaps,
194            skip_backfill=plan.skip_backfill,
195            restatements=plan.restatements,
196            notification_targets=self.notification_targets,
197            backfill_concurrent_tasks=self.backfill_concurrent_tasks,
198            ddl_concurrent_tasks=self.ddl_concurrent_tasks,
199            users=self.users,
200            is_dev=plan.is_dev,
201        )
202
203        if self.blocking:
204            plan_application_dag_id = airflow_common.plan_application_dag_id(
205                environment.name, plan_request_id
206            )
207
208            self.console.log_status_update(
209                f"Waiting for the plan application DAG '{plan_application_dag_id}' to be provisioned on Airflow"
210            )
211
212            plan_application_dag_run_id = self.airflow_client.wait_for_first_dag_run(
213                plan_application_dag_id,
214                self.dag_creation_poll_interval_secs,
215                self.dag_creation_max_retry_attempts,
216            )
217
218            self.airflow_client.print_tracking_url(
219                plan_application_dag_id,
220                plan_application_dag_run_id,
221                "plan application",
222            )
223            plan_application_succeeded = self.airflow_client.wait_for_dag_run_completion(
224                plan_application_dag_id,
225                plan_application_dag_run_id,
226                self.dag_run_poll_interval_secs,
227            )
228            if not plan_application_succeeded:
229                raise SQLMeshError("Plan application failed.")

Helper class that provides a standard way to create an ABC using inheritance.

AirflowPlanEvaluator( airflow_client: sqlmesh.schedulers.airflow.client.AirflowClient, console: Optional[sqlmesh.core.console.Console] = None, blocking: bool = True, dag_run_poll_interval_secs: int = 10, dag_creation_poll_interval_secs: int = 30, dag_creation_max_retry_attempts: int = 10, notification_targets: Optional[List[Annotated[Union[sqlmesh.core.notification_target.ConsoleNotificationTarget, sqlmesh.integrations.github.notification_target.GithubNotificationTarget], FieldInfo(default=PydanticUndefined, discriminator='type_', extra={})]]] = None, backfill_concurrent_tasks: int = 1, ddl_concurrent_tasks: int = 1, users: Optional[List[sqlmesh.core.user.User]] = None)
160    def __init__(
161        self,
162        airflow_client: AirflowClient,
163        console: t.Optional[Console] = None,
164        blocking: bool = True,
165        dag_run_poll_interval_secs: int = 10,
166        dag_creation_poll_interval_secs: int = 30,
167        dag_creation_max_retry_attempts: int = 10,
168        notification_targets: t.Optional[t.List[NotificationTarget]] = None,
169        backfill_concurrent_tasks: int = 1,
170        ddl_concurrent_tasks: int = 1,
171        users: t.Optional[t.List[User]] = None,
172    ):
173        self.airflow_client = airflow_client
174        self.blocking = blocking
175        self.dag_run_poll_interval_secs = dag_run_poll_interval_secs
176        self.dag_creation_poll_interval_secs = dag_creation_poll_interval_secs
177        self.dag_creation_max_retry_attempts = dag_creation_max_retry_attempts
178        self.console = console or get_console()
179        self.notification_targets = notification_targets or []
180        self.backfill_concurrent_tasks = backfill_concurrent_tasks
181        self.ddl_concurrent_tasks = ddl_concurrent_tasks
182        self.users = users or []
def evaluate(self, plan: sqlmesh.core.plan.definition.Plan) -> None:
184    def evaluate(self, plan: Plan) -> None:
185        environment = plan.environment
186
187        plan_request_id = random_id()
188
189        self.airflow_client.apply_plan(
190            plan.new_snapshots,
191            environment,
192            plan_request_id,
193            no_gaps=plan.no_gaps,
194            skip_backfill=plan.skip_backfill,
195            restatements=plan.restatements,
196            notification_targets=self.notification_targets,
197            backfill_concurrent_tasks=self.backfill_concurrent_tasks,
198            ddl_concurrent_tasks=self.ddl_concurrent_tasks,
199            users=self.users,
200            is_dev=plan.is_dev,
201        )
202
203        if self.blocking:
204            plan_application_dag_id = airflow_common.plan_application_dag_id(
205                environment.name, plan_request_id
206            )
207
208            self.console.log_status_update(
209                f"Waiting for the plan application DAG '{plan_application_dag_id}' to be provisioned on Airflow"
210            )
211
212            plan_application_dag_run_id = self.airflow_client.wait_for_first_dag_run(
213                plan_application_dag_id,
214                self.dag_creation_poll_interval_secs,
215                self.dag_creation_max_retry_attempts,
216            )
217
218            self.airflow_client.print_tracking_url(
219                plan_application_dag_id,
220                plan_application_dag_run_id,
221                "plan application",
222            )
223            plan_application_succeeded = self.airflow_client.wait_for_dag_run_completion(
224                plan_application_dag_id,
225                plan_application_dag_run_id,
226                self.dag_run_poll_interval_secs,
227            )
228            if not plan_application_succeeded:
229                raise SQLMeshError("Plan application failed.")

Evaluates a plan by pushing snapshots and backfilling data.

Given a plan, it pushes snapshots into the state and then kicks off the backfill process for all affected snapshots. Once backfill is done, snapshots that are part of the plan are promoted in the environment targeted by this plan.

Arguments:
  • plan: The plan to evaluate.