Thanks to visit codestin.com
Credit goes to github.com

Skip to content

Share one poll loop across sibling event triggers#66584

Open
Lee-W wants to merge 9 commits into
apache:mainfrom
astronomer:feat/asset-watcher-match
Open

Share one poll loop across sibling event triggers#66584
Lee-W wants to merge 9 commits into
apache:mainfrom
astronomer:feat/asset-watcher-match

Conversation

@Lee-W
Copy link
Copy Markdown
Member

@Lee-W Lee-W commented May 8, 2026

Why

When multiple AssetWatcher instances back triggers that read from the same upstream resource — one SQS queue, one Kafka topic, one directory — today's triggerer spins up N independent poll loops, one per trigger. For directory scans and similar idempotent / read-only sources this is wasted I/O and load on the upstream.

What

  • New opt-in path on BaseEventTrigger:
    • shared_stream_key() — return a hashable; triggers with equal keys share one poll loop. Returning None (default) keeps the existing run() loop unchanged.
    • @classmethod open_shared_stream(kwargs) — runs once per group; yields raw events for the lifetime of the group, or raises. Declared as a classmethod (not staticmethod) so subclasses can chain via super().open_shared_stream(kwargs).
    • filter_shared_stream(shared_stream) — per-subscriber; converts raw events into TriggerEvents.
  • New module airflow.triggers.shared_stream with SharedStreamManager: one poll task per distinct key, broadcasts to per-subscriber bounded queues, evicts groups synchronously before any await to close lifecycle race windows. The manager is single-event-loop and not thread-safe; TriggerRunner is its sole owner.
  • Triggerer integration in TriggerRunner.run_trigger: triggers with a non-None key route through the manager; stop_all() runs in shutdown as a safety net.
  • Reference provider trigger: DirectoryFileDeleteTrigger in the standard provider — sibling triggers watching the same directory share a single scan.
  • New config option [triggerer] shared_stream_subscriber_queue_size (default 1024) — per-subscriber buffer size; a slow subscriber that overruns its buffer fails loudly with _SubscriberOverflow rather than dropping events silently. Sibling subscribers are unaffected.

Scope

This PR targets idempotent / read-only / subscriber-side-effect upstreams only — directory scans, polling REST APIs, Kafka with enable.auto.commit=true, sources where per-event cleanup lives on the subscriber.

Manual-commit Kafka, SQS delete-on-process, Pub/Sub ack_id, and Service Bus peek-lock are out of scope — they need a producer-side ack channel from each subscriber's accept/reject decision back to the producer's handle, which filter_shared_stream does not provide today. A follow-up issue will track that work.

closes: #66476


Was generative AI tooling used to co-author this PR?
  • Yes (please specify the tool below)

Generated-by: [Claude] following the guidelines


  • Read the Pull Request Guidelines for more information. Note: commit author/co-author name and email in commits become permanently public when merged.
  • For fundamental code changes, an Airflow Improvement Proposal (AIP) is needed.
  • When adding dependency, check compliance with the ASF 3rd Party License Policy.
  • For significant user-facing changes create newsfragment: {pr_number}.significant.rst, in airflow-core/newsfragments. You can add this file in a follow-up commit after the PR is created so you know the PR number.

@Lee-W Lee-W force-pushed the feat/asset-watcher-match branch from cb5ecf0 to 0406c1f Compare May 8, 2026 13:47
@Lee-W Lee-W changed the title feat(asset): add match filter to AssetWatcher feat(triggerer): share one poll across sibling event triggers May 8, 2026
@Lee-W Lee-W force-pushed the feat/asset-watcher-match branch 4 times, most recently from dfdf876 to ac871a6 Compare May 12, 2026 12:02
@Lee-W Lee-W marked this pull request as ready for review May 12, 2026 12:03
@Lee-W Lee-W force-pushed the feat/asset-watcher-match branch 5 times, most recently from 51a57b1 to 0163983 Compare May 14, 2026 04:03
Comment thread airflow-core/src/airflow/triggers/shared_stream.py Outdated
@Lee-W Lee-W force-pushed the feat/asset-watcher-match branch from 4ba609f to 0a5af27 Compare May 19, 2026 11:01
@Lee-W Lee-W changed the title feat(triggerer): share one poll across sibling event triggers Share one poll loop across sibling event triggers May 19, 2026
@Lee-W Lee-W force-pushed the feat/asset-watcher-match branch 2 times, most recently from 59c79e5 to c2f2da8 Compare May 19, 2026 12:30
@Lee-W Lee-W marked this pull request as ready for review May 19, 2026 12:54
Comment thread airflow-core/src/airflow/jobs/triggerer_job_runner.py Outdated
Comment thread airflow-core/src/airflow/triggers/shared_stream.py Outdated
Comment thread providers/standard/src/airflow/providers/standard/triggers/file.py Outdated
Comment thread providers/standard/src/airflow/providers/standard/triggers/file.py
@Lee-W Lee-W force-pushed the feat/asset-watcher-match branch from c2f2da8 to 6a377f9 Compare May 20, 2026 06:04
Copy link
Copy Markdown
Member

