-
Notifications
You must be signed in to change notification settings - Fork 17k
AIP-76: Consume task-emitted partition keys on asset events #66782
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Open
anishgirianish
wants to merge
1
commit into
apache:main
Choose a base branch
from
anishgirianish:aip-76-partition-at-runtime-server-consumer
base: main
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
+203
−38
Open
Changes from all commits
Commits
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change | ||||||||||||||
|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|
|
|
@@ -1476,10 +1476,20 @@ def register_asset_changes_in_db( | |||||||||||||||
| SerializedAssetUriRef, | ||||||||||||||||
| ) | ||||||||||||||||
|
|
||||||||||||||||
| # TODO: AIP-76 should we provide an interface to override this, so that the task can | ||||||||||||||||
| # tell the truth if for some reason it touches a different partition? | ||||||||||||||||
| # https://github.com/apache/airflow/issues/58474 | ||||||||||||||||
| partition_key = ti.dag_run.partition_key | ||||||||||||||||
| events_by_asset: dict[SerializedAssetUniqueKey, list[tuple[dict, str | None]]] = defaultdict(list) | ||||||||||||||||
| for outlet_event in outlet_events: | ||||||||||||||||
| if "source_alias_name" in outlet_event: | ||||||||||||||||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Suggested change
|
||||||||||||||||
| continue | ||||||||||||||||
| asset_key = SerializedAssetUniqueKey(**outlet_event["dest_asset_key"]) | ||||||||||||||||
| events_by_asset[asset_key].append((outlet_event["extra"], outlet_event.get("partition_key"))) | ||||||||||||||||
|
|
||||||||||||||||
| runtime_pks: set[str] = { | ||||||||||||||||
| pk for events in events_by_asset.values() for _, pk in events if pk is not None | ||||||||||||||||
| } | ||||||||||||||||
| if len(runtime_pks) == 1 and ti.dag_run.partition_key is None: | ||||||||||||||||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. let's also add a comment here to explain what we're doing |
||||||||||||||||
| ti.dag_run.partition_key = next(iter(runtime_pks)) | ||||||||||||||||
| dag_run_partition_key = ti.dag_run.partition_key | ||||||||||||||||
|
|
||||||||||||||||
| asset_keys = { | ||||||||||||||||
| SerializedAssetUniqueKey(o.name, o.uri) | ||||||||||||||||
| for o in task_outlets | ||||||||||||||||
|
|
@@ -1506,11 +1516,25 @@ def register_asset_changes_in_db( | |||||||||||||||
| ) | ||||||||||||||||
| } | ||||||||||||||||
|
|
||||||||||||||||
| asset_event_extras: dict[SerializedAssetUniqueKey, dict] = { | ||||||||||||||||
| SerializedAssetUniqueKey(**event["dest_asset_key"]): event["extra"] | ||||||||||||||||
| for event in outlet_events | ||||||||||||||||
| if "source_alias_name" not in event | ||||||||||||||||
| } | ||||||||||||||||
| def _register(am: AssetModel, key: SerializedAssetUniqueKey) -> None: | ||||||||||||||||
| events_for_asset = events_by_asset.get(key, []) | ||||||||||||||||
| if not events_for_asset: | ||||||||||||||||
| asset_manager.register_asset_change( | ||||||||||||||||
| task_instance=ti, | ||||||||||||||||
| asset=am, | ||||||||||||||||
| extra=None, | ||||||||||||||||
| partition_key=dag_run_partition_key, | ||||||||||||||||
| session=session, | ||||||||||||||||
| ) | ||||||||||||||||
| return | ||||||||||||||||
| for extra, event_pk in events_for_asset: | ||||||||||||||||
| asset_manager.register_asset_change( | ||||||||||||||||
| task_instance=ti, | ||||||||||||||||
| asset=am, | ||||||||||||||||
| extra=extra, | ||||||||||||||||
| partition_key=event_pk if event_pk is not None else dag_run_partition_key, | ||||||||||||||||
| session=session, | ||||||||||||||||
| ) | ||||||||||||||||
|
|
||||||||||||||||
| for key in asset_keys: | ||||||||||||||||
| try: | ||||||||||||||||
|
|
@@ -1523,52 +1547,36 @@ def register_asset_changes_in_db( | |||||||||||||||
| ) | ||||||||||||||||
| continue | ||||||||||||||||
| ti.log.debug("register event for asset %s", am) | ||||||||||||||||
| asset_manager.register_asset_change( | ||||||||||||||||
| task_instance=ti, | ||||||||||||||||
| asset=am, | ||||||||||||||||
| extra=asset_event_extras.get(key), | ||||||||||||||||
| partition_key=partition_key, | ||||||||||||||||
| session=session, | ||||||||||||||||
| ) | ||||||||||||||||
| _register(am, key) | ||||||||||||||||
|
|
||||||||||||||||
| if asset_name_refs: | ||||||||||||||||
| asset_models_by_name = {key.name: am for key, am in asset_models.items()} | ||||||||||||||||
| asset_event_extras_by_name = {key.name: extra for key, extra in asset_event_extras.items()} | ||||||||||||||||
| asset_models_by_name: dict[str, tuple[SerializedAssetUniqueKey, AssetModel]] = { | ||||||||||||||||
| key.name: (key, am) for key, am in asset_models.items() | ||||||||||||||||
| } | ||||||||||||||||
| for nref in asset_name_refs: | ||||||||||||||||
| try: | ||||||||||||||||
| am = asset_models_by_name[nref.name] | ||||||||||||||||
| key, am = asset_models_by_name[nref.name] | ||||||||||||||||
| except KeyError: | ||||||||||||||||
| ti.log.warning( | ||||||||||||||||
| 'Task has inactive assets "Asset.ref(name=%s)" in inlets or outlets', nref.name | ||||||||||||||||
| ) | ||||||||||||||||
| continue | ||||||||||||||||
| ti.log.debug("register event for asset name ref %s", am) | ||||||||||||||||
| asset_manager.register_asset_change( | ||||||||||||||||
| task_instance=ti, | ||||||||||||||||
| asset=am, | ||||||||||||||||
| extra=asset_event_extras_by_name.get(nref.name), | ||||||||||||||||
| partition_key=partition_key, | ||||||||||||||||
| session=session, | ||||||||||||||||
| ) | ||||||||||||||||
| _register(am, key) | ||||||||||||||||
| if asset_uri_refs: | ||||||||||||||||
| asset_models_by_uri = {key.uri: am for key, am in asset_models.items()} | ||||||||||||||||
| asset_event_extras_by_uri = {key.uri: extra for key, extra in asset_event_extras.items()} | ||||||||||||||||
| asset_models_by_uri: dict[str, tuple[SerializedAssetUniqueKey, AssetModel]] = { | ||||||||||||||||
| key.uri: (key, am) for key, am in asset_models.items() | ||||||||||||||||
| } | ||||||||||||||||
| for uref in asset_uri_refs: | ||||||||||||||||
| try: | ||||||||||||||||
| am = asset_models_by_uri[uref.uri] | ||||||||||||||||
| key, am = asset_models_by_uri[uref.uri] | ||||||||||||||||
| except KeyError: | ||||||||||||||||
| ti.log.warning( | ||||||||||||||||
| 'Task has inactive assets "Asset.ref(uri=%s)" in inlets or outlets', uref.uri | ||||||||||||||||
| ) | ||||||||||||||||
| continue | ||||||||||||||||
| ti.log.debug("register event for asset uri ref %s", am) | ||||||||||||||||
| asset_manager.register_asset_change( | ||||||||||||||||
| task_instance=ti, | ||||||||||||||||
| asset=am, | ||||||||||||||||
| extra=asset_event_extras_by_uri.get(uref.uri), | ||||||||||||||||
| partition_key=partition_key, | ||||||||||||||||
| session=session, | ||||||||||||||||
| ) | ||||||||||||||||
| _register(am, key) | ||||||||||||||||
|
|
||||||||||||||||
| def _asset_event_extras_from_aliases() -> dict[tuple[SerializedAssetUniqueKey, str, str], set[str]]: | ||||||||||||||||
| d = defaultdict(set) | ||||||||||||||||
|
|
@@ -1607,7 +1615,7 @@ def _asset_event_extras_from_aliases() -> dict[tuple[SerializedAssetUniqueKey, s | |||||||||||||||
| asset=asset, | ||||||||||||||||
| source_alias_names=event_aliase_names, | ||||||||||||||||
| extra=asset_event_extra, | ||||||||||||||||
| partition_key=partition_key, | ||||||||||||||||
| partition_key=dag_run_partition_key, | ||||||||||||||||
| session=session, | ||||||||||||||||
| ) | ||||||||||||||||
| if event is None: | ||||||||||||||||
|
|
@@ -1619,7 +1627,7 @@ def _asset_event_extras_from_aliases() -> dict[tuple[SerializedAssetUniqueKey, s | |||||||||||||||
| asset=asset, | ||||||||||||||||
| source_alias_names=event_aliase_names, | ||||||||||||||||
| extra=asset_event_extra, | ||||||||||||||||
| partition_key=partition_key, | ||||||||||||||||
| partition_key=dag_run_partition_key, | ||||||||||||||||
| session=session, | ||||||||||||||||
| ) | ||||||||||||||||
|
|
||||||||||||||||
|
|
||||||||||||||||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
it would be easier to read this way