diff --git a/python_modules/dagster/dagster/_core/definitions/decorators/asset_decorator.py b/python_modules/dagster/dagster/_core/definitions/decorators/asset_decorator.py index c11a0d8ab45d9..eb59fa194f6c7 100644 --- a/python_modules/dagster/dagster/_core/definitions/decorators/asset_decorator.py +++ b/python_modules/dagster/dagster/_core/definitions/decorators/asset_decorator.py @@ -35,7 +35,7 @@ from ..asset_out import AssetOut from ..asset_spec import AssetSpec from ..assets import AssetsDefinition -from ..backfill_policy import BackfillPolicy, BackfillPolicyType +from ..backfill_policy import BackfillPolicy from ..decorators.graph_decorator import graph from ..events import AssetKey, CoercibleToAssetKey, CoercibleToAssetKeyPrefix from ..input import GraphIn @@ -390,19 +390,6 @@ def create_assets_def_from_fn_and_decorator_args( out_asset_key=out_asset_key, ) - with disable_dagster_warnings(): - # check backfill policy is BackfillPolicyType.SINGLE_RUN for non-partitioned asset - if args.partitions_def is None: - check.param_invariant( - ( - args.backfill_policy.policy_type is BackfillPolicyType.SINGLE_RUN - if args.backfill_policy - else True - ), - "backfill_policy", - "Non partitioned asset can only have single run backfill policy", - ) - with disable_dagster_warnings(): builder_args = DecoratorAssetsDefinitionBuilderArgs( name=args.name, diff --git a/python_modules/dagster/dagster/_core/definitions/decorators/decorator_assets_definition_builder.py b/python_modules/dagster/dagster/_core/definitions/decorators/decorator_assets_definition_builder.py index d866e5f9b03d2..02374c2535afb 100644 --- a/python_modules/dagster/dagster/_core/definitions/decorators/decorator_assets_definition_builder.py +++ b/python_modules/dagster/dagster/_core/definitions/decorators/decorator_assets_definition_builder.py @@ -31,7 +31,7 @@ AssetsDefinition, get_partition_mappings_from_deps, ) -from dagster._core.definitions.backfill_policy import BackfillPolicy +from dagster._core.definitions.backfill_policy import BackfillPolicy, BackfillPolicyType from dagster._core.definitions.input import In from dagster._core.definitions.op_definition import OpDefinition from dagster._core.definitions.output import Out @@ -268,6 +268,17 @@ def __init__( else named_ins_by_asset_key ) + if args.partitions_def is None: + check.param_invariant( + ( + args.backfill_policy.policy_type is BackfillPolicyType.SINGLE_RUN + if args.backfill_policy + else True + ), + "backfill_policy", + "Non partitioned asset can only have single run backfill policy", + ) + @staticmethod def for_multi_asset( *, fn: Callable[..., Any], args: DecoratorAssetsDefinitionBuilderArgs