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, name=None, description=None, required_resource_keys=None, resource_defs=None, config_schema=None, compute_kind=None, op_tags=None, retry_policy=None)[source]

experimental This API may break in future versions, even between dot releases.

Create a definition for how to execute an asset check.

Parameters:
  • asset (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.

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(passed=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(asset=my_asset, 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(passed=num_rows > 5, metadata={"num_rows": num_rows})
class dagster.AssetCheckResult(*, passed, asset_key=None, check_name=None, metadata=None, severity=AssetCheckSeverity.ERROR)[source]

experimental This API may break in future versions, even between dot releases.

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]

passed

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]]

severity

Severity of the check. Defaults to ERROR.

Type:

AssetCheckSeverity

class dagster.AssetCheckSpec(name, *, asset, description=None)[source]

experimental This API may break in future versions, even between dot releases.

Defines information about an asset 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 (Union[AssetKey, Sequence[str], str, AssetsDefinition, SourceAsset]) – The asset that the check applies to.

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

class dagster.AssetCheckSeverity(value)[source]

experimental This API may break in future versions, even between dot releases.

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.

class dagster.AssetCheckKey(asset_key, name)[source]

experimental This API may break in future versions, even between dot releases.

Check names are expected to be unique per-asset. Thus, this combination of asset key and check name uniquely identifies an asset check within a deployment.

dagster.load_asset_checks_from_modules(modules, asset_key_prefix=None)[source]

Constructs a list of asset checks from the given modules. This is most often used in conjunction with a call to load_assets_from_modules.

Parameters:
  • modules (Iterable[ModuleType]) – The Python modules to look for checks inside.

  • asset_key_prefix (Optional[Union[str, Sequence[str]]]) – The prefix for the asset keys targeted by the loaded checks. This should match the key_prefix argument to load_assets_from_modules.

Returns:

A list containing asset checks defined in the given modules.

Return type:

Sequence[AssetChecksDefinition]

dagster.load_asset_checks_from_current_module(asset_key_prefix=None)[source]

Constructs a list of asset checks from the module where this function is called. This is most often used in conjunction with a call to load_assets_from_current_module.

Parameters:

asset_key_prefix (Optional[Union[str, Sequence[str]]]) – The prefix for the asset keys targeted by the loaded checks. This should match the key_prefix argument to load_assets_from_current_module.

Returns:

A list containing asset checks defined in the current module.

Return type:

Sequence[AssetChecksDefinition]

dagster.load_asset_checks_from_package_module(package_module, asset_key_prefix=None)[source]

Constructs a list of asset checks from all sub-modules of the given package module. This is most often used in conjunction with a call to load_assets_from_package_module.

Parameters:
  • package_module (ModuleType) – The Python module to look for checks inside.

  • asset_key_prefix (Optional[Union[str, Sequence[str]]]) – The prefix for the asset keys targeted by the loaded checks. This should match the key_prefix argument to load_assets_from_package_module.

Returns:

A list containing asset checks defined in the package.

Return type:

Sequence[AssetChecksDefinition]

dagster.load_asset_checks_from_package_name(package_name, asset_key_prefix=None)[source]

Constructs a list of asset checks from all sub-modules of the given package. This is most often used in conjunction with a call to load_assets_from_package_name.

Parameters:
  • package_name (str) – The name of the Python package to look for checks inside.

  • asset_key_prefix (Optional[Union[str, Sequence[str]]]) – The prefix for the asset keys targeted by the loaded checks. This should match the key_prefix argument to load_assets_from_package_name.

Returns:

A list containing asset checks defined in the package.

Return type:

Sequence[AssetChecksDefinition]

class dagster.AssetChecksDefinition(*, node_def, resource_defs, specs, input_output_props)[source]

experimental This API may break in future versions, even between dot releases.

Defines a set of checks that are produced by the same op or op graph.

AssetChecksDefinition are typically not instantiated directly, but rather produced using a decorator like @asset_check.

property asset_key
property description
property name
property node_def

The op or op graph that can be executed to check the assets.

property required_resource_keys

The set of keys for resources that must be provided to this AssetsDefinition.

Type:

Set[str]

property spec
property specs