DagsterDocs

Solid Events and Exceptions#

Within the body of a solid, it is possible to communicate with the Dagster framework either by yielding an event, or raising an exception. This page describes these different possibilities and the scenarios in which you might use them.

Relevant APIs#

NameDescription
OutputDagster event used to yield an output from a solid
AssetMaterializationDagster event indicating that a solid has materialized an asset.
ExpectationResultDagster event representing the result of a data quality check
FailureDagster exception indicating that a failure has occurred
RetryRequestedDagster exception requesting the step to be retried

Overview#

Within the body of a solid, the solid can yield a stream of structured events, consisting of several different types. These events will be processed by Dagster and recorded in the event log, along with some additional context about the solid that emitted it.

It is also possible to raise Dagster-specific exceptions, which will indicate to the framework to halt the solid execution entirely and perform some action.

Event Metadata#

Often, it may be useful to attach some arbitrary information to an event or exception that is not captured by its basic parameters. Through the EventMetadataEntry object, we provide a consistent interface for specifying this metadata on a variety of events. Depending on the type of the data, these entries will be rendered in Dagit in a more useful format than a simple unstructured string.

The AssetMaterialization, ExpectationResult, and Failure objects each accept a metadata_entries parameter, which is simply a list of EventMetadataEntry. Output also accepts this parameter, although this functionality is currently experimental may change in the future.

We provide support for a wide variety of potentially useful metadata types, including simple datatypes (EventMetadataEntry.float, EventMetadataEntry.int, EventMetadataEntry.text), as well as more complex information such as markdown and json (EventMetadataEntry.md, EventMetadataEntry.json).

Check out the API Docs for EventMetadataEntry for more details.

Events#

Yielding events from within the body of a solid is a useful way of communicating with the Dagster framework. The most critical event to the functionality of Dagster is the Output event, which allows output data to be passed on from one solid to the next. However, we also provide interfaces to inform Dagster about external assets and data quality checks during the run of a solid.

Outputs#

Because returning a value from a solid is such a fundamental part of creating a data pipeline, we have a few different interfaces for this functionality, to help ease transition into writing Dagster-specific code.

For solids with a single output, you can simply return a value directly from the compute_fn. Internally, this will be converted to a Dagster Output event with the default output name result:

@solid
def my_simple_return_solid(context):
    return 1

While this is perhaps the most intuitive way to return a value from a function, once you have multiple outputs defined on your solid, or want to yield additional, non-output information from the body of your solid, explicitly returning a value is no longer an option. In these cases, you'll want to explicitly yield Output events. With that in mind, the above example can be converted to the equivalent yield pattern like so:

@solid
def my_simple_yield_solid(context):
    yield Output(1)

or, if you have a specific output name other than the default result:

@solid(
    output_defs=[
        OutputDefinition(name="my_output"),
    ]
)
def my_named_yield_solid(context):
    yield Output(1, output_name="my_output")

Check out the docs on Solid Outputs to learn more.

Attaching Metadata to Outputs
Experimental
#

If there is information specific to an Output that you would like to log, you may optionally represent that by passing in a list of EventMetadataEntry to the metadata_entries parameter.

The following example demonstrates how you might use this functionality:

@solid
def my_metadata_output(context):
    df = get_some_data()
    yield Output(
        df,
        metadata_entries=[
            EventMetadataEntry.text("Text-based metadata for this event", label="text_metadata"),
            EventMetadataEntry.url("http://mycoolsite.com/url_for_my_data", label="dashboard_url"),
            EventMetadataEntry.int(len(df), "row count"),
            EventMetadataEntry.float(calculate_bytes(df), "size (bytes)"),
        ],
    )

Asset Materializations#

AssetMaterialization events tell Dagster that you have written some data asset to an external system. The classic example would be writing to a table in a database, but really any sort of persisted object that you would want to keep track of can be considered an asset.

Generally, you'd want to send this event directly after you persist the asset to your external system. All AssetMaterialization events must define an asset_key, which is a unique identifier to describe the asset you are persisting.

