Skip to main content

Metadata Types

Metadata allows you to attach structured information to assets, ops, and events in Dagster. This information is displayed in the Dagster UI and can be used for debugging, monitoring, and documentation.

MetadataValue

Base class for all metadata values. Provides static constructor methods for creating typed metadata values.
from dagster import AssetMaterialization, MetadataValue, asset

@asset
def my_dataset(context):
    df = get_dataframe()
    
    yield AssetMaterialization(
        asset_key="my_dataset",
        metadata={
            "num_rows": MetadataValue.int(len(df)),
            "schema": MetadataValue.md(df.dtypes.to_markdown()),
            "preview_url": MetadataValue.url("https://example.com/preview"),
            "file_path": MetadataValue.path("/data/output.parquet"),
        }
    )
    return df

Static Constructors

text
staticmethod
Creates a text metadata value.Parameters:
  • text (str): The text string for a metadata entry
Returns: TextMetadataValue
metadata = {"description": MetadataValue.text("User activity data")}
url
staticmethod
Creates a URL metadata value that renders as a clickable link in the UI.Parameters:
  • url (str): The URL for a metadata entry
Returns: UrlMetadataValue
metadata = {"dashboard": MetadataValue.url("http://mycoolsite.com/dashboard")}
path
staticmethod
Creates a filesystem path metadata value.Parameters:
  • path (str | PathLike): The path for a metadata entry
Returns: PathMetadataValue
metadata = {"output_file": MetadataValue.path("/data/processed/output.csv")}
notebook
staticmethod
Creates a notebook path metadata value. The UI will render the notebook inline.Parameters:
  • path (str | PathLike): The path to a notebook
Returns: NotebookMetadataValue
metadata = {"analysis": MetadataValue.notebook("/notebooks/analysis.ipynb")}
json
staticmethod
Creates a JSON metadata value. The UI will render the JSON in a structured format.Parameters:
  • data (Sequence[Any] | Mapping[str, Any]): JSON-serializable data
Returns: JsonMetadataValue
metadata = {
    "config": MetadataValue.json({"batch_size": 100, "enabled": True})
}
md
staticmethod
Creates a Markdown metadata value. The UI will render the markdown.Parameters:
  • data (str): Markdown string
Returns: MarkdownMetadataValue
metadata = {
    "report": MetadataValue.md("# Summary\n\n- Total: 1000\n- Errors: 5")
}
python_artifact
staticmethod
Creates a reference to a Python artifact (class or function).Parameters:
  • python_artifact (Callable): The Python class or function
Returns: PythonArtifactMetadataValue
metadata = {
    "model_class": MetadataValue.python_artifact(MyMLModel),
    "transform_fn": MetadataValue.python_artifact(transform_data)
}
float
staticmethod
Creates a float metadata value.Parameters:
  • value (float): The float value
Returns: FloatMetadataValue
metadata = {"accuracy": MetadataValue.float(0.95)}
int
staticmethod
Creates an integer metadata value.Parameters:
  • value (int): The integer value
Returns: IntMetadataValue
metadata = {"num_rows": MetadataValue.int(10000)}
bool
staticmethod
Creates a boolean metadata value.Parameters:
  • value (bool): The boolean value
Returns: BoolMetadataValue
metadata = {"is_valid": MetadataValue.bool(True)}
timestamp
staticmethod
Creates a timestamp metadata value.Parameters:
  • value (float | datetime): Unix timestamp or datetime object (must have timezone)
Returns: TimestampMetadataValue
from datetime import datetime, timezone

metadata = {
    "processed_at": MetadataValue.timestamp(datetime.now(timezone.utc))
}
dagster_run
staticmethod
Creates a reference to a Dagster run.Parameters:
  • run_id (str): The ID of the run
Returns: DagsterRunMetadataValue
metadata = {"source_run": MetadataValue.dagster_run(context.run_id)}
asset
staticmethod
Creates a reference to a Dagster asset. The UI will render a link to the asset.Parameters:
  • asset_key (AssetKey): The asset key
Returns: DagsterAssetMetadataValue
from dagster import AssetKey

metadata = {"source_table": MetadataValue.asset(AssetKey("raw_data"))}
job
staticmethod
Creates a reference to a Dagster job.Parameters:
  • job_name (str): The name of the job
  • location_name (str): The code location name
  • repository_name (str, optional): The repository name
Returns: DagsterJobMetadataValue
metadata = {
    "source_job": MetadataValue.job("etl_job", "production")
}
table
staticmethod
Creates tabular metadata that renders as a table in the UI.Parameters:
  • records (Sequence[TableRecord]): List of table records
  • schema (TableSchema, optional): Optional table schema
Returns: TableMetadataValue
from dagster import TableRecord, TableSchema, TableColumn

metadata = {
    "summary": MetadataValue.table(
        records=[
            TableRecord({"category": "A", "count": 100}),
            TableRecord({"category": "B", "count": 200})
        ],
        schema=TableSchema(
            columns=[
                TableColumn(name="category", type="string"),
                TableColumn(name="count", type="int")
            ]
        )
    )
}
table_schema
staticmethod
Creates table schema metadata.Parameters:
  • schema (TableSchema): The table schema
Returns: TableSchemaMetadataValue
from dagster import TableSchema, TableColumn

schema = TableSchema(
    columns=[
        TableColumn(name="id", type="int"),
        TableColumn(name="name", type="string"),
        TableColumn(name="created_at", type="timestamp")
    ]
)
metadata = {"schema": MetadataValue.table_schema(schema)}
column_lineage
staticmethod
Creates column lineage metadata showing dependencies between columns.Parameters:
  • lineage (TableColumnLineage): The column lineage
