-
Notifications
You must be signed in to change notification settings - Fork 1.6k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[exploration] output-free multi_asset
- Loading branch information
1 parent
ed7d3ce
commit d99d0c5
Showing
5 changed files
with
282 additions
and
50 deletions.
There are no files selected for viewing
104 changes: 104 additions & 0 deletions
104
python_modules/dagster/dagster/_core/definitions/asset_node.py
This file contains 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,104 @@ | ||
from typing import AbstractSet, Any, Iterable, Mapping, NamedTuple, Optional, Union | ||
|
||
import dagster._check as check | ||
from dagster._annotations import PublicAttr | ||
from dagster._core.definitions.backfill_policy import BackfillPolicy | ||
|
||
from .auto_materialize_policy import AutoMaterializePolicy | ||
from .events import ( | ||
AssetKey, | ||
CoercibleToAssetKey, | ||
) | ||
from .freshness_policy import FreshnessPolicy | ||
from .metadata import MetadataUserInput | ||
|
||
|
||
class AssetSpec( | ||
NamedTuple( | ||
"_AssetSpec", | ||
[ | ||
("asset_key", PublicAttr[AssetKey]), | ||
("deps", PublicAttr[AbstractSet[AssetKey]]), | ||
("description", PublicAttr[Optional[str]]), | ||
("metadata", PublicAttr[Optional[Mapping[str, Any]]]), | ||
("group_name", PublicAttr[Optional[str]]), | ||
("code_version", PublicAttr[Optional[str]]), | ||
("freshness_policy", PublicAttr[Optional[FreshnessPolicy]]), | ||
("auto_materialize_policy", PublicAttr[Optional[AutoMaterializePolicy]]), | ||
("backfill_policy", PublicAttr[Optional[BackfillPolicy]]), | ||
], | ||
) | ||
): | ||
"""This object specifies the attributes of an asset. This object is attached to the decorated | ||
function that defines how it materialized. | ||
Attributes: | ||
asset_key (AssetKey): The unique identifier for this asset. | ||
deps (Optional[AbstractSet[AssetKey]]): The asset keys for the upstream assets that | ||
materializing this asset depends on. | ||
description (Optional[str]): Human-readable description of this asset. | ||
metadata (Optional[Dict[str, Any]]): A dict of static metadata for this asset. | ||
For example, users can provide information about the database table this | ||
asset corresponds to. | ||
group_name (Optional[str]): A string name used to organize multiple assets into groups. If | ||
not provided, the name "default" is used. | ||
code_version (Optional[str]): The version of the code that generates this asset. | ||
freshness_policy (Optional[FreshnessPolicy]): A policy which indicates how up to date this | ||
asset is intended to be. | ||
auto_materialize_policy (Optional[AutoMaterializePolicy]): AutoMaterializePolicy to apply to | ||
the specified asset. | ||
backfill_policy (Optional[BackfillPolicy]): BackfillPolicy to apply to the specified asset. | ||
""" | ||
|
||
def __new__( | ||
cls, | ||
asset_key: CoercibleToAssetKey, | ||
deps: Optional[ | ||
Iterable[ | ||
Union[ | ||
CoercibleToAssetKey, | ||
"AssetSpec", | ||
# AssetsDefinition, if only one-key trick | ||
] | ||
] | ||
] = None, | ||
description: Optional[str] = None, | ||
metadata: Optional[MetadataUserInput] = None, | ||
group_name: Optional[str] = None, | ||
code_version: Optional[str] = None, | ||
freshness_policy: Optional[FreshnessPolicy] = None, | ||
auto_materialize_policy: Optional[AutoMaterializePolicy] = None, | ||
backfill_policy: Optional[BackfillPolicy] = None, | ||
): | ||
dep_set = set() | ||
if deps: | ||
for dep in deps: | ||
if isinstance(dep, AssetSpec): | ||
dep_set.add(dep.asset_key) | ||
else: | ||
dep_set.add(AssetKey.from_coercible(asset_key)) | ||
|
||
return super().__new__( | ||
cls, | ||
asset_key=AssetKey.from_coercible(asset_key), | ||
deps=dep_set, | ||
description=check.opt_str_param(description, "description"), | ||
metadata=check.opt_mapping_param(metadata, "metadata", key_type=str), | ||
group_name=check.opt_str_param(group_name, "group_name"), | ||
code_version=check.opt_str_param(code_version, "code_version"), | ||
freshness_policy=check.opt_inst_param( | ||
freshness_policy, | ||
"freshness_policy", | ||
FreshnessPolicy, | ||
), | ||
auto_materialize_policy=check.opt_inst_param( | ||
auto_materialize_policy, | ||
"auto_materialize_policy", | ||
AutoMaterializePolicy, | ||
), | ||
backfill_policy=check.opt_inst_param( | ||
backfill_policy, | ||
"backfill_policy", | ||
BackfillPolicy, | ||
), | ||
) |
This file contains 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 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 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.