@solid
def my_asset_solid(context):
    df = get_some_data()
    store_to_s3(df)
    yield AssetMaterialization(
        asset_key="s3.my_asset",
        description="A df I stored in s3",
    )

    result = do_some_transform(df)
    yield Output(result)

To learn more about assets and how they are surfaced once you send this event, check out the Asset Catalog documentation.

Attaching Metadata to Asset Materializations#

Attaching metadata to Asset Materializations is an important way of tracking aspects of a given asset over time. This functions essentially identically to other events which accept a metadata_entries parameter, allowing you to attach a list of EventMetadataEntry.

@solid
def my_metadata_materialization_solid(context):
    df = read_df()
    remote_storage_path = persist_to_storage(df)
    yield AssetMaterialization(
        asset_key="my_dataset",
        description="Persisted result to storage",
        metadata_entries=[
            EventMetadataEntry.text("Text-based metadata for this event", label="text_metadata"),
            EventMetadataEntry.fspath(remote_storage_path),
            EventMetadataEntry.url("http://mycoolsite.com/url_for_my_data", label="dashboard_url"),
            EventMetadataEntry.float(calculate_bytes(df), "size (bytes)"),
        ],
    )
    yield Output(remote_storage_path)

Expectation Results#

Solids can emit structured events to represent the results of a data quality test. The data quality event class is the ExpectationResult. To generate an expectation result, we can yield an ExpectationResult event in our solid.

@solid
def my_expectation_solid(context, df):
    do_some_transform(df)
    yield ExpectationResult(success=len(df) > 0, description="ensure dataframe has rows")
    yield Output(df)

Attaching Metadata to Expectation Results#

Like many other event types in Dagster, there are a variety of types of metadata that can be associated with an expectation result event, all through the EventMetadataEntry class. Each expectation event optionally takes a list of metadata entries that are then displayed in the event log.

This example shows metadata entries of different types attached to the same expectation result:

@solid
def my_metadata_expectation_solid(context, df):
    df = do_some_transform(df)
    yield ExpectationResult(
        success=len(df) > 0,
        description="ensure dataframe has rows",
        metadata_entries=[
            EventMetadataEntry.text("Text-based metadata for this event", label="text_metadata"),
            EventMetadataEntry.url("http://mycoolsite.com/url_for_my_data", label="dashboard_url"),
            EventMetadataEntry.int(len(df), "row count"),
            EventMetadataEntry.float(calculate_bytes(df), "size (bytes)"),
        ],
    )
    yield Output(df)

Exceptions#

Dagster also provides some solid-specific exception classes, which can be raised to halt the execution of a solid. The behavior after an exception is raised depends on the exception that you use. The exceptions are documented below.

Failures#

Failure exceptions may be raised to indicate that a solid has failed in an unrecoverable way, and that execution should stop. This may be useful if, in the body of your solid, you detect an issue that would not raise an error on its own, but would still cause problems if the execution were to continue.
@solid
def my_failure_solid(_):
    path = "/path/to/files"
    my_files = get_files(path)
    if len(my_files) == 0:
        raise Failure("No files to process")
    return some_calculation(my_files)

Attaching Metadata to Failures#

As is the case with many Dagster-provided classes, you can attach a list EventMetadataEntry denoting an arbitrary set of metadata relevant to the failure.

@solid
def my_failure_metadata_solid(_):
    path = "/path/to/files"
    my_files = get_files(path)
    if len(my_files) == 0:
        raise Failure(
            description="No files to process",
            metadata_entries=[
                EventMetadataEntry.fspath(path, label="filepath"),
                EventMetadataEntry.url("http://mycoolsite.com/failures", label="dashboard_url"),
            ],
        )
    return some_calculation(my_files)

Retry Requests#

RetryRequested exceptions are useful when you experience failures that are possible to recover from. For example, if you have a flaky operation that you expect to throw an exception once in a while, you can catch the exception and throw a RetryRequested to make Dagster halt and re-start solid execution.

You can configure the number of retries to be attempted with the max_retries parameter.

@solid
def my_retry_solid(_):
    try:
        result = flaky_operation()
    except:
        raise RetryRequested(max_retries=3)
    return result