-
Notifications
You must be signed in to change notification settings - Fork 16.8k
Add "@asset" to decorate a function as a DAG and an asset #41325
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
Merged
Merged
Changes from all commits
Commits
Show all changes
29 commits
Select commit
Hold shift + click to select a range
b707d72
Implement asset definition creating a DAG
uranusjr f26e64f
Basic inlet dependency
uranusjr 80b9873
Make AssetDefinition subclass Asset
uranusjr 6d33cc1
style: fix mypy error
Lee-W c8eba16
feat(asset): allow uri to be None
Lee-W c5d1ddc
fix: temporarily serialize AssetDefintion into a string
Lee-W 34d2b16
feat(decorators/assets): rewrite how asset definition is serialized
Lee-W c7e82b9
test(decorators/assets): add test cases to check whether asset decora…
Lee-W 7cc485b
test(decorators/assets): add test cases to AssetDefinition
Lee-W 1d31d4d
test(decorators/asset): add test cases to Test_AssetMainOperator
Lee-W 5e933f4
test(decorators/assets): remove unused fixtures
Lee-W c69045a
docs(example_dag): add example dag for asset_decorator
Lee-W 6c0ec47
feat(decorators/assets): allow passing self and context into asset
Lee-W ad0c6d2
feat(decorators/assets): return actual asset in asset decorator
Lee-W 290461a
refactor(decorators/assets): extract active assets fetching logic as …
Lee-W 00ecd55
feat(decorators/assets): allow fethcing inlet events through AssetRef
Lee-W 574b602
feat(decorators/assets): reorder import paths
Lee-W 8728714
docs: update asset decorator example dag
Lee-W 6c2539a
test: fix tests
Lee-W d42ce8c
test(decorators/assets): extend test_determine_kwargs to cover active…
Lee-W 17fce69
fix: address easy to fix comments
Lee-W de73b46
fix: fix asset serialization
Lee-W 0b76c81
refactor(decorators/assets): postpone the attribute check to AssetDef…
Lee-W 69523dc
Simplify group validators
uranusjr 81ee77b
style(dag): remove _wrapped_definition
Lee-W 7ccc9ab
style(decorators/assets): change types.FunctionType to Callable
Lee-W 52c8b85
refactor(decorators/assets): make session in _fetch_active_assets_by_…
Lee-W 7e4ecf8
fix(decorators/asets): remove DAG.bulk_write_to_db and remove self ha…
Lee-W 2060ead
feat(utils/context): fetch asset_refs all at once
Lee-W 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
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 |
|---|---|---|
| @@ -0,0 +1,131 @@ | ||
| # 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. | ||
|
|
||
| from __future__ import annotations | ||
|
|
||
| import inspect | ||
| from typing import TYPE_CHECKING, Any, Callable, Iterator, Mapping | ||
|
|
||
| import attrs | ||
|
|
||
| from airflow.assets import Asset, AssetRef | ||
| from airflow.models.asset import _fetch_active_assets_by_name | ||
| from airflow.models.dag import DAG, ScheduleArg | ||
| from airflow.providers.standard.operators.python import PythonOperator | ||
| from airflow.utils.session import create_session | ||
|
|
||
| if TYPE_CHECKING: | ||
| from airflow.io.path import ObjectStoragePath | ||
|
|
||
|
|
||
| class _AssetMainOperator(PythonOperator): | ||
| def __init__(self, *, definition_name: str, uri: str | None = None, **kwargs) -> None: | ||
| super().__init__(**kwargs) | ||
| self._definition_name = definition_name | ||
| self._uri = uri | ||
|
|
||
| def _iter_kwargs( | ||
| self, context: Mapping[str, Any], active_assets: dict[str, Asset] | ||
| ) -> Iterator[tuple[str, Any]]: | ||
| value: Any | ||
| for key in inspect.signature(self.python_callable).parameters: | ||
| if key == "self": | ||
| value = active_assets.get(self._definition_name) | ||
| elif key == "context": | ||
| value = context | ||
| else: | ||
| value = active_assets.get(key, Asset(name=key)) | ||
| yield key, value | ||
|
|
||
| def determine_kwargs(self, context: Mapping[str, Any]) -> Mapping[str, Any]: | ||
| active_assets: dict[str, Asset] = {} | ||
| asset_names = [asset_ref.name for asset_ref in self.inlets if isinstance(asset_ref, AssetRef)] | ||
| if "self" in inspect.signature(self.python_callable).parameters: | ||
| asset_names.append(self._definition_name) | ||
|
|
||
| if asset_names: | ||
| with create_session() as session: | ||
| active_assets = _fetch_active_assets_by_name(asset_names, session) | ||
| return dict(self._iter_kwargs(context, active_assets)) | ||
|
|
||
|
|
||
| @attrs.define(kw_only=True) | ||
| class AssetDefinition(Asset): | ||
| """ | ||
| Asset representation from decorating a function with ``@asset``. | ||
|
|
||
| :meta private: | ||
| """ | ||
|
|
||
| function: Callable | ||
| schedule: ScheduleArg | ||
|
|
||
| def __attrs_post_init__(self) -> None: | ||
| parameters = inspect.signature(self.function).parameters | ||
|
|
||
| with DAG(dag_id=self.name, schedule=self.schedule, auto_register=True): | ||
| _AssetMainOperator( | ||
| task_id="__main__", | ||
| inlets=[ | ||
| AssetRef(name=inlet_asset_name) | ||
| for inlet_asset_name in parameters | ||
| if inlet_asset_name not in ("self", "context") | ||
| ], | ||
| outlets=[self.to_asset()], | ||
| python_callable=self.function, | ||
| definition_name=self.name, | ||
| uri=self.uri, | ||
| ) | ||
|
|
||
| def to_asset(self) -> Asset: | ||
| return Asset( | ||
| name=self.name, | ||
| uri=self.uri, | ||
| group=self.group, | ||
| extra=self.extra, | ||
| ) | ||
|
|
||
| def serialize(self): | ||
| return { | ||
| "uri": self.uri, | ||
| "name": self.name, | ||
| "group": self.group, | ||
| "extra": self.extra, | ||
| } | ||
|
|
||
|
|
||
| @attrs.define(kw_only=True) | ||
| class asset: | ||
| """Create an asset by decorating a materialization function.""" | ||
|
|
||
| schedule: ScheduleArg | ||
| uri: str | ObjectStoragePath | None = None | ||
| group: str = "" | ||
| extra: dict[str, Any] = attrs.field(factory=dict) | ||
|
|
||
| def __call__(self, f: Callable) -> AssetDefinition: | ||
| if (name := f.__name__) != f.__qualname__: | ||
| raise ValueError("nested function not supported") | ||
|
|
||
| return AssetDefinition( | ||
| name=name, | ||
| uri=name if self.uri is None else str(self.uri), | ||
| group=self.group, | ||
| extra=self.extra, | ||
| function=f, | ||
| schedule=self.schedule, | ||
| ) | ||
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 |
|---|---|---|
| @@ -0,0 +1,52 @@ | ||
| # 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. | ||
| from __future__ import annotations | ||
|
|
||
| import pendulum | ||
|
|
||
| from airflow.assets import Asset | ||
| from airflow.decorators import dag, task | ||
| from airflow.decorators.assets import asset | ||
|
|
||
|
|
||
| @asset(uri="s3://bucket/asset1_producer", schedule=None) | ||
| def asset1_producer(): | ||
| pass | ||
|
|
||
|
|
||
| @asset(uri="s3://bucket/object", schedule=None) | ||
| def asset2_producer(self, context, asset1_producer): | ||
| print(self) | ||
| print(context["inlet_events"][asset1_producer]) | ||
|
|
||
|
|
||
| @dag( | ||
| schedule=Asset(uri="s3://bucket/asset1_producer", name="asset1_producer") | ||
| | Asset(uri="s3://bucket/object", name="asset2_producer"), | ||
| start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), | ||
| catchup=False, | ||
| tags=["consumes", "asset-scheduled"], | ||
| ) | ||
| def consumes_asset_decorator(): | ||
| @task(outlets=[Asset(name="process_nothing")]) | ||
| def process_nothing(): | ||
| pass | ||
|
|
||
| process_nothing() | ||
|
|
||
|
|
||
| consumes_asset_decorator() |
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
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
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
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.
Uh oh!
There was an error while loading. Please reload this page.