sqlmesh.schedulers.airflow.operators.targets
1import abc 2import typing as t 3 4from airflow.exceptions import AirflowSkipException 5from airflow.utils.context import Context 6from airflow.utils.session import provide_session 7from sqlalchemy.orm import Session 8 9from sqlmesh.core.engine_adapter import create_engine_adapter 10from sqlmesh.core.snapshot import ( 11 Snapshot, 12 SnapshotEvaluator, 13 SnapshotId, 14 SnapshotTableInfo, 15) 16from sqlmesh.engines import commands 17from sqlmesh.schedulers.airflow import common, util 18from sqlmesh.utils.date import TimeLike 19from sqlmesh.utils.pydantic import PydanticModel 20 21CP = t.TypeVar("CP", bound=PydanticModel) 22 23 24class BaseTarget(abc.ABC, t.Generic[CP]): 25 command_type: commands.CommandType 26 command_handler: t.Callable[[SnapshotEvaluator, CP], None] 27 ddl_concurrent_tasks: int 28 29 def serialized_command_payload(self, context: Context) -> str: 30 """Returns the serialized command payload for the Spark application. 31 32 Args: 33 context: Airflow task context. 34 35 Returns: 36 The serialized command payload. 37 """ 38 return self._get_command_payload_or_skip(context).json() 39 40 def execute( 41 self, context: Context, connection_factory: t.Callable[[], t.Any], dialect: str 42 ) -> None: 43 """Executes this target. 44 45 Args: 46 context: Airflow task context. 47 connection_factory: a callable which produces a new Database API compliant 48 connection on every call. 49 dialect: The dialect with which this adapter is associated. 50 """ 51 payload = self._get_command_payload_or_skip(context) 52 snapshot_evaluator = SnapshotEvaluator( 53 create_engine_adapter( 54 connection_factory, dialect, multithreaded=self.ddl_concurrent_tasks > 1 55 ), 56 ddl_concurrent_tasks=self.ddl_concurrent_tasks, 57 ) 58 try: 59 self.command_handler(snapshot_evaluator, payload) 60 self.post_hook(context) 61 finally: 62 snapshot_evaluator.close() 63 64 def post_hook(self, context: Context, **kwargs: t.Any) -> None: 65 """The hook that should be invoked once the processing of this target 66 is complete. 67 68 Args: 69 context: Airflow task context. 70 """ 71 72 @abc.abstractmethod 73 def _get_command_payload(self, context: Context) -> t.Optional[CP]: 74 """Constructs the command payload. 75 76 Args: 77 context: Airflow task context. 78 79 Returns: 80 The command payload or None if there is no command to execute 81 and the target must be skipped. 82 """ 83 84 def _get_command_payload_or_skip(self, context: Context) -> CP: 85 payload = self._get_command_payload(context) 86 if not payload: 87 self.post_hook(context) 88 raise AirflowSkipException 89 return payload 90 91 92class SnapshotEvaluationTarget(BaseTarget[commands.EvaluateCommandPayload], PydanticModel): 93 """The target which contains attributes necessary to evaluate a given snapshot. 94 95 Args: 96 snapshot: The snapshot which should be evaluated. 97 parent_snapshots: All upstream snapshots (by model name) to use for expansion and mapping of physical locations. 98 start: The start of the interval to evaluate. 99 end: The end of the interval to evaluate. 100 latest: The latest time used for non incremental datasets. 101 is_dev: Indicates whether the evaluation happens in the development mode and temporary 102 tables / table clones should be used where applicable. 103 """ 104 105 command_type: commands.CommandType = commands.CommandType.EVALUATE 106 command_handler: t.Callable[ 107 [SnapshotEvaluator, commands.EvaluateCommandPayload], None 108 ] = commands.evaluate 109 ddl_concurrent_tasks: int = 1 110 111 snapshot: Snapshot 112 parent_snapshots: t.Dict[str, Snapshot] 113 start: t.Optional[TimeLike] 114 end: t.Optional[TimeLike] 115 latest: t.Optional[TimeLike] 116 is_dev: bool 117 118 def post_hook( 119 self, 120 context: Context, 121 **kwargs: t.Any, 122 ) -> None: 123 with util.scoped_state_sync() as state_sync: 124 state_sync.add_interval( 125 self.snapshot.snapshot_id, 126 self._get_start(context), 127 self._get_end(context), 128 is_dev=self.is_dev, 129 ) 130 131 def _get_command_payload(self, context: Context) -> t.Optional[commands.EvaluateCommandPayload]: 132 return commands.EvaluateCommandPayload( 133 snapshot=self.snapshot, 134 parent_snapshots=self.parent_snapshots, 135 start=self._get_start(context), 136 end=self._get_end(context), 137 latest=self._get_latest(context), 138 is_dev=self.is_dev, 139 ) 140 141 def _get_start(self, context: Context) -> TimeLike: 142 return self.start or context["dag_run"].data_interval_start 143 144 def _get_end(self, context: Context) -> TimeLike: 145 return self.end or context["dag_run"].data_interval_end 146 147 def _get_latest(self, context: Context) -> TimeLike: 148 return self.latest or context["dag_run"].logical_date 149 150 151class SnapshotPromotionTarget(BaseTarget[commands.PromoteCommandPayload], PydanticModel): 152 """The target which contains attributes necessary to perform snapshot promotion in a given environment. 153 154 The promotion means creation of views associated with the environment which target physical tables 155 associated with the given list of snapshots. 156 157 Args: 158 snapshots: The list of snapshots that should be promoted in the target environment. 159 environment: The target environment. 160 ddl_concurrent_tasks: The number of concurrent tasks used for DDL 161 operations (table / view creation, deletion, etc). Default: 1. 162 is_dev: Indicates whether the promotion happens in the development mode and temporary 163 tables / table clones should be used where applicable. 164 """ 165 166 command_type: commands.CommandType = commands.CommandType.PROMOTE 167 command_handler: t.Callable[ 168 [SnapshotEvaluator, commands.PromoteCommandPayload], None 169 ] = commands.promote 170 171 snapshots: t.List[SnapshotTableInfo] 172 environment: str 173 ddl_concurrent_tasks: int 174 is_dev: bool 175 176 def _get_command_payload(self, context: Context) -> t.Optional[commands.PromoteCommandPayload]: 177 return commands.PromoteCommandPayload( 178 snapshots=self.snapshots, 179 environment=self.environment, 180 is_dev=self.is_dev, 181 ) 182 183 184class SnapshotDemotionTarget(BaseTarget[commands.DemoteCommandPayload], PydanticModel): 185 """The target which contains attributes necessary to perform snapshot demotion in a given environment. 186 187 The demotion means deletion of views that match names of provided snapshots in the target environment. 188 189 Args: 190 snapshots: The list of snapshots that should be demoted in the target environment. 191 environment: The target environment. 192 """ 193 194 command_type: commands.CommandType = commands.CommandType.DEMOTE 195 command_handler: t.Callable[ 196 [SnapshotEvaluator, commands.DemoteCommandPayload], None 197 ] = commands.demote 198 199 snapshots: t.List[SnapshotTableInfo] 200 environment: str 201 ddl_concurrent_tasks: int 202 203 def _get_command_payload(self, context: Context) -> t.Optional[commands.DemoteCommandPayload]: 204 return commands.DemoteCommandPayload( 205 snapshots=self.snapshots, 206 environment=self.environment, 207 ) 208 209 210class SnapshotCleanupTarget(BaseTarget[commands.CleanupCommandPayload], PydanticModel): 211 """The target which contains attributes necessary to perform table cleanup of expired snapshots""" 212 213 command_type: commands.CommandType = commands.CommandType.CLEANUP 214 command_handler: t.Callable[ 215 [SnapshotEvaluator, commands.CleanupCommandPayload], None 216 ] = commands.cleanup 217 ddl_concurrent_tasks: int = 1 218 219 @provide_session 220 def post_hook( 221 self, 222 context: Context, 223 session: Session = util.PROVIDED_SESSION, 224 **kwargs: t.Any, 225 ) -> None: 226 _delete_xcom( 227 common.SNAPSHOT_CLEANUP_COMMAND_XCOM_KEY, 228 common.JANITOR_TASK_ID, 229 context, 230 session, 231 ) 232 233 def _get_command_payload(self, context: Context) -> t.Optional[commands.CleanupCommandPayload]: 234 command = commands.CleanupCommandPayload.parse_raw( 235 context["ti"].xcom_pull(key=common.SNAPSHOT_CLEANUP_COMMAND_XCOM_KEY) 236 ) 237 if not command.snapshots and not command.environments: 238 return None 239 return command 240 241 242class SnapshotCreateTablesTarget(BaseTarget[commands.CreateTablesCommandPayload], PydanticModel): 243 """The target which creates physical tables for the given set of new snapshots.""" 244 245 command_type: commands.CommandType = commands.CommandType.CREATE_TABLES 246 command_handler: t.Callable[ 247 [SnapshotEvaluator, commands.CreateTablesCommandPayload], None 248 ] = commands.create_tables 249 250 new_snapshots: t.List[Snapshot] 251 ddl_concurrent_tasks: int 252 253 def _get_command_payload( 254 self, context: Context 255 ) -> t.Optional[commands.CreateTablesCommandPayload]: 256 if not self.new_snapshots: 257 return None 258 259 parent_snapshot_ids = { 260 p_sid for snapshot in self.new_snapshots for p_sid in snapshot.parents 261 } 262 stored_snapshots = self._get_stored_snapshots(parent_snapshot_ids) 263 264 return commands.CreateTablesCommandPayload( 265 target_snapshot_ids=[s.snapshot_id for s in self.new_snapshots], 266 snapshots=stored_snapshots + self.new_snapshots, 267 ) 268 269 def _get_stored_snapshots(self, snapshot_ids: t.Set[SnapshotId]) -> t.List[Snapshot]: 270 with util.scoped_state_sync() as state_sync: 271 return list(state_sync.get_snapshots(snapshot_ids).values()) 272 273 274class SnapshotMigrateTablesTarget(BaseTarget[commands.MigrateTablesCommandPayload], PydanticModel): 275 """The target which updates schemas of existing physical tables to bring them in correspondance 276 with schemas of target snapshots. 277 """ 278 279 command_type: commands.CommandType = commands.CommandType.MIGRATE_TABLES 280 command_handler: t.Callable[ 281 [SnapshotEvaluator, commands.MigrateTablesCommandPayload], None 282 ] = commands.migrate_tables 283 284 snapshots: t.List[SnapshotTableInfo] 285 ddl_concurrent_tasks: int 286 287 def _get_command_payload( 288 self, context: Context 289 ) -> t.Optional[commands.MigrateTablesCommandPayload]: 290 if not self.snapshots: 291 return None 292 return commands.MigrateTablesCommandPayload(snapshots=self.snapshots) 293 294 295def _delete_xcom(key: str, task_id: str, context: Context, session: Session) -> None: 296 ti = context["ti"] 297 util.delete_xcoms( 298 ti.dag_id, 299 {key}, 300 task_id=task_id, 301 run_id=ti.run_id, 302 session=session, 303 )
25class BaseTarget(abc.ABC, t.Generic[CP]): 26 command_type: commands.CommandType 27 command_handler: t.Callable[[SnapshotEvaluator, CP], None] 28 ddl_concurrent_tasks: int 29 30 def serialized_command_payload(self, context: Context) -> str: 31 """Returns the serialized command payload for the Spark application. 32 33 Args: 34 context: Airflow task context. 35 36 Returns: 37 The serialized command payload. 38 """ 39 return self._get_command_payload_or_skip(context).json() 40 41 def execute( 42 self, context: Context, connection_factory: t.Callable[[], t.Any], dialect: str 43 ) -> None: 44 """Executes this target. 45 46 Args: 47 context: Airflow task context. 48 connection_factory: a callable which produces a new Database API compliant 49 connection on every call. 50 dialect: The dialect with which this adapter is associated. 51 """ 52 payload = self._get_command_payload_or_skip(context) 53 snapshot_evaluator = SnapshotEvaluator( 54 create_engine_adapter( 55 connection_factory, dialect, multithreaded=self.ddl_concurrent_tasks > 1 56 ), 57 ddl_concurrent_tasks=self.ddl_concurrent_tasks, 58 ) 59 try: 60 self.command_handler(snapshot_evaluator, payload) 61 self.post_hook(context) 62 finally: 63 snapshot_evaluator.close() 64 65 def post_hook(self, context: Context, **kwargs: t.Any) -> None: 66 """The hook that should be invoked once the processing of this target 67 is complete. 68 69 Args: 70 context: Airflow task context. 71 """ 72 73 @abc.abstractmethod 74 def _get_command_payload(self, context: Context) -> t.Optional[CP]: 75 """Constructs the command payload. 76 77 Args: 78 context: Airflow task context. 79 80 Returns: 81 The command payload or None if there is no command to execute 82 and the target must be skipped. 83 """ 84 85 def _get_command_payload_or_skip(self, context: Context) -> CP: 86 payload = self._get_command_payload(context) 87 if not payload: 88 self.post_hook(context) 89 raise AirflowSkipException 90 return payload
Helper class that provides a standard way to create an ABC using inheritance.
30 def serialized_command_payload(self, context: Context) -> str: 31 """Returns the serialized command payload for the Spark application. 32 33 Args: 34 context: Airflow task context. 35 36 Returns: 37 The serialized command payload. 38 """ 39 return self._get_command_payload_or_skip(context).json()
Returns the serialized command payload for the Spark application.
Arguments:
- context: Airflow task context.
Returns:
The serialized command payload.
41 def execute( 42 self, context: Context, connection_factory: t.Callable[[], t.Any], dialect: str 43 ) -> None: 44 """Executes this target. 45 46 Args: 47 context: Airflow task context. 48 connection_factory: a callable which produces a new Database API compliant 49 connection on every call. 50 dialect: The dialect with which this adapter is associated. 51 """ 52 payload = self._get_command_payload_or_skip(context) 53 snapshot_evaluator = SnapshotEvaluator( 54 create_engine_adapter( 55 connection_factory, dialect, multithreaded=self.ddl_concurrent_tasks > 1 56 ), 57 ddl_concurrent_tasks=self.ddl_concurrent_tasks, 58 ) 59 try: 60 self.command_handler(snapshot_evaluator, payload) 61 self.post_hook(context) 62 finally: 63 snapshot_evaluator.close()
Executes this target.
Arguments:
- context: Airflow task context.
- connection_factory: a callable which produces a new Database API compliant connection on every call.
- dialect: The dialect with which this adapter is associated.
65 def post_hook(self, context: Context, **kwargs: t.Any) -> None: 66 """The hook that should be invoked once the processing of this target 67 is complete. 68 69 Args: 70 context: Airflow task context. 71 """
The hook that should be invoked once the processing of this target is complete.
Arguments:
- context: Airflow task context.
93class SnapshotEvaluationTarget(BaseTarget[commands.EvaluateCommandPayload], PydanticModel): 94 """The target which contains attributes necessary to evaluate a given snapshot. 95 96 Args: 97 snapshot: The snapshot which should be evaluated. 98 parent_snapshots: All upstream snapshots (by model name) to use for expansion and mapping of physical locations. 99 start: The start of the interval to evaluate. 100 end: The end of the interval to evaluate. 101 latest: The latest time used for non incremental datasets. 102 is_dev: Indicates whether the evaluation happens in the development mode and temporary 103 tables / table clones should be used where applicable. 104 """ 105 106 command_type: commands.CommandType = commands.CommandType.EVALUATE 107 command_handler: t.Callable[ 108 [SnapshotEvaluator, commands.EvaluateCommandPayload], None 109 ] = commands.evaluate 110 ddl_concurrent_tasks: int = 1 111 112 snapshot: Snapshot 113 parent_snapshots: t.Dict[str, Snapshot] 114 start: t.Optional[TimeLike] 115 end: t.Optional[TimeLike] 116 latest: t.Optional[TimeLike] 117 is_dev: bool 118 119 def post_hook( 120 self, 121 context: Context, 122 **kwargs: t.Any, 123 ) -> None: 124 with util.scoped_state_sync() as state_sync: 125 state_sync.add_interval( 126 self.snapshot.snapshot_id, 127 self._get_start(context), 128 self._get_end(context), 129 is_dev=self.is_dev, 130 ) 131 132 def _get_command_payload(self, context: Context) -> t.Optional[commands.EvaluateCommandPayload]: 133 return commands.EvaluateCommandPayload( 134 snapshot=self.snapshot, 135 parent_snapshots=self.parent_snapshots, 136 start=self._get_start(context), 137 end=self._get_end(context), 138 latest=self._get_latest(context), 139 is_dev=self.is_dev, 140 ) 141 142 def _get_start(self, context: Context) -> TimeLike: 143 return self.start or context["dag_run"].data_interval_start 144 145 def _get_end(self, context: Context) -> TimeLike: 146 return self.end or context["dag_run"].data_interval_end 147 148 def _get_latest(self, context: Context) -> TimeLike: 149 return self.latest or context["dag_run"].logical_date
The target which contains attributes necessary to evaluate a given snapshot.
Arguments:
- snapshot: The snapshot which should be evaluated.
- parent_snapshots: All upstream snapshots (by model name) to use for expansion and mapping of physical locations.
- start: The start of the interval to evaluate.
- end: The end of the interval to evaluate.
- latest: The latest time used for non incremental datasets.
- is_dev: Indicates whether the evaluation happens in the development mode and temporary tables / table clones should be used where applicable.
119 def post_hook( 120 self, 121 context: Context, 122 **kwargs: t.Any, 123 ) -> None: 124 with util.scoped_state_sync() as state_sync: 125 state_sync.add_interval( 126 self.snapshot.snapshot_id, 127 self._get_start(context), 128 self._get_end(context), 129 is_dev=self.is_dev, 130 )
The hook that should be invoked once the processing of this target is complete.
Arguments:
- context: Airflow task context.
Inherited Members
- pydantic.main.BaseModel
- BaseModel
- parse_obj
- parse_raw
- parse_file
- from_orm
- construct
- copy
- schema
- schema_json
- validate
- update_forward_refs
152class SnapshotPromotionTarget(BaseTarget[commands.PromoteCommandPayload], PydanticModel): 153 """The target which contains attributes necessary to perform snapshot promotion in a given environment. 154 155 The promotion means creation of views associated with the environment which target physical tables 156 associated with the given list of snapshots. 157 158 Args: 159 snapshots: The list of snapshots that should be promoted in the target environment. 160 environment: The target environment. 161 ddl_concurrent_tasks: The number of concurrent tasks used for DDL 162 operations (table / view creation, deletion, etc). Default: 1. 163 is_dev: Indicates whether the promotion happens in the development mode and temporary 164 tables / table clones should be used where applicable. 165 """ 166 167 command_type: commands.CommandType = commands.CommandType.PROMOTE 168 command_handler: t.Callable[ 169 [SnapshotEvaluator, commands.PromoteCommandPayload], None 170 ] = commands.promote 171 172 snapshots: t.List[SnapshotTableInfo] 173 environment: str 174 ddl_concurrent_tasks: int 175 is_dev: bool 176 177 def _get_command_payload(self, context: Context) -> t.Optional[commands.PromoteCommandPayload]: 178 return commands.PromoteCommandPayload( 179 snapshots=self.snapshots, 180 environment=self.environment, 181 is_dev=self.is_dev, 182 )
The target which contains attributes necessary to perform snapshot promotion in a given environment.
The promotion means creation of views associated with the environment which target physical tables associated with the given list of snapshots.
Arguments:
- snapshots: The list of snapshots that should be promoted in the target environment.
- environment: The target environment.
- ddl_concurrent_tasks: The number of concurrent tasks used for DDL operations (table / view creation, deletion, etc). Default: 1.
- is_dev: Indicates whether the promotion happens in the development mode and temporary tables / table clones should be used where applicable.
Inherited Members
- pydantic.main.BaseModel
- BaseModel
- parse_obj
- parse_raw
- parse_file
- from_orm
- construct
- copy
- schema
- schema_json
- validate
- update_forward_refs
185class SnapshotDemotionTarget(BaseTarget[commands.DemoteCommandPayload], PydanticModel): 186 """The target which contains attributes necessary to perform snapshot demotion in a given environment. 187 188 The demotion means deletion of views that match names of provided snapshots in the target environment. 189 190 Args: 191 snapshots: The list of snapshots that should be demoted in the target environment. 192 environment: The target environment. 193 """ 194 195 command_type: commands.CommandType = commands.CommandType.DEMOTE 196 command_handler: t.Callable[ 197 [SnapshotEvaluator, commands.DemoteCommandPayload], None 198 ] = commands.demote 199 200 snapshots: t.List[SnapshotTableInfo] 201 environment: str 202 ddl_concurrent_tasks: int 203 204 def _get_command_payload(self, context: Context) -> t.Optional[commands.DemoteCommandPayload]: 205 return commands.DemoteCommandPayload( 206 snapshots=self.snapshots, 207 environment=self.environment, 208 )
The target which contains attributes necessary to perform snapshot demotion in a given environment.
The demotion means deletion of views that match names of provided snapshots in the target environment.
Arguments:
- snapshots: The list of snapshots that should be demoted in the target environment.
- environment: The target environment.
Inherited Members
- pydantic.main.BaseModel
- BaseModel
- parse_obj
- parse_raw
- parse_file
- from_orm
- construct
- copy
- schema
- schema_json
- validate
- update_forward_refs
211class SnapshotCleanupTarget(BaseTarget[commands.CleanupCommandPayload], PydanticModel): 212 """The target which contains attributes necessary to perform table cleanup of expired snapshots""" 213 214 command_type: commands.CommandType = commands.CommandType.CLEANUP 215 command_handler: t.Callable[ 216 [SnapshotEvaluator, commands.CleanupCommandPayload], None 217 ] = commands.cleanup 218 ddl_concurrent_tasks: int = 1 219 220 @provide_session 221 def post_hook( 222 self, 223 context: Context, 224 session: Session = util.PROVIDED_SESSION, 225 **kwargs: t.Any, 226 ) -> None: 227 _delete_xcom( 228 common.SNAPSHOT_CLEANUP_COMMAND_XCOM_KEY, 229 common.JANITOR_TASK_ID, 230 context, 231 session, 232 ) 233 234 def _get_command_payload(self, context: Context) -> t.Optional[commands.CleanupCommandPayload]: 235 command = commands.CleanupCommandPayload.parse_raw( 236 context["ti"].xcom_pull(key=common.SNAPSHOT_CLEANUP_COMMAND_XCOM_KEY) 237 ) 238 if not command.snapshots and not command.environments: 239 return None 240 return command
The target which contains attributes necessary to perform table cleanup of expired snapshots
220 @provide_session 221 def post_hook( 222 self, 223 context: Context, 224 session: Session = util.PROVIDED_SESSION, 225 **kwargs: t.Any, 226 ) -> None: 227 _delete_xcom( 228 common.SNAPSHOT_CLEANUP_COMMAND_XCOM_KEY, 229 common.JANITOR_TASK_ID, 230 context, 231 session, 232 )
The hook that should be invoked once the processing of this target is complete.
Arguments:
- context: Airflow task context.
Inherited Members
- pydantic.main.BaseModel
- BaseModel
- parse_obj
- parse_raw
- parse_file
- from_orm
- construct
- copy
- schema
- schema_json
- validate
- update_forward_refs
243class SnapshotCreateTablesTarget(BaseTarget[commands.CreateTablesCommandPayload], PydanticModel): 244 """The target which creates physical tables for the given set of new snapshots.""" 245 246 command_type: commands.CommandType = commands.CommandType.CREATE_TABLES 247 command_handler: t.Callable[ 248 [SnapshotEvaluator, commands.CreateTablesCommandPayload], None 249 ] = commands.create_tables 250 251 new_snapshots: t.List[Snapshot] 252 ddl_concurrent_tasks: int 253 254 def _get_command_payload( 255 self, context: Context 256 ) -> t.Optional[commands.CreateTablesCommandPayload]: 257 if not self.new_snapshots: 258 return None 259 260 parent_snapshot_ids = { 261 p_sid for snapshot in self.new_snapshots for p_sid in snapshot.parents 262 } 263 stored_snapshots = self._get_stored_snapshots(parent_snapshot_ids) 264 265 return commands.CreateTablesCommandPayload( 266 target_snapshot_ids=[s.snapshot_id for s in self.new_snapshots], 267 snapshots=stored_snapshots + self.new_snapshots, 268 ) 269 270 def _get_stored_snapshots(self, snapshot_ids: t.Set[SnapshotId]) -> t.List[Snapshot]: 271 with util.scoped_state_sync() as state_sync: 272 return list(state_sync.get_snapshots(snapshot_ids).values())
The target which creates physical tables for the given set of new snapshots.
Inherited Members
- pydantic.main.BaseModel
- BaseModel
- parse_obj
- parse_raw
- parse_file
- from_orm
- construct
- copy
- schema
- schema_json
- validate
- update_forward_refs
275class SnapshotMigrateTablesTarget(BaseTarget[commands.MigrateTablesCommandPayload], PydanticModel): 276 """The target which updates schemas of existing physical tables to bring them in correspondance 277 with schemas of target snapshots. 278 """ 279 280 command_type: commands.CommandType = commands.CommandType.MIGRATE_TABLES 281 command_handler: t.Callable[ 282 [SnapshotEvaluator, commands.MigrateTablesCommandPayload], None 283 ] = commands.migrate_tables 284 285 snapshots: t.List[SnapshotTableInfo] 286 ddl_concurrent_tasks: int 287 288 def _get_command_payload( 289 self, context: Context 290 ) -> t.Optional[commands.MigrateTablesCommandPayload]: 291 if not self.snapshots: 292 return None 293 return commands.MigrateTablesCommandPayload(snapshots=self.snapshots)
The target which updates schemas of existing physical tables to bring them in correspondance with schemas of target snapshots.
Inherited Members
- pydantic.main.BaseModel
- BaseModel
- parse_obj
- parse_raw
- parse_file
- from_orm
- construct
- copy
- schema
- schema_json
- validate
- update_forward_refs