dagster 1.12.12__py3-none-any.whl → 1.12.13rc0__py3-none-any.whl

This diff represents the content of publicly available package versions that have been released to one of the supported registries. The information contained in this diff is provided for informational purposes only and reflects changes between package versions as they appear in their respective public registries.
Files changed (34) hide show
  1. dagster/_core/asset_graph_view/asset_graph_view.py +83 -19
  2. dagster/_core/asset_graph_view/entity_subset.py +14 -9
  3. dagster/_core/asset_graph_view/serializable_entity_subset.py +6 -0
  4. dagster/_core/definitions/asset_checks/asset_check_evaluation.py +41 -68
  5. dagster/_core/definitions/asset_checks/asset_check_result.py +10 -0
  6. dagster/_core/definitions/asset_checks/asset_check_spec.py +11 -0
  7. dagster/_core/definitions/assets/graph/asset_graph.py +1 -0
  8. dagster/_core/definitions/assets/graph/base_asset_graph.py +29 -2
  9. dagster/_core/definitions/assets/graph/remote_asset_graph.py +9 -5
  10. dagster/_core/definitions/declarative_automation/legacy/valid_asset_subset.py +4 -4
  11. dagster/_core/definitions/declarative_automation/operands/operands.py +10 -4
  12. dagster/_core/definitions/decorators/asset_check_decorator.py +6 -0
  13. dagster/_core/event_api.py +10 -0
  14. dagster/_core/execution/context/asset_check_execution_context.py +39 -0
  15. dagster/_core/execution/plan/execute_step.py +4 -3
  16. dagster/_core/instance/runs/run_domain.py +73 -90
  17. dagster/_core/remote_representation/external_data.py +6 -0
  18. dagster/_core/storage/asset_check_execution_record.py +49 -5
  19. dagster/_core/storage/asset_check_state.py +263 -0
  20. dagster/_core/storage/dagster_run.py +77 -0
  21. dagster/_core/storage/event_log/base.py +59 -1
  22. dagster/_core/storage/event_log/sql_event_log.py +174 -7
  23. dagster/_core/storage/event_log/sqlite/sqlite_event_log.py +6 -1
  24. dagster/_core/storage/legacy_storage.py +26 -5
  25. dagster/_core/workspace/load_target.py +1 -1
  26. dagster/_daemon/monitoring/run_monitoring.py +5 -1
  27. dagster/_utils/__init__.py +11 -0
  28. dagster/version.py +1 -1
  29. {dagster-1.12.12.dist-info → dagster-1.12.13rc0.dist-info}/METADATA +3 -3
  30. {dagster-1.12.12.dist-info → dagster-1.12.13rc0.dist-info}/RECORD +34 -33
  31. {dagster-1.12.12.dist-info → dagster-1.12.13rc0.dist-info}/WHEEL +1 -1
  32. {dagster-1.12.12.dist-info → dagster-1.12.13rc0.dist-info}/entry_points.txt +0 -0
  33. {dagster-1.12.12.dist-info → dagster-1.12.13rc0.dist-info}/licenses/LICENSE +0 -0
  34. {dagster-1.12.12.dist-info → dagster-1.12.13rc0.dist-info}/top_level.txt +0 -0
@@ -21,6 +21,7 @@ from dagster._core.definitions.events import AssetKeyPartitionKey
21
21
  from dagster._core.definitions.freshness import FreshnessState
22
22
  from dagster._core.definitions.partitions.context import (
23
23
  PartitionLoadingContext,
24
+ partition_loading_context,
24
25
  use_partition_loading_context,
25
26
  )
26
27
  from dagster._core.definitions.partitions.definition import (
@@ -162,10 +163,7 @@ class AssetGraphView(LoadingContext):
162
163
  return self._queryer
163
164
 
164
165
  def _get_partitions_def(self, key: T_EntityKey) -> Optional["PartitionsDefinition"]:
165
- if isinstance(key, AssetKey):
166
- return self.asset_graph.get(key).partitions_def
167
- else:
168
- return None
166
+ return self.asset_graph.get(key).partitions_def
169
167
 
170
168
  @cached_method
171
169
  @use_partition_loading_context
@@ -374,6 +372,16 @@ class AssetGraphView(LoadingContext):
374
372
  )
375
373
  return EntitySubset(self, key=key, value=_ValidatedEntitySubsetValue(value))