Returns: TableColumnLineageMetadataValue
from dagster import TableColumnLineage, TableColumnDep, AssetKey

lineage = TableColumnLineage(
    deps_by_column={
        "full_name": [
            TableColumnDep(AssetKey("users"), "first_name"),
            TableColumnDep(AssetKey("users"), "last_name")
        ]
    }
)
metadata = {"dagster/column_lineage": lineage}
null
staticmethod
Creates a null metadata value.Returns: NullMetadataValue
metadata = {"optional_field": MetadataValue.null()}
pool
staticmethod
Creates a reference to a concurrency pool.Parameters:
  • pool (str): The pool identifier
Returns: PoolMetadataValue
metadata = {"execution_pool": MetadataValue.pool("default")}

Table Metadata Types

TableRecord

Represents one record in a table. Used with MetadataValue.table().
from dagster import TableRecord

record = TableRecord({
    "column1": "value1",
    "column2": 42,
    "column3": 3.14,
    "column4": True
})
data
Mapping[str, str | int | float | bool | None]
required
Dictionary mapping column names to values. Values must be strings, integers, floats, bools, or None.

TableSchema

Representation of a schema for tabular data.
from dagster import TableSchema, TableColumn, TableColumnConstraints, TableConstraints

schema = TableSchema(
    columns=[
        TableColumn(
            name="id",
            type="int",
            constraints=TableColumnConstraints(nullable=False, unique=True)
        ),
        TableColumn(
            name="email",
            type="string",
            description="User email address",
            constraints=TableColumnConstraints(nullable=False, unique=True)
        ),
        TableColumn(name="age", type="int"),
    ],
    constraints=TableConstraints(other=["age >= 0"])
)
columns
Sequence[TableColumn]
required
List of column definitions.
constraints
TableConstraints
Optional table-level constraints.

TableColumn

Descriptor for a table column.
name
str
required
Column name.
type
str
Column type (e.g., “string”, “int”, “float”, “bool”, “timestamp”). Defaults to “string”.
description
str
Optional column description.
constraints
TableColumnConstraints
Column-level constraints.
tags
Mapping[str, str]
Optional tags for filtering or organizing columns.

TableColumnConstraints

Constraints for a table column.
nullable
bool
Whether the column can hold null values. Defaults to True.
unique
bool
Whether all values in the column must be unique. Defaults to False.
other
Sequence[str]
List of arbitrary constraint descriptions (e.g., less than or equal to 100, starts with A).

TableConstraints

Table-level constraints.
other
Sequence[str]
required
Descriptions of arbitrary table-level constraints (e.g., [“column_a > column_b”]).

TableColumnLineage

Represents column-level lineage for an asset.
from dagster import TableColumnLineage, TableColumnDep, AssetKey, asset, MaterializeResult

@asset(deps=[AssetKey("source_a"), AssetKey("source_b")])
def combined_data():
    yield MaterializeResult(
        metadata={
            "dagster/column_lineage": TableColumnLineage(
                deps_by_column={
                    "full_name": [
                        TableColumnDep(AssetKey("source_a"), "first_name"),
                        TableColumnDep(AssetKey("source_a"), "last_name"),
                    ],
                    "total_amount": [
                        TableColumnDep(AssetKey("source_b"), "amount"),
                    ]
                }
            )
        }
    )
deps_by_column
Mapping[str, Sequence[TableColumnDep]]
required
Mapping from output column names to the columns they depend on.

TableColumnDep

Identifier for a column in an asset.
asset_key
AssetKey
required
The asset key containing the column.
column_name
str
required
The name of the column.

Usage Examples

Asset Materialization with Metadata

from dagster import asset, MaterializeResult, MetadataValue
import pandas as pd

@asset
def processed_data():
    df = pd.DataFrame({"a": [1, 2, 3], "b": [4, 5, 6]})
    
    return MaterializeResult(
        metadata={
            "num_rows": len(df),
            "columns": MetadataValue.json(list(df.columns)),
            "sample": MetadataValue.md(df.head().to_markdown()),
            "size_bytes": MetadataValue.float(df.memory_usage(deep=True).sum()),
        }
    )

Op with Metadata

from dagster import op, Output, MetadataValue

@op
def process_file(context):
    file_path = "/data/input.csv"
    result = process(file_path)
    
    yield Output(
        value=result,
        metadata={
            "input_file": MetadataValue.path(file_path),
            "rows_processed": MetadataValue.int(len(result)),
            "processing_time_ms": MetadataValue.float(123.45),
        }
    )

Table Schema Metadata

from dagster import asset, TableSchema, TableColumn, MetadataValue

@asset(
    metadata={
        "schema": MetadataValue.table_schema(
            TableSchema(
                columns=[
                    TableColumn(name="user_id", type="int"),
                    TableColumn(name="email", type="string"),
                    TableColumn(name="created_at", type="timestamp"),
                ]
            )
        )
    }
)
def users_table():
    return query_users()

Automatic Metadata Conversion

Dagster automatically converts simple Python types to metadata values:
from dagster import asset

@asset
def my_asset():
    return MaterializeResult(
        metadata={
            "count": 100,              # Converted to IntMetadataValue
            "rate": 0.95,              # Converted to FloatMetadataValue  
            "description": "text",     # Converted to TextMetadataValue
            "enabled": True,           # Converted to BoolMetadataValue
        }
    )

See Also

Build docs developers (and LLMs) love