|
11 | 11 | overload,
|
12 | 12 | )
|
13 | 13 |
|
14 |
| -from dagster_shared.serdes import whitelist_for_serdes |
| 14 | +from dagster_shared.serdes import serialize_value, whitelist_for_serdes |
15 | 15 |
|
16 | 16 | import dagster._check as check
|
17 | 17 | from dagster._annotations import (
|
|
26 | 26 | AutomationCondition,
|
27 | 27 | )
|
28 | 28 | from dagster._core.definitions.events import AssetKey, CoercibleToAssetKey
|
| 29 | +from dagster._core.definitions.freshness import ( |
| 30 | + INTERNAL_FRESHNESS_POLICY_METADATA_KEY, |
| 31 | + InternalFreshnessPolicy, |
| 32 | +) |
29 | 33 | from dagster._core.definitions.freshness_policy import FreshnessPolicy
|
30 | 34 | from dagster._core.definitions.partition import PartitionsDefinition
|
31 | 35 | from dagster._core.definitions.partition_mapping import PartitionMapping
|
@@ -108,6 +112,11 @@ def validate_kind_tags(kinds: Optional[AbstractSet[str]]) -> None:
|
108 | 112 | breaking_version="1.10.0",
|
109 | 113 | additional_warn_text="use `automation_condition` instead",
|
110 | 114 | )
|
| 115 | +@hidden_param( |
| 116 | + param="internal_freshness_policy", |
| 117 | + breaking_version="1.10.0", |
| 118 | + additional_warn_text="experimental feature, use freshness checks instead", |
| 119 | +) |
111 | 120 | @record_custom
|
112 | 121 | class AssetSpec(IHasInternalInit, IHaveNew, LegacyNamedTupleMixin):
|
113 | 122 | """Specifies the core attributes of an asset, except for the function that materializes or
|
@@ -195,6 +204,15 @@ def __new__(
|
195 | 204 | }
|
196 | 205 | validate_kind_tags(kind_tags)
|
197 | 206 |
|
| 207 | + internal_freshness_policy: Optional[InternalFreshnessPolicy] = kwargs.get( |
| 208 | + "internal_freshness_policy" |
| 209 | + ) |
| 210 | + if internal_freshness_policy: |
| 211 | + metadata = { |
| 212 | + **(metadata or {}), |
| 213 | + INTERNAL_FRESHNESS_POLICY_METADATA_KEY: serialize_value(internal_freshness_policy), |
| 214 | + } |
| 215 | + |
198 | 216 | return super().__new__(
|
199 | 217 | cls,
|
200 | 218 | key=key,
|
@@ -435,3 +453,13 @@ def map_asset_specs(
|
435 | 453 | obj.map_asset_specs(func) if isinstance(obj, AssetsDefinition) else func(obj)
|
436 | 454 | for obj in iterable
|
437 | 455 | ]
|
| 456 | + |
| 457 | + |
| 458 | +def attach_internal_freshness_policy(spec: AssetSpec, policy: InternalFreshnessPolicy) -> AssetSpec: |
| 459 | + """Apply a freshness policy to an asset spec, attaching it to the spec's metadata. |
| 460 | +
|
| 461 | + You can use this in Definitions.map_asset_specs to attach a freshness policy to an asset spec. |
| 462 | + """ |
| 463 | + return spec.merge_attributes( |
| 464 | + metadata={INTERNAL_FRESHNESS_POLICY_METADATA_KEY: serialize_value(policy)} # pyright: ignore[reportArgumentType] |
| 465 | + ) |
0 commit comments