Assets
An asset is an object in persistent storage, such as a table, file, or persisted machine learning model. An asset definition is a description, in code, of an asset that should exist and how to produce and update that asset.
Asset definitions
Refer to the Asset definitions documentation for more information.
- @dagster.asset [source]
- Create a definition for how to compute an asset. - A software-defined asset is the combination of: - An asset key, e.g. the name of a table.
- A function, which can be run to compute the contents of the asset.
- A set of upstream assets that are provided as inputs to the function when computing the asset. Unlike an op, whose dependencies are determined by the graph it lives inside, an asset knows about the upstream assets it depends on. The upstream assets are inferred from the arguments to the decorated function. The name of the argument designates the name of the upstream asset.
 - An asset has an op inside it to represent the function that computes it. The name of the op will be the segments of the asset key, separated by double-underscores. - Parameters: - name (Optional[str]) – The name of the asset. If not provided, defaults to the name of the decorated function. The asset’s name must be a valid name in dagster (ie only contains letters, numbers, and _) and may not contain python reserved keywords.
- key_prefix (Optional[Union[str, Sequence[str]]]) – If provided, the asset’s key is the concatenation of the key_prefix and the asset’s name, which defaults to the name of the decorated function. Each item in key_prefix must be a valid name in dagster (ie only contains letters, numbers, and _) and may not contain python reserved keywords.
- ins (Optional[Mapping[str, AssetIn]]) – A dictionary that maps input names to information about the input.
- deps (Optional[Sequence[Union[AssetDep, AssetsDefinition, SourceAsset, AssetKey, str]]]) – The assets that are upstream dependencies, but do not correspond to a parameter of the decorated function. If the AssetsDefinition for a multi_asset is provided, dependencies on all assets created by the multi_asset will be created.
- config_schema (Optional[ConfigSchema) – The configuration schema for the asset’s underlying op. If set, Dagster will check that config provided for the op matches this schema and fail if it does not. If not set, Dagster will accept any config provided for the op.
- metadata (Optional[Dict[str, Any]]) – A dict of metadata entries for the asset.
- tags (Optional[Mapping[str, str]]) – Tags for filtering and organizing. These tags are not attached to runs of the asset.
- required_resource_keys (Optional[Set[str]]) – Set of resource handles required by the op.
- io_manager_key (Optional[str]) – The resource key of the IOManager used for storing the output of the op as an asset, and for loading it in downstream ops (default: “io_manager”). Only one of io_manager_key and io_manager_def can be provided.
- io_manager_def (Optional[object]) – beta (Beta) The IOManager used for storing the output of the op as an asset, and for loading it in downstream ops. Only one of io_manager_def and io_manager_key can be provided.
- dagster_type (Optional[DagsterType]) – Allows specifying type validation functions that will be executed on the output of the decorated function after it runs.
- partitions_def (Optional[PartitionsDefinition]) – Defines the set of partition keys that compose the asset.
- op_tags (Optional[Dict[str, Any]]) – A dictionary of tags for the op that computes the asset. Frameworks may expect and require certain metadata to be attached to a op. Values that are not strings will be json encoded and must meet the criteria that json.loads(json.dumps(value)) == value.
- group_name (Optional[str]) – A string name used to organize multiple assets into groups. If not provided, the name “default” is used.
- resource_defs (Optional[Mapping[str, object]]) – beta (Beta) A mapping of resource keys to resources. These resources will be initialized during execution, and can be accessed from the context within the body of the function.
- hooks (Optional[AbstractSet[HookDefinition]]) – A set of hooks to attach to the asset. These hooks will be executed when the asset is materialized.
- output_required (bool) – Whether the decorated function will always materialize an asset. Defaults to True. If False, the function can conditionally not yield a result. If no result is yielded, no output will be materialized to storage and downstream assets will not be materialized. Note that for output_required to work at all, you must use yield in your asset logic rather than return. return will not respect this setting and will always produce an asset materialization, even if None is returned.
- automation_condition (AutomationCondition) – A condition describing when Dagster should materialize this asset.
- backfill_policy (BackfillPolicy) – beta (Beta) Configure Dagster to backfill this asset according to its BackfillPolicy.
- retry_policy (Optional[RetryPolicy]) – The retry policy for the op that computes the asset.
- code_version (Optional[str]) – Version of the code that generates this asset. In general, versions should be set only for code that deterministically produces the same output when given the same inputs.
- check_specs (Optional[Sequence[AssetCheckSpec]]) – Specs for asset checks that execute in the decorated function after materializing the asset.
- key (Optional[CoeercibleToAssetKey]) – The key for this asset. If provided, cannot specify key_prefix or name.
- owners (Optional[Sequence[str]]) – A list of strings representing owners of the asset. Each string can be a user’s email address, or a team name prefixed with team:, e.g. team:finops.
- kinds (Optional[Set[str]]) – A list of strings representing the kinds of the asset. These will be made visible in the Dagster UI.
- pool (Optional[str]) – A string that identifies the concurrency pool that governs this asset’s execution.
- non_argument_deps (Optional[Union[Set[AssetKey], Set[str]]]) – deprecated Deprecated, use deps instead. Set of asset keys that are upstream dependencies, but do not pass an input to the asset. Hidden parameter not exposed in the decorator signature, but passed in kwargs.
 - Examples: - @asset
 def my_upstream_asset() -> int:
 return 5
 @asset
 def my_asset(my_upstream_asset: int) -> int:
 return my_upstream_asset + 1
 should_materialize = True
 @asset(output_required=False)
 def conditional_asset():
 if should_materialize:
 yield Output(5) # you must `yield`, not `return`, the result
 # Will also only materialize if `should_materialize` is `True`
 @asset
 def downstream_asset(conditional_asset):
 return conditional_asset + 1
- classdagster.MaterializeResult [source]
- An object representing a successful materialization of an asset. These can be returned from @asset and @multi_asset decorated functions to pass metadata or specify specific assets were materialized. - Parameters: - asset_key (Optional[AssetKey]) – Optional in @asset, required in @multi_asset to discern which asset this refers to.
- metadata (Optional[RawMetadataMapping]) – Metadata to record with the corresponding AssetMaterialization event.
- check_results (Optional[Sequence[AssetCheckResult]]) – Check results to record with the corresponding AssetMaterialization event.
- data_version (Optional[DataVersion]) – The data version of the asset that was observed.
- tags (Optional[Mapping[str, str]]) – Tags to record with the corresponding AssetMaterialization event.
- value (Optional[Any]) – The output value of the asset that was materialized.
 
- classdagster.AssetSpec [source]
- Specifies the core attributes of an asset, except for the function that materializes or observes it. - An asset spec plus any materialization or observation function for the asset constitutes an “asset definition”. - Parameters: - key (AssetKey) – The unique identifier for this asset.
- deps (Optional[AbstractSet[AssetKey]]) – The asset keys for the upstream assets that materializing this asset depends on.
- description (Optional[str]) – Human-readable description of this asset.
- metadata (Optional[Dict[str, Any]]) – A dict of static metadata for this asset. For example, users can provide information about the database table this asset corresponds to.
- skippable (bool) – Whether this asset can be omitted during materialization, causing downstream dependencies to skip.
- group_name (Optional[str]) – A string name used to organize multiple assets into groups. If not provided, the name “default” is used.
- code_version (Optional[str]) – The version of the code for this specific asset, overriding the code version of the materialization function
- backfill_policy (Optional[BackfillPolicy]) – BackfillPolicy to apply to the specified asset.
- owners (Optional[Sequence[str]]) – A list of strings representing owners of the asset. Each string can be a user’s email address, or a team name prefixed with team:, e.g. team:finops.
- automation_condition (Optional[AutomationCondition]) – The automation condition to apply to the asset.
- tags (Optional[Mapping[str, str]]) – Tags for filtering and organizing. These tags are not attached to runs of the asset.
- kinds – (Optional[Set[str]]): A set of strings representing the kinds of the asset. These will be made visible in the Dagster UI.
- partitions_def (Optional[PartitionsDefinition]) – Defines the set of partition keys that compose the asset.
 - merge_attributes [source]
- Returns a new AssetSpec with the specified attributes merged with the current attributes. - Parameters: - deps (Optional[Iterable[CoercibleToAssetDep]]) – A set of asset dependencies to add to the asset self.
- metadata (Optional[Mapping[str, Any]]) – A set of metadata to add to the asset self. Will overwrite any existing metadata with the same key.
- owners (Optional[Sequence[str]]) – A set of owners to add to the asset self.
- tags (Optional[Mapping[str, str]]) – A set of tags to add to the asset self. Will overwrite any existing tags with the same key.
- kinds (Optional[Set[str]]) – A set of kinds to add to the asset self.
 - Returns: AssetSpec 
 - replace_attributes [source]
- Returns a new AssetSpec with the specified attributes replaced. 
 - with_io_manager_key [source]
- Returns a copy of this AssetSpec with an extra metadata value that dictates which I/O manager to use to load the contents of this asset in downstream computations. - Parameters: io_manager_key (str) – The I/O manager key. This will be used as the value for the “dagster/io_manager_key” metadata key.Returns: AssetSpec 
 
- classdagster.AssetsDefinition [source]
- Defines a set of assets that are produced by the same op or graph. - AssetsDefinitions are typically not instantiated directly, but rather produced using the - @assetor- @multi_assetdecorators.- staticfrom_graph [source]
- Constructs an AssetsDefinition from a GraphDefinition. - Parameters: - graph_def (GraphDefinition) – The GraphDefinition that is an asset.
- keys_by_input_name (Optional[Mapping[str, AssetKey]]) – A mapping of the input names of the decorated graph to their corresponding asset keys. If not provided, the input asset keys will be created from the graph input names.
- keys_by_output_name (Optional[Mapping[str, AssetKey]]) – A mapping of the output names of the decorated graph to their corresponding asset keys. If not provided, the output asset keys will be created from the graph output names.
- key_prefix (Optional[Union[str, Sequence[str]]]) – If provided, key_prefix will be prepended to each key in keys_by_output_name. Each item in key_prefix must be a valid name in dagster (ie only contains letters, numbers, and _) and may not contain python reserved keywords.
- internal_asset_deps (Optional[Mapping[str, Set[AssetKey]]]) – By default, it is assumed that all assets produced by the graph depend on all assets that are consumed by that graph. If this default is not correct, you pass in a map of output names to a corrected set of AssetKeys that they depend on. Any AssetKeys in this list must be either used as input to the asset or produced within the graph.
- partitions_def (Optional[PartitionsDefinition]) – Defines the set of partition keys that compose the assets.
- partition_mappings (Optional[Mapping[str, PartitionMapping]]) – Defines how to map partition keys for this asset to partition keys of upstream assets. Each key in the dictionary correponds to one of the input assets, and each value is a PartitionMapping. If no entry is provided for a particular asset dependency, the partition mapping defaults to the default partition mapping for the partitions definition, which is typically maps partition keys to the same partition keys in upstream assets.
- resource_defs (Optional[Mapping[str, ResourceDefinition]]) – beta (Beta) A mapping of resource keys to resource definitions. These resources will be initialized during execution, and can be accessed from the body of ops in the graph during execution.
- group_name (Optional[str]) – A group name for the constructed asset. Assets without a group name are assigned to a group called “default”.
- group_names_by_output_name (Optional[Mapping[str, Optional[str]]]) – Defines a group name to be associated with some or all of the output assets for this node. Keys are names of the outputs, and values are the group name. Cannot be used with the group_name argument.
- descriptions_by_output_name (Optional[Mapping[str, Optional[str]]]) – Defines a description to be associated with each of the output asstes for this graph.
- metadata_by_output_name (Optional[Mapping[str, Optional[RawMetadataMapping]]]) – Defines metadata to be associated with each of the output assets for this node. Keys are names of the outputs, and values are dictionaries of metadata to be associated with the related asset.
- tags_by_output_name (Optional[Mapping[str, Optional[Mapping[str, str]]]]) – Defines tags to be associated with each of the output assets for this node. Keys are the names of outputs, and values are dictionaries of tags to be associated with the related asset.
- legacy_freshness_policies_by_output_name (Optional[Mapping[str, Optional[FreshnessPolicy]]]) – deprecated Defines a FreshnessPolicy to be associated with some or all of the output assets for this node. Keys are the names of the outputs, and values are the FreshnessPolicies to be attached to the associated asset.
- automation_conditions_by_output_name (Optional[Mapping[str, Optional[AutomationCondition]]]) – Defines an AutomationCondition to be associated with some or all of the output assets for this node. Keys are the names of the outputs, and values are the AutoMaterializePolicies to be attached to the associated asset.
- backfill_policy (Optional[BackfillPolicy]) – Defines this asset’s BackfillPolicy
- owners_by_key (Optional[Mapping[AssetKey, Sequence[str]]]) – Defines owners to be associated with each of the asset keys for this node.
 
 - staticfrom_op [source]
- Constructs an AssetsDefinition from an OpDefinition. - Parameters: - op_def (OpDefinition) – The OpDefinition that is an asset.
- keys_by_input_name (Optional[Mapping[str, AssetKey]]) – A mapping of the input names of the decorated op to their corresponding asset keys. If not provided, the input asset keys will be created from the op input names.
- keys_by_output_name (Optional[Mapping[str, AssetKey]]) – A mapping of the output names of the decorated op to their corresponding asset keys. If not provided, the output asset keys will be created from the op output names.
- key_prefix (Optional[Union[str, Sequence[str]]]) – If provided, key_prefix will be prepended to each key in keys_by_output_name. Each item in key_prefix must be a valid name in dagster (ie only contains letters, numbers, and _) and may not contain python reserved keywords.
- internal_asset_deps (Optional[Mapping[str, Set[AssetKey]]]) – By default, it is assumed that all assets produced by the op depend on all assets that are consumed by that op. If this default is not correct, you pass in a map of output names to a corrected set of AssetKeys that they depend on. Any AssetKeys in this list must be either used as input to the asset or produced within the op.
- partitions_def (Optional[PartitionsDefinition]) – Defines the set of partition keys that compose the assets.
- partition_mappings (Optional[Mapping[str, PartitionMapping]]) – Defines how to map partition keys for this asset to partition keys of upstream assets. Each key in the dictionary correponds to one of the input assets, and each value is a PartitionMapping. If no entry is provided for a particular asset dependency, the partition mapping defaults to the default partition mapping for the partitions definition, which is typically maps partition keys to the same partition keys in upstream assets.
- group_name (Optional[str]) – A group name for the constructed asset. Assets without a group name are assigned to a group called “default”.
- group_names_by_output_name (Optional[Mapping[str, Optional[str]]]) – Defines a group name to be associated with some or all of the output assets for this node. Keys are names of the outputs, and values are the group name. Cannot be used with the group_name argument.
- descriptions_by_output_name (Optional[Mapping[str, Optional[str]]]) – Defines a description to be associated with each of the output asstes for this graph.
- metadata_by_output_name (Optional[Mapping[str, Optional[RawMetadataMapping]]]) – Defines metadata to be associated with each of the output assets for this node. Keys are names of the outputs, and values are dictionaries of metadata to be associated with the related asset.
- tags_by_output_name (Optional[Mapping[str, Optional[Mapping[str, str]]]]) – Defines tags to be associated with each othe output assets for this node. Keys are the names of outputs, and values are dictionaries of tags to be associated with the related asset.
- legacy_freshness_policies_by_output_name (Optional[Mapping[str, Optional[LegacyFreshnessPolicy]]]) – deprecated Defines a LegacyFreshnessPolicy to be associated with some or all of the output assets for this node. Keys are the names of the outputs, and values are the LegacyFreshnessPolicies to be attached to the associated asset.
- automation_conditions_by_output_name (Optional[Mapping[str, Optional[AutomationCondition]]]) – Defines an AutomationCondition to be associated with some or all of the output assets for this node. Keys are the names of the outputs, and values are the AutoMaterializePolicies to be attached to the associated asset.
- backfill_policy (Optional[BackfillPolicy]) – Defines this asset’s BackfillPolicy
 
 - get_asset_spec [source]
- Returns a representation of this asset as an - AssetSpec.- If this is a multi-asset, the “key” argument allows selecting which asset to return the spec for. - Parameters: key (Optional[AssetKey]) – If this is a multi-asset, select which asset to return its AssetSpec. If not a multi-asset, this can be left as None.Returns: AssetSpec 
 - get_partition_mapping [source]
- Returns the partition mapping between keys in this AssetsDefinition and a given input asset key (if any). 
 - to_source_asset [source]
- Returns a representation of this asset as a - SourceAsset.- If this is a multi-asset, the “key” argument allows selecting which asset to return a SourceAsset representation of. - Parameters: key (Optional[Union[str, Sequence[str], AssetKey]]]) – If this is a multi-asset, select which asset to return a SourceAsset representation of. If not a multi-asset, this can be left as None.Returns: SourceAsset 
 - to_source_assets [source]
