diff --git a/python_modules/dagster/dagster/_core/definitions/decorators/asset_check_decorator.py b/python_modules/dagster/dagster/_core/definitions/decorators/asset_check_decorator.py index e7a42b83222c9..f8b87b0bf7a70 100644 --- a/python_modules/dagster/dagster/_core/definitions/decorators/asset_check_decorator.py +++ b/python_modules/dagster/dagster/_core/definitions/decorators/asset_check_decorator.py @@ -24,7 +24,6 @@ from dagster._core.definitions.events import AssetKey, CoercibleToAssetKey from dagster._core.definitions.output import Out from dagster._core.definitions.policy import RetryPolicy -from dagster._core.definitions.resource_annotation import get_resource_args from dagster._core.definitions.source_asset import SourceAsset from dagster._core.errors import DagsterInvalidDefinitionError from dagster._core.execution.build_resources import wrap_resources_for_execution @@ -325,8 +324,9 @@ def checks(): def inner(fn: MultiAssetCheckFunction) -> AssetChecksDefinition: op_name = name or fn.__name__ - arg_resource_keys = {arg.name for arg in get_resource_args(fn)} - op_required_resource_keys = required_resource_keys - arg_resource_keys + op_required_resource_keys = compute_required_resource_keys( + required_resource_keys, resource_defs, fn=fn, decorator_name="@multi_asset_check" + ) outs = { spec.get_python_identifier(): Out(None, is_required=not can_subset) for spec in specs