376
374
 
375
+ @use_partition_loading_context
376
+ def get_subset_from_partition_keys(
377
+ self,
378
+ key: T_EntityKey,
379
+ partitions_def: "PartitionsDefinition",
380
+ partition_keys: AbstractSet[str],
381
+ ) -> EntitySubset[T_EntityKey]:
382
+ value = partitions_def.subset_with_partition_keys(partition_keys)
383
+ return EntitySubset(self, key=key, value=_ValidatedEntitySubsetValue(value))
384
+
377
385
  @use_partition_loading_context
378
386
  def compute_parent_subset_and_required_but_nonexistent_subset(
379
387
  self, parent_key, subset: EntitySubset[T_EntityKey]
@@ -548,11 +556,22 @@ class AssetGraphView(LoadingContext):
548
556
  check.failed(f"Unsupported partitions_def: {partitions_def}")
549
557
 
550
558
  async def compute_subset_with_status(
551
- self, key: AssetCheckKey, status: Optional["AssetCheckExecutionResolvedStatus"]
552
- ):
559
+ self,
560
+ key: AssetCheckKey,
561
+ status: Optional["AssetCheckExecutionResolvedStatus"],
562
+ from_subset: EntitySubset,
563
+ ) -> EntitySubset[AssetCheckKey]:
564
+ """Returns the subset of an asset check that matches a given status."""
553
565
  from dagster._core.storage.event_log.base import AssetCheckSummaryRecord
554
566
 
555
- """Returns the subset of an asset check that matches a given status."""
567
+ # Handle partitioned asset checks
568
+ if self._get_partitions_def(key):
569
+ with partition_loading_context(new_ctx=self._partition_loading_context):
570
+ return await self._get_partitioned_check_subset_with_status(
571
+ key, status, from_subset
572
+ )
573
+
574
+ # Handle non-partitioned asset checks with existing logic
556
575
  summary = await AssetCheckSummaryRecord.gen(self, key)
557
576
  latest_record = summary.last_check_execution_record if summary else None
558
577
  resolved_status = (
@@ -586,31 +605,64 @@ class AssetGraphView(LoadingContext):
586
605
  return self.get_empty_subset(key=key)
587
606
 
588
607
  async def _compute_run_in_progress_check_subset(
589
- self, key: AssetCheckKey
608
+ self, key: AssetCheckKey, from_subset: EntitySubset
590
609
  ) -> EntitySubset[AssetCheckKey]:
591
610
  from dagster._core.storage.asset_check_execution_record import (
592
611
  AssetCheckExecutionResolvedStatus,
593
612
  )
594
613
 
595
614
  return await self.compute_subset_with_status(
596
- key, AssetCheckExecutionResolvedStatus.IN_PROGRESS
615
+ key, AssetCheckExecutionResolvedStatus.IN_PROGRESS, from_subset
597
616
  )
598
617
 
599
618
  async def _compute_execution_failed_check_subset(
600
- self, key: AssetCheckKey
619
+ self, key: AssetCheckKey, from_subset: EntitySubset
601
620
  ) -> EntitySubset[AssetCheckKey]:
602
621
  from dagster._core.storage.asset_check_execution_record import (
603
622
  AssetCheckExecutionResolvedStatus,
604
623
  )
605
624
 
606
625
  return await self.compute_subset_with_status(
607
- key, AssetCheckExecutionResolvedStatus.EXECUTION_FAILED
626
+ key, AssetCheckExecutionResolvedStatus.EXECUTION_FAILED, from_subset
608
627
  )
609
628
 
610
629
  async def _compute_missing_check_subset(
611
- self, key: AssetCheckKey
630
+ self, key: AssetCheckKey, from_subset: EntitySubset
631
+ ) -> EntitySubset[AssetCheckKey]:
632
+ return await self.compute_subset_with_status(key, None, from_subset)
633
+
634
+ @use_partition_loading_context
635
+ async def _get_partitioned_check_subset_with_status(
636
+ self,
637
+ key: AssetCheckKey,
638
+ status: Optional["AssetCheckExecutionResolvedStatus"],
639
+ from_subset: EntitySubset,
612
640
  ) -> EntitySubset[AssetCheckKey]:
613
- return await self.compute_subset_with_status(key, None)
641
+ from dagster._core.storage.asset_check_state import AssetCheckState
642
+
643
+ check_node = self.asset_graph.get(key)
644
+ if not check_node or not check_node.partitions_def:
645
+ check.failed(f"Asset check {key} not found or not partitioned.")
646
+
647
+ cache_value = (
648
+ await AssetCheckState.gen(self, (key, check_node.partitions_def))
649
+ or AssetCheckState.empty()
650
+ )
651
+
652
+ if status is None:
653
+ known_statuses = self.get_empty_subset(key=key)
654
+ for serializable_subset in cache_value.subsets.values():
655
+ subset = self.get_subset_from_serializable_subset(serializable_subset)
656
+ if subset:
657
+ known_statuses = known_statuses.compute_union(subset)
658
+ return from_subset.compute_difference(known_statuses) or self.get_empty_subset(key=key)
659
+ else:
660
+ serializable_subset = cache_value.subsets.get(status)
661
+ if serializable_subset is None:
662
+ return self.get_empty_subset(key=key)
663
+ return self.get_subset_from_serializable_subset(
664
+ serializable_subset
665
+ ) or self.get_empty_subset(key=key)
614
666
 
615
667
  async def _compute_run_in_progress_asset_subset(self, key: AssetKey) -> EntitySubset[AssetKey]:
616
668
  from dagster._core.storage.partition_status_cache import AssetStatusCacheValue
@@ -735,15 +787,21 @@ class AssetGraphView(LoadingContext):
735
787
  )