- Returns a SourceAsset for each asset in this definition. - Each produced SourceAsset will have the same key, metadata, io_manager_key, etc. as the corresponding asset 
 - with_hooks [source]
- Apply a set of hooks to all op instances within the asset. 
 - propertyasset_deps [source]
- Maps assets that are produced by this definition to assets that they depend on. The dependencies can be either “internal”, meaning that they refer to other assets that are produced by this definition, or “external”, meaning that they refer to assets that aren’t produced by this definition. 
 - propertycan_subset [source]
- If True, indicates that this AssetsDefinition may materialize any subset of its asset keys in a given computation (as opposed to being required to materialize all asset keys). - Type: bool 
 - propertycheck_specs [source]
- Returns the asset check specs defined on this AssetsDefinition, i.e. the checks that can be executed while materializing the assets. - Return type: Iterable[AssetsCheckSpec] 
 - propertydependency_keys [source]
- The asset keys which are upstream of any asset included in this AssetsDefinition. - Type: Iterable[AssetKey] 
 - propertydescriptions_by_key [source]
- Returns a mapping from the asset keys in this AssetsDefinition to the descriptions assigned to them. If there is no assigned description for a given AssetKey, it will not be present in this dictionary. - Type: Mapping[AssetKey, str] 
 - propertygroup_names_by_key [source]
- Returns a mapping from the asset keys in this AssetsDefinition to the group names assigned to them. If there is no assigned group name for a given AssetKey, it will not be present in this dictionary. - Type: Mapping[AssetKey, str] 
 - propertykey [source]
- The asset key associated with this AssetsDefinition. If this AssetsDefinition has more than one asset key, this will produce an error. - Type: AssetKey 
 - propertykeys [source]
- The asset keys associated with this AssetsDefinition. - Type: AbstractSet[AssetKey] 
 - propertynode_def [source]
- Returns the OpDefinition or GraphDefinition that is used to materialize the assets in this AssetsDefinition. - Type: NodeDefinition 
 - propertyop [source]
- Returns the OpDefinition that is used to materialize the assets in this AssetsDefinition. - Type: OpDefinition 
 - propertypartitions_def [source]
- The PartitionsDefinition for this AssetsDefinition (if any). - Type: Optional[PartitionsDefinition] 
 - propertyrequired_resource_keys [source]
- The set of keys for resources that must be provided to this AssetsDefinition. - Type: Set[str] 
 - propertyresource_defs [source]
- A mapping from resource name to ResourceDefinition for the resources bound to this AssetsDefinition. - Type: Mapping[str, ResourceDefinition] 
 
- classdagster.AssetKey [source]
- Object representing the structure of an asset key. Takes in a sanitized string, list of strings, or tuple of strings. - Example usage: - from dagster import AssetKey
 AssetKey("asset1")
 AssetKey(["asset1"]) # same as the above
 AssetKey(["prefix", "asset1"])
 AssetKey(["prefix", "subprefix", "asset1"])- Parameters: path (Union[str, Sequence[str]]) – String, list of strings, or tuple of strings. A list of strings represent the hierarchical structure of the asset_key. 
- dagster.map_asset_specs [source]
- Map a function over a sequence of AssetSpecs or AssetsDefinitions, replacing specs in the sequence or specs in an AssetsDefinitions with the result of the function. - Parameters: - func (Callable[[AssetSpec], AssetSpec]) – The function to apply to each AssetSpec.
- iterable (Iterable[Union[AssetsDefinition, AssetSpec]]) – The sequence of AssetSpecs or AssetsDefinitions.
 - Returns: A sequence of AssetSpecs or AssetsDefinitions with the function applied to each spec. - Return type: Sequence[Union[AssetsDefinition, AssetSpec]] Examples: - from dagster import AssetSpec, map_asset_specs
 asset_specs = [
 AssetSpec(key="my_asset"),
 AssetSpec(key="my_asset_2"),
 ]
 mapped_specs = map_asset_specs(lambda spec: spec.replace_attributes(owners=["nelson@hooli.com"]), asset_specs)
