Assets and Materializations
Dagster solids often produce persistent assets which represent a result of some computation. In
Dagster pipelines, these assets are often by-products or side-effects of a solid's computation,
distinct from the Output
returned by a solid. Dagster solids can describe these assets by
yielding an AssetMaterialization
event and attaching useful metadata.
AssetMaterialization events include an AssetKey
, which serves as an organizational index across
pipelines and runs. These materialized assets are represented in dagit
under the Assets
tab and
cross-reference the pipelines and runs that generate them. This is useful not only for quickly
viewing the status of an asset, but also for better understanding the relationship over time between
the asset and the computation that created it.
This example sets up a pipeline that materializes an asset annotated with metadata, and builds that asset view in Dagit.
from dagster import (
AssetKey,
AssetMaterialization,
EventMetadataEntry,
Field,
Output,
pipeline,
repository,
solid,
)
@solid(config_schema={"num": Field(float, is_required=False, default_value=1.0)})
def source_float(context):
return context.solid_config["num"]
@solid
def add_one_and_materialize(_, num):
result = num + 1
yield AssetMaterialization(
description="Analytics dashboard for example pipeline",
asset_key=AssetKey(["dashboards", "analytics_dashboard"]),
metadata_entries=[
EventMetadataEntry.url(
"http://mycoolwebsite.com/dashboards/analytics", "dashboard url"
),
EventMetadataEntry.float(result, "numeric value"),
],
)
# Because we are yielding a materialization event as well as an output, we need to explicitly
# yield an `Output` instead of relying on the return value of the solid
yield Output(result)
@pipeline
def materialization_pipeline():
add_one_and_materialize(source_float())
@repository
def materializations_example_repo():
return [materialization_pipeline]
Assets in Dagit¶
The Assets
tab in Dagit is only enabled for instances configured with an "asset-aware"
event log storage, such as PostgresEventLogStorage
and ConsolidatedSqliteEventLogStorage
. Here
is an example of an asset-aware instance configuration.
run_launcher:
module: dagster.core.launcher
class: DefaultRunLauncher
event_log_storage:
module: dagster.core.storage.event_log
class: ConsolidatedSqliteEventLogStorage
config:
base_dir: "/var/shared/dagster_home"
Please refer to our Instance Documentation for more information about configuring your Dagster instance.
Once the Dagster instance is configured with an asset-aware event log storage, after you run a
pipeline which generates an asset, you'll find the asset listed on the Assets
tab in Dagit.
You can also find the details about it by clicking the asset key of it.
Open in a playground¶
Download¶
curl https://codeload.github.com/dagster-io/dagster/tar.gz/master | tar -xz --strip=2 dagster-master/examples/materializations
cd materializations