736
788
 
737
789
  @cached_method
738
- async def compute_run_in_progress_subset(self, *, key: EntityKey) -> EntitySubset:
790
+ async def compute_run_in_progress_subset(
791
+ self, *, key: EntityKey, from_subset: EntitySubset
792
+ ) -> EntitySubset:
739
793
  return await _dispatch(
740
794
  key=key,
741
- check_method=self._compute_run_in_progress_check_subset,
795
+ check_method=functools.partial(
796
+ self._compute_run_in_progress_check_subset, from_subset=from_subset
797
+ ),
742
798
  asset_method=self._compute_run_in_progress_asset_subset,
743
799
  )
744
800
 
745
801
  @cached_method
746
- async def compute_backfill_in_progress_subset(self, *, key: EntityKey) -> EntitySubset:
802
+ async def compute_backfill_in_progress_subset(
803
+ self, *, key: EntityKey, from_subset: EntitySubset
804
+ ) -> EntitySubset:
747
805
  async def get_empty_subset(key: EntityKey) -> EntitySubset:
748
806
  return self.get_empty_subset(key=key)
749
807
 
@@ -755,10 +813,14 @@ class AssetGraphView(LoadingContext):
755
813
  )
756
814
 
757
815
  @cached_method
758
- async def compute_execution_failed_subset(self, *, key: EntityKey) -> EntitySubset:
816
+ async def compute_execution_failed_subset(
817
+ self, *, key: EntityKey, from_subset: EntitySubset
818
+ ) -> EntitySubset:
759
819
  return await _dispatch(
760
820
  key=key,
761
- check_method=self._compute_execution_failed_check_subset,
821
+ check_method=functools.partial(
822
+ self._compute_execution_failed_check_subset, from_subset=from_subset
823
+ ),
762
824
  asset_method=self._compute_execution_failed_asset_subset,
763
825
  )
764
826
 
@@ -768,7 +830,9 @@ class AssetGraphView(LoadingContext):
768
830
  ) -> EntitySubset:
769
831
  return await _dispatch(
770
832
  key=key,
771
- check_method=self._compute_missing_check_subset,
833
+ check_method=functools.partial(
834
+ self._compute_missing_check_subset, from_subset=from_subset
835
+ ),
772
836
  asset_method=functools.partial(
773
837
  self._compute_missing_asset_subset, from_subset=from_subset
774
838
  ),
@@ -69,10 +69,13 @@ class EntitySubset(Generic[T_EntityKey]):
69
69
  return SerializableEntitySubset(key=self._key, value=self._value)
70
70
 
71
71
  def expensively_compute_partition_keys(self) -> AbstractSet[str]:
72
- return {
73
- check.not_none(akpk.partition_key, "No None partition keys")
74
- for akpk in self.expensively_compute_asset_partitions()
75
- }
72
+ internal_value = self.get_internal_value()
73
+ if isinstance(internal_value, PartitionsSubset):
74
+ return set(internal_value.get_partition_keys())
75
+ elif internal_value:
76
+ check.failed("Subset is not partitioned")
77
+ else:
78
+ return set()
76
79
 
77
80
  def expensively_compute_asset_partitions(self) -> AbstractSet[AssetKeyPartitionKey]:
78
81
  if not isinstance(self.key, AssetKey):
@@ -106,11 +109,13 @@ class EntitySubset(Generic[T_EntityKey]):
106
109
  return self._oper(other, operator.and_)
107
110
 
108
111
  def compute_intersection_with_partition_keys(
109
- self: "EntitySubset[AssetKey]", partition_keys: AbstractSet[str]
110
- ) -> "EntitySubset[AssetKey]":
111
- key = check.inst(self.key, AssetKey)
112
- partition_subset = self._asset_graph_view.get_asset_subset_from_asset_partitions(
113
- self.key, {AssetKeyPartitionKey(key, pk) for pk in partition_keys}
112
+ self: "EntitySubset[T_EntityKey]", partition_keys: AbstractSet[str]
113
+ ) -> "EntitySubset[T_EntityKey]":
114
+ if self.partitions_def is None:
115
+ return self._asset_graph_view.get_empty_subset(key=self.key)
116
+
117
+ partition_subset = self._asset_graph_view.get_subset_from_partition_keys(
118
+ self.key, self.partitions_def, partition_keys
114
119
  )
115
120
  return self.compute_intersection(partition_subset)
116
121
 
@@ -50,6 +50,12 @@ class SerializableEntitySubset(Generic[T_EntityKey]):
50
50
  key: T_EntityKey
51
51
  value: EntitySubsetValue
52
52
 
53
+ @classmethod
54
+ def empty(
55
+ cls, key: T_EntityKey, partitions_def: Optional[PartitionsDefinition]
56
+ ) -> "SerializableEntitySubset[T_EntityKey]":
57
+ return cls(key=key, value=partitions_def.empty_subset() if partitions_def else False)
58
+
53
59
  @classmethod
54
60
  def from_coercible_value(
55
61
  cls,
@@ -1,34 +1,27 @@
1
1
  from collections.abc import Mapping
2
- from typing import NamedTuple, Optional
2
+ from typing import Optional
3
+
4
+ import dagster_shared.check as check
5
+ from dagster_shared.record import IHaveNew, record, record_custom, replace
3
6
 
4
- import dagster._check as check
5
7
  from dagster._core.definitions.asset_checks.asset_check_spec import (
6
8
  AssetCheckKey,
7
9
  AssetCheckSeverity,
8
10
  )
9
11
  from dagster._core.definitions.events import AssetKey, MetadataValue, RawMetadataValue
10
12
  from dagster._core.definitions.metadata import normalize_metadata
13
+ from dagster._core.definitions.partitions.subset import PartitionsSubset
11
14
  from dagster._serdes import whitelist_for_serdes
12
15
 
13
16
 
14
17
  @whitelist_for_serdes
15
- class AssetCheckEvaluationPlanned(
16
- NamedTuple(
17
- "_AssetCheckEvaluationPlanned",
18
- [
19
- ("asset_key", AssetKey),
20
- ("check_name", str),
21
- ],
22
- )
23
- ):
18
+ @record
19
+ class AssetCheckEvaluationPlanned:
24
20
  """Metadata for the event when an asset check is launched."""
25
21
 
26
- def __new__(cls, asset_key: AssetKey, check_name: str):
27
- return super().__new__(
28
- cls,
29
- asset_key=check.inst_param(asset_key, "asset_key", AssetKey),
30
- check_name=check.str_param(check_name, "check_name"),
31
- )
22
+ asset_key: AssetKey
23
+ check_name: str
24
+ partitions_subset: Optional[PartitionsSubset] = None
32
25
 
33
26
  @property
34
27
  def asset_check_key(self) -> AssetCheckKey:
@@ -36,46 +29,18 @@ class AssetCheckEvaluationPlanned(
36
29
 
37
30
 
38
31
  @whitelist_for_serdes
39
- class AssetCheckEvaluationTargetMaterializationData(
40
- NamedTuple(
41
- "_AssetCheckEvaluationTargetMaterializationData",
42
- [
43
- ("storage_id", int),
44
- ("run_id", str),
45
- ("timestamp", float),
46
- ],
47
- )
48
- ):
32
+ @record
33
+ class AssetCheckEvaluationTargetMaterializationData:
49
34
  """A pointer to the latest materialization at execution time of an asset check."""
50
35
 
51
- def __new__(cls, storage_id: int, run_id: str, timestamp: float):
52
- return super().__new__(
53
- cls,
54
- storage_id=check.int_param(storage_id, "storage_id"),
55
- run_id=check.str_param(run_id, "run_id"),
56
- timestamp=check.float_param(timestamp, "timestamp"),
57
- )
36
+ storage_id: int
37
+ run_id: str
38
+ timestamp: float
58
39
 
59
40
 
60
41
  @whitelist_for_serdes(storage_field_names={"passed": "success"})
61
- class AssetCheckEvaluation(
62
- NamedTuple(
63
- "_AssetCheckEvaluation",
64
- [
65
- ("asset_key", AssetKey),
66
- ("check_name", str),
67
- ("passed", bool),
68
- ("metadata", Mapping[str, MetadataValue]),
69
- (
70
- "target_materialization_data",
71
- Optional[AssetCheckEvaluationTargetMaterializationData],
72
- ),
73
- ("severity", AssetCheckSeverity),
74
- ("description", Optional[str]),
75
- ("blocking", Optional[bool]),
76
- ],
77
- )
78
- ):
42
+ @record_custom
43
+ class AssetCheckEvaluation(IHaveNew):
79
44
  """Represents the outcome of a evaluating an asset check.
80
45
 
81
46
  Args:
@@ -97,8 +62,20 @@ class AssetCheckEvaluation(
97
62
  A text description of the result of the check evaluation.
98
63
  blocking (Optional[bool]):
99
64
  Whether the check is blocking.
65
+ partition (Optional[str]):
66
+ The partition that the check was evaluated on, if applicable.
100
67
  """
101
68
 
69
+ asset_key: AssetKey
70
+ check_name: str
71
+ passed: bool
72
+ metadata: Mapping[str, MetadataValue]
73
+ target_materialization_data: Optional[AssetCheckEvaluationTargetMaterializationData]
74
+ severity: AssetCheckSeverity
75
+ description: Optional[str]
76
+ blocking: Optional[bool]
77
+ partition: Optional[str]
78
+
102
79
  def __new__(
103
80
  cls,
104
81
  asset_key: AssetKey,
@@ -109,25 +86,21 @@ class AssetCheckEvaluation(
109
86
  severity: AssetCheckSeverity = AssetCheckSeverity.ERROR,
110
87
  description: Optional[str] = None,
111
88
  blocking: Optional[bool] = None,
89
+ partition: Optional[str] = None,
112
90
  ):
113
- normed_metadata = normalize_metadata(
114
- check.opt_mapping_param(metadata, "metadata", key_type=str),
115
- )
116
-
117
91
  return super().__new__(
118
92
  cls,
119
- asset_key=check.inst_param(asset_key, "asset_key", AssetKey),
120
- check_name=check.str_param(check_name, "check_name"),
121
- passed=check.bool_param(passed, "passed"),
122
- metadata=normed_metadata,
123
- target_materialization_data=check.opt_inst_param(
124
- target_materialization_data,
125
- "target_materialization_data",
126
- AssetCheckEvaluationTargetMaterializationData,
93
+ asset_key=asset_key,
94
+ check_name=check_name,
95
+ passed=passed,
96
+ metadata=normalize_metadata(
97
+ check.opt_mapping_param(metadata, "metadata", key_type=str)
127
98
  ),
128
- severity=check.inst_param(severity, "severity", AssetCheckSeverity),
129
- description=check.opt_str_param(description, "description"),
130
- blocking=check.opt_bool_param(blocking, "blocking"),
99
+ target_materialization_data=target_materialization_data,
100
+ severity=severity,
101
+ description=description,
102
+ blocking=blocking,
103
+ partition=partition,
131
104
  )
132
105
 
133
106
  @property
@@ -138,4 +111,4 @@ class AssetCheckEvaluation(
138
111
  normed_metadata = normalize_metadata(
139
112
  check.opt_mapping_param(metadata, "metadata", key_type=str),
140
113
  )
141
- return self._replace(metadata=normed_metadata)
114
+ return replace(self, metadata=normed_metadata)
@@ -172,6 +172,15 @@ class AssetCheckResult(
172
172
  else:
173
173
  target_materialization_data = None
174
174
 
175
+ if step_context.has_partition_key:
176
+ check_spec = assets_def_for_check.get_spec_for_check_key(check_key)
177
+ if check_spec.partitions_def is not None:
178
+ partition = step_context.partition_key
179
+ else:
180
+ partition = None
181
+ else:
182
+ partition = None
183
+
175
184
  return AssetCheckEvaluation(
176
185
  check_name=check_key.name,
177
186
  asset_key=check_key.asset_key,
@@ -181,6 +190,7 @@ class AssetCheckResult(
181
190
  severity=self.severity,
182
191
  description=self.description,
183
192
  blocking=assets_def_for_check.get_spec_for_check_key(check_key).blocking,
193
+ partition=partition,
184
194
  )
185
195
 
186
196
  def with_metadata(self, metadata: Mapping[str, RawMetadataValue]) -> "AssetCheckResult": # pyright: ignore[reportIncompatibleMethodOverride]
@@ -10,9 +10,11 @@ from dagster_shared.record import (
10
10
  replace,
11
11
  )
12
12
  from dagster_shared.serdes import whitelist_for_serdes
13
+ from dagster_shared.utils.warnings import preview_warning
13
14
 
14
15
  from dagster._annotations import PublicAttr, public
15
16
  from dagster._core.definitions.asset_key import AssetCheckKey, AssetKey, CoercibleToAssetKey
17
+ from dagster._core.definitions.partitions.definition import PartitionsDefinition
16
18
 
17
19
  if TYPE_CHECKING:
18
20
  from dagster._core.definitions.assets.definition.asset_dep import AssetDep, CoercibleToAssetDep
@@ -58,6 +60,7 @@ class AssetCheckSpec(IHaveNew, LegacyNamedTupleMixin):
58
60
  blocking: PublicAttr[bool]
59
61
  metadata: PublicAttr[Mapping[str, Any]]
60
62
  automation_condition: PublicAttr[Optional[LazyAutomationCondition]]
63
+ partitions_def: PublicAttr[Optional[PartitionsDefinition]]
61
64
 
62
65
  """Defines information about an asset check, except how to execute it.
63
66
 
@@ -80,6 +83,9 @@ class AssetCheckSpec(IHaveNew, LegacyNamedTupleMixin):
80
83
  that multi-asset is responsible for enforcing that downstream assets within the
81
84
  same step do not execute after a blocking asset check fails.
82
85
  metadata (Optional[Mapping[str, Any]]): A dict of static metadata for this asset check.
86
+ automation_condition (Optional[AutomationCondition[AssetCheckKey]]): The AutomationCondition for this asset check.
87
+ partitions_def (Optional[PartitionsDefinition]): The PartitionsDefinition for this asset check. Must be either None
88
+ or the same as the PartitionsDefinition of the asset specified by `asset`.
83
89
  """
84
90
 
85
91
  def __new__(
@@ -92,11 +98,15 @@ class AssetCheckSpec(IHaveNew, LegacyNamedTupleMixin):
92
98
  blocking: bool = False,
93
99
  metadata: Optional[Mapping[str, Any]] = None,
94
100
  automation_condition: Optional["AutomationCondition[AssetCheckKey]"] = None,
101
+ partitions_def: Optional[PartitionsDefinition] = None,
95
102
  ):
96
103
  from dagster._core.definitions.assets.definition.asset_dep import (
97
104
  coerce_to_deps_and_check_duplicates,
98
105
  )
99
106
 
107
+ if partitions_def is not None:
108
+ preview_warning("Specifying a partitions_def on an AssetCheckSpec")
109
+
100
110
  asset_key = AssetKey.from_coercible_or_definition(asset)
101
111
 
102
112
  additional_asset_deps = coerce_to_deps_and_check_duplicates(
@@ -119,6 +129,7 @@ class AssetCheckSpec(IHaveNew, LegacyNamedTupleMixin):
119
129
  blocking=blocking,
120
130
  metadata=metadata or {},
121
131
  automation_condition=automation_condition,
132
+ partitions_def=partitions_def,
122
133
  )
123
134
 
124
135
  def get_python_identifier(self) -> str:
@@ -189,6 +189,7 @@ class AssetGraph(BaseAssetGraph[AssetNode]):
189
189
  v.get_spec_for_check_key(k).description,
190
190
  v.get_spec_for_check_key(k).automation_condition,
191
191
  v.get_spec_for_check_key(k).metadata,
192
+ v.get_spec_for_check_key(k).partitions_def,
192
193
  )
193
194
  for k, v in assets_defs_by_check_key.items()
194
195
  }
@@ -217,6 +217,7 @@ class AssetCheckNode(BaseEntityNode[AssetCheckKey]):
217
217
  description: Optional[str],
218
218
  automation_condition: Optional["AutomationCondition[AssetCheckKey]"],
219
219
  metadata: ArbitraryMetadataMapping,
220
+ partitions_def: Optional[PartitionsDefinition],
220
221
  ):
221
222
  self.key = key
222
223
  self.blocking = blocking
@@ -224,6 +225,7 @@ class AssetCheckNode(BaseEntityNode[AssetCheckKey]):
224
225
  self._additional_deps = additional_deps
225
226
  self._description = description
226
227
  self._metadata = metadata
228
+ self._partitions_def = partitions_def
227
229
 
228
230
  @property
229
231
  def parent_entity_keys(self) -> AbstractSet[AssetKey]:
@@ -235,8 +237,7 @@ class AssetCheckNode(BaseEntityNode[AssetCheckKey]):
235
237
 
236
238
  @property
237
239
  def partitions_def(self) -> Optional[PartitionsDefinition]:
238
- # all checks are unpartitioned
239
- return None
240
+ return self._partitions_def
240
241
 
241
242
  @property
242
243
  def partition_mappings(self) -> Mapping[EntityKey, PartitionMapping]:
@@ -266,6 +267,10 @@ class BaseAssetGraph(ABC, Generic[T_AssetNode]):
266
267
  def asset_nodes(self) -> Iterable[T_AssetNode]:
267
268
  return self._asset_nodes_by_key.values()
268
269
 
270
+ @property
271
+ def asset_check_nodes(self) -> Iterable[AssetCheckNode]:
272
+ return self._asset_check_nodes_by_key.values()
273
+
269
274
  @property
270
275
  def nodes(self) -> Iterable[BaseEntityNode]:
271
276
  return [
@@ -668,6 +673,28 @@ class BaseAssetGraph(ABC, Generic[T_AssetNode]):
668
673
  f"Invalid partition mapping from {node.key.to_user_string()} to {parent.key.to_user_string()}"
669
674
  ) from e
670
675
 
676
+ # Validate that asset checks have compatible partitions_def with their target asset
677
+ for node in self.asset_check_nodes:
678
+ if node.partitions_def is None:
679
+ continue
680
+
681
+ target_asset_key = node.key.asset_key
682
+ if not self.has(target_asset_key):
683
+ raise DagsterInvalidDefinitionError(
684
+ f"Partitioned asset check '{node.key.to_user_string()}' targets "
685
+ f"asset '{target_asset_key.to_user_string()}' "
686
+ "but the asset does not exist in the graph."
687
+ )
688
+ # If the check is partitioned, it must have the same partitions_def as the asset
689
+ if node.partitions_def != self.get(target_asset_key).partitions_def:
690
+ raise DagsterInvalidDefinitionError(
691
+ f"Asset check '{node.key.to_user_string()}' targets asset '{target_asset_key.to_user_string()}' "
692
+ "but has a different partitions definition. "
693
+ f"Asset check partitions_def: {node.partitions_def}, "
694
+ f"Asset partitions_def: {self.get(target_asset_key).partitions_def}. "
695
+ "Partitioned asset checks must have the same partitions definition as their target asset."
696
+ )
697
+
671
698
  def upstream_key_iterator(self, asset_key: AssetKey) -> Iterator[AssetKey]:
672
699
  """Iterates through all asset keys which are upstream of the given key."""
673
700
  visited: set[AssetKey] = set()
@@ -480,6 +480,9 @@ class RemoteAssetGraph(BaseAssetGraph[TRemoteAssetNode], ABC, Generic[TRemoteAss
480
480
  remote_node.asset_check.description,
481
481
  remote_node.asset_check.automation_condition,
482
482
  {}, # metadata not yet on AssetCheckNodeSnap
483
+ remote_node.asset_check.partitions_def_snapshot.get_partitions_definition()
484
+ if remote_node.asset_check.partitions_def_snapshot
485
+ else None,
483
486
  )
484
487
 
485
488
  ##### COMMON ASSET GRAPH INTERFACE
@@ -744,12 +747,13 @@ class RemoteWorkspaceAssetGraph(RemoteAssetGraph[RemoteWorkspaceAssetNode]):
744
747
  def get_repo_scoped_node(
745
748
  self, key: EntityKey, repository_selector: "RepositorySelector"
746
749
  ) -> Optional[Union[RemoteRepositoryAssetNode, RemoteAssetCheckNode]]:
747
- if isinstance(key, AssetKey):
748
- if not self.has(key):
749
- return None
750
- return self.get(key).resolve_to_repo_scoped_node(repository_selector)
750
+ if not self.has(key):
751
+ return None
752
+ node = self.get(key)
753
+ if isinstance(node, RemoteWorkspaceAssetNode):
754
+ return node.resolve_to_repo_scoped_node(repository_selector)
751
755
  else:
752
- raise Exception("Key must be an asset key for get_repo_scoped_node")
756
+ return node # type: ignore
753
757
 
754
758
  def split_entity_keys_by_repository(
755
759
  self, keys: AbstractSet[EntityKey]
@@ -98,14 +98,14 @@ class ValidAssetSubset(SerializableEntitySubset[AssetKey]):
98
98
  key=asset_key, value=AllPartitionsSubset(partitions_def, ctx)
99
99
  )
100
100
 
101
- @staticmethod
101
+ @classmethod
102
102
  def empty(
103
- asset_key: AssetKey, partitions_def: Optional[PartitionsDefinition]
103
+ cls, key: AssetKey, partitions_def: Optional[PartitionsDefinition]
104
104
  ) -> "ValidAssetSubset":
105
105
  if partitions_def is None:
106
- return ValidAssetSubset(key=asset_key, value=False)
106
+ return cls(key=key, value=False)
107
107
  else:
108
- return ValidAssetSubset(key=asset_key, value=partitions_def.empty_subset())
108
+ return cls(key=key, value=partitions_def.empty_subset())
109
109
 
110
110
  @staticmethod
111
111
  def from_asset_partitions_set(
@@ -102,7 +102,9 @@ class RunInProgressAutomationCondition(SubsetAutomationCondition):
102
102
  return "run_in_progress"
103
103
 
104
104
  async def compute_subset(self, context: AutomationContext) -> EntitySubset: # pyright: ignore[reportIncompatibleMethodOverride]
105
- return await context.asset_graph_view.compute_run_in_progress_subset(key=context.key)
105
+ return await context.asset_graph_view.compute_run_in_progress_subset(
106
+ key=context.key, from_subset=context.candidate_subset
107
+ )
106
108
 
107
109
 
108
110
  @whitelist_for_serdes
@@ -113,7 +115,9 @@ class BackfillInProgressAutomationCondition(SubsetAutomationCondition):
113
115
  return "backfill_in_progress"
114
116
 
115
117
  async def compute_subset(self, context: AutomationContext) -> EntitySubset: # pyright: ignore[reportIncompatibleMethodOverride]
116
- return await context.asset_graph_view.compute_backfill_in_progress_subset(key=context.key)
118
+ return await context.asset_graph_view.compute_backfill_in_progress_subset(
119
+ key=context.key, from_subset=context.candidate_subset
120
+ )
117
121
 
118
122
 
119
123
  @whitelist_for_serdes(storage_name="FailedAutomationCondition")
@@ -124,7 +128,9 @@ class ExecutionFailedAutomationCondition(SubsetAutomationCondition):
124
128
  return "execution_failed"
125
129
 
126
130
  async def compute_subset(self, context: AutomationContext) -> EntitySubset: # pyright: ignore[reportIncompatibleMethodOverride]
127
- return await context.asset_graph_view.compute_execution_failed_subset(key=context.key)
131
+ return await context.asset_graph_view.compute_execution_failed_subset(
132
+ key=context.key, from_subset=context.candidate_subset
133
+ )
128
134
 
129
135
 
130
136
  @whitelist_for_serdes
@@ -322,5 +328,5 @@ class CheckResultCondition(SubsetAutomationCondition[AssetCheckKey]):
322
328
  else AssetCheckExecutionResolvedStatus.FAILED
323
329
  )
324
330
  return await context.asset_graph_view.compute_subset_with_status(
325
- key=context.key, status=target_status
331
+ key=context.key, status=target_status, from_subset=context.candidate_subset
326
332
  )