@uranusjr uranusjr left a comment

Choose a reason for hiding this comment

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

Not sure with the parameter name, but I don’t have a concrete opinion.

@Lee-W Lee-W force-pushed the feat/asset-watcher-match branch 7 times, most recently from 236ee58 to d7e9549 Compare May 22, 2026 09:49
Comment thread airflow-core/docs/authoring-and-scheduling/event-scheduling.rst
Comment thread airflow-core/docs/authoring-and-scheduling/event-scheduling.rst
return None

@classmethod
async def open_shared_stream(cls, kwargs: dict[str, Any]) -> AsyncIterator[Any]:
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Not sure if it's a good idea to introduce method-level generic so that user could define TypedDict or internal data model themself that shared for open_shared_stream and filter_shared_stream.

e.g. async def open_shared_stream(cls, kwargs: dict[str, Any]) -> AsyncIterator[MyType]:

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

The raw stream is a private contract between open_shared_stream and filter_shared_streamT never crosses an API boundary, so the TypeVar would serve only as documentation for the subclass author.
I'd rather skip it unless users ask for it, which I doubt anyone would ever do.

Comment thread airflow-core/src/airflow/triggers/shared_stream.py Outdated
"""


class _PollTerminated(Exception):
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Do these exception need to inherent from AirflowException?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

_PollTerminated and _SubscriberOverflow are internal sentinels — they never escape SharedStreamManager and the trigger-failure path catches Exception regardless of base.

I'd rather not add more AirflowException subclasses for purely internal types.

Comment thread airflow-core/src/airflow/triggers/shared_stream.py
Lee-W added 8 commits May 23, 2026 13:57
When several AssetWatcher instances back triggers that read from the
same upstream resource (one SQS queue, one Kafka topic, etc.), the
triggerer spins up N independent poll loops today — one per trigger.
Issue apache#66476 asks for one shared poller serving all of them.

Add an opt-in path on BaseEventTrigger via three new hooks
(`shared_stream_key`, `open_shared_stream`, `filter_shared_stream`)
and a new SharedStreamManager that runs one poll task per distinct
key and broadcasts events to per-subscriber queues. The key is read
once when run_trigger starts and identifies the group for the
trigger's lifetime. Per-trigger cleanup runs in run_trigger's
finally; SharedStreamManager.stop_all() runs in the triggerer's
shutdown path as a safety net. Triggers whose shared_stream_key()
returns None (the default) keep their existing run() loop unchanged.

The per-subscriber buffer size is exposed as
[triggerer] shared_stream_subscriber_queue_size (default 1024) so
deployments with a fast upstream can raise it without code changes.
- triggerer: compute shared_stream_key after render_template_fields so
  templated attributes resolve before keying
- shared_stream: extract _drain_and_offer_failure helper; reuse from
  terminal broadcast and overflow paths
- DirectoryFileDeleteTrigger: normalise directory via realpath so
  relative/absolute/symlink/trailing-slash variants share one scan
- DirectoryFileDeleteTrigger.open_shared_stream: raise on
  PermissionError / NotADirectoryError / IsADirectoryError so config
  bugs surface in the UI instead of silently spinning a warning loop;
  keep swallow + retry for the rest of OSError (transient I/O)
Round-3 doc cleanup for jason's review (C1/C2/C6):
- Drop Kafka/SQS recommendations from event-scheduling.rst and BaseEventTrigger class docstring; the producer-side ack channel is out of scope this iteration.
- Document the deterministic-key requirement on shared_stream_key and add a Slow-subscriber overflow mitigations section (raise subscriber queue size, redesign the key to narrow groups).
Address jason's C4 suggestion: collapse the get-and-None-check on SharedStreamManager.subscribe into a single walrus expression.
@Lee-W Lee-W force-pushed the feat/asset-watcher-match branch from d7e9549 to 3ddd18a Compare May 23, 2026 05:58
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Fan In: Single poller for shared-queue Asset Watchers

6 participants