diff --git a/python_modules/dagster-graphql/dagster_graphql/implementation/loader.py b/python_modules/dagster-graphql/dagster_graphql/implementation/loader.py index c239a6fa03546..c63b97fdba93d 100644 --- a/python_modules/dagster-graphql/dagster_graphql/implementation/loader.py +++ b/python_modules/dagster-graphql/dagster_graphql/implementation/loader.py @@ -7,6 +7,7 @@ DagsterInstance, _check as check, ) +from dagster._core.definitions.asset_spec import AssetExecutionType from dagster._core.definitions.data_version import CachingStaleStatusResolver from dagster._core.definitions.events import AssetKey from dagster._core.events.log import EventLogEntry @@ -315,6 +316,7 @@ def _build_cross_repo_deps( ) ], depended_by=[], + execution_type=AssetExecutionType.UNEXECUTABLE, ) return sink_assets, external_asset_deps diff --git a/python_modules/dagster-graphql/dagster_graphql/schema/asset_graph.py b/python_modules/dagster-graphql/dagster_graphql/schema/asset_graph.py index d2ffd338a10a5..11d8427821c8d 100644 --- a/python_modules/dagster-graphql/dagster_graphql/schema/asset_graph.py +++ b/python_modules/dagster-graphql/dagster_graphql/schema/asset_graph.py @@ -5,6 +5,7 @@ AssetKey, _check as check, ) +from dagster._core.definitions.asset_spec import AssetExecutionType from dagster._core.definitions.assets_job import ASSET_BASE_JOB_PREFIX from dagster._core.definitions.data_time import CachingDataTimeResolver from dagster._core.definitions.data_version import ( @@ -259,6 +260,7 @@ class GrapheneAssetNode(graphene.ObjectType): groupName = graphene.String() id = graphene.NonNull(graphene.ID) isExecutable = graphene.NonNull(graphene.Boolean) + isExternal = graphene.NonNull(graphene.Boolean) isObservable = graphene.NonNull(graphene.Boolean) isPartitioned = graphene.NonNull(graphene.Boolean) isSource = graphene.NonNull(graphene.Boolean) @@ -501,7 +503,8 @@ def is_graph_backed_asset(self) -> bool: return self.graphName is not None def is_source_asset(self) -> bool: - return self._external_asset_node.is_source + node = self._external_asset_node + return node.is_source or node.is_external and len(node.dependencies) == 0 def resolve_hasMaterializePermission( self, @@ -962,7 +965,10 @@ def resolve_isPartitioned(self, _graphene_info: ResolveInfo) -> bool: return self._external_asset_node.partitions_def_data is not None def resolve_isObservable(self, _graphene_info: ResolveInfo) -> bool: - return self._external_asset_node.is_observable + return self._external_asset_node.execution_type == AssetExecutionType.OBSERVATION + + def resolve_isExternal(self, _graphene_info: ResolveInfo) -> bool: + return self._external_asset_node.is_external def resolve_isExecutable(self, _graphene_info: ResolveInfo) -> bool: return self._external_asset_node.is_executable diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/repo.py b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/repo.py index 0085b5822b651..1b3db957428cc 100644 --- a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/repo.py +++ b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/repo.py @@ -1335,8 +1335,8 @@ def never_runs_asset( hanging_job = build_assets_job( name="hanging_job", - source_assets=[dummy_source_asset], - assets=[first_asset, hanging_asset, never_runs_asset], + other_assets=[dummy_source_asset], + assets_to_execute=[first_asset, hanging_asset, never_runs_asset], resource_defs={ "io_manager": IOManagerDefinition.hardcoded_io_manager(DummyIOManager()), "hanging_asset_resource": hanging_asset_resource, @@ -1383,7 +1383,7 @@ def downstream_asset(hanging_graph): hanging_graph_asset_job = build_assets_job( name="hanging_graph_asset_job", - assets=[hanging_graph_asset, downstream_asset], + assets_to_execute=[hanging_graph_asset, downstream_asset], resource_defs={ "hanging_asset_resource": hanging_asset_resource, "io_manager": IOManagerDefinition.hardcoded_io_manager(DummyIOManager()), @@ -1401,7 +1401,7 @@ def asset_two(asset_one): return asset_one + 1 -two_assets_job = build_assets_job(name="two_assets_job", assets=[asset_one, asset_two]) +two_assets_job = build_assets_job(name="two_assets_job", assets_to_execute=[asset_one, asset_two]) @asset @@ -1412,7 +1412,7 @@ def executable_asset() -> None: unexecutable_asset = next(iter(external_assets_from_specs([AssetSpec("unexecutable_asset")]))) executable_test_job = build_assets_job( - name="executable_test_job", assets=[executable_asset, unexecutable_asset] + name="executable_test_job", assets_to_execute=[executable_asset, unexecutable_asset] ) static_partitions_def = StaticPartitionsDefinition(["a", "b", "c", "d", "e", "f"]) @@ -1455,7 +1455,7 @@ def downstream_dynamic_partitioned_asset( dynamic_partitioned_assets_job = build_assets_job( "dynamic_partitioned_assets_job", - assets=[upstream_dynamic_partitioned_asset, downstream_dynamic_partitioned_asset], + assets_to_execute=[upstream_dynamic_partitioned_asset, downstream_dynamic_partitioned_asset], ) @@ -1529,7 +1529,7 @@ def yield_partition_materialization(): partition_materialization_job = build_assets_job( "partition_materialization_job", - assets=[yield_partition_materialization], + assets_to_execute=[yield_partition_materialization], executor_def=in_process_executor, ) @@ -1543,7 +1543,7 @@ def fail_partition_materialization(context): fail_partition_materialization_job = build_assets_job( "fail_partition_materialization_job", - assets=[fail_partition_materialization], + assets_to_execute=[fail_partition_materialization], executor_def=in_process_executor, ) @@ -1562,7 +1562,7 @@ def hanging_partition_asset(context): hanging_partition_asset_job = build_assets_job( "hanging_partition_asset_job", - assets=[hanging_partition_asset], + assets_to_execute=[hanging_partition_asset], executor_def=in_process_executor, resource_defs={ "io_manager": IOManagerDefinition.hardcoded_io_manager(DummyIOManager()), @@ -1580,7 +1580,7 @@ def asset_yields_observation(): observation_job = build_assets_job( "observation_job", - assets=[asset_yields_observation], + assets_to_execute=[asset_yields_observation], executor_def=in_process_executor, ) diff --git a/python_modules/dagster/dagster/_core/definitions/asset_checks.py b/python_modules/dagster/dagster/_core/definitions/asset_checks.py index 75e6dcf3cedbc..feebefcfea5f0 100644 --- a/python_modules/dagster/dagster/_core/definitions/asset_checks.py +++ b/python_modules/dagster/dagster/_core/definitions/asset_checks.py @@ -255,4 +255,5 @@ def blocking_asset(**kwargs): auto_materialize_policy=asset_def.auto_materialize_policies_by_key.get(asset_def.key), backfill_policy=asset_def.backfill_policy, config=None, # gets config from asset_def.op + _execution_type=asset_def.execution_type, ) diff --git a/python_modules/dagster/dagster/_core/definitions/asset_graph.py b/python_modules/dagster/dagster/_core/definitions/asset_graph.py index fbda6375a9dda..f5b07c35d225e 100644 --- a/python_modules/dagster/dagster/_core/definitions/asset_graph.py +++ b/python_modules/dagster/dagster/_core/definitions/asset_graph.py @@ -22,6 +22,7 @@ import toposort import dagster._check as check +from dagster._core.definitions.asset_spec import AssetExecutionType from dagster._core.definitions.auto_materialize_policy import AutoMaterializePolicy from dagster._core.errors import DagsterInvalidInvocationError from dagster._core.instance import DynamicPartitionsStore @@ -87,6 +88,7 @@ def __init__( required_assets_and_checks_by_key: Mapping[ AssetKeyOrCheckKey, AbstractSet[AssetKeyOrCheckKey] ], + execution_types_by_key: Mapping[AssetKey, AssetExecutionType], ): self._asset_dep_graph = asset_dep_graph self._source_asset_keys = source_asset_keys @@ -100,10 +102,8 @@ def __init__( self._is_observable_by_key = is_observable_by_key self._auto_observe_interval_minutes_by_key = auto_observe_interval_minutes_by_key # source assets keys can sometimes appear in the upstream dict - self._materializable_asset_keys = ( - self._asset_dep_graph["upstream"].keys() - self.source_asset_keys - ) self._required_assets_and_checks_by_key = required_assets_and_checks_by_key + self._execution_types_by_key = execution_types_by_key @property def asset_dep_graph(self) -> DependencyGraph[AssetKey]: @@ -117,6 +117,10 @@ def group_names_by_key(self) -> Mapping[AssetKey, Optional[str]]: def source_asset_keys(self) -> AbstractSet[AssetKey]: return self._source_asset_keys + @property + def external_asset_keys(self) -> AbstractSet[AssetKey]: + return self.all_asset_keys - self.materializable_asset_keys + @functools.cached_property def root_asset_keys(self) -> AbstractSet[AssetKey]: """Non-source asset keys that have no non-source parents.""" @@ -140,6 +144,10 @@ def root_materializable_or_observable_asset_keys(self) -> AbstractSet[AssetKey]: def freshness_policies_by_key(self) -> Mapping[AssetKey, Optional[FreshnessPolicy]]: return self._freshness_policies_by_key + @property + def observable_keys(self) -> AbstractSet[AssetKey]: + return {key for key, is_observable in self._is_observable_by_key.items() if is_observable} + @property def auto_materialize_policies_by_key( self, @@ -150,6 +158,10 @@ def auto_materialize_policies_by_key( def backfill_policies_by_key(self) -> Mapping[AssetKey, Optional[BackfillPolicy]]: return self._backfill_policies_by_key + @property + def execution_types_by_key(self) -> Mapping[AssetKey, AssetExecutionType]: + return self._execution_types_by_key + def get_auto_observe_interval_minutes(self, asset_key: AssetKey) -> Optional[float]: return self._auto_observe_interval_minutes_by_key.get(asset_key) @@ -158,6 +170,10 @@ def from_assets( all_assets: Iterable[Union[AssetsDefinition, SourceAsset]], asset_checks: Optional[Sequence[AssetChecksDefinition]] = None, ) -> "InternalAssetGraph": + from dagster._core.definitions.external_asset import ( + SYSTEM_METADATA_KEY_AUTO_OBSERVE_INTERVAL_MINUTES, + ) + assets_defs: List[AssetsDefinition] = [] source_assets: List[SourceAsset] = [] partitions_defs_by_key: Dict[AssetKey, Optional[PartitionsDefinition]] = {} @@ -165,6 +181,7 @@ def from_assets( AssetKey, Optional[Mapping[AssetKey, PartitionMapping]] ] = {} group_names_by_key: Dict[AssetKey, Optional[str]] = {} + execution_types_by_key: Dict[AssetKey, AssetExecutionType] = {} freshness_policies_by_key: Dict[AssetKey, Optional[FreshnessPolicy]] = {} auto_materialize_policies_by_key: Dict[AssetKey, Optional[AutoMaterializePolicy]] = {} backfill_policies_by_key: Dict[AssetKey, Optional[BackfillPolicy]] = {} @@ -184,6 +201,7 @@ def from_assets( auto_observe_interval_minutes_by_key[ asset.key ] = asset.auto_observe_interval_minutes + execution_types_by_key[asset.key] = AssetExecutionType.UNEXECUTABLE else: # AssetsDefinition assets_defs.append(asset) partition_mappings_by_key.update( @@ -195,6 +213,27 @@ def from_assets( auto_materialize_policies_by_key.update(asset.auto_materialize_policies_by_key) backfill_policies_by_key.update({key: asset.backfill_policy for key in asset.keys}) code_versions_by_key.update(asset.code_versions_by_key) + for key in asset.keys: + execution_types_by_key[key] = asset.execution_type + + is_observable = asset.execution_type == AssetExecutionType.OBSERVATION + is_observable_by_key.update({key: is_observable for key in asset.keys}) + + # Set auto_observe_interval_minutes for external observable assets + # This can be removed when/if we have a a solution for mapping + # `auto_observe_interval_minutes` to an AutoMaterialzePolicy + first_key = next(iter(asset.keys), None) + if ( + first_key + and SYSTEM_METADATA_KEY_AUTO_OBSERVE_INTERVAL_MINUTES + in asset.metadata_by_key[first_key] + ): + interval = asset.metadata_by_key[first_key][ + SYSTEM_METADATA_KEY_AUTO_OBSERVE_INTERVAL_MINUTES + ] + auto_observe_interval_minutes_by_key.update( + {key: interval for key in asset.keys} + ) if not asset.can_subset: all_required_keys = {*asset.check_keys, *asset.keys} @@ -218,15 +257,20 @@ def from_assets( is_observable_by_key=is_observable_by_key, auto_observe_interval_minutes_by_key=auto_observe_interval_minutes_by_key, required_assets_and_checks_by_key=required_assets_and_checks_by_key, + execution_types_by_key=execution_types_by_key, ) @property def materializable_asset_keys(self) -> AbstractSet[AssetKey]: - return self._materializable_asset_keys + return { + k + for k, v in self._execution_types_by_key.items() + if v == AssetExecutionType.MATERIALIZATION + } @property def all_asset_keys(self) -> AbstractSet[AssetKey]: - return self._materializable_asset_keys | self.source_asset_keys + return self._execution_types_by_key.keys() def get_partitions_def(self, asset_key: AssetKey) -> Optional[PartitionsDefinition]: return self._partitions_defs_by_key.get(asset_key) @@ -275,7 +319,10 @@ def have_same_or_no_partitioning(self, asset_keys: Iterable[AssetKey]) -> bool: ) def is_observable(self, asset_key: AssetKey) -> bool: - return self._is_observable_by_key.get(asset_key, False) + return ( + self._is_observable_by_key.get(asset_key, False) + or self._execution_types_by_key.get(asset_key) == AssetExecutionType.OBSERVATION + ) def get_children(self, asset_key: AssetKey) -> AbstractSet[AssetKey]: """Returns all assets that depend on the given asset.""" @@ -718,6 +765,7 @@ def __init__( required_assets_and_checks_by_key: Mapping[ AssetKeyOrCheckKey, AbstractSet[AssetKeyOrCheckKey] ], + execution_types_by_key: Mapping[AssetKey, AssetExecutionType], ): super().__init__( asset_dep_graph=asset_dep_graph, @@ -732,6 +780,7 @@ def __init__( is_observable_by_key=is_observable_by_key, auto_observe_interval_minutes_by_key=auto_observe_interval_minutes_by_key, required_assets_and_checks_by_key=required_assets_and_checks_by_key, + execution_types_by_key=execution_types_by_key, ) self._assets = assets self._source_assets = source_assets @@ -760,13 +809,15 @@ def source_assets(self) -> Sequence[SourceAsset]: def asset_checks(self) -> Sequence[AssetChecksDefinition]: return self._asset_checks - def includes_materializable_and_source_assets(self, asset_keys: AbstractSet[AssetKey]) -> bool: + def includes_materializable_and_external_assets( + self, asset_keys: AbstractSet[AssetKey] + ) -> bool: """Returns true if the given asset keys contains at least one materializable asset and at least one source asset. """ - selected_source_assets = self.source_asset_keys & asset_keys - selected_regular_assets = asset_keys - self.source_asset_keys - return len(selected_source_assets) > 0 and len(selected_regular_assets) > 0 + selected_external_assets = self.external_asset_keys & asset_keys + selected_regular_assets = asset_keys - self.external_asset_keys + return len(selected_external_assets) > 0 and len(selected_regular_assets) > 0 def sort_key_for_asset_partition( diff --git a/python_modules/dagster/dagster/_core/definitions/asset_layer.py b/python_modules/dagster/dagster/_core/definitions/asset_layer.py index f8ab025115d64..adbb55e6bcd1f 100644 --- a/python_modules/dagster/dagster/_core/definitions/asset_layer.py +++ b/python_modules/dagster/dagster/_core/definitions/asset_layer.py @@ -20,6 +20,9 @@ import dagster._check as check from dagster._core.definitions.asset_check_spec import AssetCheckKey, AssetCheckSpec from dagster._core.definitions.asset_checks import AssetChecksDefinition +from dagster._core.definitions.asset_spec import ( + AssetExecutionType, +) from dagster._core.definitions.hook_definition import HookDefinition from dagster._core.definitions.metadata import ( ArbitraryMetadataMapping, @@ -296,11 +299,11 @@ def asset_key_to_dep_node_handles( dep_nodes_by_asset_key: Dict[AssetKey, List[NodeHandle]] = {} dep_node_outputs_by_asset_key: Dict[AssetKey, List[NodeOutputHandle]] = {} - for node_handle, assets_defs in assets_defs_by_node_handle.items(): + for node_handle, assets_def in assets_defs_by_node_handle.items(): dep_node_output_handles_by_node: Dict[ NodeOutputHandle, Sequence[NodeOutputHandle] ] = {} # memoized map of node output handles to all node output handle dependencies that are from ops - for output_name, asset_key in assets_defs.keys_by_output_name.items(): + for output_name, asset_key in assets_def.keys_by_output_name.items(): dep_nodes_by_asset_key[ asset_key ] = [] # first element in list is node that outputs asset @@ -373,6 +376,7 @@ class AssetLayer(NamedTuple): keys for each asset key produced by this job. """ + asset_keys_to_execute: AbstractSet[AssetKey] assets_defs_by_key: Mapping[AssetKey, "AssetsDefinition"] assets_defs_by_node_handle: Mapping[NodeHandle, "AssetsDefinition"] asset_keys_by_node_input_handle: Mapping[NodeInputHandle, AssetKey] @@ -380,7 +384,6 @@ class AssetLayer(NamedTuple): check_key_by_node_output_handle: Mapping[NodeOutputHandle, AssetCheckKey] asset_deps: Mapping[AssetKey, AbstractSet[AssetKey]] dependency_node_handles_by_asset_key: Mapping[AssetKey, Set[NodeHandle]] - source_assets_by_key: Mapping[AssetKey, "SourceAsset"] io_manager_keys_by_asset_key: Mapping[AssetKey, str] # Used to store the asset key dependencies of op node handles within graph backed assets # See AssetLayer.downstream_dep_assets for more information @@ -395,10 +398,9 @@ class AssetLayer(NamedTuple): @staticmethod def from_graph_and_assets_node_mapping( graph_def: GraphDefinition, - assets_defs_by_outer_node_handle: Mapping[NodeHandle, "AssetsDefinition"], - asset_checks_defs_by_node_handle: Mapping[NodeHandle, "AssetChecksDefinition"], - observable_source_assets_by_node_handle: Mapping[NodeHandle, "SourceAsset"], - source_assets: Sequence["SourceAsset"], + assets_to_execute_by_node_handle: Mapping[NodeHandle, "AssetsDefinition"], + asset_checks_by_node_handle: Mapping[NodeHandle, "AssetChecksDefinition"], + other_assets: Sequence["AssetsDefinition"], resolved_asset_deps: "ResolvedAssetDependencies", ) -> "AssetLayer": """Generate asset info from a GraphDefinition and a mapping from nodes in that graph to the @@ -410,24 +412,37 @@ def from_graph_and_assets_node_mapping( assets_defs_by_outer_node_handle (Mapping[NodeHandle, AssetsDefinition]): A mapping from a NodeHandle pointing to the node in the graph where the AssetsDefinition ended up. """ + from dagster._core.definitions.assets import AssetsDefinition + asset_key_by_input: Dict[NodeInputHandle, AssetKey] = {} asset_info_by_output: Dict[NodeOutputHandle, AssetOutputInfo] = {} check_key_by_output: Dict[NodeOutputHandle, AssetCheckKey] = {} asset_deps: Dict[AssetKey, AbstractSet[AssetKey]] = {} io_manager_by_asset: Dict[AssetKey, str] = { - source_asset.key: source_asset.get_io_manager_key() for source_asset in source_assets + key: asset.get_io_manager_key_for_asset_key(key) + for asset in other_assets + for key in asset.keys } partition_mappings_by_asset_dep: Dict[Tuple[NodeHandle, AssetKey], "PartitionMapping"] = {} + assets_to_observe_by_node_handle = { + k: v for k, v in assets_to_execute_by_node_handle.items() if v.is_observable + } + + # This can be executed for just materialized assets because observed assets do not have node + # dependencies. ( dep_node_handles_by_asset_key, dep_node_output_handles_by_asset_key, - ) = asset_key_to_dep_node_handles(graph_def, assets_defs_by_outer_node_handle) + ) = asset_key_to_dep_node_handles( + graph_def, + {k: v for k, v in assets_to_execute_by_node_handle.items()}, + ) node_output_handles_by_asset_check_key: Mapping[AssetCheckKey, NodeOutputHandle] = {} check_names_by_asset_key_by_node_handle: Dict[NodeHandle, Dict[AssetKey, Set[str]]] = {} - for node_handle, assets_def in assets_defs_by_outer_node_handle.items(): + for node_handle, assets_def in assets_to_execute_by_node_handle.items(): for key in assets_def.keys: asset_deps[key] = resolved_asset_deps.get_resolved_upstream_asset_keys( assets_def, key @@ -516,7 +531,7 @@ def partitions_fn(context: "OutputContext") -> AbstractSet[str]: for node_output_handle in node_output_handles: dep_asset_keys_by_node_output_handle[node_output_handle].add(asset_key) - for node_handle, checks_def in asset_checks_defs_by_node_handle.items(): + for node_handle, checks_def in asset_checks_by_node_handle.items(): check_names_by_asset_key_by_node_handle[node_handle] = defaultdict(set) for output_name, check_spec in checks_def.specs_by_output_name.items(): inner_output_def, inner_node_handle = checks_def.node_def.resolve_output_to_origin( @@ -541,31 +556,13 @@ def partitions_fn(context: "OutputContext") -> AbstractSet[str]: assets_defs_by_key = { key: assets_def - for assets_def in assets_defs_by_outer_node_handle.values() + for assets_def in [ + *assets_to_execute_by_node_handle.values(), + *other_assets, + ] for key in assets_def.keys } - source_assets_by_key = {source_asset.key: source_asset for source_asset in source_assets} - for node_handle, source_asset in observable_source_assets_by_node_handle.items(): - node_def = cast(NodeDefinition, source_asset.node_def) - check.invariant(len(node_def.output_defs) == 1) - output_name = node_def.output_defs[0].name - # resolve graph output to the op output it comes from - inner_output_def, inner_node_handle = node_def.resolve_output_to_origin( - output_name, handle=node_handle - ) - node_output_handle = NodeOutputHandle( - check.not_none(inner_node_handle), inner_output_def.name - ) - - asset_info_by_output[node_output_handle] = AssetOutputInfo( - source_asset.key, - partitions_fn=None, - partitions_def=source_asset.partitions_def, - is_required=True, - code_version=inner_output_def.code_version, - ) - assets_defs_by_node_handle: Dict[NodeHandle, "AssetsDefinition"] = { # nodes for assets **{ @@ -573,16 +570,27 @@ def partitions_fn(context: "OutputContext") -> AbstractSet[str]: for asset_key, node_handles in dep_node_handles_by_asset_key.items() for node_handle in node_handles }, + **{ + node_handle: assets_def + for node_handle, assets_def in assets_to_observe_by_node_handle.items() + if isinstance(assets_def, AssetsDefinition) + }, # nodes for asset checks. Required for AssetsDefs that have selected checks # but not assets **{ - node_handle: assets_def - for node_handle, assets_def in assets_defs_by_outer_node_handle.items() - if assets_def.check_keys + node_handle: asset + for node_handle, asset in assets_to_execute_by_node_handle.items() + if isinstance(asset, AssetsDefinition) and asset.check_keys }, } + asset_keys_to_execute = { + key + for assets_def in assets_to_execute_by_node_handle.values() + for key in assets_def.keys + } return AssetLayer( + asset_keys_to_execute=asset_keys_to_execute, asset_keys_by_node_input_handle=asset_key_by_input, asset_info_by_node_output_handle=asset_info_by_output, check_key_by_node_output_handle=check_key_by_output, @@ -590,11 +598,10 @@ def partitions_fn(context: "OutputContext") -> AbstractSet[str]: assets_defs_by_node_handle=assets_defs_by_node_handle, dependency_node_handles_by_asset_key=dep_node_handles_by_asset_key, assets_defs_by_key=assets_defs_by_key, - source_assets_by_key=source_assets_by_key, io_manager_keys_by_asset_key=io_manager_by_asset, dep_asset_keys_by_node_output_handle=dep_asset_keys_by_node_output_handle, partition_mappings_by_asset_dep=partition_mappings_by_asset_dep, - asset_checks_defs_by_node_handle=asset_checks_defs_by_node_handle, + asset_checks_defs_by_node_handle=asset_checks_by_node_handle, node_output_handles_by_asset_check_key=node_output_handles_by_asset_check_key, check_names_by_asset_key_by_node_handle=check_names_by_asset_key_by_node_handle, ) @@ -609,9 +616,39 @@ def upstream_assets_for_asset(self, asset_key: AssetKey) -> AbstractSet[AssetKey def downstream_assets_for_asset(self, asset_key: AssetKey) -> AbstractSet[AssetKey]: return {k for k, v in self.asset_deps.items() if asset_key in v} + @property + def target_asset_keys(self) -> Iterable[AssetKey]: + return self.asset_keys_to_execute + # return set(self.dependency_node_handles_by_asset_key.keys()) & self.executable_asset_keys + @property def asset_keys(self) -> Iterable[AssetKey]: - return self.dependency_node_handles_by_asset_key.keys() + # return self.dependency_node_handles_by_asset_key.keys() + return self.assets_defs_by_key.keys() + + @property + def observable_asset_keys(self) -> Iterable[AssetKey]: + return { + asset_key + for asset_key, assets_def in self.assets_defs_by_key.items() + if assets_def.is_observable + } + + @property + def materializable_asset_keys(self) -> Iterable[AssetKey]: + return { + asset_key + for asset_key, assets_def in self.assets_defs_by_key.items() + if assets_def.is_materializable + } + + @property + def executable_asset_keys(self) -> Iterable[AssetKey]: + return { + asset_key + for asset_key, assets_def in self.assets_defs_by_key.items() + if assets_def.is_executable + } @property def has_assets_defs(self) -> bool: @@ -706,10 +743,16 @@ def input_for_asset_key(self, node_handle: NodeHandle, key: AssetKey) -> Optiona def io_manager_key_for_asset(self, asset_key: AssetKey) -> str: return self.io_manager_keys_by_asset_key.get(asset_key, "io_manager") + def execution_type_for_asset(self, asset_key: AssetKey) -> AssetExecutionType: + if asset_key in self.assets_defs_by_key: + return self.assets_defs_by_key[asset_key].execution_type + else: + check.failed(f"Couldn't find key {asset_key}") + def is_observable_for_asset(self, asset_key: AssetKey) -> bool: return ( - asset_key in self.source_assets_by_key - and self.source_assets_by_key[asset_key].is_observable + asset_key in self.assets_defs_by_key + and self.assets_defs_by_key[asset_key].execution_type == AssetExecutionType.OBSERVATION ) def is_materializable_for_asset(self, asset_key: AssetKey) -> bool: @@ -729,10 +772,7 @@ def code_version_for_asset(self, asset_key: AssetKey) -> Optional[str]: def metadata_for_asset( self, asset_key: AssetKey ) -> Optional[Mapping[str, ArbitraryMetadataMapping]]: - if asset_key in self.source_assets_by_key: - raw_metadata = self.source_assets_by_key[asset_key].raw_metadata - return raw_metadata or None - elif asset_key in self.assets_defs_by_key: + if asset_key in self.assets_defs_by_key: return self.assets_defs_by_key[asset_key].metadata_by_key[asset_key] else: check.failed(f"Couldn't find key {asset_key}") @@ -755,32 +795,15 @@ def asset_check_key_for_output( return self.check_key_by_node_output_handle.get(NodeOutputHandle(node_handle, output_name)) def group_names_by_assets(self) -> Mapping[AssetKey, str]: - group_names: Dict[AssetKey, str] = { + return { key: assets_def.group_names_by_key[key] for key, assets_def in self.assets_defs_by_key.items() if key in assets_def.group_names_by_key } - group_names.update( - { - key: source_asset_def.group_name - for key, source_asset_def in self.source_assets_by_key.items() - } - ) - - return group_names - def partitions_def_for_asset(self, asset_key: AssetKey) -> Optional["PartitionsDefinition"]: assets_def = self.assets_defs_by_key.get(asset_key) - - if assets_def is not None: - return assets_def.partitions_def - else: - source_asset = self.source_assets_by_key.get(asset_key) - if source_asset is not None: - return source_asset.partitions_def - - return None + return assets_def.partitions_def if assets_def is not None else None def partition_mapping_for_node_input( self, node_handle: NodeHandle, upstream_asset_key: AssetKey @@ -883,23 +906,24 @@ def build_asset_selection_job( f"{partitions_def}.", ) - if len(included_assets) or len(included_checks_defs) > 0: - # Job materializes assets and/or executes checks - final_assets = included_assets - final_asset_checks = included_checks_defs - final_source_assets = [*source_assets, *excluded_assets] - else: - # Job only observes source assets - final_assets = [] - final_asset_checks = [] - final_source_assets = included_source_assets + included_observables = [asset for asset in included_source_assets if asset.is_observable] + final_assets = [*included_assets, *included_observables] + final_asset_checks = included_checks_defs + final_source_assets = [ + *( + source_asset + for source_asset in source_assets + if source_asset not in included_observables + ), + *excluded_assets, + ] return build_assets_job( name=name, - assets=final_assets, + assets_to_execute=final_assets, asset_checks=final_asset_checks, config=config, - source_assets=final_source_assets, + other_assets=final_source_assets, resource_defs=resource_defs, executor_def=executor_def, partitions_def=partitions_def, diff --git a/python_modules/dagster/dagster/_core/definitions/asset_spec.py b/python_modules/dagster/dagster/_core/definitions/asset_spec.py index 62c4d4a8589fa..ae86a034261b8 100644 --- a/python_modules/dagster/dagster/_core/definitions/asset_spec.py +++ b/python_modules/dagster/dagster/_core/definitions/asset_spec.py @@ -3,6 +3,7 @@ import dagster._check as check from dagster._annotations import PublicAttr +from dagster._serdes.serdes import whitelist_for_serdes from .auto_materialize_policy import AutoMaterializePolicy from .events import ( @@ -15,15 +16,8 @@ if TYPE_CHECKING: from dagster._core.definitions.asset_dep import AssetDep, CoercibleToAssetDep -# SYSTEM_METADATA_KEY_ASSET_EXECUTION_TYPE lives on the metadata of an asset -# (which currently ends up on the Output associated with the asset key) -# whih encodes the execution type the of asset. "Unexecutable" assets are assets -# that cannot be materialized in Dagster, but can have events in the event -# log keyed off of them, making Dagster usable as a observability and lineage tool -# for externally materialized assets. -SYSTEM_METADATA_KEY_ASSET_EXECUTION_TYPE = "dagster/asset_execution_type" - +@whitelist_for_serdes class AssetExecutionType(Enum): OBSERVATION = "OBSERVATION" UNEXECUTABLE = "UNEXECUTABLE" diff --git a/python_modules/dagster/dagster/_core/definitions/assets.py b/python_modules/dagster/dagster/_core/definitions/assets.py index 270f62a3a632c..bf438ac8e6952 100644 --- a/python_modules/dagster/dagster/_core/definitions/assets.py +++ b/python_modules/dagster/dagster/_core/definitions/assets.py @@ -30,6 +30,7 @@ from dagster._core.definitions.op_selection import get_graph_subset from dagster._core.definitions.partition_mapping import MultiPartitionMapping from dagster._core.definitions.resource_requirement import ( + ExternalAssetIOManagerRequirement, RequiresResources, ResourceAddable, ResourceRequirement, @@ -94,6 +95,7 @@ class AssetsDefinition(ResourceAddable, RequiresResources, IHasInternalInit): _descriptions_by_key: Mapping[AssetKey, str] _selected_asset_check_keys: AbstractSet[AssetCheckKey] _is_subset: bool + _execution_type: AssetExecutionType def __init__( self, @@ -116,6 +118,7 @@ def __init__( check_specs_by_output_name: Optional[Mapping[str, AssetCheckSpec]] = None, selected_asset_check_keys: Optional[AbstractSet[AssetCheckKey]] = None, is_subset: bool = False, + _execution_type: AssetExecutionType = AssetExecutionType.MATERIALIZATION, # if adding new fields, make sure to handle them in the with_attributes, from_graph, and # get_attributes_dict methods ): @@ -315,6 +318,9 @@ def __init__( ) self._is_subset = check.bool_param(is_subset, "is_subset") + self._execution_type = check.inst_param( + _execution_type, "_execution_type", AssetExecutionType + ) @staticmethod def dagster_internal_init( @@ -337,6 +343,7 @@ def dagster_internal_init( check_specs_by_output_name: Optional[Mapping[str, AssetCheckSpec]], selected_asset_check_keys: Optional[AbstractSet[AssetCheckKey]], is_subset: bool, + _execution_type: AssetExecutionType, ) -> "AssetsDefinition": return AssetsDefinition( keys_by_input_name=keys_by_input_name, @@ -357,6 +364,7 @@ def dagster_internal_init( check_specs_by_output_name=check_specs_by_output_name, selected_asset_check_keys=selected_asset_check_keys, is_subset=is_subset, + _execution_type=_execution_type, ) def __call__(self, *args: object, **kwargs: object) -> object: @@ -394,6 +402,7 @@ def from_graph( backfill_policy: Optional[BackfillPolicy] = None, can_subset: bool = False, check_specs: Optional[Sequence[AssetCheckSpec]] = None, + _execution_type: AssetExecutionType = AssetExecutionType.MATERIALIZATION, ) -> "AssetsDefinition": """Constructs an AssetsDefinition from a GraphDefinition. @@ -466,6 +475,7 @@ def from_graph( backfill_policy=backfill_policy, can_subset=can_subset, check_specs=check_specs, + _execution_type=_execution_type, ) @public @@ -489,6 +499,7 @@ def from_op( ] = None, backfill_policy: Optional[BackfillPolicy] = None, can_subset: bool = False, + _execution_type: AssetExecutionType = AssetExecutionType.MATERIALIZATION, ) -> "AssetsDefinition": """Constructs an AssetsDefinition from an OpDefinition. @@ -554,6 +565,7 @@ def from_op( auto_materialize_policies_by_output_name=auto_materialize_policies_by_output_name, backfill_policy=backfill_policy, can_subset=can_subset, + _execution_type=_execution_type, ) @staticmethod @@ -578,6 +590,7 @@ def _from_node( backfill_policy: Optional[BackfillPolicy] = None, can_subset: bool = False, check_specs: Optional[Sequence[AssetCheckSpec]] = None, + _execution_type: AssetExecutionType = AssetExecutionType.MATERIALIZATION, ) -> "AssetsDefinition": from dagster._core.definitions.decorators.asset_decorator import ( _assign_output_names_to_check_specs, @@ -715,6 +728,7 @@ def _from_node( check_specs_by_output_name=check_specs_by_output_name, selected_asset_check_keys=None, is_subset=False, + _execution_type=_execution_type, ) @public @@ -908,33 +922,25 @@ def check_keys(self) -> AbstractSet[AssetCheckKey]: """ return self._selected_asset_check_keys - def is_asset_executable(self, asset_key: AssetKey) -> bool: - """Returns True if the asset key is materializable by this AssetsDefinition. - - Args: - asset_key (AssetKey): The asset key to check. + @property + def execution_type(self) -> AssetExecutionType: + return self._execution_type - Returns: - bool: True if the asset key is materializable by this AssetsDefinition. - """ - from dagster._core.definitions.asset_spec import ( - SYSTEM_METADATA_KEY_ASSET_EXECUTION_TYPE, - AssetExecutionType, - ) + @property + def is_external(self) -> bool: + return self.execution_type != AssetExecutionType.MATERIALIZATION - return AssetExecutionType.is_executable( - self._metadata_by_key.get(asset_key, {}).get(SYSTEM_METADATA_KEY_ASSET_EXECUTION_TYPE) - ) + @property + def is_observable(self) -> bool: + return self.execution_type == AssetExecutionType.OBSERVATION - def asset_execution_type_for_asset(self, asset_key: AssetKey) -> AssetExecutionType: - from dagster._core.definitions.asset_spec import ( - SYSTEM_METADATA_KEY_ASSET_EXECUTION_TYPE, - AssetExecutionType, - ) + @property + def is_materializable(self) -> bool: + return self.execution_type == AssetExecutionType.MATERIALIZATION - return AssetExecutionType.str_to_enum( - self._metadata_by_key.get(asset_key, {}).get(SYSTEM_METADATA_KEY_ASSET_EXECUTION_TYPE) - ) + @property + def is_executable(self) -> bool: + return self.execution_type != AssetExecutionType.UNEXECUTABLE def get_partition_mapping_for_input(self, input_name: str) -> Optional[PartitionMapping]: return self._partition_mappings.get(self._keys_by_input_name[input_name]) @@ -1133,6 +1139,7 @@ def with_attributes( selected_asset_check_keys=selected_asset_check_keys if selected_asset_check_keys else self._selected_asset_check_keys, + _execution_type=self.execution_type, ) return self.__class__(**merge_dicts(self.get_attributes_dict(), replaced_attributes)) @@ -1328,7 +1335,14 @@ def get_io_manager_key_for_asset_key(self, key: AssetKey) -> str: )[0].io_manager_key def get_resource_requirements(self) -> Iterator[ResourceRequirement]: - yield from self.node_def.get_resource_requirements() # type: ignore[attr-defined] + if self.is_executable: + yield from self.node_def.get_resource_requirements() # type: ignore[attr-defined] + else: + for key in self.keys: + yield ExternalAssetIOManagerRequirement( + key=self.get_io_manager_key_for_asset_key(key), + asset_key=key.to_string(), + ) for source_key, resource_def in self.resource_defs.items(): yield from resource_def.get_resource_requirements(outer_context=source_key) @@ -1378,6 +1392,7 @@ def get_attributes_dict(self) -> Dict[str, Any]: descriptions_by_key=self._descriptions_by_key, check_specs_by_output_name=self._check_specs_by_output_name, selected_asset_check_keys=self._selected_asset_check_keys, + _execution_type=self._execution_type, ) diff --git a/python_modules/dagster/dagster/_core/definitions/assets_job.py b/python_modules/dagster/dagster/_core/definitions/assets_job.py index b5529f79efa85..4ac6f3169f8b2 100644 --- a/python_modules/dagster/dagster/_core/definitions/assets_job.py +++ b/python_modules/dagster/dagster/_core/definitions/assets_job.py @@ -63,56 +63,53 @@ def is_base_asset_job_name(name: str) -> bool: def get_base_asset_jobs( assets: Sequence[AssetsDefinition], - source_assets: Sequence[SourceAsset], asset_checks: Sequence[AssetChecksDefinition], resource_defs: Optional[Mapping[str, ResourceDefinition]], executor_def: Optional[ExecutorDefinition], ) -> Sequence[JobDefinition]: - assets_by_partitions_def: Dict[ - Optional[PartitionsDefinition], List[AssetsDefinition] + executable_assets = [ad for ad in assets if ad.is_executable] + unexecutable_assets = [ad for ad in assets if not ad.is_executable] + + executable_assets_by_partitions_def: Dict[ + Optional[PartitionsDefinition], List[Union[AssetsDefinition, SourceAsset]] ] = defaultdict(list) - for assets_def in assets: - assets_by_partitions_def[assets_def.partitions_def].append(assets_def) - - # We need to create "empty" jobs for each partitions def that is used by an observable but no - # materializable asset. They are empty because we don't assign the source asset to the `assets`, - # but rather the `source_assets` argument of `build_assets_job`. - for observable in [sa for sa in source_assets if sa.is_observable]: - if observable.partitions_def not in assets_by_partitions_def: - assets_by_partitions_def[observable.partitions_def] = [] - if len(assets_by_partitions_def.keys()) == 0 or assets_by_partitions_def.keys() == {None}: + for asset in executable_assets: + executable_assets_by_partitions_def[asset.partitions_def].append(asset) + # sort to ensure some stability in the ordering + all_partitions_defs = sorted( + [p for p in executable_assets_by_partitions_def.keys() if p], key=repr + ) + + if len(all_partitions_defs) == 0: return [ build_assets_job( name=ASSET_BASE_JOB_PREFIX, - assets=assets, + assets_to_execute=executable_assets, + other_assets=unexecutable_assets, asset_checks=asset_checks, - source_assets=source_assets, executor_def=executor_def, resource_defs=resource_defs, ) ] else: - unpartitioned_assets = assets_by_partitions_def.get(None, []) - partitioned_assets_by_partitions_def = { - k: v for k, v in assets_by_partitions_def.items() if k is not None - } + unpartitioned_executable_assets = executable_assets_by_partitions_def.get(None, []) jobs = [] - # sort to ensure some stability in the ordering - for i, (partitions_def, assets_with_partitions) in enumerate( - sorted(partitioned_assets_by_partitions_def.items(), key=lambda item: repr(item[0])) - ): + # all partition base jobs contain all unpartitioned assets + for i, partitions_def in enumerate(all_partitions_defs): + partitioned_executable_assets = executable_assets_by_partitions_def[partitions_def] + assets_to_execute = [*partitioned_executable_assets, *unpartitioned_executable_assets] jobs.append( build_assets_job( f"{ASSET_BASE_JOB_PREFIX}_{i}", - assets=[*assets_with_partitions, *unpartitioned_assets], - source_assets=[*source_assets, *assets], + assets_to_execute=assets_to_execute, + other_assets=[ + *(asset for asset in executable_assets if asset not in assets_to_execute), + *unexecutable_assets, + ], asset_checks=asset_checks, resource_defs=resource_defs, executor_def=executor_def, - # Only explicitly set partitions_def for observable-only jobs since it can't be - # auto-detected from the passed assets (which is an empty list). - partitions_def=partitions_def if len(assets_with_partitions) == 0 else None, ) ) return jobs @@ -120,8 +117,8 @@ def get_base_asset_jobs( def build_assets_job( name: str, - assets: Sequence[AssetsDefinition], - source_assets: Optional[Sequence[Union[SourceAsset, AssetsDefinition]]] = None, + assets_to_execute: Sequence[Union[AssetsDefinition, SourceAsset]], + other_assets: Optional[Sequence[Union[SourceAsset, AssetsDefinition]]] = None, asset_checks: Optional[Sequence[AssetChecksDefinition]] = None, resource_defs: Optional[Mapping[str, object]] = None, description: Optional[str] = None, @@ -170,12 +167,13 @@ def asset2(asset1): Returns: JobDefinition: A job that materializes the given assets. """ + from dagster._core.definitions.external_asset import create_external_asset_from_source_asset from dagster._core.execution.build_resources import wrap_resources_for_execution check.str_param(name, "name") - check.iterable_param(assets, "assets", of_type=(AssetsDefinition, SourceAsset)) - source_assets = check.opt_sequence_param( - source_assets, "source_assets", of_type=(SourceAsset, AssetsDefinition) + check.iterable_param(assets_to_execute, "assets", of_type=(AssetsDefinition, SourceAsset)) + other_assets = check.opt_sequence_param( + other_assets, "source_assets", of_type=(SourceAsset, AssetsDefinition) ) asset_checks = check.opt_sequence_param( asset_checks, "asset_checks", of_type=AssetChecksDefinition @@ -184,53 +182,48 @@ def asset2(asset1): check.opt_inst_param(_asset_selection_data, "_asset_selection_data", AssetSelectionData) # figure out what partitions (if any) exist for this job - partitions_def = partitions_def or build_job_partitions_from_assets(assets) + partitions_def = partitions_def or build_job_partitions_from_assets(assets_to_execute) resource_defs = check.opt_mapping_param(resource_defs, "resource_defs") resource_defs = merge_dicts({DEFAULT_IO_MANAGER_KEY: default_job_io_manager}, resource_defs) wrapped_resource_defs = wrap_resources_for_execution(resource_defs) - # turn any AssetsDefinitions into SourceAssets - resolved_source_assets: List[SourceAsset] = [] - for asset in source_assets or []: - if isinstance(asset, AssetsDefinition): - resolved_source_assets += asset.to_source_assets() - elif isinstance(asset, SourceAsset): - resolved_source_assets.append(asset) - - resolved_asset_deps = ResolvedAssetDependencies(assets, resolved_source_assets) - deps, assets_defs_by_node_handle, asset_checks_defs_by_node_handle = build_node_deps( - assets, asset_checks, resolved_asset_deps + # normalize SourceAssets to AssetsDefinition (external assets) + assets_to_execute = [ + create_external_asset_from_source_asset(a) if isinstance(a, SourceAsset) else a + for a in assets_to_execute + ] + other_assets = [ + create_external_asset_from_source_asset(a) if isinstance(a, SourceAsset) else a + for a in other_assets or [] + ] + + all_assets = [*assets_to_execute, *other_assets] + + resolved_asset_deps = ResolvedAssetDependencies( + assets_defs=all_assets, + source_assets=[], + ) + + deps, assets_by_node_handle, asset_checks_by_node_handle = _build_node_deps( + assets_to_execute, asset_checks, resolved_asset_deps ) # attempt to resolve cycles using multi-asset subsetting if _has_cycles(deps): - assets = _attempt_resolve_cycles(assets, resolved_source_assets) - resolved_asset_deps = ResolvedAssetDependencies(assets, resolved_source_assets) + assets_to_execute = _attempt_resolve_cycles(assets_to_execute, other_assets) - deps, assets_defs_by_node_handle, asset_checks_defs_by_node_handle = build_node_deps( - assets, asset_checks, resolved_asset_deps - ) + # assets_to_execute = _attempt_resolve_cycles(all_assets, []) + resolved_asset_deps = ResolvedAssetDependencies([*assets_to_execute, *other_assets], []) - if len(assets) > 0 or len(asset_checks) > 0: - node_defs = [ - *(asset.node_def for asset in assets), - *(asset_check.node_def for asset_check in asset_checks), - ] - observable_source_assets_by_node_handle = {} - else: - node_defs = [] - observable_source_assets_by_node_handle: Mapping[NodeHandle, SourceAsset] = {} - for asset in source_assets: - if ( - isinstance(asset, SourceAsset) - and asset.is_observable - and asset.node_def is not None - ): - node_defs.append(asset.node_def) - node_handle = NodeHandle(asset.node_def.name, parent=None) - observable_source_assets_by_node_handle[node_handle] = asset + deps, assets_by_node_handle, asset_checks_by_node_handle = _build_node_deps( + assets_to_execute, asset_checks, resolved_asset_deps + ) + node_defs = [ + *(asset.node_def for asset in assets_to_execute), + *(asset_check.node_def for asset_check in asset_checks), + ] graph = GraphDefinition( name=name, node_defs=node_defs, @@ -243,16 +236,13 @@ def asset2(asset1): asset_layer = AssetLayer.from_graph_and_assets_node_mapping( graph_def=graph, - asset_checks_defs_by_node_handle=asset_checks_defs_by_node_handle, - source_assets=resolved_source_assets, + assets_to_execute_by_node_handle=assets_by_node_handle, + asset_checks_by_node_handle=asset_checks_by_node_handle, + other_assets=other_assets, resolved_asset_deps=resolved_asset_deps, - assets_defs_by_outer_node_handle=assets_defs_by_node_handle, - observable_source_assets_by_node_handle=observable_source_assets_by_node_handle, ) - all_resource_defs = get_all_resource_defs( - assets, asset_checks, resolved_source_assets, wrapped_resource_defs - ) + all_resource_defs = get_all_resource_defs(all_assets, asset_checks, [], wrapped_resource_defs) if _asset_selection_data: original_job = _asset_selection_data.parent_job_def @@ -288,7 +278,7 @@ def asset2(asset1): def build_job_partitions_from_assets( assets: Iterable[Union[AssetsDefinition, SourceAsset]], ) -> Optional[PartitionsDefinition]: - assets_with_partitions_defs = [assets_def for assets_def in assets if assets_def.partitions_def] + assets_with_partitions_defs = [asset for asset in assets if asset.partitions_def] if len(assets_with_partitions_defs) == 0: return None @@ -317,18 +307,19 @@ def _key_for_asset(asset: Union[AssetsDefinition, SourceAsset]) -> AssetKey: def _get_blocking_asset_check_output_handles_by_asset_key( - assets_defs_by_node_handle, asset_checks_defs_by_node_handle + asset_checks_by_node_handle, asset_checks_defs_by_node_handle ) -> Mapping[AssetKey, AbstractSet[NodeOutputHandle]]: """For each asset key, returns the set of node output handles that correspond to asset check specs that should block the execution of downstream assets if they fail. """ check_specs_by_node_output_handle: Mapping[NodeOutputHandle, AssetCheckSpec] = {} - for node_handle, assets_def in assets_defs_by_node_handle.items(): - for output_name, check_spec in assets_def.check_specs_by_output_name.items(): - check_specs_by_node_output_handle[ - NodeOutputHandle(node_handle, output_name=output_name) - ] = check_spec + for node_handle, assets_def in asset_checks_by_node_handle.items(): + if isinstance(assets_def, AssetsDefinition) and assets_def.is_materializable: + for output_name, check_spec in assets_def.check_specs_by_output_name.items(): + check_specs_by_node_output_handle[ + NodeOutputHandle(node_handle, output_name=output_name) + ] = check_spec for node_handle, asset_checks_def in asset_checks_defs_by_node_handle.items(): for output_name, check_spec in asset_checks_def.specs_by_output_name.items(): @@ -348,9 +339,9 @@ def _get_blocking_asset_check_output_handles_by_asset_key( return blocking_asset_check_output_handles_by_asset_key -def build_node_deps( - assets_defs: Iterable[AssetsDefinition], - asset_checks_defs: Sequence[AssetChecksDefinition], +def _build_node_deps( + assets_to_execute: Iterable[AssetsDefinition], + asset_checks_to_execute: Sequence[AssetChecksDefinition], resolved_asset_deps: ResolvedAssetDependencies, ) -> Tuple[ DependencyMapping[NodeInvocation], @@ -358,16 +349,21 @@ def build_node_deps( Mapping[NodeHandle, AssetChecksDefinition], ]: # sort so that nodes get a consistent name - assets_defs = sorted(assets_defs, key=lambda ad: (sorted((ak for ak in ad.keys)))) + def sort_key_fn(asset: Union[AssetsDefinition, SourceAsset]): + keys = asset.keys if isinstance(asset, AssetsDefinition) else [asset.key] + return sorted((ak for ak in keys)) + + assets_to_execute = sorted(assets_to_execute, key=sort_key_fn) # if the same graph/op is used in multiple assets_definitions, their invocations must have # different names. we keep track of definitions that share a name and add a suffix to their # invocations to solve this issue collisions: Dict[str, int] = {} - assets_defs_by_node_handle: Dict[NodeHandle, AssetsDefinition] = {} + node_handle_to_asset: Dict[NodeHandle, AssetsDefinition] = {} node_alias_and_output_by_asset_key: Dict[AssetKey, Tuple[str, str]] = {} - for assets_def in assets_defs: - node_name = assets_def.node_def.name + for asset in assets_to_execute: + node_def = check.not_none(asset.node_def) + node_name = node_def.name if collisions.get(node_name): collisions[node_name] += 1 node_alias = f"{node_name}_{collisions[node_name]}" @@ -376,70 +372,75 @@ def build_node_deps( node_alias = node_name # unique handle for each AssetsDefinition - assets_defs_by_node_handle[NodeHandle(node_alias, parent=None)] = assets_def - for output_name, key in assets_def.keys_by_output_name.items(): - node_alias_and_output_by_asset_key[key] = (node_alias, output_name) + node_handle_to_asset[NodeHandle(node_alias, parent=None)] = asset + + # Observation outputs are not recorded because they aren't a node dependency + if isinstance(asset, AssetsDefinition) and asset.is_materializable: + for output_name, key in asset.keys_by_output_name.items(): + node_alias_and_output_by_asset_key[key] = (node_alias, output_name) asset_checks_defs_by_node_handle: Dict[NodeHandle, AssetChecksDefinition] = {} - for asset_checks_def in asset_checks_defs: + for asset_checks_def in asset_checks_to_execute: node_def_name = asset_checks_def.node_def.name node_key = NodeInvocation(node_def_name) asset_checks_defs_by_node_handle[NodeHandle(node_def_name, parent=None)] = asset_checks_def blocking_asset_check_output_handles_by_asset_key = ( _get_blocking_asset_check_output_handles_by_asset_key( - assets_defs_by_node_handle, asset_checks_defs_by_node_handle + node_handle_to_asset, asset_checks_defs_by_node_handle ) ) deps: Dict[NodeInvocation, Dict[str, IDependencyDefinition]] = {} - for node_handle, assets_def in assets_defs_by_node_handle.items(): + for node_handle, asset in node_handle_to_asset.items(): # the key that we'll use to reference the node inside this AssetsDefinition - node_def_name = assets_def.node_def.name + node_def_name = check.not_none(asset.node_def).name alias = node_handle.name if node_handle.name != node_def_name else None node_key = NodeInvocation(node_def_name, alias=alias) deps[node_key] = {} # connect each input of this AssetsDefinition to the proper upstream node - for input_name in assets_def.input_names: - upstream_asset_key = resolved_asset_deps.get_resolved_asset_key_for_input( - assets_def, input_name - ) + if isinstance(asset, AssetsDefinition) and asset.is_materializable: + for input_name in asset.input_names: + upstream_asset_key = resolved_asset_deps.get_resolved_asset_key_for_input( + asset, input_name + ) - # ignore self-deps - if upstream_asset_key in assets_def.keys: - continue + # ignore self-deps + if upstream_asset_key in asset.keys: + continue - blocking_asset_check_output_handles = ( - blocking_asset_check_output_handles_by_asset_key.get(upstream_asset_key) - ) - asset_check_deps = [ - DependencyDefinition( - node_output_handle.node_handle.name, node_output_handle.output_name + blocking_asset_check_output_handles = ( + blocking_asset_check_output_handles_by_asset_key.get(upstream_asset_key) ) - for node_output_handle in blocking_asset_check_output_handles or [] - ] - - if upstream_asset_key in node_alias_and_output_by_asset_key: - upstream_node_alias, upstream_output_name = node_alias_and_output_by_asset_key[ - upstream_asset_key + asset_check_deps = [ + DependencyDefinition( + node_output_handle.node_handle.name, node_output_handle.output_name + ) + for node_output_handle in blocking_asset_check_output_handles or [] ] - asset_dep_def = DependencyDefinition(upstream_node_alias, upstream_output_name) - if blocking_asset_check_output_handles: + if upstream_asset_key in node_alias_and_output_by_asset_key: + upstream_node_alias, upstream_output_name = node_alias_and_output_by_asset_key[ + upstream_asset_key + ] + + asset_dep_def = DependencyDefinition(upstream_node_alias, upstream_output_name) + if blocking_asset_check_output_handles: + deps[node_key][input_name] = BlockingAssetChecksDependencyDefinition( + asset_check_dependencies=asset_check_deps, + other_dependency=asset_dep_def, + ) + else: + deps[node_key][input_name] = asset_dep_def + elif asset_check_deps: deps[node_key][input_name] = BlockingAssetChecksDependencyDefinition( - asset_check_dependencies=asset_check_deps, other_dependency=asset_dep_def + asset_check_dependencies=asset_check_deps, other_dependency=None ) - else: - deps[node_key][input_name] = asset_dep_def - elif asset_check_deps: - deps[node_key][input_name] = BlockingAssetChecksDependencyDefinition( - asset_check_dependencies=asset_check_deps, other_dependency=None - ) # put asset checks downstream of the assets they're checking asset_checks_defs_by_node_handle: Dict[NodeHandle, AssetChecksDefinition] = {} - for asset_checks_def in asset_checks_defs: + for asset_checks_def in asset_checks_to_execute: node_def_name = asset_checks_def.node_def.name node_key = NodeInvocation(node_def_name) deps[node_key] = {} @@ -454,7 +455,7 @@ def build_node_deps( upstream_node_alias, upstream_output_name ) - return deps, assets_defs_by_node_handle, asset_checks_defs_by_node_handle + return deps, node_handle_to_asset, asset_checks_defs_by_node_handle def _has_cycles( @@ -484,8 +485,8 @@ def _has_cycles( def _attempt_resolve_cycles( - assets_defs: Iterable["AssetsDefinition"], - source_assets: Iterable["SourceAsset"], + assets_to_execute: Iterable["AssetsDefinition"], + other_assets: Iterable["AssetsDefinition"], ) -> Sequence["AssetsDefinition"]: """DFS starting at root nodes to color the asset dependency graph. Each time you leave your current AssetsDefinition, the color increments. @@ -502,14 +503,8 @@ def _attempt_resolve_cycles( """ from dagster._core.selector.subset_selector import generate_asset_dep_graph - # get asset dependencies - asset_deps = generate_asset_dep_graph(assets_defs, source_assets) - - # index AssetsDefinitions by their asset names - assets_defs_by_asset_key: Dict[AssetKey, AssetsDefinition] = {} - for assets_def in assets_defs: - for asset_key in assets_def.keys: - assets_defs_by_asset_key[asset_key] = assets_def + asset_deps = generate_asset_dep_graph([*assets_to_execute, *other_assets], []) + assets_to_execute_by_asset_key = {k: ad for ad in assets_to_execute for k in ad.keys} # color for each asset colors = {} @@ -517,10 +512,11 @@ def _attempt_resolve_cycles( # recursively color an asset and all of its downstream assets def _dfs(key, cur_color): colors[key] = cur_color - if key in assets_defs_by_asset_key: - cur_node_asset_keys = assets_defs_by_asset_key[key].keys + + assets_def = assets_to_execute_by_asset_key.get(key) + if assets_def is not None: + cur_node_asset_keys = assets_def.keys else: - # in a SourceAsset, treat all downstream as if they're in the same node cur_node_asset_keys = asset_deps["downstream"][key] for downstream_key in asset_deps["downstream"][key]: @@ -545,10 +541,10 @@ def _dfs(key, cur_color): lambda: defaultdict(set) ) for key, color in colors.items(): - # ignore source assets - if key not in assets_defs_by_asset_key: + assets_def = assets_to_execute_by_asset_key.get(key) + if assets_def is None: continue - color_mapping_by_assets_defs[assets_defs_by_asset_key[key]][color].add(key) + color_mapping_by_assets_defs[assets_def][color].add(key) ret = [] for assets_def, color_mapping in color_mapping_by_assets_defs.items(): diff --git a/python_modules/dagster/dagster/_core/definitions/decorators/asset_decorator.py b/python_modules/dagster/dagster/_core/definitions/decorators/asset_decorator.py index ddb9b46a0f258..648d817b40387 100644 --- a/python_modules/dagster/dagster/_core/definitions/decorators/asset_decorator.py +++ b/python_modules/dagster/dagster/_core/definitions/decorators/asset_decorator.py @@ -40,7 +40,7 @@ from ..asset_check_spec import AssetCheckSpec from ..asset_in import AssetIn from ..asset_out import AssetOut -from ..asset_spec import AssetSpec +from ..asset_spec import AssetExecutionType, AssetSpec from ..assets import ASSET_SUBSET_INPUT_PREFIX, AssetsDefinition from ..backfill_policy import BackfillPolicy, BackfillPolicyType from ..decorators.graph_decorator import graph @@ -71,7 +71,7 @@ def asset( metadata: Optional[Mapping[str, Any]] = ..., description: Optional[str] = ..., config_schema: Optional[UserConfigSchema] = None, - required_resource_keys: Optional[Set[str]] = ..., + required_resource_keys: Optional[AbstractSet[str]] = ..., resource_defs: Optional[Mapping[str, object]] = ..., io_manager_def: Optional[object] = ..., io_manager_key: Optional[str] = ..., @@ -89,6 +89,7 @@ def asset( key: Optional[CoercibleToAssetKey] = None, non_argument_deps: Optional[Union[Set[AssetKey], Set[str]]] = ..., check_specs: Optional[Sequence[AssetCheckSpec]] = ..., + _execution_type: AssetExecutionType = ..., ) -> Callable[[Callable[..., Any]], AssetsDefinition]: ... @@ -110,7 +111,7 @@ def asset( metadata: Optional[ArbitraryMetadataMapping] = None, description: Optional[str] = None, config_schema: Optional[UserConfigSchema] = None, - required_resource_keys: Optional[Set[str]] = None, + required_resource_keys: Optional[AbstractSet[str]] = None, resource_defs: Optional[Mapping[str, object]] = None, io_manager_def: Optional[object] = None, io_manager_key: Optional[str] = None, @@ -128,6 +129,7 @@ def asset( key: Optional[CoercibleToAssetKey] = None, non_argument_deps: Optional[Union[Set[AssetKey], Set[str]]] = None, check_specs: Optional[Sequence[AssetCheckSpec]] = None, + _execution_type: AssetExecutionType = AssetExecutionType.MATERIALIZATION, ) -> Union[AssetsDefinition, Callable[[Callable[..., Any]], AssetsDefinition]]: """Create a definition for how to compute an asset. @@ -241,6 +243,7 @@ def create_asset(): code_version=code_version, check_specs=check_specs, key=key, + execution_type=_execution_type, ) if compute_fn is not None: @@ -313,6 +316,7 @@ def __init__( code_version: Optional[str] = None, key: Optional[CoercibleToAssetKey] = None, check_specs: Optional[Sequence[AssetCheckSpec]] = None, + execution_type: AssetExecutionType = AssetExecutionType.MATERIALIZATION, ): self.name = name self.key_prefix = key_prefix @@ -340,6 +344,7 @@ def __init__( self.code_version = code_version self.check_specs = check_specs self.key = key + self.execution_type = execution_type def __call__(self, fn: Callable) -> AssetsDefinition: from dagster._config.pythonic_config import ( @@ -484,6 +489,7 @@ def __call__(self, fn: Callable) -> AssetsDefinition: check_specs_by_output_name=check_specs_by_output_name, selected_asset_check_keys=None, # no subselection in decorator is_subset=False, + _execution_type=self.execution_type, ) @@ -512,6 +518,7 @@ def multi_asset( code_version: Optional[str] = None, specs: Optional[Sequence[AssetSpec]] = None, check_specs: Optional[Sequence[AssetCheckSpec]] = None, + _execution_type: AssetExecutionType = AssetExecutionType.MATERIALIZATION, # deprecated non_argument_deps: Optional[Union[Set[AssetKey], Set[str]]] = None, ) -> Callable[[Callable[..., Any]], AssetsDefinition]: @@ -882,6 +889,7 @@ def inner(fn: Callable[..., Any]) -> AssetsDefinition: check_specs_by_output_name=check_specs_by_output_name, selected_asset_check_keys=None, # no subselection in decorator is_subset=False, + _execution_type=_execution_type, ) return inner @@ -1019,6 +1027,7 @@ def graph_asset( resource_defs: Optional[Mapping[str, ResourceDefinition]] = ..., check_specs: Optional[Sequence[AssetCheckSpec]] = None, key: Optional[CoercibleToAssetKey] = None, + _execution_type: AssetExecutionType = ..., ) -> Callable[[Callable[..., Any]], AssetsDefinition]: ... @@ -1040,6 +1049,7 @@ def graph_asset( resource_defs: Optional[Mapping[str, ResourceDefinition]] = None, check_specs: Optional[Sequence[AssetCheckSpec]] = None, key: Optional[CoercibleToAssetKey] = None, + _execution_type: AssetExecutionType = AssetExecutionType.MATERIALIZATION, ) -> Union[AssetsDefinition, Callable[[Callable[..., Any]], AssetsDefinition]]: """Creates a software-defined asset that's computed using a graph of ops. @@ -1117,6 +1127,7 @@ def slack_files_table(): resource_defs=resource_defs, check_specs=check_specs, key=key, + _execution_type=_execution_type, ) else: return graph_asset_no_defaults( @@ -1135,6 +1146,7 @@ def slack_files_table(): resource_defs=resource_defs, check_specs=check_specs, key=key, + _execution_type=_execution_type, ) @@ -1155,6 +1167,7 @@ def graph_asset_no_defaults( resource_defs: Optional[Mapping[str, ResourceDefinition]], check_specs: Optional[Sequence[AssetCheckSpec]], key: Optional[CoercibleToAssetKey], + _execution_type: AssetExecutionType, ) -> AssetsDefinition: ins = ins or {} asset_ins = build_asset_ins(compose_fn, ins or {}, set()) @@ -1210,6 +1223,7 @@ def graph_asset_no_defaults( descriptions_by_output_name={"result": description} if description else None, resource_defs=resource_defs, check_specs=check_specs, + _execution_type=_execution_type, ) @@ -1225,6 +1239,7 @@ def graph_multi_asset( resource_defs: Optional[Mapping[str, ResourceDefinition]] = None, check_specs: Optional[Sequence[AssetCheckSpec]] = None, config: Optional[Union[ConfigMapping, Mapping[str, Any]]] = None, + _execution_type: AssetExecutionType = AssetExecutionType.MATERIALIZATION, ) -> Callable[[Callable[..., Any]], AssetsDefinition]: """Create a combined definition of multiple assets that are computed using the same graph of ops, and the same upstream assets. @@ -1337,6 +1352,7 @@ def inner(fn: Callable) -> AssetsDefinition: descriptions_by_output_name=descriptions_by_output_name, resource_defs=resource_defs, check_specs=check_specs, + _execution_type=_execution_type, ) return inner diff --git a/python_modules/dagster/dagster/_core/definitions/external_asset.py b/python_modules/dagster/dagster/_core/definitions/external_asset.py index dea5cf03085bc..60f199239c869 100644 --- a/python_modules/dagster/dagster/_core/definitions/external_asset.py +++ b/python_modules/dagster/dagster/_core/definitions/external_asset.py @@ -2,7 +2,6 @@ from dagster import _check as check from dagster._core.definitions.asset_spec import ( - SYSTEM_METADATA_KEY_ASSET_EXECUTION_TYPE, AssetExecutionType, AssetSpec, ) @@ -16,6 +15,7 @@ ) from dagster._core.errors import DagsterInvariantViolationError from dagster._core.execution.context.compute import AssetExecutionContext +from dagster._utils.warnings import disable_dagster_warnings def external_asset_from_spec(spec: AssetSpec) -> AssetsDefinition: @@ -109,17 +109,11 @@ def external_assets_from_specs(specs: Sequence[AssetSpec]) -> List[AssetsDefinit key=spec.key, description=spec.description, group_name=spec.group_name, - metadata={ - **(spec.metadata or {}), - **{ - SYSTEM_METADATA_KEY_ASSET_EXECUTION_TYPE: ( - AssetExecutionType.UNEXECUTABLE.value - ) - }, - }, + metadata=spec.metadata, deps=spec.deps, ) ], + _execution_type=AssetExecutionType.UNEXECUTABLE, ) def _external_assets_def(context: AssetExecutionContext) -> None: raise DagsterInvariantViolationError( @@ -132,48 +126,60 @@ def _external_assets_def(context: AssetExecutionContext) -> None: return assets_defs -def create_external_asset_from_source_asset(source_asset: SourceAsset) -> AssetsDefinition: - check.invariant( - source_asset.auto_observe_interval_minutes is None, - "Automatically observed external assets not supported yet: auto_observe_interval_minutes" - " should be None", - ) - - injected_metadata = ( - {SYSTEM_METADATA_KEY_ASSET_EXECUTION_TYPE: AssetExecutionType.UNEXECUTABLE.value} - if source_asset.observe_fn is None - else {SYSTEM_METADATA_KEY_ASSET_EXECUTION_TYPE: AssetExecutionType.OBSERVATION.value} - ) - - kwargs = { - "key": source_asset.key, - "metadata": { - **source_asset.metadata, - **injected_metadata, - }, - "group_name": source_asset.group_name, - "description": source_asset.description, - "partitions_def": source_asset.partitions_def, - } +# SYSTEM_METADATA_KEY_AUTO_OBSERVE_INTERVAL_MINUTES lives on the metadata of +# external assets resulting from a source asset conversion. It contains the +# `auto_observe_interval_minutes` value from the source asset and is consulted +# in the auto-materialize daemon. It should eventually be eliminated in favor +# of an implementation of `auto_observe_interval_minutes` in terms of +# `AutoMaterializeRule`. +SYSTEM_METADATA_KEY_AUTO_OBSERVE_INTERVAL_MINUTES = "dagster/auto_observe_interval_minutes" - if source_asset.io_manager_def: - kwargs["io_manager_def"] = source_asset.io_manager_def - elif source_asset.io_manager_key: - kwargs["io_manager_key"] = source_asset.io_manager_key - @asset(**kwargs) - def _shim_assets_def(context: AssetExecutionContext): - if not source_asset.observe_fn: - raise NotImplementedError(f"Asset {source_asset.key} is not executable") +def create_external_asset_from_source_asset(source_asset: SourceAsset) -> AssetsDefinition: + observe_interval = source_asset.auto_observe_interval_minutes + metadata = { + **source_asset.raw_metadata, + **( + {SYSTEM_METADATA_KEY_AUTO_OBSERVE_INTERVAL_MINUTES: observe_interval} + if observe_interval + else {} + ), + } - op_function = wrap_source_asset_observe_fn_in_op_compute_fn(source_asset) - return_value = op_function.decorated_fn(context) - check.invariant( - isinstance(return_value, Output) - and SYSTEM_METADATA_KEY_SOURCE_ASSET_OBSERVATION in return_value.metadata, - "The wrapped decorated_fn should return an Output with a special metadata key.", + with disable_dagster_warnings(): + + @asset( + key=source_asset.key, + metadata=metadata, + group_name=source_asset.group_name, + description=source_asset.description, + partitions_def=source_asset.partitions_def, + _execution_type=( + AssetExecutionType.UNEXECUTABLE + if source_asset.observe_fn is None + else AssetExecutionType.OBSERVATION + ), + io_manager_key=source_asset.io_manager_key, + # We don't pass the `io_manager_def` because it will already be present in + # `resource_defs` (it is added during `SourceAsset` initialization). + resource_defs=source_asset.resource_defs, + # We need to access the raw attribute because the property will return a computed value that + # includes requirements for the io manager. Those requirements will be inferred again when + # we create an AssetsDefinition. + required_resource_keys=source_asset._required_resource_keys, # noqa: SLF001 ) - return return_value + def _shim_assets_def(context: AssetExecutionContext): + if not source_asset.observe_fn: + raise NotImplementedError(f"Asset {source_asset.key} is not executable") + + op_function = wrap_source_asset_observe_fn_in_op_compute_fn(source_asset) + return_value = op_function.decorated_fn(context) + check.invariant( + isinstance(return_value, Output) + and SYSTEM_METADATA_KEY_SOURCE_ASSET_OBSERVATION in return_value.metadata, + "The wrapped decorated_fn should return an Output with a special metadata key.", + ) + return return_value check.invariant(isinstance(_shim_assets_def, AssetsDefinition)) assert isinstance(_shim_assets_def, AssetsDefinition) # appease pyright diff --git a/python_modules/dagster/dagster/_core/definitions/external_asset_graph.py b/python_modules/dagster/dagster/_core/definitions/external_asset_graph.py index ff09eef6b3f2f..7d8fb25533421 100644 --- a/python_modules/dagster/dagster/_core/definitions/external_asset_graph.py +++ b/python_modules/dagster/dagster/_core/definitions/external_asset_graph.py @@ -13,6 +13,7 @@ ) import dagster._check as check +from dagster._core.definitions.asset_spec import AssetExecutionType from dagster._core.definitions.assets_job import ASSET_BASE_JOB_PREFIX from dagster._core.definitions.auto_materialize_policy import AutoMaterializePolicy from dagster._core.host_representation.external import ExternalRepository @@ -53,6 +54,7 @@ def __init__( required_assets_and_checks_by_key: Mapping[ AssetKeyOrCheckKey, AbstractSet[AssetKeyOrCheckKey] ], + execution_types_by_key: Mapping[AssetKey, AssetExecutionType], ): super().__init__( asset_dep_graph=asset_dep_graph, @@ -67,6 +69,7 @@ def __init__( is_observable_by_key=is_observable_by_key, auto_observe_interval_minutes_by_key=auto_observe_interval_minutes_by_key, required_assets_and_checks_by_key=required_assets_and_checks_by_key, + execution_types_by_key=execution_types_by_key, ) self._repo_handles_by_key = repo_handles_by_key self._materialization_job_names_by_key = job_names_by_key @@ -148,23 +151,25 @@ def from_repository_handles_and_external_asset_nodes( for _, node in repo_handle_external_asset_nodes if not node.is_source } + execution_types_by_key = { + node.asset_key: node.execution_type for _, node in repo_handle_external_asset_nodes + } all_non_source_keys = { node.asset_key for _, node in repo_handle_external_asset_nodes if not node.is_source } - is_observable_by_key = {key: False for key in all_non_source_keys} + is_observable_by_key = {} auto_observe_interval_minutes_by_key = {} for repo_handle, node in repo_handle_external_asset_nodes: + is_observable_by_key[node.asset_key] = ( + node.execution_type == AssetExecutionType.OBSERVATION + ) + auto_observe_interval_minutes_by_key[ + node.asset_key + ] = node.auto_observe_interval_minutes if node.is_source: - # We need to set this even if the node is a regular asset in another code location. - # `is_observable` will only ever be consulted in the source asset context. - is_observable_by_key[node.asset_key] = node.is_observable - auto_observe_interval_minutes_by_key[ - node.asset_key - ] = node.auto_observe_interval_minutes - if node.asset_key in all_non_source_keys: # one location's source is another location's non-source continue @@ -224,6 +229,7 @@ def from_repository_handles_and_external_asset_nodes( is_observable_by_key=is_observable_by_key, auto_observe_interval_minutes_by_key=auto_observe_interval_minutes_by_key, required_assets_and_checks_by_key=required_assets_and_checks_by_key, + execution_types_by_key=execution_types_by_key, ) @property diff --git a/python_modules/dagster/dagster/_core/definitions/job_definition.py b/python_modules/dagster/dagster/_core/definitions/job_definition.py index 58df3c0a3037a..d778275acea75 100644 --- a/python_modules/dagster/dagster/_core/definitions/job_definition.py +++ b/python_modules/dagster/dagster/_core/definitions/job_definition.py @@ -770,10 +770,7 @@ def _get_job_def_for_asset_selection( check.opt_set_param(asset_check_selection, "asset_check_selection", AssetCheckKey) nonexistent_assets = [ - asset - for asset in asset_selection - if asset not in self.asset_layer.asset_keys - and asset not in self.asset_layer.source_assets_by_key + asset for asset in asset_selection if asset not in self.asset_layer.asset_keys ] nonexistent_asset_strings = [ asset_str @@ -818,7 +815,7 @@ def _get_job_def_for_asset_selection( new_job = build_asset_selection_job( name=self.name, assets=set(self.asset_layer.assets_defs_by_key.values()), - source_assets=self.asset_layer.source_assets_by_key.values(), + source_assets=set(), executor_def=self.executor_def, resource_defs=self.resource_defs, description=self.description, @@ -1219,6 +1216,8 @@ def _infer_asset_layer_from_source_asset_deps(job_graph_def: GraphDefinition) -> """For non-asset jobs that have some inputs that are fed from SourceAssets, constructs an AssetLayer that includes those SourceAssets. """ + from dagster._core.definitions.external_asset import create_external_asset_from_source_asset + asset_keys_by_node_input_handle: Dict[NodeInputHandle, AssetKey] = {} source_assets_list = [] source_asset_keys_set = set() @@ -1250,17 +1249,18 @@ def _infer_asset_layer_from_source_asset_deps(job_graph_def: GraphDefinition) -> for node_name, node in graph_def.node_dict.items(): if isinstance(node.definition, GraphDefinition): stack.append((node.definition, NodeHandle(node_name, parent_node_handle))) - + assets_defs_by_key = { + source_asset.key: create_external_asset_from_source_asset(source_asset) + for source_asset in source_assets_list + } return AssetLayer( + asset_keys_to_execute=set(), assets_defs_by_node_handle={}, asset_keys_by_node_input_handle=asset_keys_by_node_input_handle, asset_info_by_node_output_handle={}, asset_deps={}, dependency_node_handles_by_asset_key={}, - assets_defs_by_key={}, - source_assets_by_key={ - source_asset.key: source_asset for source_asset in source_assets_list - }, + assets_defs_by_key=assets_defs_by_key, io_manager_keys_by_asset_key=io_manager_keys_by_asset_key, dep_asset_keys_by_node_output_handle={}, partition_mappings_by_asset_dep={}, diff --git a/python_modules/dagster/dagster/_core/definitions/observe.py b/python_modules/dagster/dagster/_core/definitions/observe.py index 975e3d7209175..5c46ec667dd69 100644 --- a/python_modules/dagster/dagster/_core/definitions/observe.py +++ b/python_modules/dagster/dagster/_core/definitions/observe.py @@ -1,9 +1,12 @@ -from typing import TYPE_CHECKING, Any, Mapping, Optional, Sequence +from typing import TYPE_CHECKING, Any, Mapping, Optional, Sequence, Union import dagster._check as check +from dagster._annotations import deprecated_param +from dagster._core.definitions.assets import AssetsDefinition from dagster._core.definitions.assets_job import build_assets_job from dagster._core.definitions.definitions_class import Definitions -from dagster._utils.warnings import disable_dagster_warnings +from dagster._core.definitions.external_asset import create_external_asset_from_source_asset +from dagster._utils.warnings import disable_dagster_warnings, normalize_renamed_param from ..instance import DagsterInstance from .source_asset import SourceAsset @@ -12,22 +15,27 @@ from ..execution.execute_in_process_result import ExecuteInProcessResult +@deprecated_param( + param="source_assets", breaking_version="2.0", additional_warn_text="Use `assets` instead." +) def observe( - source_assets: Sequence[SourceAsset], + assets: Optional[Sequence[Union[SourceAsset, AssetsDefinition]]] = None, run_config: Any = None, instance: Optional[DagsterInstance] = None, resources: Optional[Mapping[str, object]] = None, partition_key: Optional[str] = None, raise_on_error: bool = True, tags: Optional[Mapping[str, str]] = None, + *, + source_assets: Optional[Sequence[Union[SourceAsset, AssetsDefinition]]] = None, ) -> "ExecuteInProcessResult": - """Executes a single-threaded, in-process run which observes provided source assets. + """Executes a single-threaded, in-process run which observes provided observable assets. By default, will materialize assets to the local filesystem. Args: - source_assets (Sequence[SourceAsset]): - The source assets to materialize. + assets (Sequence[Union[SourceAsset, AssetsDefinition]]): + The assets to observe. Assets must be observable. resources (Optional[Mapping[str, object]]): The resources needed for execution. Can provide resource instances directly, or resource definitions. Note that if provided resources @@ -37,19 +45,33 @@ def observe( The string partition key that specifies the run config to execute. Can only be used to select run config for assets with partitioned config. tags (Optional[Mapping[str, str]]): Tags for the run. + source_assets (Sequence[Union[SourceAsset, AssetsDefinition]]): + The assets to observe. Returns: ExecuteInProcessResult: The result of the execution. """ - source_assets = check.sequence_param(source_assets, "assets", of_type=(SourceAsset)) + assets = check.not_none( + normalize_renamed_param( + assets, + "assets", + source_assets, + "source_assets", + ) + ) + assets = check.sequence_param(assets, "assets", of_type=(AssetsDefinition, SourceAsset)) instance = check.opt_inst_param(instance, "instance", DagsterInstance) partition_key = check.opt_str_param(partition_key, "partition_key") resources = check.opt_mapping_param(resources, "resources", key_type=str) + normalized_assets = [ + *(x for x in assets if isinstance(x, AssetsDefinition)), + *(create_external_asset_from_source_asset(x) for x in assets if isinstance(x, SourceAsset)), + ] with disable_dagster_warnings(): - observation_job = build_assets_job("in_process_observation_job", [], source_assets) + observation_job = build_assets_job("in_process_observation_job", [], normalized_assets) defs = Definitions( - assets=source_assets, + assets=normalized_assets, jobs=[observation_job], resources=resources, ) diff --git a/python_modules/dagster/dagster/_core/definitions/repository_definition/repository_data_builder.py b/python_modules/dagster/dagster/_core/definitions/repository_definition/repository_data_builder.py index be03eb7e8d914..ee51e1c32e2a7 100644 --- a/python_modules/dagster/dagster/_core/definitions/repository_definition/repository_data_builder.py +++ b/python_modules/dagster/dagster/_core/definitions/repository_definition/repository_data_builder.py @@ -32,6 +32,7 @@ AutomationPolicySensorDefinition, ) from dagster._core.definitions.executor_definition import ExecutorDefinition +from dagster._core.definitions.external_asset import create_external_asset_from_source_asset from dagster._core.definitions.graph_definition import GraphDefinition from dagster._core.definitions.job_definition import JobDefinition from dagster._core.definitions.logger_definition import LoggerDefinition @@ -223,17 +224,19 @@ def build_caching_repository_data_from_list( asset_keys.update(definition.keys) assets_defs.append(definition) + # All source assets are converted to external assets here elif isinstance(definition, SourceAsset): source_assets.append(definition) + external_assets_def = create_external_asset_from_source_asset(definition) + assets_defs.append(external_assets_def) elif isinstance(definition, AssetChecksDefinition): asset_checks_defs.append(definition) else: check.failed(f"Unexpected repository entry {definition}") - if assets_defs or source_assets or asset_checks_defs: + if assets_defs or asset_checks_defs: for job_def in get_base_asset_jobs( assets=assets_defs, - source_assets=source_assets, executor_def=default_executor_def, resource_defs=top_level_resources, asset_checks=asset_checks_defs, @@ -261,9 +264,7 @@ def build_caching_repository_data_from_list( schedule_def, coerced_graphs, unresolved_jobs, jobs, target ) - asset_graph = AssetGraph.from_assets( - [*assets_defs, *source_assets], asset_checks=asset_checks_defs - ) + asset_graph = AssetGraph.from_assets(assets_defs, asset_checks=asset_checks_defs) _validate_automation_policy_sensors(sensors.values(), asset_graph) if unresolved_partitioned_asset_schedules: diff --git a/python_modules/dagster/dagster/_core/definitions/repository_definition/repository_definition.py b/python_modules/dagster/dagster/_core/definitions/repository_definition/repository_definition.py index 5ad4e62bb1d12..f17f33ad3a288 100644 --- a/python_modules/dagster/dagster/_core/definitions/repository_definition/repository_definition.py +++ b/python_modules/dagster/dagster/_core/definitions/repository_definition/repository_definition.py @@ -280,7 +280,7 @@ def get_implicit_job_def_for_assets( if self.has_job(ASSET_BASE_JOB_PREFIX): base_job = self.get_job(ASSET_BASE_JOB_PREFIX) if all( - key in base_job.asset_layer.assets_defs_by_key + key in base_job.asset_layer.target_asset_keys or base_job.asset_layer.is_observable_for_asset(key) for key in asset_keys ): @@ -289,9 +289,8 @@ def get_implicit_job_def_for_assets( i = 0 while self.has_job(f"{ASSET_BASE_JOB_PREFIX}_{i}"): base_job = self.get_job(f"{ASSET_BASE_JOB_PREFIX}_{i}") - if all( - key in base_job.asset_layer.assets_defs_by_key + key in base_job.asset_layer.target_asset_keys or base_job.asset_layer.is_observable_for_asset(key) for key in asset_keys ): diff --git a/python_modules/dagster/dagster/_core/definitions/resource_requirement.py b/python_modules/dagster/dagster/_core/definitions/resource_requirement.py index 7ddb26aa60ee4..bb8610ab06633 100644 --- a/python_modules/dagster/dagster/_core/definitions/resource_requirement.py +++ b/python_modules/dagster/dagster/_core/definitions/resource_requirement.py @@ -104,6 +104,29 @@ def describe_requirement(self) -> str: ) +class ExternalAssetIOManagerRequirement( + NamedTuple( + "_ExternalAssetIOManagerRequirement", + [ + ("key", str), + ("asset_key", Optional[str]), + ], + ), + ResourceRequirement, +): + @property + def expected_type(self) -> Type: + from ..storage.io_manager import IOManagerDefinition + + return IOManagerDefinition + + def describe_requirement(self) -> str: + external_asset_descriptor = ( + f"external asset with key {self.asset_key}" if self.asset_key else "external asset" + ) + return f"io manager with key '{self.key}' required by {external_asset_descriptor}" + + class SourceAssetIOManagerRequirement( NamedTuple( "_InputManagerRequirement", diff --git a/python_modules/dagster/dagster/_core/definitions/source_asset.py b/python_modules/dagster/dagster/_core/definitions/source_asset.py index 5751ad26ca716..f121d3e273f9c 100644 --- a/python_modules/dagster/dagster/_core/definitions/source_asset.py +++ b/python_modules/dagster/dagster/_core/definitions/source_asset.py @@ -15,6 +15,7 @@ import dagster._check as check from dagster._annotations import PublicAttr, experimental_param, public from dagster._core.decorator_utils import get_function_params +from dagster._core.definitions.asset_spec import AssetExecutionType from dagster._core.definitions.data_version import ( DATA_VERSION_TAG, DataVersion, @@ -255,12 +256,25 @@ def op(self) -> OpDefinition: ) return cast(OpDefinition, self.node_def) + @property + def execution_type(self) -> AssetExecutionType: + return ( + AssetExecutionType.OBSERVATION + if self.is_observable + else AssetExecutionType.UNEXECUTABLE + ) + @public @property def is_observable(self) -> bool: """bool: Whether the asset is observable.""" return self.node_def is not None + @property + def is_materializable(self) -> bool: + """bool: Whether the asset is materializable.""" + return False + @property def required_resource_keys(self) -> AbstractSet[str]: return {requirement.key for requirement in self.get_resource_requirements()} diff --git a/python_modules/dagster/dagster/_core/definitions/unresolved_asset_job_definition.py b/python_modules/dagster/dagster/_core/definitions/unresolved_asset_job_definition.py index 0269dbae15486..b33c53b3c0d90 100644 --- a/python_modules/dagster/dagster/_core/definitions/unresolved_asset_job_definition.py +++ b/python_modules/dagster/dagster/_core/definitions/unresolved_asset_job_definition.py @@ -184,13 +184,6 @@ def resolve( assets = asset_graph.assets source_assets = asset_graph.source_assets selected_asset_keys = self.selection.resolve(asset_graph) - if asset_graph.includes_materializable_and_source_assets(selected_asset_keys): - raise DagsterInvalidDefinitionError( - f"Asset selection for job '{self.name}' specified both regular assets and source " - "assets. This is not currently supported. Selections must be all regular " - "assets or all source assets.", - ) - selected_asset_checks = self.selection.resolve_checks(asset_graph) asset_keys_by_partitions_def = defaultdict(set) diff --git a/python_modules/dagster/dagster/_core/execution/plan/execute_step.py b/python_modules/dagster/dagster/_core/execution/plan/execute_step.py index 311b808db893b..f1a69555959eb 100644 --- a/python_modules/dagster/dagster/_core/execution/plan/execute_step.py +++ b/python_modules/dagster/dagster/_core/execution/plan/execute_step.py @@ -578,20 +578,15 @@ def _type_check_and_store_output( yield evt -def _materializing_asset_key_and_partitions_for_output( +def _get_asset_key_and_partitions_for_output( output_context: OutputContext, ) -> Tuple[Optional[AssetKey], AbstractSet[str]]: output_asset_info = output_context.asset_info - if ( - output_asset_info - and not output_context.step_context.job_def.asset_layer.is_observable_for_asset( - output_asset_info.key - ) - ): + if output_asset_info: if not output_asset_info.is_required: output_context.log.warning( - f"Materializing unexpected asset key: {output_asset_info.key}." + f"Materializing or observing unexpected asset key: {output_asset_info.key}." ) return ( output_asset_info.key, @@ -619,7 +614,7 @@ def _get_output_asset_events( if ( execution_type == AssetExecutionType.MATERIALIZATION and step_context.is_external_input_asset_version_info_loaded - and asset_key in step_context.job_def.asset_layer.asset_keys + and asset_key in step_context.job_def.asset_layer.target_asset_keys ): assert isinstance(output, Output) code_version = _get_code_version(asset_key, step_context) @@ -773,7 +768,7 @@ def _store_output( or (step_context.output_observes_source_asset(step_output_handle.output_name)) or output_context.dagster_type.is_nothing ): - yield from _log_asset_materialization_events_for_asset( + yield from _log_asset_events_for_asset( step_context=step_context, output_context=output_context, output=output, @@ -857,7 +852,7 @@ def _gen_fn(): yield DagsterEvent.asset_materialization(step_context, materialization) - yield from _log_asset_materialization_events_for_asset( + yield from _log_asset_events_for_asset( step_context=step_context, output_context=output_context, output=output, @@ -873,57 +868,53 @@ def _gen_fn(): ) -def _log_asset_materialization_events_for_asset( - step_context, output_context, output, output_def, manager_metadata -): - asset_key, partitions = _materializing_asset_key_and_partitions_for_output(output_context) +def _log_asset_events_for_asset( + step_context: StepExecutionContext, + output_context: OutputContext, + output: Union[Output, DynamicOutput], + output_def: OutputDefinition, + manager_metadata: Mapping[str, MetadataValue], +) -> Iterator[DagsterEvent]: + # This is a temporary workaround to prevent duplicate observation events from external + # observable assets that were auto-converted from source assets. These assets yield + # observation events through the context in their body, and will continue to do so until we + # can convert them to using ObserveResult, which requires a solution to partition-scoped + # metadata and data version on output. We identify these auto-converted assets by looking + # for OBSERVATION-type asset that have this special metadata key (added in + # `wrap_source_asset_observe_fn_in_op_compute_fn`), which should only occur for these + # auto-converted source assets. This can be removed when source asset observation functions + # are converted to use ObserveResult. + if SYSTEM_METADATA_KEY_SOURCE_ASSET_OBSERVATION in output.metadata: + return + + asset_key, partitions = _get_asset_key_and_partitions_for_output(output_context) if asset_key: asset_layer = step_context.job_def.asset_layer execution_type = ( - asset_layer.assets_def_for_asset(asset_key).asset_execution_type_for_asset(asset_key) + asset_layer.assets_def_for_asset(asset_key).execution_type if asset_layer.has_assets_def_for_asset(asset_key) else AssetExecutionType.MATERIALIZATION ) - check.invariant( - execution_type != AssetExecutionType.UNEXECUTABLE, - "There should never be unexecutable assets here", - ) - check.invariant( execution_type in {AssetExecutionType.MATERIALIZATION, AssetExecutionType.OBSERVATION}, f"Unexpected asset execution type {execution_type}", ) - # This is a temporary workaround to prevent duplicate observation events from external - # observable assets that were auto-converted from source assets. These assets yield - # observation events through the context in their body, and will continue to do so until we - # can convert them to using ObserveResult, which requires a solution to partition-scoped - # metadata and data version on output. We identify these auto-converted assets by looking - # for OBSERVATION-type asset that have this special metadata key (added in - # `wrap_source_asset_observe_fn_in_op_compute_fn`), which should only occur for these - # auto-converted source assets. This can be removed when source asset observation functions - # are converted to use ObserveResult. - if ( - execution_type == AssetExecutionType.OBSERVATION - and SYSTEM_METADATA_KEY_SOURCE_ASSET_OBSERVATION in output.metadata - ): - pass - else: - yield from ( - ( - _dagster_event_for_asset_event(step_context, event) - for event in _get_output_asset_events( - asset_key, - partitions, - output, - output_def, - manager_metadata, - step_context, - execution_type, - ) + yield from ( + ( + _dagster_event_for_asset_event(step_context, event) + for event in _get_output_asset_events( + asset_key, + partitions, + output, + output_def, + manager_metadata, + step_context, + execution_type, ) ) + ) def _dagster_event_for_asset_event( diff --git a/python_modules/dagster/dagster/_core/host_representation/external.py b/python_modules/dagster/dagster/_core/host_representation/external.py index 772edff995d8a..eb5cedfbf18a7 100644 --- a/python_modules/dagster/dagster/_core/host_representation/external.py +++ b/python_modules/dagster/dagster/_core/host_representation/external.py @@ -216,7 +216,7 @@ def _external_sensors(self) -> Dict[str, "ExternalSensor"]: if asset_key not in covered_asset_keys: default_sensor_asset_keys.add(asset_key) - for asset_key in asset_graph.source_asset_keys: + for asset_key in asset_graph.observable_keys: if asset_graph.get_auto_observe_interval_minutes(asset_key) is None: continue diff --git a/python_modules/dagster/dagster/_core/host_representation/external_data.py b/python_modules/dagster/dagster/_core/host_representation/external_data.py index 2a1c58267232f..2f247fd6527dd 100644 --- a/python_modules/dagster/dagster/_core/host_representation/external_data.py +++ b/python_modules/dagster/dagster/_core/host_representation/external_data.py @@ -44,13 +44,12 @@ PartitionsDefinition, RepositoryDefinition, ScheduleDefinition, - SourceAsset, ) from dagster._core.definitions.asset_check_spec import AssetCheckKey from dagster._core.definitions.asset_checks import AssetChecksDefinition from dagster._core.definitions.asset_sensor_definition import AssetSensorDefinition from dagster._core.definitions.asset_spec import ( - SYSTEM_METADATA_KEY_ASSET_EXECUTION_TYPE, + SYSTEM_METADATA_KEY_AUTO_OBSERVE_INTERVAL_MINUTES, AssetExecutionType, ) from dagster._core.definitions.assets import AssetsDefinition @@ -69,6 +68,7 @@ OpNode, ) from dagster._core.definitions.events import AssetKey +from dagster._core.definitions.external_asset import is_external_asset from dagster._core.definitions.freshness_policy import FreshnessPolicy from dagster._core.definitions.metadata import ( MetadataFieldSerializer, @@ -99,7 +99,9 @@ from dagster._core.snap.mode import ResourceDefSnap, build_resource_def_snap from dagster._core.storage.io_manager import IOManagerDefinition from dagster._serdes import whitelist_for_serdes -from dagster._serdes.serdes import is_whitelisted_for_serdes_object +from dagster._serdes.serdes import ( + is_whitelisted_for_serdes_object, +) from dagster._utils.error import SerializableErrorInfo if TYPE_CHECKING: @@ -1160,6 +1162,11 @@ def key(self) -> AssetCheckKey: return AssetCheckKey(asset_key=self.asset_key, name=self.name) +# SYSTEM_METADATA_KEY_ASSET_EXECUTION_TYPE used to live on the metadata of external assets and +# encoded the execution type the of asset. +_SYSTEM_METADATA_KEY_ASSET_EXECUTION_TYPE = "dagster/asset_execution_type" + + @whitelist_for_serdes( storage_field_names={"metadata": "metadata_entries"}, field_serializers={"metadata": MetadataFieldSerializer}, @@ -1171,6 +1178,7 @@ class ExternalAssetNode( ("asset_key", AssetKey), ("dependencies", Sequence[ExternalAssetDependency]), ("depended_by", Sequence[ExternalAssetDependedBy]), + ("execution_type", AssetExecutionType), ("compute_kind", Optional[str]), ("op_name", Optional[str]), ("op_names", Sequence[str]), @@ -1210,6 +1218,7 @@ def __new__( asset_key: AssetKey, dependencies: Sequence[ExternalAssetDependency], depended_by: Sequence[ExternalAssetDependedBy], + execution_type: Optional[AssetExecutionType] = None, compute_kind: Optional[str] = None, op_name: Optional[str] = None, op_names: Optional[Sequence[str]] = None, @@ -1232,6 +1241,32 @@ def __new__( backfill_policy: Optional[BackfillPolicy] = None, auto_observe_interval_minutes: Optional[float] = None, ): + metadata = normalize_metadata(check.opt_mapping_param(metadata, "metadata", key_type=str)) + + # backcompat logic for execution type specified via metadata + if _SYSTEM_METADATA_KEY_ASSET_EXECUTION_TYPE in metadata: + val = metadata[_SYSTEM_METADATA_KEY_ASSET_EXECUTION_TYPE] + if not isinstance(val, TextMetadataValue): + check.failed( + f"Expected metadata value for key {_SYSTEM_METADATA_KEY_ASSET_EXECUTION_TYPE} to be a TextMetadataValue, got {val}" + ) + metadata_execution_type = AssetExecutionType.str_to_enum(val.value) + if execution_type is not None: + check.invariant( + execution_type == metadata_execution_type, + f"Execution type {execution_type} in metadata does not match type inferred from metadata {metadata_execution_type}", + ) + execution_type = metadata_execution_type + else: + execution_type = ( + check.opt_inst_param( + execution_type, + "execution_type", + AssetExecutionType, + ) + or AssetExecutionType.MATERIALIZATION + ) + # backcompat logic to handle ExternalAssetNodes serialized without op_names/graph_name if not op_names: op_names = list(filter(None, [op_name])) @@ -1266,9 +1301,7 @@ def __new__( ), output_name=check.opt_str_param(output_name, "output_name"), output_description=check.opt_str_param(output_description, "output_description"), - metadata=normalize_metadata( - check.opt_mapping_param(metadata, "metadata", key_type=str) - ), + metadata=metadata, group_name=check.opt_str_param(group_name, "group_name"), freshness_policy=check.opt_inst_param( freshness_policy, "freshness_policy", FreshnessPolicy @@ -1292,10 +1325,11 @@ def __new__( auto_observe_interval_minutes=check.opt_numeric_param( auto_observe_interval_minutes, "auto_observe_interval_minutes" ), + execution_type=check.inst_param(execution_type, "execution_type", AssetExecutionType), ) @property - def is_executable(self) -> bool: + def execution_type(self) -> AssetExecutionType: metadata_value = self.metadata.get(SYSTEM_METADATA_KEY_ASSET_EXECUTION_TYPE) if not metadata_value: varietal_text = None @@ -1303,8 +1337,19 @@ def is_executable(self) -> bool: check.inst(metadata_value, TextMetadataValue) # for guaranteed runtime error assert isinstance(metadata_value, TextMetadataValue) # for type checker varietal_text = metadata_value.value + return ( + AssetExecutionType.MATERIALIZATION + if varietal_text is None + else AssetExecutionType(varietal_text) + ) - return AssetExecutionType.is_executable(varietal_text) + @property + def is_external(self) -> bool: + return self.execution_type != AssetExecutionType.MATERIALIZATION + + @property + def is_executable(self) -> bool: + return AssetExecutionType.is_executable(self.execution_type) ResourceJobUsageMap = Dict[str, List[ResourceJobUsageEntry]] @@ -1372,10 +1417,7 @@ def external_repository_data_from_def( job_refs = None resource_datas = repository_def.get_top_level_resources() - asset_graph = external_asset_nodes_from_defs( - jobs, - source_assets_by_key=repository_def.source_assets_by_key, - ) + asset_graph = external_asset_nodes_from_defs(jobs, repository_def.assets_defs_by_key) nested_resource_map = _get_nested_resources_map( resource_datas, repository_def.get_resource_key_mapping() @@ -1393,12 +1435,6 @@ def external_repository_data_from_def( for resource_key in asset.required_top_level_resources: resource_asset_usage_map[resource_key].append(asset.asset_key) - # collect resource usage from source assets - for source_asset_key, source_asset in repository_def.source_assets_by_key.items(): - if source_asset.required_resource_keys: - for resource_key in source_asset.required_resource_keys: - resource_asset_usage_map[resource_key].append(source_asset_key) - resource_schedule_usage_map: Dict[str, List[str]] = defaultdict(list) for schedule in repository_def.schedule_defs: if schedule.required_resource_keys: @@ -1525,10 +1561,127 @@ def external_asset_checks_from_defs( return sorted(external_checks, key=lambda check: (check.asset_key, check.name)) +# def external_asset_nodes_from_assets_defs( +# assets_defs: Sequence[AssetsDefinition], +# job_defs: Sequence[JobDefinition], +# ) -> Sequence[ExternalAssetNode]: +# asset_nodes: List[ExternalAssetNode] = [] +# +# # An asset may appear in a job as a node output, a node input, or both a node output and a node +# # input. +# # +# # - An asset is only considered "part" of a job if it appears as a node output, i.e. is +# # materialized by the job. +# # When an asset appears as a node output, it will can appear in multiple jobs. +# # `ExternalAssetNode` contains fields (e.g. `op_name`, `graph_name`) that implicitly assume a +# # single job. We therefore need to find the canonical job for each asset that will be used to +# # resolve these fields. +# # build job-dependent data structures +# processed_job_asset_keys: Set[AssetKey] = set() +# graph_names_by_key: Dict[AssetKey, Optional[str]] = {} +# op_names_by_key: Dict[AssetKey, List[str]] = {} +# op_name_by_key: Dict[AssetKey, str] = {} +# job_names_by_key: DefaultDict[AssetKey, List[str]] = defaultdict(list) +# output_names_by_key: Dict[AssetKey, str] = {} +# for job_def in job_defs: +# asset_layer = job_def.asset_layer +# for asset_key in asset_layer.asset_keys: +# job_names_by_key[asset_key].append(job_def.name) +# +# asset_info_by_node_output = asset_layer.asset_info_by_node_output_handle +# for node_output_handle, asset_info in asset_info_by_node_output.items(): +# asset_key = asset_info.key +# node_def = job_def.graph.get_node(node_output_handle.node_handle).definition +# if asset_key not in processed_job_asset_keys: +# op_names = [ +# str(handle) +# for handle in asset_layer.dependency_node_handles_by_asset_key.get( +# asset_key, [] +# ) +# ] +# # if the asset is produced by an op at the top level of the +# # graph, graph_name should be None +# graph_name = None +# node_handle = node_output_handle.node_handle +# while node_handle.parent: +# node_handle = node_handle.parent +# graph_name = node_handle.name +# op_name = graph_name or next(iter(op_names), None) or node_def.name +# +# processed_job_asset_keys.add(asset_key) +# graph_names_by_key[asset_key] = graph_name +# op_names_by_key[asset_key] = op_names +# op_name_by_key[asset_key] = op_name +# +# # build dependency data structures +# deps: Dict[AssetKey, Dict[AssetKey, ExternalAssetDependency]] = defaultdict(dict) +# dep_by: Dict[AssetKey, Dict[AssetKey, ExternalAssetDependedBy]] = defaultdict(dict) +# for assets_def in assets_defs: +# for asset_key in assets_def.keys: +# for upstream_key in assets_def.asset_deps[asset_key]: +# partition_mapping = assets_def.get_partition_mapping(upstream_key) +# deps[asset_key][upstream_key] = ExternalAssetDependency( +# upstream_asset_key=upstream_key, +# partition_mapping=( +# partition_mapping +# if isinstance(partition_mapping, get_builtin_partition_mapping_types()) +# else None +# ), +# ) +# dep_by[upstream_key][asset_key] = ExternalAssetDependedBy( +# downstream_asset_key=asset_key +# ) +# +# # build asset nodes +# for assets_def in assets_defs: +# partitions_data = ( +# external_partitions_definition_from_def(assets_def.partitions_def) +# if assets_def.partitions_def +# else None +# ) +# for asset_key in assets_def.keys: +# asset_nodes.append( +# ExternalAssetNode( +# asset_key=asset_key, +# dependencies=list(deps[asset_key].values()), +# depended_by=list(dep_by[asset_key].values()), +# compute_kind=assets_def.node_def.tags.get("kind"), +# op_name=op_name_by_key.get(asset_key), +# op_names=op_names_by_key.get(asset_key), +# code_version=assets_def.code_versions_by_key.get(asset_key), +# node_definition_name=assets_def.node_def.name, +# graph_name=graph_names_by_key.get(asset_key), +# op_description=assets_def.node_def.description, +# job_names=job_names_by_key[asset_key], +# partitions_def_data=partitions_data, +# output_name=None, +# output_description=None, +# metadata=assets_def.metadata_by_key.get(asset_key, {}), +# group_name=assets_def.group_names_by_key.get(asset_key), +# freshness_policy=assets_def.freshness_policies_by_key.get(asset_key), +# is_source=False, +# is_observable=False, +# atomic_execution_unit_id=assets_def.unique_id, +# required_top_level_resources=assets_def.required_resource_keys, +# auto_materialize_policy=assets_def.auto_materialize_policies_by_key.get( +# asset_key +# ), +# backfill_policy=assets_def.backfill_policy, +# auto_observe_interval_minutes=assets_def.metadata_by_key.get(asset_key, {}).get( +# SYSTEM_METADATA_KEY_AUTO_OBSERVE_INTERVAL_MINUTES +# ), +# ) +# ) + + def external_asset_nodes_from_defs( job_defs: Sequence[JobDefinition], - source_assets_by_key: Mapping[AssetKey, SourceAsset], + assets_defs_by_key: Mapping[AssetKey, AssetsDefinition], ) -> Sequence[ExternalAssetNode]: + external_assets_defs = [ + assets_def for assets_def in assets_defs_by_key.values() if is_external_asset(assets_def) + ] + external_asset_keys = {key for assets_def in external_assets_defs for key in assets_def.keys} node_defs_by_asset_key: Dict[ AssetKey, List[Tuple[NodeOutputHandle, JobDefinition]] ] = defaultdict(list) @@ -1546,6 +1699,7 @@ def external_asset_nodes_from_defs( group_name_by_asset_key: Dict[AssetKey, str] = {} descriptions_by_asset_key: Dict[AssetKey, str] = {} atomic_execution_unit_ids_by_key: Dict[Union[AssetKey, AssetCheckKey], str] = {} + execution_types_by_asset_key: Dict[AssetKey, AssetExecutionType] = {} for job_def in job_defs: asset_layer = job_def.asset_layer @@ -1569,6 +1723,9 @@ def external_asset_nodes_from_defs( all_upstream_asset_keys.update(upstream_asset_keys) node_defs_by_asset_key[output_key].append((node_output_handle, job_def)) asset_info_by_asset_key[output_key] = asset_info + execution_types_by_asset_key[output_key] = asset_layer.execution_type_for_asset( + output_key + ) for upstream_key in upstream_asset_keys: partition_mapping = asset_layer.partition_mapping_for_node_input( @@ -1607,13 +1764,14 @@ def external_asset_nodes_from_defs( asset_keys_without_definitions = all_upstream_asset_keys.difference( node_defs_by_asset_key.keys() - ).difference(source_assets_by_key.keys()) + ).difference(external_asset_keys) asset_nodes = [ ExternalAssetNode( asset_key=asset_key, dependencies=list(deps[asset_key].values()), depended_by=list(dep_by[asset_key].values()), + execution_type=AssetExecutionType.UNEXECUTABLE, job_names=[], group_name=group_name_by_asset_key.get(asset_key), code_version=code_version_by_asset_key.get(asset_key), @@ -1621,46 +1779,55 @@ def external_asset_nodes_from_defs( for asset_key in asset_keys_without_definitions ] - for source_asset in source_assets_by_key.values(): - if source_asset.key not in node_defs_by_asset_key: - job_names = ( - [ - job_def.name - for job_def in job_defs - if source_asset.key in job_def.asset_layer.source_assets_by_key - and ( - # explicit source-asset observation job - not job_def.asset_layer.has_assets_defs - # "base asset job" will have both source and materializable assets - or is_base_asset_job_name(job_def.name) + # for source_asset in source_assets_by_key.values(): + for external_assets_def in external_assets_defs: + for asset_key in external_assets_def.keys: + is_observable = ( + external_assets_def.execution_type == AssetExecutionType.OBSERVATION + ) + if asset_key not in node_defs_by_asset_key: + job_names = ( + [ + job_def.name + for job_def in job_defs + if asset_key in job_def.asset_layer.target_asset_keys and ( - source_asset.partitions_def is None - or source_asset.partitions_def == job_def.partitions_def + # explicit source-asset observation job + not job_def.asset_layer.has_assets_defs + # "base asset job" will have both source and materializable assets + or is_base_asset_job_name(job_def.name) + and ( + external_assets_def.partitions_def is None + or external_assets_def.partitions_def == job_def.partitions_def + ) ) + ] + if is_observable + else [] + ) + asset_nodes.append( + ExternalAssetNode( + asset_key=asset_key, + dependencies=list(deps[asset_key].values()), + depended_by=list(dep_by[asset_key].values()), + job_names=job_names, + op_description=external_assets_def.descriptions_by_key.get(asset_key), + metadata=external_assets_def.metadata_by_key.get(asset_key, {}), + group_name=external_assets_def.group_names_by_key.get(asset_key), + is_source=True, + is_observable=is_observable, + auto_observe_interval_minutes=external_assets_def.metadata_by_key.get( + asset_key, {} + ).get(SYSTEM_METADATA_KEY_AUTO_OBSERVE_INTERVAL_MINUTES), + partitions_def_data=( + external_partitions_definition_from_def( + external_assets_def.partitions_def + ) + if external_assets_def.partitions_def + else None + ), ) - ] - if source_asset.node_def is not None - else [] - ) - asset_nodes.append( - ExternalAssetNode( - asset_key=source_asset.key, - dependencies=list(deps[source_asset.key].values()), - depended_by=list(dep_by[source_asset.key].values()), - job_names=job_names, - op_description=source_asset.description, - metadata=source_asset.metadata, - group_name=source_asset.group_name, - is_source=True, - is_observable=source_asset.is_observable, - auto_observe_interval_minutes=source_asset.auto_observe_interval_minutes, - partitions_def_data=( - external_partitions_definition_from_def(source_asset.partitions_def) - if source_asset.partitions_def - else None - ), ) - ) for asset_key, node_tuple_list in node_defs_by_asset_key.items(): node_output_handle, job_def = node_tuple_list[0] @@ -1698,11 +1865,20 @@ def external_asset_nodes_from_defs( node_handle = node_handle.parent graph_name = node_handle.name + if asset_key in source_assets_by_key: + source_asset = source_assets_by_key[asset_key] + is_observable = source_asset.is_observable + auto_observe_interval_minutes = source_asset.auto_observe_interval_minutes + else: + is_observable = False + auto_observe_interval_minutes = None + asset_nodes.append( ExternalAssetNode( asset_key=asset_key, dependencies=list(deps[asset_key].values()), depended_by=list(dep_by[asset_key].values()), + execution_type=execution_types_by_asset_key[asset_key], compute_kind=node_def.tags.get("kind"), # backcompat op_name=graph_name @@ -1717,6 +1893,8 @@ def external_asset_nodes_from_defs( partitions_def_data=partitions_def_data, output_name=output_def.name, metadata=asset_metadata, + is_observable=is_observable, + auto_observe_interval_minutes=auto_observe_interval_minutes, # assets defined by Out(asset_key="k") do not have any group # name specified we default to DEFAULT_GROUP_NAME here to ensure # such assets are part of the default group diff --git a/python_modules/dagster/dagster/_utils/caching_instance_queryer.py b/python_modules/dagster/dagster/_utils/caching_instance_queryer.py index 5cf3327a3c750..6c2374d72ee35 100644 --- a/python_modules/dagster/dagster/_utils/caching_instance_queryer.py +++ b/python_modules/dagster/dagster/_utils/caching_instance_queryer.py @@ -184,10 +184,10 @@ def get_asset_record(self, asset_key: AssetKey) -> Optional["AssetRecord"]: return self._asset_record_cache[asset_key] def _event_type_for_key(self, asset_key: AssetKey) -> DagsterEventType: - if self.asset_graph.is_source(asset_key): - return DagsterEventType.ASSET_OBSERVATION - else: + if self.asset_graph.is_materializable(asset_key): return DagsterEventType.ASSET_MATERIALIZATION + else: + return DagsterEventType.ASSET_OBSERVATION @cached_method def _get_latest_materialization_or_observation_record( @@ -203,7 +203,7 @@ def _get_latest_materialization_or_observation_record( if ( before_cursor is None and asset_partition.partition_key is None - and not self.asset_graph.is_observable(asset_partition.asset_key) + and self.asset_graph.is_materializable(asset_partition.asset_key) ): asset_record = self.get_asset_record(asset_partition.asset_key) if asset_record is None: @@ -283,7 +283,7 @@ def asset_partition_has_materialization_or_observation( after_cursor (Optional[int]): Filter parameter such that only records with a storage_id greater than this value will be considered. """ - if not self.asset_graph.is_source(asset_partition.asset_key): + if self.asset_graph.is_materializable(asset_partition.asset_key): asset_record = self.get_asset_record(asset_partition.asset_key) if ( asset_record is None @@ -319,7 +319,6 @@ def get_latest_materialization_or_observation_record( "before_cursor", "Cannot set both before_cursor and after_cursor", ) - # first, do a quick check to eliminate the case where we know there is no record if not self.asset_partition_has_materialization_or_observation( asset_partition, after_cursor @@ -534,7 +533,7 @@ def asset_partitions_with_newly_updated_parents_and_new_cursor( """Finds asset partitions of the given child whose parents have been materialized since latest_storage_id. """ - if self.asset_graph.is_source(child_asset_key): + if not self.asset_graph.is_materializable(child_asset_key): return set(), latest_storage_id child_partitions_def = self.asset_graph.get_partitions_def(child_asset_key) @@ -548,10 +547,8 @@ def asset_partitions_with_newly_updated_parents_and_new_cursor( ) ] for parent_asset_key in self.asset_graph.get_parents(child_asset_key): - # ignore non-observable sources - if self.asset_graph.is_source(parent_asset_key) and not self.asset_graph.is_observable( - parent_asset_key - ): + # ignore non-executable assets + if not self.asset_graph.is_executable(parent_asset_key): continue # if the parent has not been updated at all since the latest_storage_id, then skip @@ -803,7 +800,8 @@ def get_asset_partitions_updated_after_cursor( if not updated_after_cursor: return set() if after_cursor is None or ( - not self.asset_graph.is_source(asset_key) and not respect_materialization_data_versions + self.asset_graph.is_materializable(asset_key) + and not respect_materialization_data_versions ): return updated_after_cursor @@ -862,10 +860,8 @@ def get_parent_asset_partitions_updated_after_child( if parent_key in ignored_parent_keys: continue - # ignore non-observable source parents - if self.asset_graph.is_source(parent_key) and not self.asset_graph.is_observable( - parent_key - ): + # ignore non-executable parents + if not self.asset_graph.is_executable(parent_key): continue # when mapping from unpartitioned assets to time partitioned assets, we ignore diff --git a/python_modules/dagster/dagster_tests/asset_defs_tests/partition_mapping_tests/test_asset_partition_mappings.py b/python_modules/dagster/dagster_tests/asset_defs_tests/partition_mapping_tests/test_asset_partition_mappings.py index 0a93730ca1001..618871682803f 100644 --- a/python_modules/dagster/dagster_tests/asset_defs_tests/partition_mapping_tests/test_asset_partition_mappings.py +++ b/python_modules/dagster/dagster_tests/asset_defs_tests/partition_mapping_tests/test_asset_partition_mappings.py @@ -119,7 +119,7 @@ def downstream_asset(context: AssetExecutionContext, upstream_asset): my_job = build_assets_job( "my_job", - assets=[upstream_asset, downstream_asset], + assets_to_execute=[upstream_asset, downstream_asset], resource_defs={"io_manager": IOManagerDefinition.hardcoded_io_manager(MyIOManager())}, ) result = my_job.execute_in_process(partition_key="2") @@ -168,7 +168,7 @@ def load_input(self, context): my_job = build_assets_job( "my_job", - assets=[ + assets_to_execute=[ AssetsDefinition.from_graph(upstream_asset, partitions_def=partitions_def), AssetsDefinition.from_graph( downstream_asset, @@ -204,7 +204,7 @@ def load_input(self, context): my_job = build_assets_job( "my_job", - assets=[upstream, downstream], + assets_to_execute=[upstream, downstream], resource_defs={"io_manager": IOManagerDefinition.hardcoded_io_manager(MyIOManager())}, ) result = my_job.execute_in_process(partition_key="b") @@ -313,7 +313,7 @@ def load_input(self, context): my_job = build_assets_job( "my_job", - assets=[upstream, downstream], + assets_to_execute=[upstream, downstream], resource_defs={"io_manager": IOManagerDefinition.hardcoded_io_manager(MyIOManager())}, ) result = my_job.execute_in_process(partition_key="b") @@ -363,15 +363,15 @@ def load_input(self, context): upstream_job = build_assets_job( "upstream_job", - assets=[upstream], + assets_to_execute=[upstream], resource_defs={"io_manager": IOManagerDefinition.hardcoded_io_manager(MyIOManager())}, ) upstream_job.execute_in_process(partition_key="2022-09-11") downstream_job = build_assets_job( "downstream_job", - assets=[downstream], - source_assets=[upstream], + assets_to_execute=[downstream], + other_assets=[upstream], resource_defs={"io_manager": IOManagerDefinition.hardcoded_io_manager(MyIOManager())}, ) downstream_job.execute_in_process(partition_key="2022-09-11") diff --git a/python_modules/dagster/dagster_tests/asset_defs_tests/test_asset_graph.py b/python_modules/dagster/dagster_tests/asset_defs_tests/test_asset_graph.py index b53d1a4768656..e0708f3617bb5 100644 --- a/python_modules/dagster/dagster_tests/asset_defs_tests/test_asset_graph.py +++ b/python_modules/dagster/dagster_tests/asset_defs_tests/test_asset_graph.py @@ -51,7 +51,7 @@ def repo(): return assets + (asset_checks or []) external_asset_nodes = external_asset_nodes_from_defs( - repo.get_all_jobs(), source_assets_by_key={} + repo.get_all_jobs(), assets_defs_by_key=repo.assets_defs_by_key ) return ExternalAssetGraph.from_repository_handles_and_external_asset_nodes( [(MagicMock(), asset_node) for asset_node in external_asset_nodes], diff --git a/python_modules/dagster/dagster_tests/asset_defs_tests/test_assets_job.py b/python_modules/dagster/dagster_tests/asset_defs_tests/test_assets_job.py index d1168f18d3a2e..c563b88b7b199 100644 --- a/python_modules/dagster/dagster_tests/asset_defs_tests/test_assets_job.py +++ b/python_modules/dagster/dagster_tests/asset_defs_tests/test_assets_job.py @@ -39,8 +39,10 @@ from dagster._core.definitions.asset_graph import AssetGraph from dagster._core.definitions.asset_selection import AssetSelection, CoercibleToAssetSelection from dagster._core.definitions.assets_job import get_base_asset_jobs +from dagster._core.definitions.data_version import DataVersion from dagster._core.definitions.dependency import NodeHandle, NodeInvocation from dagster._core.definitions.executor_definition import in_process_executor +from dagster._core.definitions.external_asset import create_external_asset_from_source_asset from dagster._core.definitions.load_assets_from_modules import prefix_assets from dagster._core.errors import DagsterInvalidSubsetError from dagster._core.execution.api import execute_run_iterator @@ -279,7 +281,7 @@ def my_io_manager(_): job = build_assets_job( "a", [asset1], - source_assets=[ + other_assets=[ SourceAsset( AssetKey("source1"), io_manager_key="special_io_manager", metadata={"a": "b"} ) @@ -303,14 +305,14 @@ def asset1(source1): with pytest.raises( DagsterInvalidDefinitionError, match=( - r"io manager with key 'special_io_manager' required by SourceAsset with key" + r"io manager with key 'special_io_manager' required by external asset with key" r" \[\"source1\"\] was not provided." ), ): build_assets_job( "a", [asset1], - source_assets=[SourceAsset(AssetKey("source1"), io_manager_key="special_io_manager")], + other_assets=[SourceAsset(AssetKey("source1"), io_manager_key="special_io_manager")], ) @@ -338,7 +340,7 @@ def my_io_manager(_): job = build_assets_job( "a", [asset1], - source_assets=[source1], + other_assets=[source1], resource_defs={"special_io_manager": my_io_manager}, ) assert job.graph.node_defs == [asset1.op] @@ -1298,7 +1300,7 @@ def test_subset_of_asset_job(): def test_subset_of_build_assets_job(): - foo_job = build_assets_job("foo_job", assets=[foo, bar, foo_bar, baz]) + foo_job = build_assets_job("foo_job", assets_to_execute=[foo, bar, foo_bar, baz]) with instance_for_test() as instance: result = foo_job.execute_in_process( instance=instance, @@ -1702,7 +1704,7 @@ def my_derived_asset(my_source_asset): return my_source_asset + 4 source_asset_job = build_assets_job( - name="test", assets=[my_derived_asset], source_assets=[my_source_asset] + name="test", assets_to_execute=[my_derived_asset], other_assets=[my_source_asset] ) result = source_asset_job.execute_in_process(asset_selection=[AssetKey("my_derived_asset")]) @@ -1730,8 +1732,8 @@ def my_derived_asset(my_source_asset): source_asset_job = build_assets_job( "the_job", - assets=[my_derived_asset], - source_assets=[my_source_asset], + assets_to_execute=[my_derived_asset], + other_assets=[my_source_asset], resource_defs={"io_manager": the_manager}, ) @@ -1760,8 +1762,8 @@ def my_derived_asset(my_source_asset): source_asset_job = build_assets_job( "the_job", - assets=[my_derived_asset], - source_assets=[my_source_asset], + assets_to_execute=[my_derived_asset], + other_assets=[my_source_asset], resource_defs={"some_key": the_manager}, ) @@ -1800,8 +1802,8 @@ def my_derived_asset(my_source_asset): source_asset_job = build_assets_job( "the_job", - assets=[my_derived_asset], - source_assets=[my_source_asset], + assets_to_execute=[my_derived_asset], + other_assets=[my_source_asset], ) result = source_asset_job.execute_in_process(asset_selection=[AssetKey("my_derived_asset")]) @@ -1825,7 +1827,7 @@ def asset_provides_foo(): DagsterInvalidDefinitionError, match="resource with key 'foo' required by op 'asset_reqs_foo' was not provided.", ): - build_assets_job(name="test", assets=[asset_reqs_foo, asset_provides_foo]) + build_assets_job(name="test", assets_to_execute=[asset_reqs_foo, asset_provides_foo]) @ignore_warning("Parameter `resource_defs` .* is experimental") @@ -1842,7 +1844,7 @@ def the_asset(): DagsterInvalidDefinitionError, match="resource with key 'foo' required by resource with key 'unused' was not provided.", ): - build_assets_job(name="test", assets=[the_asset]) + build_assets_job(name="test", assets_to_execute=[the_asset]) @ignore_warning("Parameter `resource_defs` .* is experimental") @@ -1857,7 +1859,7 @@ def used_resource(context): def the_asset(): pass - the_job = build_assets_job(name="test", assets=[the_asset]) + the_job = build_assets_job(name="test", assets_to_execute=[the_asset]) assert the_job.execute_in_process().success @@ -1883,7 +1885,9 @@ def my_derived_asset(my_source_asset): " was not provided." ), ): - build_assets_job(name="test", assets=[my_derived_asset], source_assets=[my_source_asset]) + build_assets_job( + name="test", assets_to_execute=[my_derived_asset], other_assets=[my_source_asset] + ) def test_resolve_dependency_in_group(): @@ -1976,7 +1980,6 @@ def unpartitioned_asset(): hourly_asset, unpartitioned_asset, ], - source_assets=[], executor_def=None, resource_defs={}, asset_checks=[], @@ -2020,10 +2023,8 @@ def asset_x(asset_b: B): jobs = get_base_asset_jobs( assets=[ asset_x, - ], - source_assets=[ - asset_a, - asset_b, + create_external_asset_from_source_asset(asset_a), + create_external_asset_from_source_asset(asset_b), ], executor_def=None, resource_defs={}, @@ -2048,7 +2049,9 @@ def an_asset(context) -> None: executed["yes"] = True a_job = build_assets_job( - "my_job", assets=[an_asset], resource_defs={"bare_resource": BareResourceObject()} + "my_job", + assets_to_execute=[an_asset], + resource_defs={"bare_resource": BareResourceObject()}, ) assert a_job.execute_in_process().success @@ -2091,7 +2094,7 @@ async def aio_gen_asset(context): context.log.info(v.output_name) yield v - aio_job = build_assets_job(name="test", assets=[aio_gen_asset]) + aio_job = build_assets_job(name="test", assets_to_execute=[aio_gen_asset]) result = aio_job.execute_in_process() assert result.success @@ -2105,7 +2108,7 @@ def asset1(): assert Definitions( assets=[source_asset, asset1], jobs=[define_asset_job("something", selection="abc/asset1")] - ).get_job_def("something").asset_layer.asset_keys == {AssetKey(["abc", "asset1"])} + ).get_job_def("something").asset_layer.target_asset_keys == {AssetKey(["abc", "asset1"])} @pytest.mark.parametrize( @@ -2885,3 +2888,36 @@ def python(): result = job.execute_in_process() assert result.success assert _all_asset_keys(result) == {AssetKey("a"), AssetKey("b")} + + +def test_mixed_asset_job(): + with disable_dagster_warnings(): + + class MyIOManager(IOManager): + def handle_output(self, context, obj): + pass + + def load_input(self, context): + return 5 + + @observable_source_asset + def foo(): + return DataVersion("alpha") + + @asset + def bar(foo): + return foo + 1 + + defs = Definitions( + assets=[foo, bar], + jobs=[define_asset_job("mixed_assets_job", [foo, bar])], + resources={"io_manager": MyIOManager()}, + ) + + job_def = defs.get_job_def("mixed_assets_job") + result = job_def.execute_in_process() + assert result.success + assert len(result.asset_materializations_for_node("foo")) == 0 + assert len(result.asset_observations_for_node("foo")) == 1 + assert len(result.asset_materializations_for_node("bar")) == 1 + assert len(result.asset_observations_for_node("bar")) == 0 diff --git a/python_modules/dagster/dagster_tests/asset_defs_tests/test_partitioned_assets.py b/python_modules/dagster/dagster_tests/asset_defs_tests/test_partitioned_assets.py index 45162314975aa..6439e9b182476 100644 --- a/python_modules/dagster/dagster_tests/asset_defs_tests/test_partitioned_assets.py +++ b/python_modules/dagster/dagster_tests/asset_defs_tests/test_partitioned_assets.py @@ -163,7 +163,7 @@ def my_asset(context: AssetExecutionContext): my_job = build_assets_job( "my_job", - assets=[my_asset], + assets_to_execute=[my_asset], resource_defs={"io_manager": IOManagerDefinition.hardcoded_io_manager(MyIOManager())}, ) result = my_job.execute_in_process(partition_key="b") @@ -183,7 +183,7 @@ def upstream(): def downstream(upstream): assert upstream is None - my_job = build_assets_job("my_job", assets=[upstream, downstream]) + my_job = build_assets_job("my_job", assets_to_execute=[upstream, downstream]) result = my_job.execute_in_process(partition_key="b") assert_namedtuple_lists_equal( result.asset_materializations_for_node("upstream"), @@ -208,7 +208,7 @@ def upstream(): def downstream(upstream): assert upstream is None - build_assets_job("my_job", assets=[upstream, downstream]) + build_assets_job("my_job", assets_to_execute=[upstream, downstream]) def test_access_partition_keys_from_context_direct_invocation(): @@ -304,7 +304,7 @@ def my_asset(): my_job = build_assets_job( "my_job", - assets=[my_asset], + assets_to_execute=[my_asset], resource_defs={"io_manager": IOManagerDefinition.hardcoded_io_manager(MyIOManager())}, ) my_job.execute_in_process(partition_key="2021-06-06") @@ -368,8 +368,8 @@ def load_input(self, context): daily_job = build_assets_job( name="daily_job", - assets=[daily_asset], - source_assets=[hourly_asset], + assets_to_execute=[daily_asset], + other_assets=[hourly_asset], resource_defs={"io_manager": IOManagerDefinition.hardcoded_io_manager(CustomIOManager())}, ) assert daily_job.execute_in_process(partition_key="2021-06-06").success @@ -396,8 +396,8 @@ def load_input(self, context): daily_job = build_assets_job( name="daily_job", - assets=[daily_asset], - source_assets=[hourly_asset], + assets_to_execute=[daily_asset], + other_assets=[hourly_asset], resource_defs={"io_manager": IOManagerDefinition.hardcoded_io_manager(CustomIOManager())}, ) assert daily_job.execute_in_process(partition_key="2021-06-06").success @@ -475,7 +475,7 @@ def my_asset(): my_job = build_assets_job( "my_job", - assets=[my_asset], + assets_to_execute=[my_asset], resource_defs={"io_manager": IOManagerDefinition.hardcoded_io_manager(MyIOManager())}, ) result = my_job.execute_in_process(partition_key="b") @@ -512,7 +512,7 @@ def downstream_asset_2(upstream_asset_2: int): del upstream_asset_2 my_job = build_assets_job( - "my_job", assets=[upstream_asset, downstream_asset_1, downstream_asset_2] + "my_job", assets_to_execute=[upstream_asset, downstream_asset_1, downstream_asset_2] ) result = my_job.execute_in_process(partition_key="b") diff --git a/python_modules/dagster/dagster_tests/asset_defs_tests/test_source_asset_observation_job.py b/python_modules/dagster/dagster_tests/asset_defs_tests/test_source_asset_observation_job.py index 0470cdcba7529..de2d7dbe38c45 100644 --- a/python_modules/dagster/dagster_tests/asset_defs_tests/test_source_asset_observation_job.py +++ b/python_modules/dagster/dagster_tests/asset_defs_tests/test_source_asset_observation_job.py @@ -92,24 +92,6 @@ def baz(): ) -def test_mixed_source_asset_observation_job(): - @observable_source_asset - def foo(_context) -> DataVersion: - return DataVersion("alpha") - - @asset(deps=["foo"]) - def bar(context): - return 1 - - with pytest.raises( - DagsterInvalidDefinitionError, match=r"specified both regular assets and source assets" - ): - Definitions( - assets=[foo, bar], - jobs=[define_asset_job("mixed_job", [foo, bar])], - ).get_all_job_defs() - - @pytest.mark.parametrize( "is_valid,resource_defs", [(True, {"bar": ResourceDefinition.hardcoded_resource("bar")}), (False, {})], diff --git a/python_modules/dagster/dagster_tests/asset_defs_tests/test_unresolved_asset_job.py b/python_modules/dagster/dagster_tests/asset_defs_tests/test_unresolved_asset_job.py index d9ce98e639609..f8b37f1cbc929 100644 --- a/python_modules/dagster/dagster_tests/asset_defs_tests/test_unresolved_asset_job.py +++ b/python_modules/dagster/dagster_tests/asset_defs_tests/test_unresolved_asset_job.py @@ -463,7 +463,7 @@ def asset3(): job1 = define_asset_job("job1", selection=[asset1, asset2]).resolve( asset_graph=AssetGraph.from_assets(all_assets) ) - asset_keys = list(job1.asset_layer.asset_keys) + asset_keys = list(job1.asset_layer.target_asset_keys) assert len(asset_keys) == 2 assert set(asset_keys) == {asset1.key, asset2.key} job1.execute_in_process() diff --git a/python_modules/dagster/dagster_tests/core_tests/execution_tests/test_asset_backfill.py b/python_modules/dagster/dagster_tests/core_tests/execution_tests/test_asset_backfill.py index a7fd0ff5ff506..7e26c856511e7 100644 --- a/python_modules/dagster/dagster_tests/core_tests/execution_tests/test_asset_backfill.py +++ b/python_modules/dagster/dagster_tests/core_tests/execution_tests/test_asset_backfill.py @@ -707,7 +707,7 @@ def external_asset_graph_from_assets_by_repo_name( repo = Definitions(assets=assets).get_repository_def() external_asset_nodes = external_asset_nodes_from_defs( - repo.get_all_jobs(), source_assets_by_key=repo.source_assets_by_key + repo.get_all_jobs(), assets_defs_by_key=repo.assets_defs_by_key ) repo_handle = MagicMock(repository_name=repo_name) from_repository_handles_and_external_asset_nodes.extend( diff --git a/python_modules/dagster/dagster_tests/core_tests/host_representation_tests/test_external_data.py b/python_modules/dagster/dagster_tests/core_tests/host_representation_tests/test_external_data.py index a8a873f15a6a7..5a3df08261775 100644 --- a/python_modules/dagster/dagster_tests/core_tests/host_representation_tests/test_external_data.py +++ b/python_modules/dagster/dagster_tests/core_tests/host_representation_tests/test_external_data.py @@ -1,4 +1,5 @@ from datetime import datetime +from typing import Sequence, Union import pendulum import pytest @@ -20,9 +21,12 @@ from dagster._check import ParameterCheckError from dagster._core.definitions import AssetIn, SourceAsset, asset, build_assets_job, multi_asset from dagster._core.definitions.asset_graph import AssetGraph -from dagster._core.definitions.asset_spec import AssetSpec +from dagster._core.definitions.asset_spec import AssetExecutionType, AssetSpec +from dagster._core.definitions.asset_spec import SYSTEM_METADATA_KEY_ASSET_EXECUTION_TYPE, AssetSpec from dagster._core.definitions.backfill_policy import BackfillPolicy +from dagster._core.definitions.definitions_class import Definitions from dagster._core.definitions.external_asset import external_assets_from_specs +from dagster._core.definitions.job_definition import JobDefinition from dagster._core.definitions.metadata import MetadataValue, TextMetadataValue, normalize_metadata from dagster._core.definitions.multi_dimensional_partitions import MultiPartitionsDefinition from dagster._core.definitions.partition import ScheduleType @@ -44,19 +48,32 @@ from dagster._utils.partitions import DEFAULT_HOURLY_FORMAT_WITHOUT_TIMEZONE +def _get_external_asset_nodes( + assets_jobs: Sequence[JobDefinition], + assets_defs: Sequence[Union[AssetsDefinition, SourceAsset]], +) -> Sequence[ExternalAssetNode]: + defs = Definitions( + assets=assets_defs, + jobs=assets_jobs, + ) + assets_defs_by_key = defs.get_repository_def().assets_defs_by_key + return external_asset_nodes_from_defs(assets_jobs, assets_defs_by_key=assets_defs_by_key) + + def test_single_asset_job(): @asset(description="hullo") def asset1(): return 1 assets_job = build_assets_job("assets_job", [asset1]) - external_asset_nodes = external_asset_nodes_from_defs([assets_job], source_assets_by_key={}) + external_asset_nodes = _get_external_asset_nodes([assets_job], [asset1]) assert external_asset_nodes == [ ExternalAssetNode( asset_key=AssetKey("asset1"), dependencies=[], depended_by=[], + execution_type=AssetExecutionType.MATERIALIZATION, op_name="asset1", graph_name=None, op_names=["asset1"], @@ -75,13 +92,14 @@ def asset1(): return 1 assets_job = build_assets_job("assets_job", [asset1]) - external_asset_nodes = external_asset_nodes_from_defs([assets_job], source_assets_by_key={}) + external_asset_nodes = _get_external_asset_nodes([assets_job], [asset1]) assert external_asset_nodes == [ ExternalAssetNode( asset_key=AssetKey("asset1"), dependencies=[], depended_by=[], + execution_type=AssetExecutionType.MATERIALIZATION, op_name="asset1", graph_name=None, op_names=["asset1"], @@ -101,13 +119,14 @@ def asset1(): return 1 assets_job = build_assets_job("assets_job", [asset1]) - external_asset_nodes = external_asset_nodes_from_defs([assets_job], source_assets_by_key={}) + external_asset_nodes = _get_external_asset_nodes([assets_job], [asset1]) assert external_asset_nodes == [ ExternalAssetNode( asset_key=AssetKey("asset1"), dependencies=[], depended_by=[], + execution_type=AssetExecutionType.MATERIALIZATION, op_name="asset1", graph_name=None, op_names=["asset1"], @@ -145,13 +164,14 @@ def asset1(): return 1 assets_job = build_assets_job("assets_job", [asset1]) - external_asset_nodes = external_asset_nodes_from_defs([assets_job], source_assets_by_key={}) + external_asset_nodes = _get_external_asset_nodes([assets_job], [asset1]) assert external_asset_nodes == [ ExternalAssetNode( asset_key=AssetKey("asset1"), dependencies=[], depended_by=[], + execution_type=AssetExecutionType.MATERIALIZATION, op_name="asset1", graph_name=None, op_names=["asset1"], @@ -182,7 +202,7 @@ def asset1(): return 1 assets_job = build_assets_job("assets_job", [asset1]) - external_asset_nodes = external_asset_nodes_from_defs([assets_job], source_assets_by_key={}) + external_asset_nodes = _get_external_asset_nodes([assets_job], [asset1]) assert external_asset_nodes[0].group_name == "group1" @@ -193,7 +213,7 @@ def asset1(): return 1 assets_job = build_assets_job("assets_job", [asset1]) - external_asset_nodes = external_asset_nodes_from_defs([assets_job], source_assets_by_key={}) + external_asset_nodes = _get_external_asset_nodes([assets_job], [asset1]) assert external_asset_nodes[0].group_name == DEFAULT_GROUP_NAME @@ -222,13 +242,14 @@ def asset2(asset1): assert asset1 == 1 assets_job = build_assets_job("assets_job", [asset1, asset2]) - external_asset_nodes = external_asset_nodes_from_defs([assets_job], source_assets_by_key={}) + external_asset_nodes = _get_external_asset_nodes([assets_job], [asset1, asset2]) assert external_asset_nodes == [ ExternalAssetNode( asset_key=AssetKey("asset1"), dependencies=[], depended_by=[ExternalAssetDependedBy(downstream_asset_key=AssetKey("asset2"))], + execution_type=AssetExecutionType.MATERIALIZATION, op_name="asset1", node_definition_name="asset1", graph_name=None, @@ -242,6 +263,7 @@ def asset2(asset1): asset_key=AssetKey("asset2"), dependencies=[ExternalAssetDependency(upstream_asset_key=AssetKey("asset1"))], depended_by=[], + execution_type=AssetExecutionType.MATERIALIZATION, op_name="asset2", node_definition_name="asset2", graph_name=None, @@ -261,21 +283,28 @@ def test_input_name_matches_output_name(): def something(result): pass - assets_job = build_assets_job("assets_job", [something], source_assets=[not_result]) - external_asset_nodes = external_asset_nodes_from_defs([assets_job], source_assets_by_key={}) + assets_job = build_assets_job("assets_job", [something], other_assets=[not_result]) + external_asset_nodes = _get_external_asset_nodes([assets_job], [something, not_result]) assert external_asset_nodes == [ ExternalAssetNode( asset_key=AssetKey("not_result"), dependencies=[], depended_by=[ExternalAssetDependedBy(downstream_asset_key=AssetKey("something"))], + execution_type=AssetExecutionType.UNEXECUTABLE, job_names=[], group_name=DEFAULT_GROUP_NAME, + metadata=normalize_metadata( + { + SYSTEM_METADATA_KEY_ASSET_EXECUTION_TYPE: "UNEXECUTABLE", + } + ), ), ExternalAssetNode( asset_key=AssetKey("something"), dependencies=[ExternalAssetDependency(upstream_asset_key=AssetKey("not_result"))], depended_by=[], + execution_type=AssetExecutionType.MATERIALIZATION, op_name="something", node_definition_name="something", graph_name=None, @@ -313,13 +342,14 @@ def c2(c): asset_graph=AssetGraph.from_assets(all_assets) ) - external_asset_nodes = external_asset_nodes_from_defs([as_job, cs_job], source_assets_by_key={}) + external_asset_nodes = _get_external_asset_nodes([as_job, cs_job], all_assets) assert ( ExternalAssetNode( asset_key=AssetKey("a"), dependencies=[], depended_by=[ExternalAssetDependedBy(downstream_asset_key=AssetKey("a2"))], + execution_type=AssetExecutionType.MATERIALIZATION, op_name="abc", node_definition_name="abc", graph_name=None, @@ -337,6 +367,7 @@ def c2(c): asset_key=AssetKey("c"), dependencies=[], depended_by=[ExternalAssetDependedBy(downstream_asset_key=AssetKey("c2"))], + execution_type=AssetExecutionType.MATERIALIZATION, op_name="abc", node_definition_name="abc", graph_name=None, @@ -363,7 +394,7 @@ def asset2_b(asset1): assert asset1 == 1 assets_job = build_assets_job("assets_job", [asset1, asset2_a, asset2_b]) - external_asset_nodes = external_asset_nodes_from_defs([assets_job], source_assets_by_key={}) + external_asset_nodes = _get_external_asset_nodes([assets_job], [asset1, asset2_a, asset2_b]) assert external_asset_nodes == [ ExternalAssetNode( @@ -373,6 +404,7 @@ def asset2_b(asset1): ExternalAssetDependedBy(downstream_asset_key=AssetKey("asset2_a")), ExternalAssetDependedBy(downstream_asset_key=AssetKey("asset2_b")), ], + execution_type=AssetExecutionType.MATERIALIZATION, op_name="asset1", node_definition_name="asset1", graph_name=None, @@ -386,6 +418,7 @@ def asset2_b(asset1): asset_key=AssetKey("asset2_a"), dependencies=[ExternalAssetDependency(upstream_asset_key=AssetKey("asset1"))], depended_by=[], + execution_type=AssetExecutionType.MATERIALIZATION, op_name="asset2_a", node_definition_name="asset2_a", graph_name=None, @@ -399,6 +432,7 @@ def asset2_b(asset1): asset_key=AssetKey("asset2_b"), dependencies=[ExternalAssetDependency(upstream_asset_key=AssetKey("asset1"))], depended_by=[], + execution_type=AssetExecutionType.MATERIALIZATION, op_name="asset2_b", node_definition_name="asset2_b", graph_name=None, @@ -421,16 +455,15 @@ def asset2(asset1): assert asset1 == 1 assets_job1 = build_assets_job("assets_job1", [asset1]) - assets_job2 = build_assets_job("assets_job2", [asset2], source_assets=[asset1]) - external_asset_nodes = external_asset_nodes_from_defs( - [assets_job1, assets_job2], source_assets_by_key={} - ) + assets_job2 = build_assets_job("assets_job2", [asset2], other_assets=[asset1]) + external_asset_nodes = _get_external_asset_nodes([assets_job1, assets_job2], [asset1, asset2]) assert external_asset_nodes == [ ExternalAssetNode( asset_key=AssetKey("asset1"), dependencies=[], depended_by=[ExternalAssetDependedBy(downstream_asset_key=AssetKey("asset2"))], + execution_type=AssetExecutionType.MATERIALIZATION, op_name="asset1", node_definition_name="asset1", graph_name=None, @@ -444,6 +477,7 @@ def asset2(asset1): asset_key=AssetKey("asset2"), dependencies=[ExternalAssetDependency(upstream_asset_key=AssetKey("asset1"))], depended_by=[], + execution_type=AssetExecutionType.MATERIALIZATION, op_name="asset2", node_definition_name="asset2", graph_name=None, @@ -464,13 +498,14 @@ def asset1(): job1 = build_assets_job("job1", [asset1]) job2 = build_assets_job("job2", [asset1]) - external_asset_nodes = external_asset_nodes_from_defs([job1, job2], source_assets_by_key={}) + external_asset_nodes = _get_external_asset_nodes([job1, job2], [asset1]) assert external_asset_nodes == [ ExternalAssetNode( asset_key=AssetKey("asset1"), dependencies=[], depended_by=[], + execution_type=AssetExecutionType.MATERIALIZATION, op_name="asset1", node_definition_name="asset1", graph_name=None, @@ -496,7 +531,7 @@ def assets(): assets_job = build_assets_job("assets_job", [assets]) - external_asset_nodes = external_asset_nodes_from_defs([assets_job], source_assets_by_key={}) + external_asset_nodes = _get_external_asset_nodes([assets_job], [assets]) atomic_execution_unit_id = assets.unique_id @@ -505,6 +540,7 @@ def assets(): asset_key=AssetKey(f"asset{i}"), dependencies=[], depended_by=[], + execution_type=AssetExecutionType.MATERIALIZATION, op_name="assets", node_definition_name="assets", graph_name=None, @@ -549,8 +585,8 @@ def assets(in1, in2): ) all_assets_job = build_assets_job("assets_job", [in1, in2, assets, downstream]) - external_asset_nodes = external_asset_nodes_from_defs( - [subset_job, all_assets_job], source_assets_by_key={} + external_asset_nodes = _get_external_asset_nodes( + [subset_job, all_assets_job], [in1, in2, assets, downstream] ) # sort so that test is deterministic sorted_nodes = sorted( @@ -573,6 +609,7 @@ def assets(in1, in2): ExternalAssetDependency(upstream_asset_key=AssetKey(["only_out"])), ], depended_by=[], + execution_type=AssetExecutionType.MATERIALIZATION, op_name="downstream", node_definition_name="downstream", graph_name=None, @@ -590,6 +627,7 @@ def assets(in1, in2): ExternalAssetDependedBy(downstream_asset_key=AssetKey(["mixed"])), ExternalAssetDependedBy(downstream_asset_key=AssetKey(["only_in"])), ], + execution_type=AssetExecutionType.MATERIALIZATION, op_name="in1", node_definition_name="in1", graph_name=None, @@ -604,6 +642,7 @@ def assets(in1, in2): asset_key=AssetKey(["in2"]), dependencies=[], depended_by=[ExternalAssetDependedBy(downstream_asset_key=AssetKey(["only_in"]))], + execution_type=AssetExecutionType.MATERIALIZATION, op_name="in2", node_definition_name="in2", graph_name=None, @@ -624,6 +663,7 @@ def assets(in1, in2): ExternalAssetDependedBy(downstream_asset_key=AssetKey(["downstream"])), ExternalAssetDependedBy(downstream_asset_key=AssetKey(["only_out"])), ], + execution_type=AssetExecutionType.MATERIALIZATION, op_name="assets", node_definition_name="assets", graph_name=None, @@ -644,6 +684,7 @@ def assets(in1, in2): ExternalAssetDependedBy(downstream_asset_key=AssetKey(["mixed"])), ExternalAssetDependedBy(downstream_asset_key=AssetKey(["only_out"])), ], + execution_type=AssetExecutionType.MATERIALIZATION, op_name="assets", node_definition_name="assets", graph_name=None, @@ -663,6 +704,7 @@ def assets(in1, in2): depended_by=[ ExternalAssetDependedBy(downstream_asset_key=AssetKey(["downstream"])), ], + execution_type=AssetExecutionType.MATERIALIZATION, op_name="assets", node_definition_name="assets", graph_name=None, @@ -682,20 +724,27 @@ def test_source_asset_with_op(): def bar(foo): pass - assets_job = build_assets_job("assets_job", [bar], source_assets=[foo]) + assets_job = build_assets_job("assets_job", [bar], other_assets=[foo]) - external_asset_nodes = external_asset_nodes_from_defs([assets_job], source_assets_by_key={}) + external_asset_nodes = _get_external_asset_nodes([assets_job], [foo, bar]) assert external_asset_nodes == [ ExternalAssetNode( asset_key=AssetKey("foo"), + execution_type=AssetExecutionType.UNEXECUTABLE, op_description=None, dependencies=[], depended_by=[ExternalAssetDependedBy(AssetKey("bar"))], job_names=[], group_name=DEFAULT_GROUP_NAME, + metadata=normalize_metadata( + { + SYSTEM_METADATA_KEY_ASSET_EXECUTION_TYPE: "UNEXECUTABLE", + } + ), ), ExternalAssetNode( asset_key=AssetKey("bar"), + execution_type=AssetExecutionType.MATERIALIZATION, op_name="bar", node_definition_name="bar", graph_name=None, @@ -714,27 +763,37 @@ def test_unused_source_asset(): foo = SourceAsset(key=AssetKey("foo"), description="abc") bar = SourceAsset(key=AssetKey("bar"), description="def") - external_asset_nodes = external_asset_nodes_from_defs( - [], source_assets_by_key={AssetKey("foo"): foo, AssetKey("bar"): bar} - ) + external_asset_nodes = _get_external_asset_nodes([], [foo, bar]) assert external_asset_nodes == [ ExternalAssetNode( asset_key=AssetKey("foo"), op_description="abc", dependencies=[], depended_by=[], + execution_type=AssetExecutionType.UNEXECUTABLE, job_names=[], group_name=DEFAULT_GROUP_NAME, is_source=True, + metadata=normalize_metadata( + { + SYSTEM_METADATA_KEY_ASSET_EXECUTION_TYPE: "UNEXECUTABLE", + } + ), ), ExternalAssetNode( asset_key=AssetKey("bar"), op_description="def", dependencies=[], depended_by=[], + execution_type=AssetExecutionType.UNEXECUTABLE, job_names=[], group_name=DEFAULT_GROUP_NAME, is_source=True, + metadata=normalize_metadata( + { + SYSTEM_METADATA_KEY_ASSET_EXECUTION_TYPE: "UNEXECUTABLE", + } + ), ), ] @@ -746,20 +805,24 @@ def test_used_source_asset(): def foo(bar): assert bar - job1 = build_assets_job("job1", [foo], source_assets=[bar]) + job1 = build_assets_job("job1", [foo], other_assets=[bar]) - external_asset_nodes = external_asset_nodes_from_defs( - [job1], source_assets_by_key={AssetKey("bar"): bar} - ) + external_asset_nodes = _get_external_asset_nodes([job1], [bar, foo]) assert external_asset_nodes == [ ExternalAssetNode( asset_key=AssetKey("bar"), op_description="def", dependencies=[], depended_by=[ExternalAssetDependedBy(downstream_asset_key=AssetKey(["foo"]))], + execution_type=AssetExecutionType.UNEXECUTABLE, job_names=[], group_name=DEFAULT_GROUP_NAME, is_source=True, + metadata=normalize_metadata( + { + SYSTEM_METADATA_KEY_ASSET_EXECUTION_TYPE: "UNEXECUTABLE", + } + ), ), ExternalAssetNode( asset_key=AssetKey("foo"), @@ -770,6 +833,7 @@ def foo(bar): op_description=None, dependencies=[ExternalAssetDependency(upstream_asset_key=AssetKey(["bar"]))], depended_by=[], + execution_type=AssetExecutionType.MATERIALIZATION, job_names=["job1"], output_name="result", group_name=DEFAULT_GROUP_NAME, @@ -811,7 +875,7 @@ def zero(): assets_job = build_assets_job("assets_job", [zero, three_asset]) - external_asset_nodes = external_asset_nodes_from_defs([assets_job], source_assets_by_key={}) + external_asset_nodes = _get_external_asset_nodes([assets_job], [zero, three_asset]) # sort so that test is deterministic sorted_nodes = sorted( @@ -832,6 +896,7 @@ def zero(): asset_key=AssetKey(["three"]), dependencies=[ExternalAssetDependency(AssetKey(["zero"]))], depended_by=[], + execution_type=AssetExecutionType.MATERIALIZATION, op_name="three", node_definition_name="add_one", graph_name="three", @@ -846,6 +911,7 @@ def zero(): asset_key=AssetKey(["zero"]), dependencies=[], depended_by=[ExternalAssetDependedBy(AssetKey(["three"]))], + execution_type=AssetExecutionType.MATERIALIZATION, op_name="zero", node_definition_name="zero", graph_name=None, @@ -923,7 +989,9 @@ def create_twenty(thirteen, six): assets_job = build_assets_job("assets_job", [zero, eight_and_five, thirteen_and_six, twenty]) - external_asset_nodes = external_asset_nodes_from_defs([assets_job], source_assets_by_key={}) + external_asset_nodes = _get_external_asset_nodes( + [assets_job], [zero, eight_and_five, thirteen_and_six, twenty] + ) # sort so that test is deterministic sorted_nodes = sorted( [ @@ -946,6 +1014,7 @@ def create_twenty(thirteen, six): ExternalAssetDependency(AssetKey(["zero"])), ], depended_by=[ExternalAssetDependedBy(AssetKey(["twenty"]))], + execution_type=AssetExecutionType.MATERIALIZATION, op_name="create_thirteen_and_six", node_definition_name="add_one", graph_name="create_thirteen_and_six", @@ -969,6 +1038,7 @@ def create_twenty(thirteen, six): ExternalAssetDependency(AssetKey(["thirteen"])), ], depended_by=[], + execution_type=AssetExecutionType.MATERIALIZATION, op_name="create_twenty", node_definition_name="add_one", graph_name="create_twenty", @@ -988,6 +1058,7 @@ def create_twenty(thirteen, six): ExternalAssetDependedBy(AssetKey(["six"])), ExternalAssetDependedBy(AssetKey(["thirteen"])), ], + execution_type=AssetExecutionType.MATERIALIZATION, op_name="zero", node_definition_name="zero", graph_name=None, @@ -1011,13 +1082,14 @@ def asset2(asset1): del asset1 assets_job = build_assets_job("assets_job", [asset1, asset2]) - external_asset_nodes = external_asset_nodes_from_defs([assets_job], source_assets_by_key={}) + external_asset_nodes = _get_external_asset_nodes([assets_job], [asset1, asset2]) assert external_asset_nodes == [ ExternalAssetNode( asset_key=AssetKey(["abc", "asset1"]), dependencies=[], depended_by=[ExternalAssetDependedBy(downstream_asset_key=AssetKey("asset2"))], + execution_type=AssetExecutionType.MATERIALIZATION, op_name="abc__asset1", node_definition_name="abc__asset1", graph_name=None, @@ -1031,6 +1103,7 @@ def asset2(asset1): asset_key=AssetKey("asset2"), dependencies=[ExternalAssetDependency(upstream_asset_key=AssetKey(["abc", "asset1"]))], depended_by=[], + execution_type=AssetExecutionType.MATERIALIZATION, op_name="asset2", node_definition_name="asset2", graph_name=None, @@ -1141,7 +1214,7 @@ def foo(): assets_job = build_assets_job("assets_job", [foo]) - external_asset_nodes = external_asset_nodes_from_defs([assets_job], source_assets_by_key={}) + external_asset_nodes = _get_external_asset_nodes([assets_job], [foo]) assert external_asset_nodes[0].op_description == "bar" @@ -1167,7 +1240,7 @@ def foo(): external_asset_nodes = { asset_node.asset_key: asset_node - for asset_node in external_asset_nodes_from_defs([assets_job], source_assets_by_key={}) + for asset_node in _get_external_asset_nodes([assets_job], [foo]) } assert external_asset_nodes[AssetKey("asset1")].op_description == "bar" assert external_asset_nodes[AssetKey("asset2")].op_description == "baz" @@ -1193,7 +1266,7 @@ def test_external_assets_def_to_external_asset_graph() -> None: asset_one = next(iter(external_assets_from_specs([AssetSpec("asset_one")]))) assets_job = build_assets_job("assets_job", [asset_one]) - external_asset_nodes = external_asset_nodes_from_defs([assets_job], source_assets_by_key={}) + external_asset_nodes = _get_external_asset_nodes([assets_job], [asset_one]) assert len(external_asset_nodes) == 1 assert next(iter(external_asset_nodes)).is_executable is False diff --git a/python_modules/dagster/dagster_tests/core_tests/instance_tests/test_instance.py b/python_modules/dagster/dagster_tests/core_tests/instance_tests/test_instance.py index adef331448e64..a346afee39b7e 100644 --- a/python_modules/dagster/dagster_tests/core_tests/instance_tests/test_instance.py +++ b/python_modules/dagster/dagster_tests/core_tests/instance_tests/test_instance.py @@ -260,7 +260,7 @@ def noop_asset(): pass -noop_asset_job = build_assets_job(assets=[noop_asset], name="noop_asset_job") +noop_asset_job = build_assets_job(assets_to_execute=[noop_asset], name="noop_asset_job") def test_create_job_snapshot(): diff --git a/python_modules/dagster/dagster_tests/core_tests/resource_tests/test_with_resources.py b/python_modules/dagster/dagster_tests/core_tests/resource_tests/test_with_resources.py index 62344ce0fc59f..1ce1788558099 100644 --- a/python_modules/dagster/dagster_tests/core_tests/resource_tests/test_with_resources.py +++ b/python_modules/dagster/dagster_tests/core_tests/resource_tests/test_with_resources.py @@ -136,7 +136,7 @@ def my_derived_asset(my_source_asset): # generic key is used as the io manager key for the source asset. assert transformed_source.get_io_manager_key() == "io_manager" - the_job = build_assets_job("the_job", [transformed_derived], source_assets=[transformed_source]) + the_job = build_assets_job("the_job", [transformed_derived], other_assets=[transformed_source]) result = the_job.execute_in_process() assert result.success @@ -169,7 +169,7 @@ def my_derived_asset(my_source_asset): # generic key is used as the io manager key for the source asset. assert transformed_source.get_io_manager_key() == "the_manager" - the_job = build_assets_job("the_job", [transformed_derived], source_assets=[transformed_source]) + the_job = build_assets_job("the_job", [transformed_derived], other_assets=[transformed_source]) result = the_job.execute_in_process() assert result.success @@ -202,7 +202,7 @@ def my_derived_asset(my_source_asset): # override key. assert transformed_source.io_manager_def == the_manager - the_job = build_assets_job("the_job", [transformed_derived], source_assets=[transformed_source]) + the_job = build_assets_job("the_job", [transformed_derived], other_assets=[transformed_source]) result = the_job.execute_in_process() assert result.success @@ -354,7 +354,7 @@ def foo_resource(context): def my_derived_asset(my_source_asset): return my_source_asset + 4 - the_job = build_assets_job("the_job", [my_derived_asset], source_assets=[transformed_source]) + the_job = build_assets_job("the_job", [my_derived_asset], other_assets=[transformed_source]) result = the_job.execute_in_process() assert result.success diff --git a/python_modules/dagster/dagster_tests/core_tests/test_data_time.py b/python_modules/dagster/dagster_tests/core_tests/test_data_time.py index 66c0ca49a6ce2..0cabbbd36eeec 100644 --- a/python_modules/dagster/dagster_tests/core_tests/test_data_time.py +++ b/python_modules/dagster/dagster_tests/core_tests/test_data_time.py @@ -333,7 +333,8 @@ def observe_sources(*args): def observe_sources_fn(*, instance, times_by_key, **kwargs): for arg in args: key = AssetKey(arg) - observe(source_assets=[versioned_repo.source_assets_by_key[key]], instance=instance) + # observe(source_assets=[versioned_repo.source_assets_by_key[key]], instance=instance) + observe(assets=[versioned_repo.assets_defs_by_key[key]], instance=instance) latest_record = instance.get_latest_data_version_record(key, is_source=True) latest_timestamp = latest_record.timestamp times_by_key[key].append( diff --git a/python_modules/dagster/dagster_tests/definitions_tests/auto_materialize_tests/test_auto_observe.py b/python_modules/dagster/dagster_tests/definitions_tests/auto_materialize_tests/test_auto_observe.py index 5fcb4088a7b19..6500202aef0ed 100644 --- a/python_modules/dagster/dagster_tests/definitions_tests/auto_materialize_tests/test_auto_observe.py +++ b/python_modules/dagster/dagster_tests/definitions_tests/auto_materialize_tests/test_auto_observe.py @@ -7,6 +7,7 @@ ) from dagster._core.definitions.asset_daemon_cursor import AssetDaemonCursor from dagster._core.definitions.asset_graph import AssetGraph +from dagster._core.definitions.external_asset import create_external_asset_from_source_asset from pytest import fixture @@ -44,25 +45,26 @@ def asset1(): ) -@fixture -def single_auto_observe_source_asset_graph(): +@fixture(params=[True, False], ids=["use_external_asset", "use_source_asset"]) +def single_auto_observe_asset_graph(request): @observable_source_asset(auto_observe_interval_minutes=30) def asset1(): ... - asset_graph = AssetGraph.from_assets([asset1]) + observable = create_external_asset_from_source_asset(asset1) if request.param else asset1 + asset_graph = AssetGraph.from_assets([observable]) return asset_graph def test_single_observable_source_asset_no_prior_observe_requests( - single_auto_observe_source_asset_graph, + single_auto_observe_asset_graph, ): run_requests = get_auto_observe_run_requests( - asset_graph=single_auto_observe_source_asset_graph, + asset_graph=single_auto_observe_asset_graph, current_timestamp=1000, last_observe_request_timestamp_by_asset_key={}, run_tags={}, - auto_observe_asset_keys=single_auto_observe_source_asset_graph.source_asset_keys, + auto_observe_asset_keys=single_auto_observe_asset_graph.observable_keys, ) assert len(run_requests) == 1 run_request = run_requests[0] @@ -70,16 +72,16 @@ def test_single_observable_source_asset_no_prior_observe_requests( def test_single_observable_source_asset_prior_observe_requests( - single_auto_observe_source_asset_graph, + single_auto_observe_asset_graph, ): last_timestamp = 1000 run_requests = get_auto_observe_run_requests( - asset_graph=single_auto_observe_source_asset_graph, + asset_graph=single_auto_observe_asset_graph, current_timestamp=last_timestamp + 30 * 60 + 5, last_observe_request_timestamp_by_asset_key={AssetKey("asset1"): last_timestamp}, run_tags={}, - auto_observe_asset_keys=single_auto_observe_source_asset_graph.source_asset_keys, + auto_observe_asset_keys=single_auto_observe_asset_graph.observable_keys, ) assert len(run_requests) == 1 run_request = run_requests[0] @@ -87,16 +89,16 @@ def test_single_observable_source_asset_prior_observe_requests( def test_single_observable_source_asset_prior_recent_observe_requests( - single_auto_observe_source_asset_graph, + single_auto_observe_asset_graph, ): last_timestamp = 1000 run_requests = get_auto_observe_run_requests( - asset_graph=single_auto_observe_source_asset_graph, + asset_graph=single_auto_observe_asset_graph, current_timestamp=last_timestamp + 30 * 60 - 5, last_observe_request_timestamp_by_asset_key={AssetKey("asset1"): last_timestamp}, run_tags={}, - auto_observe_asset_keys=single_auto_observe_source_asset_graph.source_asset_keys, + auto_observe_asset_keys=single_auto_observe_asset_graph.observable_keys, ) assert len(run_requests) == 0 diff --git a/python_modules/dagster/dagster_tests/definitions_tests/decorators_tests/test_source_asset_decorator.py b/python_modules/dagster/dagster_tests/definitions_tests/decorators_tests/test_source_asset_decorator.py index 898462dfa99b9..420a930bb9bf3 100644 --- a/python_modules/dagster/dagster_tests/definitions_tests/decorators_tests/test_source_asset_decorator.py +++ b/python_modules/dagster/dagster_tests/definitions_tests/decorators_tests/test_source_asset_decorator.py @@ -48,9 +48,7 @@ def observable_asset_no_context(): executed["yes"] = True return DataVersion("version-string") - asset_job = build_assets_job( - "source_job", source_assets=[observable_asset_no_context], assets=[] - ) + asset_job = build_assets_job("source_job", assets_to_execute=[observable_asset_no_context]) defs = Definitions(jobs=[asset_job], assets=[observable_asset_no_context]) diff --git a/python_modules/dagster/dagster_tests/definitions_tests/test_external_assets.py b/python_modules/dagster/dagster_tests/definitions_tests/test_external_assets.py index 720bc97f36e2f..01188795a561a 100644 --- a/python_modules/dagster/dagster_tests/definitions_tests/test_external_assets.py +++ b/python_modules/dagster/dagster_tests/definitions_tests/test_external_assets.py @@ -51,7 +51,7 @@ def test_external_asset_basic_creation() -> None: assert assets_def.metadata_by_key[expected_key]["user_metadata"] == "value" assert assets_def.group_names_by_key[expected_key] == "a_group" assert assets_def.descriptions_by_key[expected_key] == "desc" - assert assets_def.is_asset_executable(expected_key) is False + assert not assets_def.is_executable def test_multi_external_asset_basic_creation() -> None: @@ -92,7 +92,7 @@ def test_normal_asset_materializeable() -> None: def an_asset() -> None: ... - assert an_asset.is_asset_executable(AssetKey(["an_asset"])) is True + assert an_asset.is_executable def test_external_asset_creation_with_deps() -> None: @@ -209,18 +209,14 @@ def an_asset(context: AssetExecutionContext, source_asset: str) -> str: assert result_one.output_for_node("an_asset") == "hardcoded-computed-2021-01-02" shimmed_source_asset = create_external_asset_from_source_asset(source_asset) - defs_with_shim = Definitions( - assets=[create_external_asset_from_source_asset(source_asset), an_asset] - ) + defs_with_shim = Definitions(assets=[shimmed_source_asset, an_asset]) assert isinstance(defs_with_shim.get_assets_def("source_asset"), AssetsDefinition) - job_def_with_shim = get_job_for_assets(defs_with_shim, an_asset, shimmed_source_asset) + job_def_with_shim = get_job_for_assets(defs_with_shim, an_asset) result_two = job_def_with_shim.execute_in_process( instance=instance, - # currently we have to explicitly select the asset to exclude the source from execution - asset_selection=[AssetKey("an_asset")], partition_key="2021-01-03", ) @@ -234,7 +230,8 @@ def an_observable_source_asset() -> DataVersion: return DataVersion("foo") assets_def = create_external_asset_from_source_asset(an_observable_source_asset) - assert assets_def.is_asset_executable(an_observable_source_asset.key) + assert assets_def.is_executable + assert assets_def.is_observable defs = Definitions(assets=[assets_def]) instance = DagsterInstance.ephemeral() @@ -267,9 +264,9 @@ def _downstream_asset(context: AssetExecutionContext) -> None: defs = Definitions(assets=[_upstream_def, _downstream_asset]) assert defs - assert defs.get_implicit_global_asset_job_def().asset_layer.asset_deps[ - AssetKey("downstream_asset") - ] == {AssetKey("upstream_asset")} + assert defs.get_asset_graph().asset_dep_graph["upstream"][AssetKey("downstream_asset")] == { + AssetKey("upstream_asset") + } def test_external_asset_multi_asset() -> None: @@ -283,9 +280,9 @@ def _generated_asset_def(context: AssetExecutionContext): defs = Definitions(assets=[_generated_asset_def]) assert defs - assert defs.get_implicit_global_asset_job_def().asset_layer.asset_deps[ - AssetKey("downstream_asset") - ] == {AssetKey("upstream_asset")} + assert defs.get_asset_graph().asset_dep_graph["upstream"][AssetKey("downstream_asset")] == { + AssetKey("upstream_asset") + } def test_external_assets_with_dependencies() -> None: @@ -295,6 +292,6 @@ def test_external_assets_with_dependencies() -> None: defs = Definitions(assets=external_assets_from_specs([upstream_asset, downstream_asset])) assert defs - assert defs.get_implicit_global_asset_job_def().asset_layer.asset_deps[ - AssetKey("downstream_asset") - ] == {AssetKey("upstream_asset")} + assert defs.get_asset_graph().asset_dep_graph["upstream"][AssetKey("downstream_asset")] == { + AssetKey("upstream_asset") + } diff --git a/python_modules/dagster/dagster_tests/definitions_tests/test_observe_result.py b/python_modules/dagster/dagster_tests/definitions_tests/test_observe_result.py index 099f903ff7774..0d60148b586d2 100644 --- a/python_modules/dagster/dagster_tests/definitions_tests/test_observe_result.py +++ b/python_modules/dagster/dagster_tests/definitions_tests/test_observe_result.py @@ -1,5 +1,5 @@ import asyncio -from typing import Any, Callable, Dict, Generator, Tuple +from typing import Generator, Tuple import pytest from dagster import ( @@ -19,10 +19,9 @@ ) from dagster._core.definitions.asset_check_spec import AssetCheckKey from dagster._core.definitions.asset_spec import ( - SYSTEM_METADATA_KEY_ASSET_EXECUTION_TYPE, AssetExecutionType, ) -from dagster._core.definitions.assets import AssetsDefinition +from dagster._core.definitions.observe import observe from dagster._core.definitions.result import ObserveResult from dagster._core.errors import DagsterInvariantViolationError, DagsterStepOutputNotFoundError from dagster._core.execution.context.invocation import build_asset_context @@ -30,43 +29,13 @@ def _exec_asset(asset_def, selection=None, partition_key=None): - result = materialize([asset_def], selection=selection, partition_key=partition_key) + result = observe([asset_def], partition_key=partition_key) assert result.success return result.asset_observations_for_node(asset_def.node_def.name) -def _with_observe_metadata(kwargs: Dict[str, Any]) -> Dict[str, Any]: - metadata = kwargs.pop("metadata", {}) - metadata[SYSTEM_METADATA_KEY_ASSET_EXECUTION_TYPE] = AssetExecutionType.OBSERVATION.value - return {**kwargs, "metadata": metadata} - - -def _external_observable_asset(**kwargs) -> Callable[..., AssetsDefinition]: - def _decorator(fn: Callable[..., Any]) -> AssetsDefinition: - new_kwargs = _with_observe_metadata(kwargs) - return asset(**new_kwargs)(fn) - - return _decorator - - -def _external_observable_multi_asset(**kwargs) -> Callable[..., AssetsDefinition]: - def _decorator(fn: Callable[..., Any]) -> AssetsDefinition: - if "outs" in kwargs: - kwargs["outs"] = { - name: AssetOut(**_with_observe_metadata(out._asdict())) - for name, out in kwargs["outs"].items() - } - elif "specs" in kwargs: - kwargs["specs"] = [ - AssetSpec(**_with_observe_metadata(spec._asdict())) for spec in kwargs["specs"] - ] - return multi_asset(**kwargs)(fn) - - return _decorator - - def test_observe_result_asset(): - @_external_observable_asset() + @asset(_execution_type=AssetExecutionType.OBSERVATION) def ret_untyped(context: AssetExecutionContext): return ObserveResult( metadata={"one": 1}, @@ -77,7 +46,7 @@ def ret_untyped(context: AssetExecutionContext): assert "one" in observations[0].metadata # key mismatch - @_external_observable_asset() + @asset(_execution_type=AssetExecutionType.OBSERVATION) def ret_mismatch(context: AssetExecutionContext): return ObserveResult( asset_key="random", @@ -89,7 +58,7 @@ def ret_mismatch(context: AssetExecutionContext): DagsterInvariantViolationError, match="Asset key random not found in AssetsDefinition", ): - materialize([ret_mismatch]) + observe([ret_mismatch]) # direct invocation with pytest.raises( @@ -99,12 +68,12 @@ def ret_mismatch(context: AssetExecutionContext): ret_mismatch(build_asset_context()) # tuple - @_external_observable_asset() + @asset(_execution_type=AssetExecutionType.OBSERVATION) def ret_two(): return ObserveResult(metadata={"one": 1}), ObserveResult(metadata={"two": 2}) # core execution - result = materialize([ret_two]) + result = observe([ret_two]) assert result.success # direct invocation @@ -115,8 +84,9 @@ def ret_two(): def test_return_observe_result_with_asset_checks(): with instance_for_test() as instance: - @_external_observable_asset( - check_specs=[AssetCheckSpec(name="foo_check", asset=AssetKey("ret_checks"))] + @asset( + check_specs=[AssetCheckSpec(name="foo_check", asset=AssetKey("ret_checks"))], + _execution_type=AssetExecutionType.OBSERVATION, ) def ret_checks(context: AssetExecutionContext): return ObserveResult( @@ -126,7 +96,7 @@ def ret_checks(context: AssetExecutionContext): ) # core execution - materialize([ret_checks], instance=instance) + observe([ret_checks], instance=instance) asset_check_executions = instance.event_log_storage.get_asset_check_execution_history( AssetCheckKey(asset_key=ret_checks.key, name="foo_check"), limit=1, @@ -141,30 +111,33 @@ def ret_checks(context: AssetExecutionContext): def test_multi_asset_observe_result(): - @_external_observable_multi_asset(outs={"one": AssetOut(), "two": AssetOut()}) + @multi_asset( + outs={"one": AssetOut(), "two": AssetOut()}, _execution_type=AssetExecutionType.OBSERVATION + ) def outs_multi_asset(): return ObserveResult(asset_key="one", metadata=({"foo": "bar"})), ObserveResult( asset_key="two", metadata={"baz": "qux"} ) - assert materialize([outs_multi_asset]).success + assert observe([outs_multi_asset]).success res = outs_multi_asset() assert res[0].metadata["foo"] == "bar" assert res[1].metadata["baz"] == "qux" - @_external_observable_multi_asset( + @multi_asset( specs=[ AssetSpec(["prefix", "one"]), AssetSpec(["prefix", "two"]), - ] + ], + _execution_type=AssetExecutionType.OBSERVATION, ) def specs_multi_asset(): return ObserveResult(asset_key=["prefix", "one"], metadata={"foo": "bar"}), ObserveResult( asset_key=["prefix", "two"], metadata={"baz": "qux"} ) - assert materialize([specs_multi_asset]).success + assert observe([specs_multi_asset]).success res = specs_multi_asset() assert res[0].metadata["foo"] == "bar" @@ -175,7 +148,10 @@ def test_yield_materialization_multi_asset(): # # yield successful # - @_external_observable_multi_asset(outs={"one": AssetOut(), "two": AssetOut()}) + @multi_asset( + outs={"one": AssetOut(), "two": AssetOut()}, + _execution_type=AssetExecutionType.OBSERVATION, + ) def multi(): yield ObserveResult( asset_key="one", @@ -198,7 +174,10 @@ def multi(): # # missing a non optional out # - @_external_observable_multi_asset(outs={"one": AssetOut(), "two": AssetOut()}) + @multi_asset( + outs={"one": AssetOut(), "two": AssetOut()}, + _execution_type=AssetExecutionType.OBSERVATION, + ) def missing(): yield ObserveResult( asset_key="one", @@ -223,7 +202,10 @@ def missing(): # # missing asset_key # - @_external_observable_multi_asset(outs={"one": AssetOut(), "two": AssetOut()}) + @multi_asset( + outs={"one": AssetOut(), "two": AssetOut()}, + _execution_type=AssetExecutionType.OBSERVATION, + ) def no_key(): yield ObserveResult( metadata={"one": 1}, @@ -253,7 +235,10 @@ def no_key(): # # return tuple success # - @_external_observable_multi_asset(outs={"one": AssetOut(), "two": AssetOut()}) + @multi_asset( + outs={"one": AssetOut(), "two": AssetOut()}, + _execution_type=AssetExecutionType.OBSERVATION, + ) def ret_multi(): return ( ObserveResult( @@ -278,7 +263,10 @@ def ret_multi(): # # return list error # - @_external_observable_multi_asset(outs={"one": AssetOut(), "two": AssetOut()}) + @multi_asset( + outs={"one": AssetOut(), "two": AssetOut()}, + _execution_type=AssetExecutionType.OBSERVATION, + ) def ret_list(): return [ ObserveResult( @@ -323,44 +311,60 @@ def handle_output(self, context, obj): def load_input(self, context): return 1 - @_external_observable_asset() + @asset(_execution_type=AssetExecutionType.OBSERVATION) def asset_with_type_annotation() -> ObserveResult: return ObserveResult(metadata={"foo": "bar"}) - assert materialize( + assert observe( [asset_with_type_annotation], resources={"io_manager": TestingIOManager()} ).success - @_external_observable_multi_asset(outs={"one": AssetOut(), "two": AssetOut()}) + @multi_asset( + outs={"one": AssetOut(), "two": AssetOut()}, + _execution_type=AssetExecutionType.OBSERVATION, + ) def multi_asset_with_outs_and_type_annotation() -> Tuple[ObserveResult, ObserveResult]: return ObserveResult(asset_key="one"), ObserveResult(asset_key="two") - assert materialize( + assert observe( [multi_asset_with_outs_and_type_annotation], resources={"io_manager": TestingIOManager()} ).success - @_external_observable_multi_asset(specs=[AssetSpec("one"), AssetSpec("two")]) + @multi_asset( + specs=[ + AssetSpec(["one"]), + AssetSpec(["two"]), + ], + _execution_type=AssetExecutionType.OBSERVATION, + ) def multi_asset_with_specs_and_type_annotation() -> Tuple[ObserveResult, ObserveResult]: return ObserveResult(asset_key="one"), ObserveResult(asset_key="two") - assert materialize( + assert observe( [multi_asset_with_specs_and_type_annotation], resources={"io_manager": TestingIOManager()} ).success - @_external_observable_multi_asset(specs=[AssetSpec("one"), AssetSpec("two")]) + @multi_asset( + specs=[ + AssetSpec(["one"]), + AssetSpec(["two"]), + ], + _execution_type=AssetExecutionType.OBSERVATION, + ) def multi_asset_with_specs_and_no_type_annotation(): return ObserveResult(asset_key="one"), ObserveResult(asset_key="two") - assert materialize( + assert observe( [multi_asset_with_specs_and_no_type_annotation], resources={"io_manager": TestingIOManager()}, ).success - @_external_observable_asset( + @asset( check_specs=[ AssetCheckSpec(name="check_one", asset="with_checks"), AssetCheckSpec(name="check_two", asset="with_checks"), - ] + ], + _execution_type=AssetExecutionType.OBSERVATION, ) def with_checks(context: AssetExecutionContext) -> ObserveResult: return ObserveResult( @@ -381,7 +385,7 @@ def with_checks(context: AssetExecutionContext) -> ObserveResult: resources={"io_manager": TestingIOManager()}, ).success - @_external_observable_multi_asset( + @multi_asset( specs=[ AssetSpec("asset_one"), AssetSpec("asset_two"), @@ -390,6 +394,7 @@ def with_checks(context: AssetExecutionContext) -> ObserveResult: AssetCheckSpec(name="check_one", asset="asset_one"), AssetCheckSpec(name="check_two", asset="asset_two"), ], + _execution_type=AssetExecutionType.OBSERVATION, ) def multi_checks(context: AssetExecutionContext) -> Tuple[ObserveResult, ObserveResult]: return ObserveResult( @@ -439,7 +444,7 @@ def generator_asset() -> Generator[ObserveResult, None, None]: def test_observe_result_generators(): - @_external_observable_asset() + @asset(_execution_type=AssetExecutionType.OBSERVATION) def generator_asset() -> Generator[ObserveResult, None, None]: yield ObserveResult(metadata={"foo": "bar"}) @@ -451,7 +456,10 @@ def generator_asset() -> Generator[ObserveResult, None, None]: assert len(res) == 1 assert res[0].metadata["foo"] == "bar" - @_external_observable_multi_asset(specs=[AssetSpec("one"), AssetSpec("two")]) + @multi_asset( + specs=[AssetSpec("one"), AssetSpec("two")], + _execution_type=AssetExecutionType.OBSERVATION, + ) def generator_specs_multi_asset(): yield ObserveResult(asset_key="one", metadata={"foo": "bar"}) yield ObserveResult(asset_key="two", metadata={"baz": "qux"}) @@ -466,7 +474,10 @@ def generator_specs_multi_asset(): assert res[0].metadata["foo"] == "bar" assert res[1].metadata["baz"] == "qux" - @_external_observable_multi_asset(outs={"one": AssetOut(), "two": AssetOut()}) + @multi_asset( + outs={"one": AssetOut(), "two": AssetOut()}, + _execution_type=AssetExecutionType.OBSERVATION, + ) def generator_outs_multi_asset(): yield ObserveResult(asset_key="one", metadata={"foo": "bar"}) yield ObserveResult(asset_key="two", metadata={"baz": "qux"}) @@ -481,7 +492,10 @@ def generator_outs_multi_asset(): assert res[0].metadata["foo"] == "bar" assert res[1].metadata["baz"] == "qux" - @_external_observable_multi_asset(specs=[AssetSpec("one"), AssetSpec("two")]) + @multi_asset( + specs=[AssetSpec("one"), AssetSpec("two")], + _execution_type=AssetExecutionType.OBSERVATION, + ) async def async_specs_multi_asset(): return ObserveResult(asset_key="one", metadata={"foo": "bar"}), ObserveResult( asset_key="two", metadata={"baz": "qux"} @@ -497,7 +511,10 @@ async def async_specs_multi_asset(): assert res[0].metadata["foo"] == "bar" assert res[1].metadata["baz"] == "qux" - @_external_observable_multi_asset(specs=[AssetSpec("one"), AssetSpec("two")]) + @multi_asset( + specs=[AssetSpec("one"), AssetSpec("two")], + _execution_type=AssetExecutionType.OBSERVATION, + ) async def async_gen_specs_multi_asset(): yield ObserveResult(asset_key="one", metadata={"foo": "bar"}) yield ObserveResult(asset_key="two", metadata={"baz": "qux"}) @@ -520,8 +537,9 @@ async def _run_async_gen(): def test_observe_result_with_partitions(): - @_external_observable_asset( - partitions_def=StaticPartitionsDefinition(["red", "blue", "yellow"]) + @asset( + partitions_def=StaticPartitionsDefinition(["red", "blue", "yellow"]), + _execution_type=AssetExecutionType.OBSERVATION, ) def partitioned_asset(context: AssetExecutionContext) -> ObserveResult: return ObserveResult(metadata={"key": context.partition_key}) @@ -532,8 +550,9 @@ def partitioned_asset(context: AssetExecutionContext) -> ObserveResult: def test_observe_result_with_partitions_direct_invocation(): - @_external_observable_asset( - partitions_def=StaticPartitionsDefinition(["red", "blue", "yellow"]) + @asset( + partitions_def=StaticPartitionsDefinition(["red", "blue", "yellow"]), + _execution_type=AssetExecutionType.OBSERVATION, ) def partitioned_asset(context: AssetExecutionContext) -> ObserveResult: return ObserveResult(metadata={"key": context.partition_key}) diff --git a/python_modules/dagster/dagster_tests/definitions_tests/test_repository_definition.py b/python_modules/dagster/dagster_tests/definitions_tests/test_repository_definition.py index 0094bad5a0f75..70448bedb5b4d 100644 --- a/python_modules/dagster/dagster_tests/definitions_tests/test_repository_definition.py +++ b/python_modules/dagster/dagster_tests/definitions_tests/test_repository_definition.py @@ -7,7 +7,6 @@ AssetsDefinition, DagsterInvalidDefinitionError, DailyPartitionsDefinition, - Definitions, GraphDefinition, IOManager, JobDefinition, @@ -677,7 +676,7 @@ def my_repo(): return [foo, asset1, asset2] assert len(my_repo.get_all_jobs()) == 1 - assert set(my_repo.get_all_jobs()[0].asset_layer.asset_keys) == { + assert set(my_repo.get_all_jobs()[0].asset_layer.target_asset_keys) == { AssetKey(["asset1"]), AssetKey(["asset2"]), } @@ -1181,7 +1180,7 @@ def assets_repo(): return [all_assets] assert len(assets_repo.get_all_jobs()) == 1 - assert set(assets_repo.get_all_jobs()[0].asset_layer.asset_keys) == { + assert set(assets_repo.get_all_jobs()[0].asset_layer.target_asset_keys) == { AssetKey(["asset1"]), AssetKey(["asset2"]), } @@ -1411,11 +1410,10 @@ def repo(): assert sorted(repo.get_implicit_asset_job_names()) == ["__ASSET_JOB_0", "__ASSET_JOB_1"] assert repo.get_implicit_job_def_for_assets( [asset1.key, asset2.key] - ).asset_layer.asset_keys == { + ).asset_layer.target_asset_keys == { asset1.key, asset2.key, } - assert repo.get_implicit_job_def_for_assets([asset2.key, asset3.key]) is None def test_automation_policy_sensors_do_not_conflict(): @@ -1478,25 +1476,3 @@ def repo(): AutomationPolicySensorDefinition("a", asset_selection=[asset1]), AutomationPolicySensorDefinition("b", asset_selection=[asset1, asset2]), ] - - -def test_invalid_asset_selection(): - source_asset = SourceAsset("source_asset") - - @asset - def asset1(): - ... - - @sensor(asset_selection=[source_asset, asset1]) - def sensor1(): - ... - - Definitions(assets=[source_asset, asset1], sensors=[sensor1]) - - with pytest.raises( - DagsterInvalidDefinitionError, match="specified both regular assets and source" - ): - Definitions( - assets=[source_asset, asset1], - jobs=[define_asset_job("foo", selection=[source_asset, asset1])], - ).get_all_job_defs() diff --git a/python_modules/dagster/dagster_tests/storage_tests/test_asset_value_loader.py b/python_modules/dagster/dagster_tests/storage_tests/test_asset_value_loader.py index d37008b77b708..607c78a9a66e2 100644 --- a/python_modules/dagster/dagster_tests/storage_tests/test_asset_value_loader.py +++ b/python_modules/dagster/dagster_tests/storage_tests/test_asset_value_loader.py @@ -77,7 +77,7 @@ def load_input(self, context): assert context.asset_key == AssetKey("asset1") assert context.upstream_output.asset_key == AssetKey("asset1") assert context.upstream_output.metadata["a"] == "b" - assert context.upstream_output.name == "asset1" + assert context.upstream_output.name == "result" assert context.dagster_type.typing_type == int return 5 diff --git a/python_modules/dagster/dagster_tests/storage_tests/test_fs_io_manager.py b/python_modules/dagster/dagster_tests/storage_tests/test_fs_io_manager.py index 5e17cfb25c2f7..5a2be8a3a3d45 100644 --- a/python_modules/dagster/dagster_tests/storage_tests/test_fs_io_manager.py +++ b/python_modules/dagster/dagster_tests/storage_tests/test_fs_io_manager.py @@ -224,7 +224,7 @@ def asset2(asset1): return asset1 + [4] return build_assets_job( - name="a", assets=[asset1, asset2], resource_defs={"io_manager": io_manager_def} + name="a", assets_to_execute=[asset1, asset2], resource_defs={"io_manager": io_manager_def} ) @@ -399,7 +399,7 @@ def four(inp): job_def = build_assets_job( name="a", - assets=[one, four_asset], + assets_to_execute=[one, four_asset], resource_defs={"io_manager": io_manager_def}, ) diff --git a/python_modules/dagster/dagster_tests/storage_tests/test_upath_io_manager.py b/python_modules/dagster/dagster_tests/storage_tests/test_upath_io_manager.py index a42fd94df00e7..ca66fc6853b77 100644 --- a/python_modules/dagster/dagster_tests/storage_tests/test_upath_io_manager.py +++ b/python_modules/dagster/dagster_tests/storage_tests/test_upath_io_manager.py @@ -223,7 +223,7 @@ def downstream_asset(upstream_asset: Dict[str, str]) -> Dict[str, str]: my_job = build_assets_job( "my_job", - assets=[upstream_asset, downstream_asset], + assets_to_execute=[upstream_asset, downstream_asset], resource_defs={"io_manager": dummy_io_manager}, ) result = my_job.execute_in_process(partition_key="A") @@ -249,7 +249,7 @@ def downstream_asset(upstream_asset): my_job = build_assets_job( "my_job", - assets=[upstream_asset, downstream_asset], + assets_to_execute=[upstream_asset, downstream_asset], resource_defs={"io_manager": dummy_io_manager}, ) result = my_job.execute_in_process(partition_key=MultiPartitionKey({"a": "a", "1": "1"})) @@ -312,7 +312,7 @@ def my_asset(context: AssetExecutionContext) -> str: my_job = build_assets_job( "my_job", - assets=[my_asset], + assets_to_execute=[my_asset], resource_defs={"io_manager": tracking_io_manager}, ) my_job.execute_in_process(partition_key="0.0-to-1.0") @@ -350,7 +350,7 @@ def my_asset(context: AssetExecutionContext) -> str: my_job = build_assets_job( "my_job", - assets=[my_asset], + assets_to_execute=[my_asset], resource_defs={"io_manager": tracking_io_manager}, ) my_job.execute_in_process(partition_key="0.0-to-1.0") diff --git a/python_modules/libraries/dagster-airbyte/dagster_airbyte_tests/test_asset_defs.py b/python_modules/libraries/dagster-airbyte/dagster_airbyte_tests/test_asset_defs.py index 87c8e41e7f107..e709ee6c3b399 100644 --- a/python_modules/libraries/dagster-airbyte/dagster_airbyte_tests/test_asset_defs.py +++ b/python_modules/libraries/dagster-airbyte/dagster_airbyte_tests/test_asset_defs.py @@ -157,7 +157,7 @@ def test_assets_with_normalization(schema_prefix, source_asset, freshness_policy ab_job = build_assets_job( "ab_job", ab_assets, - source_assets=[SourceAsset(AssetKey(source_asset))] if source_asset else None, + other_assets=[SourceAsset(AssetKey(source_asset))] if source_asset else None, resource_defs={ "airbyte": airbyte_resource.configured( { diff --git a/python_modules/libraries/dagster-aws/dagster_aws_tests/s3_tests/test_io_manager.py b/python_modules/libraries/dagster-aws/dagster_aws_tests/s3_tests/test_io_manager.py index 0a6bf8eaa5e5c..28d6c4bcdf852 100644 --- a/python_modules/libraries/dagster-aws/dagster_aws_tests/s3_tests/test_io_manager.py +++ b/python_modules/libraries/dagster-aws/dagster_aws_tests/s3_tests/test_io_manager.py @@ -185,8 +185,8 @@ def partitioned(): return build_assets_job( name="assets", - assets=[asset1, asset2, AssetsDefinition.from_graph(graph_asset), partitioned], - source_assets=[source1], + assets_to_execute=[asset1, asset2, AssetsDefinition.from_graph(graph_asset), partitioned], + other_assets=[source1], resource_defs={ "io_manager": s3_pickle_io_manager.configured({"s3_bucket": bucket}), "s3": s3_test_resource, diff --git a/python_modules/libraries/dagster-fivetran/dagster_fivetran_tests/test_asset_defs.py b/python_modules/libraries/dagster-fivetran/dagster_fivetran_tests/test_asset_defs.py index 78703eab85e66..061dec387e878 100644 --- a/python_modules/libraries/dagster-fivetran/dagster_fivetran_tests/test_asset_defs.py +++ b/python_modules/libraries/dagster-fivetran/dagster_fivetran_tests/test_asset_defs.py @@ -82,7 +82,7 @@ def test_fivetran_asset_run(tables, infer_missing_tables, should_error, schema_p fivetran_assets_job = build_assets_job( name="fivetran_assets_job", - assets=fivetran_assets, + assets_to_execute=fivetran_assets, resource_defs={"fivetran": ft_resource}, ) diff --git a/python_modules/libraries/dagster-fivetran/dagster_fivetran_tests/test_load_from_instance.py b/python_modules/libraries/dagster-fivetran/dagster_fivetran_tests/test_load_from_instance.py index 82bfeb938f1f0..a73edf4b6b7b8 100644 --- a/python_modules/libraries/dagster-fivetran/dagster_fivetran_tests/test_load_from_instance.py +++ b/python_modules/libraries/dagster-fivetran/dagster_fivetran_tests/test_load_from_instance.py @@ -199,7 +199,7 @@ def downstream_asset(xyz): final_data = {"succeeded_at": "2021-01-01T02:00:00.0Z"} fivetran_sync_job = build_assets_job( name="fivetran_assets_job", - assets=all_assets, + assets_to_execute=all_assets, ) with responses.RequestsMock() as rsps: