Software-Defined Assets

An asset is an object in persistent storage, such as a table, file, or persisted machine learning model. A software-defined asset is a Dagster object that couples an asset to the function and upstream assets that are used to produce its contents.

@dagster.asset(name=None, namespace=None, key_prefix=None, ins=None, non_argument_deps=None, metadata=None, description=None, config_schema=None, required_resource_keys=None, resource_defs=None, io_manager_def=None, io_manager_key=None, compute_kind=None, dagster_type=None, partitions_def=None, partition_mappings=None, op_tags=None, group_name=None)[source]

Create a definition for how to compute an asset.

A software-defined asset is the combination of: 1. An asset key, e.g. the name of a table. 2. A function, which can be run to compute the contents of the asset. 3. 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.

Parameters
  • name (Optional[str]) – The name of the asset. If not provided, defaults to the name of the decorated function.

  • namespace (Optional[Sequence[str]]) – Deprecated (use `key_prefix`). The namespace that the asset resides in. The namespace + the name forms the asset key.

  • 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.

  • ins (Optional[Mapping[str, AssetIn]]) – A dictionary that maps input names to their metadata and namespaces.

  • non_argument_deps (Optional[Union[Set[AssetKey], Set[str]]]) – Set of asset keys that are upstream dependencies, but do not pass an input to the asset.

  • 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.

  • 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[IOManagerDefinition]) – The definition of 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.

  • compute_kind (Optional[str]) – A string to represent the kind of computation that produces the asset, e.g. “dbt” or “spark”. It will be displayed in Dagit as a badge on the asset.

  • 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.

  • 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.

  • 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.

Examples

@asset
def my_asset(my_upstream_asset: int) -> int:
    return my_upstream_asset + 1
@dagster.multi_asset(outs, name=None, ins=None, non_argument_deps=None, description=None, config_schema=None, required_resource_keys=None, compute_kind=None, internal_asset_deps=None, partitions_def=None, partition_mappings=None, op_tags=None, can_subset=False)[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.

Parameters
  • name (Optional[str]) – The name of the op.

  • outs – (Optional[Dict[str, Out]]): The Outs representing the produced assets.

  • ins (Optional[Mapping[str, AssetIn]]) – A dictionary that maps input names to their metadata and namespaces.

  • non_argument_deps (Optional[Union[Set[AssetKey], Set[str]]]) – Set of asset keys that are upstream dependencies, but do not pass an input to the multi_asset.

  • 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 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”).

  • compute_kind (Optional[str]) – A string to represent the kind of computation that produces the asset, e.g. “dbt” or “spark”. It will be displayed in Dagit as a badge on the asset.

  • 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.

  • 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.

  • 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_assets argument. Defaults to False.

class dagster.AssetIn(key=None, metadata=None, namespace=None, key_prefix=None, asset_key=None)[source]

Defines an asset dependency.

key_prefix

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.

Type

Optional[Union[str, Sequence[str]]]

key

The asset’s key. Only one of the “key_prefix” and “key” arguments should be provided.

Type

Optional[Union[str, Sequence[str], AssetKey]]

metadata

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.

Type

Optional[Dict[str, Any]]

class dagster.SourceAsset(key, metadata=None, io_manager_key=None, io_manager_def=None, description=None, partitions_def=None, _metadata_entries=None, group_name=None, resource_defs=None)[source]

A SourceAsset represents an asset that will be loaded by (but not updated by) Dagster.

key

The key of the asset.

Type

Union[AssetKey, Sequence[str], str]

metadata_entries

Metadata associated with the asset.

Type

List[MetadataEntry]

io_manager_key

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.

Type

Optional[str]

io_manager_def

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.

Type

Optional[IOManagerDefinition]

description

The description of the asset.

Type

Optional[str]

partitions_def

Defines the set of partition keys that compose the asset.

Type

Optional[PartitionsDefinition]

dagster.define_asset_job(name, selection=None, config=None, description=None, tags=None, partitions_def=None)[source]

Creates a definition of a job which will materialize a selection of assets. This will only be resolved to a JobDefinition once placed in a repository.

Parameters
  • name (str) – The name for the job.

  • selection (Union[str, Sequence[str], AssetSelection]) –

    A selection over the set of Assets available on your repository. This can be a string such as “my_asset*”, a list of such strings (representing a union of these selections), or an AssetSelection object.

    This selection will be resolved to a set of Assets once the repository is loaded with a set of AssetsDefinitions.

  • 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 solids 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 Dagit playground, so be careful with secrets.

    If a ConfigMapping object is provided, then the schema for the job’s run config is determined by the config mapping, and the ConfigMapping, which should return configuration in the standard format to configure the job.

  • tags (Optional[Mapping[str, Any]]) – Arbitrary information that will be attached to the execution of the Job. Values that are not strings will be json encoded and must meet the criteria that json.loads(json.dumps(value)) == value. These tag values may be overwritten by tag values provided at invocation time.

  • description (Optional[str]) – A description for the Job.

  • partitions_def (Optional[PartitionsDefinition]) – Defines the set of partitions for this job. All AssetDefinitions selected for this job must have a matching PartitionsDefinition.

dagster.load_assets_from_modules(modules, group_name=None, key_prefix=None)[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, List[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.

Returns

A list containing assets and source assets defined in the given modules.

Return type

List[Union[AssetsDefinition, SourceAsset]]

dagster.load_assets_from_current_module(group_name=None, key_prefix=None)[source]

Constructs a list of assets and source 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, List[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.

Returns

A list containing assets and source assets defined in the module.

Return type

List[Union[AssetsDefinition, SourceAsset]]

dagster.load_assets_from_package_module(package_module, group_name=None, key_prefix=None)[source]

Constructs a list of assets and source assets that includes all asset definitions and source 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, List[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.

Returns

A list containing assets and source assets defined in the module.

Return type

List[Union[AssetsDefinition, SourceAsset]]

dagster.load_assets_from_package_name(package_name, group_name=None, key_prefix=None)[source]

Constructs a list of assets and source assets that include 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, List[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.

Returns

A list containing assets and source assets defined in the module.

Return type

List[Union[AssetsDefinition, SourceAsset]]