Ask AI

Asset metadata#

Attaching metadata to assets can help make your pipelines easier for you and other team members to understand. Data about your data assets can be attached to both asset definitions and materializations.

By the end of this guide, you'll understand how to attach metadata to assets and view that metadata in the Dagster UI.


How it works#

There are two main types of metadata in Dagster:

  • Definition metadata is information that's fixed or doesn't frequently change. For example, definition metadata could be the storage location of a table, a link the asset's definition in GitHub, or who owns the asset.
  • Runtime, or materialization metadata is information that changes after a materialization occurs. This could be how many records were processed or how long an asset took to materialize.

How metadata is attached to an asset depends on the type of metadata being attached. Refer to the following sections for more details.


Attaching definition metadata#

Dagster supports attaching a few different types of definition metadata:

Arbitrary metadata using the metadata parameter#

Attaching arbitrary metadata to an asset is done using the metadata argument and a dictionary of key/value pairs. Keys must be a string, but values can:

  • Be any of the MetadataValue classes provided by Dagster
  • Primitive Python types, which Dagster will convert to the appropriate MetadataValue

In the following example, we attached the storage location of the table produced by the asset:

from dagster_duckdb import DuckDBResource

from dagster import asset

# ... other assets


@asset(
    deps=[iris_dataset],
    metadata={"dataset_name": "iris.small_petals"},
)
def small_petals(duckdb: DuckDBResource) -> None:
    with duckdb.get_connection() as conn:
        conn.execute(
            "CREATE TABLE iris.small_petals AS SELECT * FROM iris.iris_dataset WHERE"
            " 'petal_length_cm' < 1 AND 'petal_width_cm' < 1"
        )

Asset owners#

Did you know? If using Dagster+ Pro, you can create asset-based alerts that will automatically notify an asset's owners when triggered. Refer to the Dagster+ alert documentation for more information.

An asset can have multiple owners, defined using the owners argument on the @asset decorator. This argument accepts a dictionary of owners, where each value is either an individual email address or a team. Teams must include a team: prefix; for example: team:data-eng.

The asset in the following example has two owners: richard.hendricks@hooli.com and the data-eng team.

from dagster import asset


@asset(owners=["richard.hendricks@hooli.com", "team:data-eng"])
def leads(): ...

Attaching materialization metadata#

Attaching materialization metadata to an asset is accomplished by returning a MaterializeResult object containing a metadata parameter. This parameter accepts a dictionary of key/value pairs, where keys must be a string.

When specifying values, use the MetadataValue utility class to wrap the data, ensuring it displays correctly in the UI. Values can also be primitive Python types, which Dagster will convert to the appropriate MetadataValue.

Arbitrary metadata#

In the following example, we added a row count and preview to a topstories asset:

import json
import requests
import pandas as pd
from dagster import AssetExecutionContext, MetadataValue, asset, MaterializeResult


@asset(deps=[topstory_ids])
def topstories(context: AssetExecutionContext) -> MaterializeResult:
    with open("data/topstory_ids.json", "r") as f:
        topstory_ids = json.load(f)

    results = []
    for item_id in topstory_ids:
        item = requests.get(
            f"https://hacker-news.firebaseio.com/v0/item/{item_id}.json"
        ).json()
        results.append(item)

        if len(results) % 20 == 0:
            context.log.info(f"Got {len(results)} items so far.")

    df = pd.DataFrame(results)
    df.to_csv("data/topstories.csv")

    return MaterializeResult(
        metadata={
            "num_records": len(df),  # Metadata can be any key-value pair
            "preview": MetadataValue.md(df.head().to_markdown()),
            # The `MetadataValue` class has useful static methods to build Metadata
        }
    )

Column-level lineage#

For assets that produce database tables, column-level lineage can be a powerful tool for improving collaboration and debugging issues. Column lineage enables data and analytics engineers alike to understand how a column is created and used in your data platform. Refer to the Column-level lineage documentation for more information.


Viewing asset metadata in the Dagster UI#

Metadata attached to assets shows up in a few places in the Dagster UI.

Global asset lineage#

In the Global asset lineage page, click on an asset to open the asset details in the sidepanel:

Asset details sidepanel showing metadata in the Global asset lineage page of the Dagster UI

If materialization metadata is numeric, it will display as a plot in the Metadata plots section of the sidepanel.


APIs in this guide#

NameDescription
@assetA decorator used to define assets.
MaterializeResultAn object representing a successful materialization of an asset.
MetadataValueUtility class to wrap metadata values passed into Dagster events, which allows them to be displayed in the Dagster UI and other tooling.