Graph-backed asset definitions
Refer to the Graph-backed asset documentation for more information.
- @dagster.graph_asset [source]
- Creates a software-defined asset that’s computed using a graph of ops. - This decorator is meant to decorate a function that composes a set of ops or graphs to define the dependencies between them. - Parameters: - 
name (Optional[str]) – The name of the asset. If not provided, defaults to the name of the decorated function. The asset’s name must be a valid name in Dagster (ie only contains letters, numbers, and underscores) and may not contain Python reserved keywords. 
- 
description (Optional[str]) – A human-readable description of the asset. 
- 
ins (Optional[Mapping[str, AssetIn]]) – A dictionary that maps input names to information about the input. 
- 
config (Optional[Union[ConfigMapping], Mapping[str, Any]) – Describes how the graph underlying the asset is configured at runtime. If a ConfigMappingobject is provided, then the graph takes on the config schema of this object. The mapping will be applied at runtime to generate the config for the graph’s constituent nodes.If a dictionary is provided, then it will be used as the default run config for the graph. This means it must conform to the config schema of the underlying nodes. Note that the values provided will be viewable and editable in the Dagster UI, so be careful with secrets. 
- 
key_prefix (Optional[Union[str, Sequence[str]]]) – If provided, the asset’s key is the concatenation of the key_prefix and the asset’s name, which defaults to the name of the decorated function. Each item in key_prefix must be a valid name in Dagster (ie only contains letters, numbers, and underscores) and may not contain Python reserved keywords. 
- 
group_name (Optional[str]) – A string name used to organize multiple assets into groups. If not provided, the name “default” is used. 
- 
partitions_def (Optional[PartitionsDefinition]) – Defines the set of partition keys that compose the asset. 
- 
hooks (Optional[AbstractSet[HookDefinition]]) – A set of hooks to attach to the asset. These hooks will be executed when the asset is materialized. 
- 
metadata (Optional[RawMetadataMapping]) – Dictionary of metadata to be associated with the asset. 
- 
tags (Optional[Mapping[str, str]]) – Tags for filtering and organizing. These tags are not attached to runs of the asset. 
- 
owners (Optional[Sequence[str]]) – A list of strings representing owners of the asset. Each string can be a user’s email address, or a team name prefixed with team:, e.g. team:finops. 
- 
kinds (Optional[Set[str]]) – A list of strings representing the kinds of the asset. These will be made visible in the Dagster UI. 
- 
automation_condition (Optional[AutomationCondition]) – The AutomationCondition to use for this asset. 
- 
backfill_policy (Optional[BackfillPolicy]) – The BackfillPolicy to use for this asset. 
- 
code_version (Optional[str]) – Version of the code that generates this asset. In general, versions should be set only for code that deterministically produces the same output when given the same inputs. 
- 
key (Optional[CoeercibleToAssetKey]) – The key for this asset. If provided, cannot specify key_prefix or name. 
 - Examples: - @op
 def fetch_files_from_slack(context) -> pd.DataFrame:
 ...
 @op
 def store_files(files) -> None:
 files.to_sql(name="slack_files", con=create_db_connection())
 @graph_asset
 def slack_files_table():
 return store_files(fetch_files_from_slack())
- 
- @dagster.graph_multi_asset [source]
- Create a combined definition of multiple assets that are computed using the same graph of ops, and the same upstream assets. - Each argument to the decorated function references an upstream asset that this asset depends on. The name of the argument designates the name of the upstream asset. - Parameters: - 
name (Optional[str]) – The name of the graph. 
- 
outs – (Optional[Dict[str, AssetOut]]): The AssetOuts representing the produced assets. 
- 
ins (Optional[Mapping[str, AssetIn]]) – A dictionary that maps input names to information about the input. 
- 
partitions_def (Optional[PartitionsDefinition]) – Defines the set of partition keys that compose the assets. 
- 
hooks (Optional[AbstractSet[HookDefinition]]) – A list of hooks to attach to the asset. 
- 
backfill_policy (Optional[BackfillPolicy]) – The backfill policy for the asset. 
- 
group_name (Optional[str]) – A string name used to organize multiple assets into groups. This group name will be applied to all assets produced by this multi_asset. 
- 
can_subset (bool) – Whether this asset’s computation can emit a subset of the asset keys based on the context.selected_assets argument. Defaults to False. 
- 
config (Optional[Union[ConfigMapping], Mapping[str, Any]) – Describes how the graph underlying the asset is configured at runtime. If a ConfigMappingobject is provided, then the graph takes on the config schema of this object. The mapping will be applied at runtime to generate the config for the graph’s constituent nodes.If a dictionary is provided, then it will be used as the default run config for the graph. This means it must conform to the config schema of the underlying nodes. Note that the values provided will be viewable and editable in the Dagster UI, so be careful with secrets. If no value is provided, then the config schema for the graph is the default (derived 
 
- 
Multi-asset definitions
Refer to the Multi-asset documentation for more information.
- @dagster.multi_asset [source]
- Create a combined definition of multiple assets that are computed using the same op and same upstream assets. - Each argument to the decorated function references an upstream asset that this asset depends on. The name of the argument designates the name of the upstream asset. - You can set I/O managers keys, auto-materialize policies, freshness policies, group names, etc. on an individual asset within the multi-asset by attaching them to the - AssetOutcorresponding to that asset in the outs parameter.- Parameters: - name (Optional[str]) – The name of the op.
- outs – (Optional[Dict[str, AssetOut]]): The AssetOuts representing the assets materialized by this function. AssetOuts detail the output, IO management, and core asset properties. This argument is required except when AssetSpecs are used.
- ins (Optional[Mapping[str, AssetIn]]) – A dictionary that maps input names to information about the input.
- deps (Optional[Sequence[Union[AssetsDefinition, SourceAsset, AssetKey, str]]]) – The assets that are upstream dependencies, but do not correspond to a parameter of the decorated function. If the AssetsDefinition for a multi_asset is provided, dependencies on all assets created by the multi_asset will be created.
- config_schema (Optional[ConfigSchema) – The configuration schema for the asset’s underlying op. If set, Dagster will check that config provided for the op matches this schema and fail if it does not. If not set, Dagster will accept any config provided for the op.
- required_resource_keys (Optional[Set[str]]) – Set of resource handles required by the underlying op.
- internal_asset_deps (Optional[Mapping[str, Set[AssetKey]]]) – By default, it is assumed that all assets produced by a multi_asset depend on all assets that are consumed by that multi asset. If this default is not correct, you pass in a map of output names to a corrected set of AssetKeys that they depend on. Any AssetKeys in this list must be either used as input to the asset or produced within the op.
- partitions_def (Optional[PartitionsDefinition]) – Defines the set of partition keys that compose the assets.
- hooks (Optional[AbstractSet[HookDefinition]]) – A set of hooks to attach to the asset. These hooks will be executed when the asset is materialized.
- backfill_policy (Optional[BackfillPolicy]) – The backfill policy for the op that computes the asset.
- op_tags (Optional[Dict[str, Any]]) – A dictionary of tags for the op that computes the asset. Frameworks may expect and require certain metadata to be attached to a op. Values that are not strings will be json encoded and must meet the criteria that json.loads(json.dumps(value)) == value.
- can_subset (bool) – If this asset’s computation can emit a subset of the asset keys based on the context.selected_asset_keys argument. Defaults to False.
- resource_defs (Optional[Mapping[str, object]]) – beta (Beta) A mapping of resource keys to resources. These resources will be initialized during execution, and can be accessed from the context within the body of the function.
- group_name (Optional[str]) – A string name used to organize multiple assets into groups. This group name will be applied to all assets produced by this multi_asset.
- retry_policy (Optional[RetryPolicy]) – The retry policy for the op that computes the asset.
- code_version (Optional[str]) – Version of the code encapsulated by the multi-asset. If set, this is used as a default code version for all defined assets.
- specs (Optional[Sequence[AssetSpec]]) – The specifications for the assets materialized by this function.
- check_specs (Optional[Sequence[AssetCheckSpec]]) – Specs for asset checks that execute in the decorated function after materializing the assets.
- pool (Optional[str]) – A string that identifies the concurrency pool that governs this multi-asset’s execution.
- non_argument_deps (Optional[Union[Set[AssetKey], Set[str]]]) – deprecated Deprecated, use deps instead. Set of asset keys that are upstream dependencies, but do not pass an input to the multi_asset.
 - Examples: - @multi_asset(
 specs=[
 AssetSpec("asset1", deps=["asset0"]),
 AssetSpec("asset2", deps=["asset0"]),
 ]
 )
 def my_function():
 asset0_value = load(path="asset0")
 asset1_result, asset2_result = do_some_transformation(asset0_value)
 write(asset1_result, path="asset1")
 write(asset2_result, path="asset2")
 # Or use IO managers to handle I/O:
 @multi_asset(
 outs={
 "asset1": AssetOut(),
 "asset2": AssetOut(),
 }
 )
 def my_function(asset0):
 asset1_value = do_some_transformation(asset0)
 asset2_value = do_some_other_transformation(asset0)
 return asset1_value, asset2_value
- @dagster.multi_observable_source_asset [source]
- betaThis API is currently in beta, and may have breaking changes in minor version releases, with behavior changes in patch releases. Defines a set of assets that can be observed together with the same function. Parameters: - name (Optional[str]) – The name of the op.
- required_resource_keys (Optional[Set[str]]) – Set of resource handles required by the underlying op.
- partitions_def (Optional[PartitionsDefinition]) – Defines the set of partition keys that compose the assets.
- can_subset (bool) – If this asset’s computation can emit a subset of the asset keys based on the context.selected_assets argument. Defaults to False.
- resource_defs (Optional[Mapping[str, object]]) – beta (Beta) A mapping of resource keys to resources. These resources will be initialized during execution, and can be accessed from the context within the body of the function.
- group_name (Optional[str]) – A string name used to organize multiple assets into groups. This group name will be applied to all assets produced by this multi_asset.
- specs (Optional[Sequence[AssetSpec]]) – The specifications for the assets observed by this function.
- check_specs (Optional[Sequence[AssetCheckSpec]]) – Specs for asset checks that execute in the decorated function after observing the assets.
 Examples: @multi_observable_source_asset(
 specs=[AssetSpec("asset1"), AssetSpec("asset2")],
 )
 def my_function():
 yield ObserveResult(asset_key="asset1", metadata={"foo": "bar"})
 yield ObserveResult(asset_key="asset2", metadata={"baz": "qux"})
- classdagster.AssetOut [source]
- Defines one of the assets produced by a - @multi_asset.- Parameters: - key_prefix (Optional[Union[str, Sequence[str]]]) – If provided, the asset’s key is the concatenation of the key_prefix and the asset’s name. When using @multi_asset, the asset name defaults to the key of the “outs” dictionary Only one of the “key_prefix” and “key” arguments should be provided.
- key (Optional[Union[str, Sequence[str], AssetKey]]) – The asset’s key. Only one of the “key_prefix” and “key” arguments should be provided.
- dagster_type (Optional[Union[Type, DagsterType]]]) – The type of this output. Should only be set if the correct type can not be inferred directly from the type signature of the decorated function.
- description (Optional[str]) – Human-readable description of the output.
- is_required (bool) – Whether the presence of this field is required. (default: True)
- io_manager_key (Optional[str]) – The resource key of the IO manager used for this output. (default: “io_manager”).
- metadata (Optional[Dict[str, Any]]) – A dict of the metadata for the output. For example, users can provide a file path if the data object will be stored in a filesystem, or provide information of a database table when it is going to load the data into the table.
- group_name (Optional[str]) – A string name used to organize multiple assets into groups. If not provided, the name “default” is used.
- code_version (Optional[str]) – The version of the code that generates this asset.
- automation_condition (Optional[AutomationCondition]) – AutomationCondition to apply to the specified asset.
- backfill_policy (Optional[BackfillPolicy]) – BackfillPolicy to apply to the specified asset.
- owners (Optional[Sequence[str]]) – A list of strings representing owners of the asset. Each string can be a user’s email address, or a team name prefixed with team:, e.g. team:finops.
- tags (Optional[Mapping[str, str]]) – Tags for filtering and organizing. These tags are not attached to runs of the asset.
- kinds (Optional[set[str]]) – A set of strings representing the kinds of the asset. These
 - will be made visible in the Dagster UI. - staticfrom_spec [source]
- Builds an AssetOut from the passed spec. - Parameters: - spec (AssetSpec) – The spec to build the AssetOut from.
- dagster_type (Optional[Union[Type, DagsterType]]) – The type of this output. Should only be set if the correct type can not be inferred directly from the type signature of the decorated function.
- is_required (bool) – Whether the presence of this field is required. (default: True)
- io_manager_key (Optional[str]) – The resource key of the IO manager used for this output. (default: “io_manager”).
- backfill_policy (Optional[BackfillPolicy]) – BackfillPolicy to apply to the specified asset.
 - Returns: The AssetOut built from the spec.Return type: AssetOut 
 
- key_prefix (Optional[Union[str, Sequence[str]]]) – If provided, the asset’s key is the concatenation of the key_prefix and the asset’s name. When using 
Source assets
Refer to the External asset dependencies documentation for more information.
- classdagster.SourceAsset [source]
- deprecatedThis API will be removed in version 2.0.0. Use AssetSpec instead. If using the SourceAsset io_manager_key property, use AssetSpec(...).with_io_manager_key(...).. A SourceAsset represents an asset that will be loaded by (but not updated by) Dagster. Parameters: - key (Union[AssetKey, Sequence[str], str]) – The key of the asset.
- metadata (Mapping[str, MetadataValue]) – Metadata associated with the asset.
- io_manager_key (Optional[str]) – The key for the IOManager that will be used to load the contents of the asset when it’s used as an input to other assets inside a job.
- io_manager_def (Optional[IOManagerDefinition]) – beta (Beta) The definition of the IOManager that will be used to load the contents of the asset when it’s used as an input to other assets inside a job.
- resource_defs (Optional[Mapping[str, ResourceDefinition]]) – beta  (Beta) resource definitions that may be required by the dagster.IOManagerDefinitionprovided in the io_manager_def argument.
- description (Optional[str]) – The description of the asset.
- partitions_def (Optional[PartitionsDefinition]) – Defines the set of partition keys that compose the asset.
- observe_fn (Optional[SourceAssetObserveFunction])
- op_tags (Optional[Dict[str, Any]]) – A dictionary of tags for the op that computes the asset. Frameworks may expect and require certain metadata to be attached to a op. Values that are not strings will be json encoded and must meet the criteria that json.loads(json.dumps(value)) == value.
- auto_observe_interval_minutes (Optional[float]) – While the asset daemon is turned on, a run of the observation function for this asset will be launched at this interval. observe_fn must be provided.
- freshness_policy (FreshnessPolicy) – A constraint telling Dagster how often this asset is intended to be updated with respect to its root data.
- tags (Optional[Mapping[str, str]]) – Tags for filtering and organizing. These tags are not attached to runs of the asset.
 - propertyis_observable [source]
- Whether the asset is observable. - Type: bool 
 - propertyop [source]
- The OpDefinition associated with the observation function of an observable source asset. - Throws an error if the asset is not observable. - Type: OpDefinition 
 
- @dagster.observable_source_asset [source]
- betaThis API is currently in beta, and may have breaking changes in minor version releases, with behavior changes in patch releases. Create a SourceAsset with an associated observation function. The observation function of a source asset is wrapped inside of an op and can be executed as part of a job. Each execution generates an AssetObservation event associated with the source asset. The source asset observation function should return a DataVersion, a ~dagster.DataVersionsByPartition, or anObserveResult.Parameters: - name (Optional[str]) – The name of the source asset. If not provided, defaults to the name of the decorated function. The asset’s name must be a valid name in dagster (ie only contains letters, numbers, and _) and may not contain python reserved keywords.
- key_prefix (Optional[Union[str, Sequence[str]]]) – If provided, the source asset’s key is the concatenation of the key_prefix and the asset’s name, which defaults to the name of the decorated function. Each item in key_prefix must be a valid name in dagster (ie only contains letters, numbers, and _) and may not contain python reserved keywords.
- metadata (Mapping[str, RawMetadataValue]) – Metadata associated with the asset.
- io_manager_key (Optional[str]) – The key for the IOManager that will be used to load the contents of the source asset when it’s used as an input to other assets inside a job.
- io_manager_def (Optional[IOManagerDefinition]) – beta (Beta) The definition of the IOManager that will be used to load the contents of the source asset when it’s used as an input to other assets inside a job.
- description (Optional[str]) – The description of the asset.
- group_name (Optional[str]) – A string name used to organize multiple assets into groups. If not provided, the name “default” is used.
- required_resource_keys (Optional[Set[str]]) – Set of resource keys required by the observe op.
- resource_defs (Optional[Mapping[str, ResourceDefinition]]) – beta  (Beta) resource definitions that may be required by the dagster.IOManagerDefinitionprovided in the io_manager_def argument.
- partitions_def (Optional[PartitionsDefinition]) – Defines the set of partition keys that compose the asset.
- op_tags (Optional[Dict[str, Any]]) – A dictionary of tags for the op that computes the asset. Frameworks may expect and require certain metadata to be attached to a op. Values that are not strings will be json encoded and must meet the criteria that json.loads(json.dumps(value)) == value.
- tags (Optional[Mapping[str, str]]) – Tags for filtering and organizing. These tags are not attached to runs of the asset.
- observe_fn (Optional[SourceAssetObserveFunction]) – Observation function for the source asset.
- automation_condition (Optional[AutomationCondition]) – A condition describing when Dagster should materialize this asset.
 
- classdagster.ObserveResult [source]
- An object representing a successful observation of an asset. These can be returned from an @observable_source_asset decorated function to pass metadata. - Parameters: - asset_key (Optional[AssetKey]) – The asset key. Optional to include.
- metadata (Optional[RawMetadataMapping]) – Metadata to record with the corresponding AssetObservation event.
- check_results (Optional[Sequence[AssetCheckResult]]) – Check results to record with the corresponding AssetObservation event.
- data_version (Optional[DataVersion]) – The data version of the asset that was observed.
- tags (Optional[Mapping[str, str]]) – Tags to record with the corresponding AssetObservation event.
 
Dependencies
- classdagster.AssetDep [source]
- Specifies a dependency on an upstream asset. - Parameters: - asset (Union[AssetKey, str, AssetSpec, AssetsDefinition, SourceAsset]) – The upstream asset to depend on.
- partition_mapping (Optional[PartitionMapping]) – Defines what partitions to depend on in the upstream asset. If not provided and the upstream asset is partitioned, defaults to the default partition mapping for the partitions definition, which is typically maps partition keys to the same partition keys in upstream assets.
 - Examples: - upstream_asset = AssetSpec("upstream_asset")
 downstream_asset = AssetSpec(
 "downstream_asset",
 deps=[
 AssetDep(
 upstream_asset,
 partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1)
 )
 ]
 )
- classdagster.AssetIn [source]
- Defines an asset dependency. - Parameters: - key_prefix (Optional[Union[str, Sequence[str]]]) – If provided, the asset’s key is the concatenation of the key_prefix and the input name. Only one of the “key_prefix” and “key” arguments should be provided.
- key (Optional[Union[str, Sequence[str], AssetKey]]) – The asset’s key. Only one of the “key_prefix” and “key” arguments should be provided.
- metadata (Optional[Dict[str, Any]]) – A dict of the metadata for the input. For example, if you only need a subset of columns from an upstream table, you could include that in metadata and the IO manager that loads the upstream table could use the metadata to determine which columns to load.
- partition_mapping (Optional[PartitionMapping]) – Defines what partitions to depend on in the upstream asset. If not provided, defaults to the default partition mapping for the partitions definition, which is typically maps partition keys to the same partition keys in upstream assets.
- dagster_type (DagsterType) – Allows specifying type validation functions that will be executed on the input of the decorated function before it runs.
 
Asset jobs
Asset jobs enable the automation of asset materializations. Dagster’s asset selection syntax can be used to select assets and assign them to a job.
- dagster.define_asset_job [source]
- Creates a definition of a job which will either materialize a selection of assets or observe a selection of source assets. This will only be resolved to a JobDefinition once placed in a project. - Parameters: - 
name (str) – The name for the job. 
- 
selection (Union[str, Sequence[str], Sequence[AssetKey], Sequence[Union[AssetsDefinition, SourceAsset]], AssetSelection]) – The assets that will be materialized or observed when the job is run. The selected assets must all be included in the assets that are passed to the assets argument of the Definitions object that this job is included on. The string “my_asset*” selects my_asset and all downstream assets within the code location. A list of strings represents the union of all assets selected by strings within the list. 
- 
config – Describes how the Job is parameterized at runtime. If no value is provided, then the schema for the job’s run config is a standard format based on its ops and resources. If a dictionary is provided, then it must conform to the standard config schema, and it will be used as the job’s run config for the job whenever the job is executed. The values provided will be viewable and editable in the Dagster UI, so be careful with secrets. 
- 
tags (Optional[Mapping[str, object]]) – A set of key-value tags that annotate the job and can be used for searching and filtering in the UI. Values that are not already strings will be serialized as JSON. If run_tags is not set, then the content of tags will also be automatically appended to the tags of any runs of this job. 
- 
run_tags (Optional[Mapping[str, object]]) – A set of key-value tags that will be automatically attached to runs launched by this job. Values that are not already strings will be serialized as JSON. These tag values may be overwritten by tag values provided at invocation time. If run_tags is set, then tags are not automatically appended to the tags of any runs of this job. 
- 
metadata (Optional[Mapping[str, RawMetadataValue]]) – Arbitrary metadata about the job. Keys are displayed string labels, and values are one of the following: string, float, int, JSON-serializable dict, JSON-serializable list, and one of the data classes returned by a MetadataValue static method. 
- 
description (Optional[str]) – A description for the Job. 
- 
executor_def (Optional[ExecutorDefinition]) – How this Job will be executed. Defaults to multi_or_in_process_executor, which can be switched between multi-process and in-process modes of execution. The default mode of execution is multi-process.
- 
hooks (Optional[AbstractSet[HookDefinition]]) – A set of hooks to be attached to each asset in the job. These hooks define logic that runs in response to events such as success or failure during the execution of individual assets. 
- 
op_retry_policy (Optional[RetryPolicy]) – The default retry policy for all ops that compute assets in this job. Only used if retry policy is not defined on the asset definition. 
- 
partitions_def (Optional[PartitionsDefinition]) – deprecated (Deprecated) Defines the set of partitions for this job. Deprecated because partitioning is inferred from the selected assets, so setting this is redundant. 
 - Returns: The job, which can be placed inside a project.Return type: UnresolvedAssetJobDefinition Examples: - # A job that targets all assets in the project:
 @asset
 def asset1():
 ...
 Definitions(
 assets=[asset1],
 jobs=[define_asset_job("all_assets")],
 )
 # A job that targets a single asset
 @asset
 def asset1():
 ...
 Definitions(
 assets=[asset1],
 jobs=[define_asset_job("all_assets", selection=[asset1])],
 )
 # A job that targets all the assets in a group:
 Definitions(
 assets=assets,
 jobs=[define_asset_job("marketing_job", selection=AssetSelection.groups("marketing"))],
 )
 @observable_source_asset
 def source_asset():
 ...
 # A job that observes a source asset:
 Definitions(
 assets=assets,
 jobs=[define_asset_job("observation_job", selection=[source_asset])],
 )
 # Resources are supplied to the assets, not the job:
 @asset(required_resource_keys={"slack_client"})
 def asset1():
 ...
 Definitions(
 assets=[asset1],
 jobs=[define_asset_job("all_assets")],
 resources={"slack_client": prod_slack_client},
 )
- 
- classdagster.AssetSelection [source]
- An AssetSelection defines a query over a set of assets and asset checks, normally all that are defined in a project. - You can use the “|”, “&”, and “-” operators to create unions, intersections, and differences of selections, respectively. - AssetSelections are typically used with - define_asset_job().- By default, selecting assets will also select all of the asset checks that target those assets. - Examples: - # Select all assets in group "marketing":
 AssetSelection.groups("marketing")
 # Select all assets in group "marketing", as well as the asset with key "promotion":
 AssetSelection.groups("marketing") | AssetSelection.assets("promotion")
 # Select all assets in group "marketing" that are downstream of asset "leads":
 AssetSelection.groups("marketing") & AssetSelection.assets("leads").downstream()
 # Select a list of assets:
 AssetSelection.assets(*my_assets_list)
 # Select all assets except for those in group "marketing"
 AssetSelection.all() - AssetSelection.groups("marketing")
 # Select all assets which are materialized by the same op as "projections":
 AssetSelection.assets("projections").required_multi_asset_neighbors()
 # Select all assets in group "marketing" and exclude their asset checks:
 AssetSelection.groups("marketing") - AssetSelection.all_asset_checks()
 # Select all asset checks that target a list of assets:
 AssetSelection.checks_for_assets(*my_assets_list)
 # Select a specific asset check:
 AssetSelection.checks(my_asset_check)- staticall [source]
- Returns a selection that includes all assets and their asset checks. - Parameters: include_sources (bool) – beta If True, then include all external assets. 
 - staticall_asset_checks [source]
- Returns a selection that includes all asset checks. 
 - staticassets [source]
- Returns a selection that includes all of the provided assets and asset checks that target them. - Parameters: *assets_defs (Union[AssetsDefinition, str, Sequence[str], AssetKey]) – The assets to select. Examples: - AssetSelection.assets(AssetKey(["a"]))
 AssetSelection.assets("a")
 AssetSelection.assets(AssetKey(["a"]), AssetKey(["b"]))
 AssetSelection.assets("a", "b")
 @asset
 def asset1():
 ...
 AssetSelection.assets(asset1)
 asset_key_list = [AssetKey(["a"]), AssetKey(["b"])]
 AssetSelection.assets(*asset_key_list)
 - staticchecks [source]
- Returns a selection that includes all of the provided asset checks or check keys. 
 - staticchecks_for_assets [source]
- Returns a selection with the asset checks that target the provided assets. - Parameters: *assets_defs (Union[AssetsDefinition, str, Sequence[str], AssetKey]) – The assets to select checks for. 
 - staticgroups [source]
- Returns a selection that includes materializable assets that belong to any of the provided groups and all the asset checks that target them. - Parameters: include_sources (bool) – beta If True, then include external assets matching the group in the selection. 
 - statickey_prefixes [source]
- Returns a selection that includes assets that match any of the provided key prefixes and all the asset checks that target them. - Parameters: include_sources (bool) – beta If True, then include external assets matching the key prefix(es) in the selection. Examples: - # match any asset key where the first segment is equal to "a" or "b"
 # e.g. AssetKey(["a", "b", "c"]) would match, but AssetKey(["abc"]) would not.
 AssetSelection.key_prefixes("a", "b")
 # match any asset key where the first two segments are ["a", "b"] or ["a", "c"]
 AssetSelection.key_prefixes(["a", "b"], ["a", "c"])
 - statickeys [source]
- deprecatedThis API will be removed in version 2.0. Use AssetSelection.assets instead.. Returns a selection that includes assets with any of the provided keys and all asset checks that target them. Deprecated: use AssetSelection.assets instead. Examples: AssetSelection.keys(AssetKey(["a"]))
 AssetSelection.keys("a")
 AssetSelection.keys(AssetKey(["a"]), AssetKey(["b"]))
 AssetSelection.keys("a", "b")
 asset_key_list = [AssetKey(["a"]), AssetKey(["b"])]
 AssetSelection.keys(*asset_key_list)
 - statictag [source]
- Returns a selection that includes materializable assets that have the provided tag, and all the asset checks that target them. - Parameters: include_sources (bool) – beta If True, then include external assets matching the group in the selection. 
 - downstream [source]
- Returns a selection that includes all assets that are downstream of any of the assets in this selection, selecting the assets in this selection by default. Includes the asset checks targeting the returned assets. Iterates through each asset in this selection and returns the union of all downstream assets. - depth (Optional[int]): If provided, then only include assets to the given depth. A depth of 2 means all assets that are children or grandchildren of the assets in this selection. - include_self (bool): If True, then include the assets in this selection in the result. If the include_self flag is False, return each downstream asset that is not part of the original selection. By default, set to True. 
 - materializable [source]
- Given an asset selection, returns a new asset selection that contains all of the assets that are materializable. Removes any assets which are not materializable. 
 - required_multi_asset_neighbors [source]
- Given an asset selection in which some assets are output from a multi-asset compute op which cannot be subset, returns a new asset selection that contains all of the assets required to execute the original asset selection. Includes the asset checks targeting the returned assets. 
 - roots [source]
- Given an asset selection, returns a new asset selection that contains all of the root assets within the original asset selection. Includes the asset checks targeting the returned assets. - A root asset is an asset that has no upstream dependencies within the asset selection. The root asset can have downstream dependencies outside of the asset selection. - Because mixed selections of external and materializable assets are currently not supported, keys corresponding to external assets will not be included as roots. To select external assets, use the upstream_source_assets method. 
 - sinks [source]
- Given an asset selection, returns a new asset selection that contains all of the sink assets within the original asset selection. Includes the asset checks targeting the returned assets. - A sink asset is an asset that has no downstream dependencies within the asset selection. The sink asset can have downstream dependencies outside of the asset selection. 
 - sources [source]
- deprecatedThis API will be removed in version 2.0. Use AssetSelection.roots instead.. Given an asset selection, returns a new asset selection that contains all of the root assets within the original asset selection. Includes the asset checks targeting the returned assets. A root asset is a materializable asset that has no upstream dependencies within the asset selection. The root asset can have downstream dependencies outside of the asset selection. Because mixed selections of external and materializable assets are currently not supported, keys corresponding to external assets will not be included as roots. To select external assets, use the upstream_source_assets method. 
 - upstream [source]
- Returns a selection that includes all materializable assets that are upstream of any of the assets in this selection, selecting the assets in this selection by default. Includes the asset checks targeting the returned assets. Iterates through each asset in this selection and returns the union of all upstream assets. - Because mixed selections of external and materializable assets are currently not supported, keys corresponding to external assets will not be included as upstream of regular assets. - Parameters: - depth (Optional[int]) – If provided, then only include assets to the given depth. A depth of 2 means all assets that are parents or grandparents of the assets in this selection.
- include_self (bool) – If True, then include the assets in this selection in the result. If the include_self flag is False, return each upstream asset that is not part of the original selection. By default, set to True.
 
 - upstream_source_assets [source]
- Given an asset selection, returns a new asset selection that contains all of the external assets that are parents of assets in the original selection. Includes the asset checks targeting the returned assets. 
 - without_checks [source]
- Removes all asset checks in the selection. 
 
Code locations
Loading assets and asset jobs into a code location makes them available to Dagster tools like the UI, CLI, and GraphQL API.
- dagster.load_assets_from_modules [source]
- Constructs a list of assets and source assets from the given modules. - Parameters: - modules (Iterable[ModuleType]) – The Python modules to look for assets inside.
- group_name (Optional[str]) – Group name to apply to the loaded assets. The returned assets will be copies of the loaded objects, with the group name added.
- key_prefix (Optional[Union[str, Sequence[str]]]) – Prefix to prepend to the keys of the loaded assets. The returned assets will be copies of the loaded objects, with the prefix prepended.
- legacy_freshness_policy (Optional[LegacyFreshnessPolicy]) – LegacyFreshnessPolicy to apply to all the loaded assets.
- automation_condition (Optional[AutomationCondition]) – AutomationCondition to apply to all the loaded assets.
- backfill_policy (Optional[AutoMaterializePolicy]) – BackfillPolicy to apply to all the loaded assets.
- source_key_prefix (bool) – Prefix to prepend to the keys of loaded SourceAssets. The returned assets will be copies of the loaded objects, with the prefix prepended.
 - Returns: A list containing assets and source assets defined in the given modules.Return type: Sequence[Union[AssetsDefinition, SourceAsset]] 
- dagster.load_assets_from_current_module [source]
- Constructs a list of assets, source assets, and cacheable assets from the module where this function is called. - Parameters: - group_name (Optional[str]) – Group name to apply to the loaded assets. The returned assets will be copies of the loaded objects, with the group name added.
- key_prefix (Optional[Union[str, Sequence[str]]]) – Prefix to prepend to the keys of the loaded assets. The returned assets will be copies of the loaded objects, with the prefix prepended.
- legacy_freshness_policy (Optional[LegacyFreshnessPolicy]) – LegacyFreshnessPolicy to apply to assets.
- automation_condition (Optional[AutomationCondition]) – AutomationCondition to apply to all the loaded assets.
- backfill_policy (Optional[AutoMaterializePolicy]) – BackfillPolicy to apply to all the loaded assets.
- source_key_prefix (bool) – Prefix to prepend to the keys of loaded SourceAssets. The returned assets will be copies of the loaded objects, with the prefix prepended.
 - Returns: A list containing assets, source assets, and cacheable assets defined in the module.Return type: Sequence[Union[AssetsDefinition, SourceAsset, CachableAssetsDefinition]] 
- dagster.load_assets_from_package_module [source]
- Constructs a list of assets and source assets that includes all asset definitions, source assets, and cacheable assets in all sub-modules of the given package module. - A package module is the result of importing a package. - Parameters: - package_module (ModuleType) – The package module to looks for assets inside.
- group_name (Optional[str]) – Group name to apply to the loaded assets. The returned assets will be copies of the loaded objects, with the group name added.
- key_prefix (Optional[Union[str, Sequence[str]]]) – Prefix to prepend to the keys of the loaded assets. The returned assets will be copies of the loaded objects, with the prefix prepended.
- legacy_freshness_policy (Optional[LegacyFreshnessPolicy]) – LegacyFreshnessPolicy to apply to all the loaded assets.
- automation_condition (Optional[AutomationCondition]) – AutomationCondition to apply to all the loaded assets.
- backfill_policy (Optional[AutoMaterializePolicy]) – BackfillPolicy to apply to all the loaded assets.
- source_key_prefix (bool) – Prefix to prepend to the keys of loaded SourceAssets. The returned assets will be copies of the loaded objects, with the prefix prepended.
 - Returns: A list containing assets, source assets, and cacheable assets defined in the module.Return type: Sequence[Union[AssetsDefinition, SourceAsset, CacheableAssetsDefinition]] 
- dagster.load_assets_from_package_name [source]
- Constructs a list of assets, source assets, and cacheable assets that includes all asset definitions and source assets in all sub-modules of the given package. - Parameters: - package_name (str) – The name of a Python package to look for assets inside.
- group_name (Optional[str]) – Group name to apply to the loaded assets. The returned assets will be copies of the loaded objects, with the group name added.
- key_prefix (Optional[Union[str, Sequence[str]]]) – Prefix to prepend to the keys of the loaded assets. The returned assets will be copies of the loaded objects, with the prefix prepended.
- legacy_freshness_policy (Optional[LegacyFreshnessPolicy]) – LegacyFreshnessPolicy to apply to all the loaded assets.
- auto_materialize_policy (Optional[AutoMaterializePolicy]) – AutoMaterializePolicy to apply to all the loaded assets.
- backfill_policy (Optional[AutoMaterializePolicy]) – BackfillPolicy to apply to all the loaded assets.
- source_key_prefix (bool) – Prefix to prepend to the keys of loaded SourceAssets. The returned assets will be copies of the loaded objects, with the prefix prepended.
 - Returns: A list containing assets, source assets, and cacheable assets defined in the module.Return type: Sequence[Union[AssetsDefinition, SourceAsset, CacheableAssetsDefinition]] 
Observations
Refer to the Asset observation documentation for more information.
- classdagster.AssetObservation [source]
- Event that captures metadata about an asset at a point in time. - Parameters: - asset_key (Union[str, List[str], AssetKey]) – A key to identify the asset.
- partition (Optional[str]) – The name of a partition of the asset that the metadata corresponds to.
- tags (Optional[Mapping[str, str]]) – A mapping containing tags for the observation.
- metadata (Optional[Dict[str, Union[str, float, int, MetadataValue]]]) – Arbitrary metadata about the asset. Keys are displayed string labels, and values are one of the following: string, float, int, JSON-serializable dict, JSON-serializable list, and one of the data classes returned by a MetadataValue static method.
 
Declarative Automation
Refer to the Declarative Automation documentation for more information.
- classdagster.AutomationCondition [source]
- An AutomationCondition represents a condition of an asset that impacts whether it should be automatically executed. For example, you can have a condition which becomes true whenever the code version of the asset is changed, or whenever an upstream dependency is updated. - from dagster import AutomationCondition, asset
 @asset(automation_condition=AutomationCondition.on_cron("0 0 * * *"))
 def my_asset(): ...- AutomationConditions may be combined together into expressions using a variety of operators. - from dagster import AssetSelection, AutomationCondition, asset
 # any dependencies from the "important" group are missing
 any_important_deps_missing = AutomationCondition.any_deps_match(
 AutomationCondition.missing(),
 ).allow(AssetSelection.groups("important"))
 # there is a new code version for this asset since the last time it was requested
 new_code_version = AutomationCondition.code_version_changed().since(
 AutomationCondition.newly_requested()
 )
 # there is a new code version and no important dependencies are missing
 my_condition = new_code_version & ~any_important_deps_missing
 @asset(automation_condition=my_condition)
 def my_asset(): ...- staticall_checks_match [source]
- Returns an AutomationCondition that is true for an asset partition if all of its checks evaluate to True for the given condition. - Parameters: - condition (AutomationCondition) – The AutomationCondition that will be evaluated against this asset’s checks.
- blocking_only (bool) – Determines if this condition will only be evaluated against blocking checks. Defaults to False.
 
 - staticall_deps_blocking_checks_passed [source]
- Returns an AutomationCondition that is true for any partition where all upstream blocking checks have passed, or will be requested on this tick. - In-tick requests are allowed to enable creating runs that target both a parent with blocking checks and a child. Even though the checks have not currently passed, if they fail within the run, the run machinery will prevent the child from being materialized. 
 - staticall_deps_match [source]
- Returns an AutomationCondition that is true for a if at least one partition of the all of the target’s dependencies evaluate to True for the given condition. - Parameters: condition (AutomationCondition) – The AutomationCondition that will be evaluated against this target’s dependencies. 
 - staticall_deps_updated_since_cron [source]
- Returns an AutomatonCondition that is true if all of the target’s dependencies have updated since the latest tick of the provided cron schedule. 
 - staticany_checks_match [source]
- Returns an AutomationCondition that is true for if at least one of the target’s checks evaluate to True for the given condition. - Parameters: - condition (AutomationCondition) – The AutomationCondition that will be evaluated against this asset’s checks.
- blocking_only (bool) – Determines if this condition will only be evaluated against blocking checks. Defaults to False.
 
 - staticany_deps_in_progress [source]
- Returns an AutomationCondition that is true if the target has at least one dependency that is in progress. 
 - staticany_deps_match [source]
- Returns an AutomationCondition that is true for a if at least one partition of the any of the target’s dependencies evaluate to True for the given condition. - Parameters: condition (AutomationCondition) – The AutomationCondition that will be evaluated against this target’s dependencies. 
 - staticany_deps_missing [source]
- Returns an AutomationCondition that is true if the target has at least one dependency that is missing, and will not be requested on this tick. 
 - staticany_deps_updated [source]
- Returns an AutomationCondition that is true if the target has at least one dependency that has updated since the previous tick, or will be requested on this tick. - Will ignore parent updates if the run that updated the parent also plans to update the asset or check that this condition is applied to. 
 - staticany_downstream_conditions [source]
- betaThis API is currently in beta, and may have breaking changes in minor version releases, with behavior changes in patch releases. Returns an AutomationCondition which represents the union of all distinct downstream conditions. 
 - staticasset_matches [source]
- Returns an AutomationCondition that is true if this condition is true for the given key. 
 - staticbackfill_in_progress [source]
- Returns an AutomationCondition that is true if the target is part of an in-progress backfill that has not yet executed it. 
 - staticcheck_failed [source]
- Returns an AutomationCondition that is true for an asset check if it has evaluated against the latest materialization of an asset and failed. 
 - staticcheck_passed [source]
- Returns an AutomationCondition that is true for an asset check if it has evaluated against the latest materialization of an asset and passed. 
 - staticcode_version_changed [source]
- Returns an AutomationCondition that is true if the target’s code version has been changed since the previous tick. 
 - staticcron_tick_passed [source]
- Returns an AutomationCondition that is whenever a cron tick of the provided schedule is passed. 
 - staticdata_version_changed [source]
- Returns an AutomationCondition that is true if the target’s data version has been changed since the previous tick. 
 - staticeager [source]
- Returns an AutomationCondition which will cause a target to be executed if any of its dependencies update, and will execute missing partitions if they become missing after this condition is applied to the target. - This will not execute targets that have any missing or in progress dependencies, or are currently in progress. - For time partitioned assets, only the latest time partition will be considered. 
 - staticexecution_failed [source]
- Returns an AutomationCondition that is true if the latest execution of the target failed. 
 - staticin_latest_time_window [source]
- Returns an AutomationCondition that is true when the target it is within the latest time window. - Parameters: lookback_delta (Optional, datetime.timedelta) – If provided, the condition will return all partitions within the provided delta of the end of the latest time window. For example, if this is used on a daily-partitioned asset with a lookback_delta of 48 hours, this will return the latest two partitions. 
 - staticin_progress [source]
- Returns an AutomationCondition that is true for an asset partition if it is part of an in-progress run or backfill that has not yet executed it. 
 - staticinitial_evaluation [source]
- Returns an AutomationCondition that is true on the first evaluation of the expression. 
 - staticmissing [source]
- Returns an AutomationCondition that is true if the target has not been executed. 
 - staticnewly_missing [source]
- Returns an AutomationCondition that is true on the tick that the target becomes missing. 
 - staticnewly_requested [source]
- Returns an AutomationCondition that is true if the target was requested on the previous tick. 
 - staticnewly_updated [source]
- Returns an AutomationCondition that is true if the target has been updated since the previous tick. 
 - staticon_cron [source]
- Returns an AutomationCondition which will cause a target to be executed on a given cron schedule, after all of its dependencies have been updated since the latest tick of that cron schedule. - For time partitioned assets, only the latest time partition will be considered. 
 - staticon_missing [source]
- Returns an AutomationCondition which will execute partitions of the target that are added after this condition is applied to the asset. - This will not execute targets that have any missing dependencies. - For time partitioned assets, only the latest time partition will be considered. 
 - staticrun_in_progress [source]
- Returns an AutomationCondition that is true if the target is part of an in-progress run that has not yet executed it. 
 - staticwill_be_requested [source]
- Returns an AutomationCondition that is true if the target will be requested this tick. 
 - replace [source]
- Replaces all instances of - oldacross any sub-conditions with- new.- If - oldis a string, then conditions with a label or name matching that string will be replaced.- Parameters: - old (Union[AutomationCondition, str]) – The condition to replace.
- new (AutomationCondition) – The condition to replace with.
 
 
- classdagster.AutomationResult [source]
- The result of evaluating an AutomationCondition. 
- classdagster.AutomationConditionSensorDefinition [source]
- Targets a set of assets and repeatedly evaluates all the AutomationConditions on all of those assets to determine which to request runs for. - Parameters: - name – The name of the sensor.
- target (Union[str, Sequence[str], Sequence[AssetKey], Sequence[Union[AssetsDefinition, SourceAsset]], AssetSelection]) – A selection of assets to evaluate AutomationConditions of and request runs for.
- tags (Optional[Mapping[str, str]]) – A set of key-value tags that annotate the sensor and can be used for searching and filtering in the UI.
- run_tags (Optional[Mapping[str, Any]]) – Tags that will be automatically attached to runs launched by this sensor.
- metadata (Optional[Mapping[str, object]]) – A set of metadata entries that annotate the sensor. Values will be normalized to typed MetadataValue objects.
- default_status (DefaultSensorStatus) – Whether the sensor starts as running or not. The default status can be overridden from the Dagster UI or via the GraphQL API.
- minimum_interval_seconds (Optional[int]) – The frequency at which to try to evaluate the sensor. The actual interval will be longer if the sensor evaluation takes longer than the provided interval.
- description (Optional[str]) – A human-readable description of the sensor.
- emit_backfills (bool) – If set to True, will emit a backfill on any tick where more than one partition of any single asset is requested, rather than individual runs. Defaults to True.
- use_user_code_server (bool) – beta (Beta) If set to True, this sensor will be evaluated in the user code server, rather than the AssetDaemon. This enables evaluating custom AutomationCondition subclasses, and ensures that the condition definitions will remain in sync with your user code version, eliminating version skew. Note: currently a maximum of 500 assets or checks may be targeted at a time by a sensor that has this value set.
- default_condition (Optional[AutomationCondition]) – beta (Beta) If provided, this condition will be used for any selected assets or asset checks which do not have an automation condition defined. Requires use_user_code_server to be set to True.
 - Examples: - import dagster as dg
 # automation condition sensor that defaults to running
 defs1 = dg.Definitions(
 assets=...,
 sensors=[
 dg.AutomationConditionSensorDefinition(
 name="automation_condition_sensor",
 target=dg.AssetSelection.all(),
 default_status=dg.DefaultSensorStatus.RUNNING,
 ),
 ]
 )
 # one automation condition sensor per group
 defs2 = dg.Definitions(
 assets=...,
 sensors=[
 dg.AutomationConditionSensorDefinition(
 name="raw_data_automation_condition_sensor",
 target=dg.AssetSelection.groups("raw_data"),
 ),
 dg.AutomationConditionSensorDefinition(
 name="ml_automation_condition_sensor",
 target=dg.AssetSelection.groups("machine_learning"),
 ),
 ]
 )
Asset values
- classdagster.AssetValueLoader [source]
- Caches resource definitions that are used to load asset values across multiple load invocations. - Should not be instantiated directly. Instead, use - get_asset_value_loader().- load_asset_value [source]
- Loads the contents of an asset as a Python object. - Invokes load_input on the - IOManagerassociated with the asset.- Parameters: - asset_key (Union[AssetKey, Sequence[str], str]) – The key of the asset to load.
- python_type (Optional[Type]) – The python type to load the asset as. This is what will be returned inside load_input by context.dagster_type.typing_type.
- partition_key (Optional[str]) – The partition of the asset to load.
- input_definition_metadata (Optional[Dict[str, Any]]) – Input metadata to pass to the IOManager(is equivalent to setting the metadata argument in In or AssetIn).
- resource_config (Optional[Any]) – A dictionary of resource configurations to be passed to the IOManager.
 - Returns: The contents of an asset as a Python object.