-
Notifications
You must be signed in to change notification settings - Fork 1.3k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[freshness-policy] [1/n] FreshnessPolicy object (#10024)
- Loading branch information
1 parent
174dd99
commit 3c89b70
Showing
4 changed files
with
263 additions
and
0 deletions.
There are no files selected for viewing
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
137 changes: 137 additions & 0 deletions
137
python_modules/dagster/dagster/_core/definitions/freshness_policy.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,137 @@ | ||
from abc import ABC, abstractmethod | ||
from datetime import datetime | ||
from typing import Mapping, Optional | ||
|
||
import pendulum | ||
from croniter import croniter | ||
|
||
from dagster._annotations import experimental | ||
|
||
from .events import AssetKey | ||
|
||
|
||
@experimental | ||
class FreshnessPolicy(ABC): | ||
"""A FreshnessPolicy is a policy that defines how up-to-date a given asset is expected to be. | ||
We calculate the current time of the data within an asset by traversing the history of asset | ||
materializations of upstream assets which occured before the most recent materialization. | ||
This gives a lower bound on the most recent records that could possibly be incorporated into the | ||
current state of the asset to which this policy is attached. | ||
""" | ||
|
||
@abstractmethod | ||
def minutes_late( | ||
self, | ||
evaluation_time: datetime, | ||
upstream_materialization_times: Mapping[AssetKey, Optional[datetime]], | ||
) -> Optional[float]: | ||
raise NotImplementedError() | ||
|
||
@staticmethod | ||
def minimum_freshness(minimum_freshness_minutes: float) -> "MinimumFreshnessPolicy": | ||
"""Static constructor for a freshness policy which specifies that the upstream data that | ||
was used for the most recent asset materialization must have been materialized no more than | ||
`minimum_freshness_minutes` ago, relative to the current time. | ||
""" | ||
return MinimumFreshnessPolicy(minimum_freshness_minutes=minimum_freshness_minutes) | ||
|
||
@staticmethod | ||
def cron_minimum_freshness( | ||
minimum_freshness_minutes: float, cron_schedule: str | ||
) -> "CronMinimumFreshnessPolicy": | ||
"""Static constructor for a freshness policy which specifies that the upstream data that | ||
was used for the most recent asset materialization must have been materialized no more than | ||
`minimum_freshness_minutes` ago, relative to the most recent cron schedule tick. | ||
""" | ||
return CronMinimumFreshnessPolicy( | ||
minimum_freshness_minutes=minimum_freshness_minutes, | ||
cron_schedule=cron_schedule, | ||
) | ||
|
||
|
||
@experimental | ||
class MinimumFreshnessPolicy(FreshnessPolicy): | ||
"""A freshness policy which specifies that the upstream data that was used for the most recent | ||
asset materialization must have been materialized no more than `minimum_freshness_minutes` ago, | ||
relative to the current time. | ||
""" | ||
|
||
def __init__(self, minimum_freshness_minutes: float): | ||
self._minimum_freshness_minutes = minimum_freshness_minutes | ||
|
||
@property | ||
def minimum_freshness_minutes(self) -> float: | ||
return self._minimum_freshness_minutes | ||
|
||
def minutes_late( | ||
self, | ||
evaluation_time: datetime, | ||
upstream_materialization_times: Mapping[AssetKey, Optional[datetime]], | ||
) -> Optional[float]: | ||
minimum_time = evaluation_time - pendulum.duration(minutes=self.minimum_freshness_minutes) | ||
|
||
minutes_late = 0.0 | ||
for upstream_time in upstream_materialization_times.values(): | ||
# if any upstream materialization data is missing, then exit early | ||
if upstream_time is None: | ||
return None | ||
|
||
if upstream_time < minimum_time: | ||
minutes_late = max( | ||
minutes_late, (minimum_time - upstream_time).total_seconds() / 60 | ||
) | ||
return minutes_late | ||
|
||
|
||
@experimental | ||
class CronMinimumFreshnessPolicy(FreshnessPolicy): | ||
"""A freshness policy which specifies that the upstream data that was used for the most recent | ||
asset materialization must have been materialized no more than `minimum_freshness_minutes` ago, | ||
relative to the most recent cron schedule tick. | ||
""" | ||
|
||
def __init__(self, minimum_freshness_minutes: float, cron_schedule: str): | ||
self._minimum_freshness_minutes = minimum_freshness_minutes | ||
self._cron_schedule = cron_schedule | ||
|
||
@property | ||
def minimum_freshness_minutes(self) -> float: | ||
return self._minimum_freshness_minutes | ||
|
||
@property | ||
def cron_schedule(self) -> str: | ||
return self._cron_schedule | ||
|
||
def minutes_late( | ||
self, | ||
evaluation_time: datetime, | ||
upstream_materialization_times: Mapping[AssetKey, Optional[datetime]], | ||
) -> Optional[float]: | ||
minimum_freshness_duration = pendulum.duration(minutes=self.minimum_freshness_minutes) | ||
|
||
# find the most recent schedule tick which is more than minimum_freshness_duration old, | ||
# i.e. the most recent schedule tick which could be failing this constraint | ||
schedule_ticks = croniter( | ||
self.cron_schedule, evaluation_time, ret_type=datetime, is_prev=True | ||
) | ||
latest_required_tick = next(schedule_ticks) | ||
while latest_required_tick + minimum_freshness_duration > evaluation_time: | ||
latest_required_tick = next(schedule_ticks) | ||
|
||
minutes_late = 0.0 | ||
for upstream_materialization_time in upstream_materialization_times.values(): | ||
|
||
# if any upstream materialization data is missing, then exit early | ||
if upstream_materialization_time is None: | ||
return None | ||
|
||
if upstream_materialization_time < latest_required_tick: | ||
# find the difference between the actual data time and the latest time that you would | ||
# have expected to get this data by | ||
expected_by_time = latest_required_tick + minimum_freshness_duration | ||
minutes_late = max( | ||
minutes_late, (evaluation_time - expected_by_time).total_seconds() / 60 | ||
) | ||
|
||
return minutes_late |
95 changes: 95 additions & 0 deletions
95
python_modules/dagster/dagster_tests/core_tests/asset_defs_tests/test_freshness_policy.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,95 @@ | ||
import pytest | ||
|
||
from dagster import AssetKey | ||
from dagster._core.definitions.freshness_policy import FreshnessPolicy | ||
from dagster._seven.compat.pendulum import create_pendulum_time | ||
|
||
|
||
@pytest.mark.parametrize( | ||
["policy", "materialization_time", "evaluation_time", "expected_minutes_late"], | ||
[ | ||
( | ||
FreshnessPolicy.minimum_freshness(30), | ||
create_pendulum_time(2022, 1, 1, 0), | ||
create_pendulum_time(2022, 1, 1, 0, 25), | ||
0, | ||
), | ||
( | ||
FreshnessPolicy.minimum_freshness(120), | ||
create_pendulum_time(2022, 1, 1, 0), | ||
create_pendulum_time(2022, 1, 1, 1), | ||
0, | ||
), | ||
( | ||
FreshnessPolicy.minimum_freshness(30), | ||
create_pendulum_time(2022, 1, 1, 0), | ||
create_pendulum_time(2022, 1, 1, 1), | ||
30, | ||
), | ||
( | ||
FreshnessPolicy.minimum_freshness(500), | ||
None, | ||
create_pendulum_time(2022, 1, 1, 0, 25), | ||
None, | ||
), | ||
# materialization happened before SLA | ||
( | ||
FreshnessPolicy.cron_minimum_freshness( | ||
cron_schedule="@daily", minimum_freshness_minutes=15 | ||
), | ||
create_pendulum_time(2022, 1, 1, 0, 5), | ||
create_pendulum_time(2022, 1, 1, 0, 10), | ||
0, | ||
), | ||
# materialization happened after SLA, but is fine now | ||
( | ||
FreshnessPolicy.cron_minimum_freshness( | ||
cron_schedule="@daily", minimum_freshness_minutes=15 | ||
), | ||
create_pendulum_time(2022, 1, 1, 0, 30), | ||
create_pendulum_time(2022, 1, 1, 1, 0), | ||
0, | ||
), | ||
# materialization for this data has not happened yet (day before) | ||
( | ||
FreshnessPolicy.cron_minimum_freshness( | ||
cron_schedule="@daily", minimum_freshness_minutes=15 | ||
), | ||
create_pendulum_time(2022, 1, 1, 23, 0), | ||
create_pendulum_time(2022, 1, 2, 2, 0), | ||
# expected data by is 2022-01-02T00:15, so you are 1 hour, 45 minutes late | ||
60 + 45, | ||
), | ||
# weird one, basically want to have a materialization every hour no more than 5 hours after | ||
# that data arrives -- edge case probably not useful in practice? | ||
( | ||
FreshnessPolicy.cron_minimum_freshness( | ||
cron_schedule="@hourly", minimum_freshness_minutes=60 * 5 | ||
), | ||
create_pendulum_time(2022, 1, 1, 1, 0), | ||
create_pendulum_time(2022, 1, 1, 4, 0), | ||
0, | ||
), | ||
( | ||
FreshnessPolicy.cron_minimum_freshness( | ||
cron_schedule="@hourly", minimum_freshness_minutes=60 * 5 | ||
), | ||
create_pendulum_time(2022, 1, 1, 1, 15), | ||
create_pendulum_time(2022, 1, 1, 7, 45), | ||
# the data for 2AM is considered missing if it is not there by 7AM (5 hours later). | ||
# we evaluate at 7:45, so at this point it is 45 minutes late | ||
45, | ||
), | ||
], | ||
) | ||
def test_policies(policy, materialization_time, evaluation_time, expected_minutes_late): | ||
if materialization_time: | ||
upstream_materialization_times = {AssetKey("root"): materialization_time} | ||
else: | ||
upstream_materialization_times = {AssetKey("root"): None} | ||
minutes_late = policy.minutes_late( | ||
evaluation_time=evaluation_time, | ||
upstream_materialization_times=upstream_materialization_times, | ||
) | ||
|
||
assert minutes_late == expected_minutes_late |