From 3234086dd2492186d36b3c3aa0d162b3d3c3d82a Mon Sep 17 00:00:00 2001 From: Marijn Valk Date: Tue, 15 Oct 2024 18:36:34 +0200 Subject: [PATCH] Recreate `AssetChecksDefinition`s after setting prefix (#25271) ## Summary & Motivation This makes sure that the `load_asset_checks_from_*` functions return `[AssetChecksDefinition]` by replacing the typecast with the creation of an actual `AssetChecksDefinition`. Alternatively, this issue could be addressed by adding a specialized `with_attributes()` method to `AssetChecksDefinition` to make it return the correct type. I'd be happy to try this path as well! But it feels like this change would be sufficient and minimally impactful across the codebase. Closes https://github.com/dagster-io/dagster/issues/22064 ## How I Tested These Changes Added an `isinstance` check to the `load_asset_checks_from_*` tests --- .../definitions/load_asset_checks_from_modules.py | 12 ++++++++++-- .../asset_check_tests/test_load_from_modules.py | 5 +++++ 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/python_modules/dagster/dagster/_core/definitions/load_asset_checks_from_modules.py b/python_modules/dagster/dagster/_core/definitions/load_asset_checks_from_modules.py index b036c2b39b17d..5a5d3ec99af40 100644 --- a/python_modules/dagster/dagster/_core/definitions/load_asset_checks_from_modules.py +++ b/python_modules/dagster/dagster/_core/definitions/load_asset_checks_from_modules.py @@ -32,10 +32,18 @@ def _checks_with_attributes( checks_defs: Sequence[AssetChecksDefinition], asset_key_prefix: Optional[CoercibleToAssetKeyPrefix] = None, ) -> Sequence[AssetChecksDefinition]: - modified_checks = [] if asset_key_prefix: modified_checks, _ = prefix_assets(checks_defs, asset_key_prefix, [], None) - return cast(Sequence[AssetChecksDefinition], modified_checks) + return [ + AssetChecksDefinition.create( + keys_by_input_name=c.keys_by_input_name, + node_def=c.op, + check_specs_by_output_name=c.check_specs_by_output_name, + resource_defs=c.resource_defs, + can_subset=c.can_subset, + ) + for c in modified_checks + ] else: return checks_defs diff --git a/python_modules/dagster/dagster_tests/definitions_tests/asset_check_tests/test_load_from_modules.py b/python_modules/dagster/dagster_tests/definitions_tests/asset_check_tests/test_load_from_modules.py index 319558ced4adc..d3176efa6e449 100644 --- a/python_modules/dagster/dagster_tests/definitions_tests/asset_check_tests/test_load_from_modules.py +++ b/python_modules/dagster/dagster_tests/definitions_tests/asset_check_tests/test_load_from_modules.py @@ -1,5 +1,6 @@ import pytest from dagster import ( + AssetChecksDefinition, AssetKey, Definitions, asset_check, @@ -23,6 +24,7 @@ def test_load_asset_checks_from_modules(): checks = load_asset_checks_from_modules([checks_module]) assert len(checks) == 1 + assert all(isinstance(check, AssetChecksDefinition) for check in checks) asset_check_1_key = next(iter(asset_check_1.check_keys)) @@ -49,6 +51,7 @@ def test_load_asset_checks_from_modules_prefix(): checks = load_asset_checks_from_modules([checks_module], asset_key_prefix="foo") assert len(checks) == 1 + assert all(isinstance(check, AssetChecksDefinition) for check in checks) check_key = next(iter(checks[0].check_keys)) assert check_key.asset_key == AssetKey(["foo", "asset_1"]) @@ -76,6 +79,7 @@ def check_in_current_module(): def test_load_asset_checks_from_current_module(): checks = load_asset_checks_from_current_module(asset_key_prefix="foo") assert len(checks) == 1 + assert all(isinstance(check, AssetChecksDefinition) for check in checks) check_key = next(iter(checks[0].check_keys)) assert check_key.name == "check_in_current_module" assert check_key.asset_key == AssetKey(["foo", "asset_1"]) @@ -100,6 +104,7 @@ def test_load_asset_checks_from_package(load_fns): checks = checks_load_fn(checks_module, asset_key_prefix="foo") assert len(checks) == 2 + assert all(isinstance(check, AssetChecksDefinition) for check in checks) check_key_0 = next(iter(checks[0].check_keys)) assert check_key_0.name == "asset_check_1" assert check_key_0.asset_key == AssetKey(["foo", "asset_1"])