-
Notifications
You must be signed in to change notification settings - Fork 1.4k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Create MaterializedSchedulingCondition
- Loading branch information
1 parent
7ba9e41
commit ea93cfa
Showing
4 changed files
with
85 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
33 changes: 33 additions & 0 deletions
33
python_modules/dagster/dagster/_core/definitions/declarative_scheduling/status_condition.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,33 @@ | ||
from abc import abstractmethod | ||
|
||
from dagster._core.asset_graph_view.asset_graph_view import AssetSlice | ||
|
||
from .asset_condition import AssetCondition, AssetConditionResult | ||
from .scheduling_condition_evaluation_context import SchedulingConditionEvaluationContext | ||
|
||
|
||
class StatusSchedulingCondition(AssetCondition): | ||
@abstractmethod | ||
def compute_slice_with_status( | ||
self, context: SchedulingConditionEvaluationContext | ||
) -> AssetSlice: ... | ||
|
||
def evaluate(self, context: SchedulingConditionEvaluationContext) -> AssetConditionResult: | ||
# don't compute anything if there are no candidates | ||
if context.candidate_slice.is_empty: | ||
true_slice = context.asset_graph_view.create_empty_slice(context.asset_key) | ||
else: | ||
true_slice = self.compute_slice_with_status(context) | ||
|
||
return AssetConditionResult.create(context, true_slice.convert_to_valid_asset_subset()) | ||
|
||
|
||
class MaterializedSchedulingCondition(StatusSchedulingCondition): | ||
@property | ||
def description(self) -> str: | ||
return "Materialized" | ||
|
||
def compute_slice_with_status( | ||
self, context: SchedulingConditionEvaluationContext | ||
) -> AssetSlice: | ||
return context.asset_graph_view.compute_materialized_asset_slice(context.asset_key) |
40 changes: 40 additions & 0 deletions
40
...nitions_tests/auto_materialize_tests/asset_condition_tests/test_materialized_condition.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,40 @@ | ||
from dagster._core.definitions.declarative_scheduling.asset_condition import ( | ||
AssetCondition, | ||
) | ||
|
||
from ..base_scenario import run_request | ||
from ..scenario_specs import one_asset, two_partitions_def | ||
from .asset_condition_scenario import AssetConditionScenarioState | ||
|
||
|
||
def test_materialized_unpartitioned() -> None: | ||
state = AssetConditionScenarioState(one_asset, asset_condition=AssetCondition.materialized()) | ||
|
||
state, result = state.evaluate("A") | ||
assert result.true_subset.size == 0 | ||
|
||
state = state.with_runs(run_request("A")) | ||
_, result = state.evaluate("A") | ||
assert result.true_subset.size == 1 | ||
|
||
|
||
def test_materialized_partitioned() -> None: | ||
state = AssetConditionScenarioState( | ||
one_asset, asset_condition=AssetCondition.materialized() | ||
).with_asset_properties(partitions_def=two_partitions_def) | ||
|
||
state, result = state.evaluate("A") | ||
assert result.true_subset.size == 0 | ||
|
||
state = state.with_runs(run_request("A", "1")) | ||
state, result = state.evaluate("A") | ||
assert result.true_subset.size == 1 | ||
|
||
# same partition materialized again | ||
state = state.with_runs(run_request("A", "1")) | ||
state, result = state.evaluate("A") | ||
assert result.true_subset.size == 1 | ||
|
||
state = state.with_runs(run_request("A", "2")) | ||
_, result = state.evaluate("A") | ||
assert result.true_subset.size == 2 |