Skip to content
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
3 changes: 3 additions & 0 deletions airflow-core/src/airflow/executors/workloads/trigger.py
Original file line number Diff line number Diff line change
Expand Up @@ -46,3 +46,6 @@ class RunTrigger(BaseModel):
dag_run_data: dict | None = (
None # Serialized DagRun data in dict format so it can be deserialized in trigger subprocess.
)

# name: uri of all "watched" Assets
watched_assets: dict[str, str] | None = None # Set for BaseEventTrigger asset watchers only
17 changes: 17 additions & 0 deletions airflow-core/src/airflow/jobs/triggerer_job_runner.py
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The _handle_request should be handling this in here?

Copy link
Copy Markdown
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What do you mean?

Original file line number Diff line number Diff line change
Expand Up @@ -741,11 +741,18 @@ def _create_workload(
render_log_fname: Callable[..., str],
session: Session,
) -> workloads.RunTrigger | None:
# Pass the "watched" Assets through for downstream use in BaseEventTrigger
if trigger.task_instance is None:
watched_assets: dict[str, str] | None = None

if trigger.assets:
watched_assets = {a.name: a.uri for a in trigger.assets}

return workloads.RunTrigger(
id=trigger.id,
classpath=trigger.classpath,
encrypted_kwargs=trigger.encrypted_kwargs,
watched_assets=watched_assets,
)

if not trigger.task_instance.dag_version_id:
Expand Down Expand Up @@ -1266,6 +1273,16 @@ async def create_triggers(self):
trigger_instance.triggerer_job_id = self.job_id
trigger_instance.timeout_after = workload.timeout_after

if isinstance(trigger_instance, BaseEventTrigger) and workload.watched_assets:
# Reconstruct AssetStateAccessors from watched_assets
from airflow.sdk.definitions.asset import Asset
from airflow.sdk.execution_time.context import AssetStoreAccessors

# Potentially address Asset vs. AssetRef, AssetUriRef, etc.
trigger_instance.asset_store = AssetStoreAccessors(
inlets=[Asset(name=name, uri=uri) for name, uri in workload.watched_assets.items()]
)

self.triggers[trigger_id] = {
"task": asyncio.create_task(
self.run_trigger(trigger_id, trigger_instance, workload.timeout_after, context),
Expand Down
7 changes: 7 additions & 0 deletions airflow-core/src/airflow/triggers/base.py
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@
from airflow.models.mappedoperator import MappedOperator
from airflow.models.taskinstance import TaskInstance
from airflow.sdk.definitions.context import Context
from airflow.sdk.execution_time.context import AssetStoreAccessors
from airflow.serialization.serialized_objects import SerializedBaseOperator

Operator: TypeAlias = MappedOperator | SerializedBaseOperator
Expand Down Expand Up @@ -297,6 +298,12 @@ class BaseEventTrigger(BaseTrigger):

supports_triggerer_queue: bool = False

def __init__(self, **kwargs):
super().__init__(**kwargs)

# Injected by the triggerer before run() is called; mirrors how trigger_id is set
self.asset_store: AssetStoreAccessors | None = None

@staticmethod
def hash(classpath: str, kwargs: dict[str, Any]) -> int:
"""
Expand Down
Loading