Skip to content

Commit

Permalink
Implement Asset.ref for name or URI references
Browse files Browse the repository at this point in the history
This allows us to refer to an asset without needing the original object,
making it easier to schedule against an asset.
  • Loading branch information
uranusjr committed Dec 18, 2024
1 parent 1ce34cc commit 15cffbc
Show file tree
Hide file tree
Showing 20 changed files with 2,274 additions and 1,719 deletions.
2 changes: 1 addition & 1 deletion airflow/api_connexion/endpoints/asset_endpoint.py
Original file line number Diff line number Diff line change
Expand Up @@ -349,7 +349,7 @@ def create_asset_event(session: Session = NEW_SESSION) -> APIResponse:
extra = json_body.get("extra", {})
extra["from_rest_api"] = True
asset_event = asset_manager.register_asset_change(
asset=asset_model.to_public(),
asset=asset_model,
timestamp=timestamp,
extra=extra,
session=session,
Expand Down
2 changes: 1 addition & 1 deletion airflow/api_fastapi/core_api/routes/public/assets.py
Original file line number Diff line number Diff line change
Expand Up @@ -232,7 +232,7 @@ def create_asset_event(
timestamp = timezone.utcnow()

assets_event = asset_manager.register_asset_change(
asset=asset_model.to_public(),
asset=asset_model,
timestamp=timestamp,
extra=body.extra,
session=session,
Expand Down
33 changes: 27 additions & 6 deletions airflow/assets/manager.py
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@
from collections.abc import Collection, Iterable
from typing import TYPE_CHECKING

from sqlalchemy import exc, select
from sqlalchemy import exc, or_, select
from sqlalchemy.orm import joinedload

from airflow.configuration import conf
Expand All @@ -31,7 +31,9 @@
AssetEvent,
AssetModel,
DagScheduleAssetAliasReference,
DagScheduleAssetNameReference,
DagScheduleAssetReference,
DagScheduleAssetUriReference,
)
from airflow.models.dagbag import DagPriorityParsingRequest
from airflow.stats import Stats
Expand All @@ -42,7 +44,7 @@

from airflow.models.dag import DagModel
from airflow.models.taskinstance import TaskInstance
from airflow.sdk.definitions.asset import Asset, AssetAlias
from airflow.sdk.definitions.asset import Asset, AssetAlias, AssetUniqueKey


class AssetManager(LoggingMixin):
Expand Down Expand Up @@ -106,7 +108,7 @@ def register_asset_change(
cls,
*,
task_instance: TaskInstance | None = None,
asset: Asset,
asset: Asset | AssetModel | AssetUniqueKey,
extra=None,
aliases: Collection[AssetAlias] = (),
source_alias_names: Iterable[str] | None = None,
Expand All @@ -119,7 +121,9 @@ def register_asset_change(
For local assets, look them up, record the asset event, queue dagruns, and broadcast
the asset event
"""
asset_model = session.scalar(
from airflow.models.dag import DagModel

asset_model: AssetModel | None = session.scalar(
select(AssetModel)
.where(AssetModel.name == asset.name, AssetModel.uri == asset.uri)
.options(
Expand Down Expand Up @@ -154,6 +158,7 @@ def register_asset_change(
dags_to_queue_from_asset = {
ref.dag for ref in asset_model.consuming_dags if ref.dag.is_active and not ref.dag.is_paused
}

dags_to_queue_from_asset_alias = set()
if source_alias_names:
asset_alias_models = session.scalars(
Expand All @@ -174,11 +179,27 @@ def register_asset_change(
if alias_ref.dag.is_active and not alias_ref.dag.is_paused
}

cls.notify_asset_changed(asset=asset)
dags_to_queue_from_asset_ref = set(
session.scalars(
select(DagModel)
.join(DagModel.schedule_asset_name_references, isouter=True)
.join(DagModel.schedule_asset_uri_references, isouter=True)
.where(
or_(
DagScheduleAssetNameReference.name == asset.name,
DagScheduleAssetUriReference.uri == asset.uri,
)
)
)
)

cls.notify_asset_changed(asset=asset_model.to_public())

Stats.incr("asset.updates")

dags_to_queue = dags_to_queue_from_asset | dags_to_queue_from_asset_alias
dags_to_queue = (
dags_to_queue_from_asset | dags_to_queue_from_asset_alias | dags_to_queue_from_asset_ref
)
cls._queue_dagruns(asset_id=asset_model.id, dags_to_queue=dags_to_queue, session=session)
return asset_event

Expand Down
71 changes: 69 additions & 2 deletions airflow/dag_processing/collection.py
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@
import traceback
from typing import TYPE_CHECKING, NamedTuple

from sqlalchemy import and_, delete, exists, func, select, tuple_
from sqlalchemy import and_, delete, exists, func, insert, select, tuple_
from sqlalchemy.exc import OperationalError
from sqlalchemy.orm import joinedload, load_only

Expand All @@ -42,15 +42,17 @@
AssetAliasModel,
AssetModel,
DagScheduleAssetAliasReference,
DagScheduleAssetNameReference,
DagScheduleAssetReference,
DagScheduleAssetUriReference,
TaskOutletAssetReference,
)
from airflow.models.dag import DAG, DagModel, DagOwnerAttributes, DagTag
from airflow.models.dagrun import DagRun
from airflow.models.dagwarning import DagWarningType
from airflow.models.errors import ParseImportError
from airflow.models.trigger import Trigger
from airflow.sdk.definitions.asset import Asset, AssetAlias
from airflow.sdk.definitions.asset import Asset, AssetAlias, AssetNameRef, AssetUriRef
from airflow.triggers.base import BaseTrigger
from airflow.utils.retries import MAX_DB_RETRIES, run_with_db_retries
from airflow.utils.sqlalchemy import with_row_locks
Expand Down Expand Up @@ -495,6 +497,8 @@ class AssetModelOperation(NamedTuple):

schedule_asset_references: dict[str, list[Asset]]
schedule_asset_alias_references: dict[str, list[AssetAlias]]
schedule_asset_name_references: set[tuple[str, str]] # dag_id, ref_name.
schedule_asset_uri_references: set[tuple[str, str]] # dag_id, ref_uri.
outlet_references: dict[str, list[tuple[str, Asset]]]
assets: dict[tuple[str, str], Asset]
asset_aliases: dict[str, AssetAlias]
Expand All @@ -510,6 +514,18 @@ def collect(cls, dags: dict[str, DAG]) -> Self:
dag_id: [alias for _, alias in dag.timetable.asset_condition.iter_asset_aliases()]
for dag_id, dag in dags.items()
},
schedule_asset_name_references={
(dag_id, ref.name)
for dag_id, dag in dags.items()
for ref in dag.timetable.asset_condition.iter_asset_refs()
if isinstance(ref, AssetNameRef)
},
schedule_asset_uri_references={
(dag_id, ref.uri)
for dag_id, dag in dags.items()
for ref in dag.timetable.asset_condition.iter_asset_refs()
if isinstance(ref, AssetUriRef)
},
outlet_references={
dag_id: [
(task_id, outlet)
Expand Down Expand Up @@ -614,6 +630,57 @@ def add_dag_asset_alias_references(
if alias_id not in orm_refs
)

def add_dag_asset_name_uri_references(self, *, session: Session) -> None:
orm_name_refs = set(
session.scalars(
select(DagScheduleAssetNameReference.dag_id, DagScheduleAssetNameReference.name).where(
DagScheduleAssetNameReference.dag_id.in_(
dag_id for dag_id, _ in self.schedule_asset_name_references
)
)
)
)
new_name_refs = self.schedule_asset_name_references - orm_name_refs
old_name_refs = orm_name_refs - self.schedule_asset_name_references
if old_name_refs:
session.execute(
delete(DagScheduleAssetNameReference).where(
tuple_(DagScheduleAssetNameReference.dag_id, DagScheduleAssetNameReference.name).in_(
old_name_refs
)
)
)
if new_name_refs:
session.execute(
insert(DagScheduleAssetNameReference),
[{"dag_id": d, "name": n} for d, n in new_name_refs],
)

orm_uri_refs = set(
session.scalars(
select(DagScheduleAssetUriReference.dag_id, DagScheduleAssetUriReference.uri).where(
DagScheduleAssetUriReference.dag_id.in_(
dag_id for dag_id, _ in self.schedule_asset_uri_references
)
)
)
)
new_uri_refs = self.schedule_asset_uri_references - orm_uri_refs
old_uri_refs = orm_uri_refs - self.schedule_asset_uri_references
if old_uri_refs:
session.execute(
delete(DagScheduleAssetUriReference).where(
tuple_(DagScheduleAssetUriReference.dag_id, DagScheduleAssetUriReference.uri).in_(
old_uri_refs
)
)
)
if new_uri_refs:
session.execute(
insert(DagScheduleAssetUriReference),
[{"dag_id": d, "uri": u} for d, u in new_uri_refs],
)

def add_task_asset_references(
self,
dags: dict[str, DagModel],
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,93 @@
#
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.

"""
Add asset reference models.
Revision ID: 38770795785f
Revises: 038dc8bc6284
Create Date: 2024-12-18 11:12:50.639369
"""

from __future__ import annotations

import sqlalchemy as sa
from alembic import op

from airflow.migrations.db_types import StringID
from airflow.utils.sqlalchemy import UtcDateTime

# revision identifiers, used by Alembic.
revision = "38770795785f"
down_revision = "038dc8bc6284"
branch_labels = None
depends_on = None
airflow_version = "3.0.0"

ASSET_STR_FIELD = sa.String(length=1500).with_variant(
sa.String(length=1500, collation="latin1_general_cs"), "mysql"
)


def upgrade():
"""Add asset reference models."""
op.create_table(
"dag_schedule_asset_name_reference",
sa.Column("name", ASSET_STR_FIELD, primary_key=True, nullable=False),
sa.Column("dag_id", StringID(), primary_key=True, nullable=False),
sa.Column("created_at", UtcDateTime(timezone=True), nullable=False),
sa.PrimaryKeyConstraint("name", "dag_id", name="dsanr_pkey"),
sa.ForeignKeyConstraint(
columns=("dag_id",),
refcolumns=["dag.dag_id"],
name="dsanr_dag_id_fkey",
ondelete="CASCADE",
),
)
op.create_index(
"idx_dag_schedule_asset_name_reference_dag_id",
"dag_schedule_asset_name_reference",
["dag_id"],
unique=False,
)

op.create_table(
"dag_schedule_asset_uri_reference",
sa.Column("uri", ASSET_STR_FIELD, primary_key=True, nullable=False),
sa.Column("dag_id", StringID(), primary_key=True, nullable=False),
sa.Column("created_at", UtcDateTime(timezone=True), nullable=False),
sa.PrimaryKeyConstraint("uri", "dag_id", name="dsaur_pkey"),
sa.ForeignKeyConstraint(
columns=("dag_id",),
refcolumns=["dag.dag_id"],
name="dsaur_dag_id_fkey",
ondelete="CASCADE",
),
)
op.create_index(
"idx_dag_schedule_asset_uri_reference_dag_id",
"dag_schedule_asset_uri_reference",
["dag_id"],
unique=False,
)


def downgrade():
"""Unadd asset reference models."""
op.drop_table("dag_schedule_asset_name_reference")
op.drop_table("dag_schedule_asset_uri_reference")
Loading

0 comments on commit 15cffbc

Please sign in to comment.