You are viewing an unreleased or outdated version of the documentation

Asset Checks (Experimental)

Dagster allows you to define and execute checks on your software-defined assets. Each asset check verifies some property of a data asset, e.g. that is has no null values in a particular column.

@dagster.asset_check(*, asset=None, name=None, description=None, required_resource_keys=None, resource_defs=None, config_schema=None, compute_kind=None, op_tags=None, retry_policy=None, severity=AssetCheckSeverity.WARN)[source]

Create a definition for how to execute an asset check.

Parameters:
  • asset (Optional[Union[AssetKey, Sequence[str], str, AssetsDefinition, SourceAsset]]) – The asset that the check applies to.

  • name (Optional[str]) – The name of the check. If not specified, the name of the decorated function will be used. Checks for the same asset must have unique names.

  • description (Optional[str]) – The description of the check.

  • required_resource_keys (Optional[Set[str]]) – A set of keys for resources that are required by the function that execute the check. These can alternatively be specified by including resource-typed parameters in the function signature.

  • config_schema (Optional[ConfigSchema) – The configuration schema for the check’s underlying op. If set, Dagster will check that config provided for the op matches this schema and fail if it does not. If not set, Dagster will accept any config provided for the op.

  • op_tags (Optional[Dict[str, Any]]) – A dictionary of tags for the op that executes the check. Frameworks may expect and require certain metadata to be attached to a op. Values that are not strings will be json encoded and must meet the criteria that json.loads(json.dumps(value)) == value.

  • compute_kind (Optional[str]) – A string to represent the kind of computation that executes the check, e.g. “dbt” or “spark”.

  • retry_policy (Optional[RetryPolicy]) – The retry policy for the op that executes the check.

  • severity (AssetCheckSeverity) – Severity of the check. Defaults to WARN.

Produces an AssetChecksDefinition object.

Example

from dagster import asset, asset_check, AssetCheckResult

@asset
def my_asset() -> None:
    ...

@asset_check(asset=my_asset, description="Check that my asset has enough rows")
def my_asset_has_enough_rows() -> AssetCheckResult:
    num_rows = ...
    return AssetCheckResult(success=num_rows > 5, metadata={"num_rows": num_rows})
Example with a DataFrame Output:
from dagster import asset, asset_check, AssetCheckResult
from pandas import DataFrame

@asset
def my_asset() -> DataFrame:
    ...

@asset_check(description="Check that my asset has enough rows")
def my_asset_has_enough_rows(my_asset: DataFrame) -> AssetCheckResult:
    num_rows = my_asset.shape[0]
    return AssetCheckResult(success=num_rows > 5, metadata={"num_rows": num_rows})
class dagster.AssetCheckResult(*, success, asset_key=None, check_name=None, metadata=None)[source]

The result of an asset check.

asset_key

The asset key that was checked.

Type:

Optional[AssetKey]

check_name

The name of the check.

Type:

Optional[str]

success

The pass/fail result of the check.

Type:

bool

metadata

Arbitrary metadata about the asset. Keys are displayed string labels, and values are one of the following: string, float, int, JSON-serializable dict, JSON-serializable list, and one of the data classes returned by a MetadataValue static method.

Type:

Optional[Dict[str, RawMetadataValue]]

class dagster.AssetCheckSpec(name, *, asset_key, description=None, severity=AssetCheckSeverity.WARN)[source]

Defines information about an check, except how to execute it.

AssetCheckSpec is often used as an argument to decorators that decorator a function that can execute multiple checks - e.g. @asset, and @multi_asset. It defines one of the checks that will be executed inside that function.

Parameters:
  • name (str) – Name of the check.

  • asset_key (AssetKey) – The key of the asset that the check applies to.

  • description (Optional[str]) – Description for the check.

  • severity (AssetCheckSeverity) – Severity of the check. Defaults to WARN

class dagster.AssetCheckSeverity(value)[source]

Severity level for an asset check.

Severities:

  • WARN: If the check fails, don’t fail the step.

  • ERROR: If the check fails, fail the step and, within the run, skip materialization of any assets that are downstream of the asset being checked.