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 7cb4529997eb9..3e0837265645d 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 @@ -11,13 +11,11 @@ from dagster._core.definitions.asset_in import AssetIn from dagster._core.definitions.assets import AssetsDefinition 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.source_asset import SourceAsset from dagster._core.definitions.utils import DEFAULT_OUTPUT from dagster._core.errors import DagsterInvalidDefinitionError from dagster._core.execution.build_resources import wrap_resources_for_execution -from dagster._core.storage.tags import COMPUTE_KIND_TAG from dagster._utils.warnings import disable_dagster_warnings from .asset_decorator import make_asset_deps @@ -29,7 +27,6 @@ create_check_specs_by_output_name, get_function_params_without_context_or_config_or_resources, ) -from .op_decorator import _Op AssetCheckFunctionReturn: TypeAlias = AssetCheckResult AssetCheckFunction: TypeAlias = Callable[..., AssetCheckFunctionReturn] @@ -321,9 +318,6 @@ def checks(): def inner(fn: MultiAssetCheckFunction) -> AssetChecksDefinition: op_name = name or fn.__name__ - outs = { - spec.get_python_identifier(): Out(None, is_required=not can_subset) for spec in specs - } named_ins_by_asset_key = build_named_ins( fn=fn, asset_ins={}, @@ -334,14 +328,13 @@ def inner(fn: MultiAssetCheckFunction) -> AssetChecksDefinition: builder_args = DecoratorAssetsDefinitionBuilderArgs( decorator_name="@multi_asset_check", name=name, - # @asset_check previous behavior is to not set description on underlying op - op_description=None, + op_description=description, required_resource_keys=required_resource_keys or set(), config_schema=config_schema, retry_policy=retry_policy, specs=[], check_specs_by_output_name=create_check_specs_by_output_name(specs), - can_subset=False, + can_subset=can_subset, compute_kind=compute_kind, op_tags=op_tags, op_def_resource_defs=resource_defs, @@ -368,19 +361,7 @@ def inner(fn: MultiAssetCheckFunction) -> AssetChecksDefinition: ) with disable_dagster_warnings(): - op_def = _Op( - name=op_name, - description=description, - ins=dict(named_ins_by_asset_key.values()), - out=outs, - required_resource_keys=builder.required_resource_keys, - tags={ - **({COMPUTE_KIND_TAG: compute_kind} if compute_kind else {}), - **(op_tags or {}), - }, - config_schema=config_schema, - retry_policy=retry_policy, - )(fn) + op_def = builder.create_op_definition() return AssetChecksDefinition.create( node_def=op_def,