Skip to main content
The @asset decorator is used to define a software-defined asset - a combination of an asset key, a compute function, and upstream dependencies.

Signature

@asset(
    name: Optional[str] = None,
    key_prefix: Optional[Union[str, Sequence[str]]] = None,
    ins: Optional[Mapping[str, AssetIn]] = None,
    deps: Optional[Iterable[CoercibleToAssetDep]] = None,
    metadata: Optional[Mapping[str, Any]] = None,
    tags: Optional[Mapping[str, str]] = None,
    description: Optional[str] = None,
    config_schema: Optional[UserConfigSchema] = None,
    required_resource_keys: Optional[AbstractSet[str]] = None,
    resource_defs: Optional[Mapping[str, object]] = None,
    hooks: Optional[AbstractSet[HookDefinition]] = None,
    io_manager_def: Optional[object] = None,
    io_manager_key: Optional[str] = None,
    dagster_type: Optional[DagsterType] = None,
    partitions_def: Optional[PartitionsDefinition] = None,
    op_tags: Optional[Mapping[str, Any]] = None,
    group_name: Optional[str] = None,
    output_required: bool = True,
    automation_condition: Optional[AutomationCondition] = None,
    backfill_policy: Optional[BackfillPolicy] = None,
    retry_policy: Optional[RetryPolicy] = None,
    code_version: Optional[str] = None,
    key: Optional[CoercibleToAssetKey] = None,
    check_specs: Optional[Sequence[AssetCheckSpec]] = None,
    owners: Optional[Sequence[str]] = None,
    kinds: Optional[AbstractSet[str]] = None,
    pool: Optional[str] = None,
) -> AssetsDefinition

Parameters

name
Optional[str]
The name of the asset. If not provided, defaults to the name of the decorated function. Must be a valid Dagster name (only contains letters, numbers, and underscore).
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. Each item must be a valid Dagster name.
ins
Optional[Mapping[str, AssetIn]]
A dictionary that maps input names to information about the input.
deps
Optional[Iterable[CoercibleToAssetDep]]
The assets that are upstream dependencies, but do not correspond to a parameter of the decorated function. Can include AssetDep, AssetsDefinition, SourceAsset, AssetKey, or str.
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.
description
Optional[str]
A human-readable description of the asset.
config_schema
Optional[ConfigSchema]
The configuration schema for the asset’s underlying op. If set, Dagster will check that config provided matches this schema.
required_resource_keys
Optional[Set[str]]
Set of resource handles required by the op.
resource_defs
Optional[Mapping[str, object]]
A mapping of resource keys to resources. These resources will be initialized during execution and can be accessed from the context.
hooks
Optional[AbstractSet[HookDefinition]]
A set of hooks to attach to the asset. These hooks will be executed when the asset is materialized.
io_manager_def
Optional[object]
The IOManager used for storing the output of the op as an asset. Only one of io_manager_def and io_manager_key can be provided.
io_manager_key
Optional[str]
The resource key of the IOManager used for storing the output. Defaults to “io_manager”.
dagster_type
Optional[DagsterType]
Allows specifying type validation functions that will be executed on the output 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. Values that are not strings will be JSON encoded.
group_name
Optional[str]
A string name used to organize multiple assets into groups. If not provided, defaults to “default”.
output_required
bool
default:"True"
Whether the decorated function will always materialize an asset. If False, the function can conditionally not yield a result.
automation_condition
Optional[AutomationCondition]
A condition describing when Dagster should materialize this asset.
backfill_policy
Optional[BackfillPolicy]
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. Should be set only for code that deterministically produces the same output when given the same inputs.
key
Optional[CoercibleToAssetKey]
The key for this asset. If provided, cannot specify key_prefix or name.
check_specs
Optional[Sequence[AssetCheckSpec]]
Specs for asset checks that execute in the decorated function after materializing 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.
pool
Optional[str]
A string that identifies the concurrency pool that governs this asset’s execution.

Returns

Type: AssetsDefinition An assets definition object representing the software-defined asset.

Examples

Basic Asset

from dagster import asset

@asset
def my_asset():
    return [1, 2, 3]

Asset with Dependencies

import pandas as pd
from dagster import asset

@asset
def upstream_asset():
    return pd.DataFrame({"column": [1, 2, 3]})

@asset
def downstream_asset(upstream_asset: pd.DataFrame) -> pd.DataFrame:
    # upstream_asset is automatically loaded and passed as an argument
    return upstream_asset * 2

Asset with Non-Argument Dependencies

from dagster import asset

@asset
def source_data():
    return {"data": [1, 2, 3]}

@asset(deps=[source_data])  # depends on source_data but doesn't load it
def processed_data():
    # Load the data yourself if needed
    return {"processed": True}

Asset with Configuration

from dagster import asset, Config
from pydantic import Field

class MyAssetConfig(Config):
    api_key: str = Field(description="API key for external service")
    num_records: int = Field(default=100)

@asset
def configurable_asset(config: MyAssetConfig):
    # Access config.api_key and config.num_records
    return f"Fetched {config.num_records} records"

Conditional Asset Materialization

from dagster import asset, Output

should_materialize = True

@asset(output_required=False)
def conditional_asset():
    if should_materialize:
        yield Output(5)  # must use yield, not return
    # Otherwise, nothing is materialized

@asset
def downstream_asset(conditional_asset):
    # Only runs if conditional_asset materialized
    return conditional_asset + 1

Asset with Metadata and Tags

from dagster import asset, MetadataValue

@asset(
    metadata={
        "owner": "[email protected]",
        "priority": "high",
    },
    tags={"domain": "finance", "pii": "true"},
    group_name="finance_assets",
    owners=["team:data-platform"],
)
def important_asset():
    return calculate_metrics()

Partitioned Asset

from dagster import asset, DailyPartitionsDefinition

@asset(
    partitions_def=DailyPartitionsDefinition(start_date="2024-01-01")
)
def daily_sales(context):
    partition_date = context.partition_key
    return load_sales_for_date(partition_date)

Build docs developers (and LLMs) love