From 6edff7496f529dbbe412cd48950e7a8ee3374212 Mon Sep 17 00:00:00 2001 From: Sean Mackesey Date: Tue, 17 Dec 2024 15:48:20 -0500 Subject: [PATCH] [components] Add cache to dagster-dg (#26517) ## Summary & Motivation Initial implementation of a cache for `dagster-dg`. Currently just caches the registered components for a code location. Cache key is (code loc abs path, env hash, type of payload). The environment hash is calculated over the file metadata of `uv.lock` and the content of `pkg.lib` (i.e. where local component types can be defined). ## How I Tested These Changes New unit tests + manual testing (running `dg list component-types` multiple times in the same code loc, observing speedup). --- .../libraries/dagster-dg/dagster_dg/cache.py | 74 +++++++++++++++ .../dagster-dg/dagster_dg/cli/__init__.py | 59 ++++++++++-- .../libraries/dagster-dg/dagster_dg/config.py | 21 +++++ .../dagster-dg/dagster_dg/context.py | 58 ++++++++++-- .../libraries/dagster-dg/dagster_dg/utils.py | 23 +++++ .../cli_tests/test_generate_commands.py | 71 ++++++-------- .../cli_tests/test_info_commands.py | 15 +-- .../cli_tests/test_list_commands.py | 18 ++-- .../dagster-dg/dagster_dg_tests/test_cache.py | 93 +++++++++++++++++++ .../dagster-dg/dagster_dg_tests/utils.py | 21 ++++- 10 files changed, 372 insertions(+), 81 deletions(-) create mode 100644 python_modules/libraries/dagster-dg/dagster_dg/cache.py create mode 100644 python_modules/libraries/dagster-dg/dagster_dg_tests/test_cache.py diff --git a/python_modules/libraries/dagster-dg/dagster_dg/cache.py b/python_modules/libraries/dagster-dg/dagster_dg/cache.py new file mode 100644 index 0000000000000..71afef32046e9 --- /dev/null +++ b/python_modules/libraries/dagster-dg/dagster_dg/cache.py @@ -0,0 +1,74 @@ +import shutil +import sys +from pathlib import Path +from typing import Final, Literal, Optional, Tuple + +from typing_extensions import Self, TypeAlias + +from dagster_dg.config import DgConfig + +_CACHE_CONTAINER_DIR_NAME: Final = "dg-cache" + +CachableDataType: TypeAlias = Literal["component_registry_data"] + + +def get_default_cache_dir() -> Path: + if sys.platform == "win32": + return Path.home() / "AppData" / "dg" / "cache" + elif sys.platform == "darwin": + return Path.home() / "Library" / "Caches" / "dg" + else: + return Path.home() / ".cache" / "dg" + + +class DgCache: + @classmethod + def from_default(cls) -> Self: + return cls.from_parent_path(get_default_cache_dir()) + + @classmethod + def from_config(cls, config: DgConfig) -> Self: + return cls.from_parent_path( + parent_path=config.cache_dir, + logging_enabled=config.verbose, + ) + + # This is the preferred constructor to use when creating a cache. It ensures that all data is + # stored inside an additional container directory inside the user-specified cache directory. + # When we clear the cache, we only delete this container directory. This is to avoid accidents + # when the user mistakenly specifies a cache directory that contains other data. + @classmethod + def from_parent_path(cls, parent_path: Path, logging_enabled: bool = False) -> Self: + root_path = parent_path / _CACHE_CONTAINER_DIR_NAME + return cls(root_path, logging_enabled) + + def __init__(self, root_path: Path, logging_enabled: bool): + self._root_path = root_path + self._root_path.mkdir(parents=True, exist_ok=True) + self._logging_enabled = logging_enabled + + def clear(self) -> None: + shutil.rmtree(self._root_path) + self.log(f"CACHE [clear]: {self._root_path}") + + def get(self, key: Tuple[str, ...]) -> Optional[str]: + path = self._get_path(key) + if path.exists(): + self.log(f"CACHE [hit]: {path}") + return path.read_text() + else: + self.log(f"CACHE [miss]: {path}") + return None + + def set(self, key: Tuple[str, ...], value: str) -> None: + path = self._get_path(key) + path.parent.mkdir(parents=True, exist_ok=True) + path.write_text(value) + self.log(f"CACHE [write]: {path}") + + def _get_path(self, key: Tuple[str, ...]) -> Path: + return Path(self._root_path, *key) + + def log(self, message: str) -> None: + if self._logging_enabled: + print(message) # noqa: T201 diff --git a/python_modules/libraries/dagster-dg/dagster_dg/cli/__init__.py b/python_modules/libraries/dagster-dg/dagster_dg/cli/__init__.py index ea2e565da889e..d8cd4041ec23c 100644 --- a/python_modules/libraries/dagster-dg/dagster_dg/cli/__init__.py +++ b/python_modules/libraries/dagster-dg/dagster_dg/cli/__init__.py @@ -1,5 +1,8 @@ +from pathlib import Path + import click +from dagster_dg.cache import DgCache from dagster_dg.cli.generate import generate_cli from dagster_dg.cli.info import info_cli from dagster_dg.cli.list import list_cli @@ -14,27 +17,69 @@ def create_dg_cli(): "list": list_cli, } + # Defaults are defined on the DgConfig object. @click.group( commands=commands, context_settings={"max_content_width": 120, "help_option_names": ["-h", "--help"]}, + invoke_without_command=True, ) - @click.version_option(__version__, "--version", "-v") @click.option( "--builtin-component-lib", type=str, default=DgConfig.builtin_component_lib, help="Specify a builitin component library to use.", ) + @click.option( + "--verbose", + is_flag=True, + default=DgConfig.verbose, + help="Enable verbose output for debugging.", + ) + @click.option( + "--disable-cache", + is_flag=True, + default=DgConfig.disable_cache, + help="Disable caching of component registry data.", + ) + @click.option( + "--clear-cache", + is_flag=True, + help="Clear the cache before running the command.", + default=False, + ) + @click.option( + "--cache-dir", + type=Path, + default=DgConfig.cache_dir, + help="Specify a directory to use for the cache.", + ) + @click.version_option(__version__, "--version", "-v") @click.pass_context - def group(context: click.Context, builtin_component_lib: str): + def group( + context: click.Context, + builtin_component_lib: str, + verbose: bool, + disable_cache: bool, + cache_dir: Path, + clear_cache: bool, + ): """CLI tools for working with Dagster components.""" context.ensure_object(dict) - set_config_on_cli_context( - context, - DgConfig( - builtin_component_lib=builtin_component_lib, - ), + config = DgConfig( + builtin_component_lib=builtin_component_lib, + verbose=verbose, + disable_cache=disable_cache, + cache_dir=cache_dir, ) + if clear_cache: + DgCache.from_config(config).clear() + if context.invoked_subcommand is None: + context.exit(0) + elif context.invoked_subcommand is None: + click.echo(context.get_help()) + context.exit(0) + + set_config_on_cli_context(context, config) return group diff --git a/python_modules/libraries/dagster-dg/dagster_dg/config.py b/python_modules/libraries/dagster-dg/dagster_dg/config.py index 1266ff1b050e8..d5a7abd9adaa4 100644 --- a/python_modules/libraries/dagster-dg/dagster_dg/config.py +++ b/python_modules/libraries/dagster-dg/dagster_dg/config.py @@ -1,4 +1,6 @@ +import sys from dataclasses import dataclass +from pathlib import Path import click from typing_extensions import Self @@ -8,14 +10,33 @@ DEFAULT_BUILTIN_COMPONENT_LIB = "dagster_components" +def _get_default_cache_dir() -> Path: + if sys.platform == "win32": + return Path.home() / "AppData" / "dg" / "cache" + elif sys.platform == "darwin": + return Path.home() / "Library" / "Caches" / "dg" + else: + return Path.home() / ".cache" / "dg" + + +DEFAULT_CACHE_DIR = _get_default_cache_dir() + + @dataclass class DgConfig: """Global configuration for Dg. Attributes: + disable_cache (bool): If True, disable caching. Defaults to False. + cache_dir (Optional[str]): The directory to use for caching. If None, the default cache will + be used. + verbose (bool): If True, log debug information. builitin_component_lib (str): The name of the builtin component library to load. """ + disable_cache: bool = False + cache_dir: Path = DEFAULT_CACHE_DIR + verbose: bool = False builtin_component_lib: str = DEFAULT_BUILTIN_COMPONENT_LIB @classmethod diff --git a/python_modules/libraries/dagster-dg/dagster_dg/context.py b/python_modules/libraries/dagster-dg/dagster_dg/context.py index 6cd7aa6aa8add..af25fcd73ff06 100644 --- a/python_modules/libraries/dagster-dg/dagster_dg/context.py +++ b/python_modules/libraries/dagster-dg/dagster_dg/context.py @@ -1,3 +1,4 @@ +import hashlib import json import os from dataclasses import dataclass @@ -8,10 +9,15 @@ import tomli from typing_extensions import Self +from dagster_dg.cache import CachableDataType, DgCache from dagster_dg.component import RemoteComponentRegistry, RemoteComponentType from dagster_dg.config import DgConfig from dagster_dg.error import DgError -from dagster_dg.utils import execute_code_location_command +from dagster_dg.utils import ( + execute_code_location_command, + hash_directory_metadata, + hash_file_metadata, +) def is_inside_deployment_directory(path: Path) -> bool: @@ -64,13 +70,14 @@ def _is_code_location_root_directory(path: Path) -> bool: _DEPLOYMENT_CODE_LOCATIONS_DIR: Final = "code_locations" # Code location -_CODE_LOCATION_CUSTOM_COMPONENTS_DIR: Final = "lib" +_CODE_LOCATION_COMPONENTS_LIB_DIR: Final = "lib" _CODE_LOCATION_COMPONENT_INSTANCES_DIR: Final = "components" @dataclass class DgContext: config: DgConfig + cache: Optional[DgCache] = None @classmethod def from_cli_context(cls, cli_context: click.Context) -> Self: @@ -78,7 +85,8 @@ def from_cli_context(cls, cli_context: click.Context) -> Self: @classmethod def from_config(cls, config: DgConfig) -> Self: - return cls(config=config) + cache = None if config.disable_cache else DgCache.from_config(config) + return cls(config=config, cache=cache) @classmethod def default(cls) -> Self: @@ -105,6 +113,27 @@ def get_code_location_names(self) -> Iterable[str]: return [loc.name for loc in sorted((self.root_path / "code_locations").iterdir())] +def get_code_location_env_hash(code_location_root_path: Path) -> str: + uv_lock_path = code_location_root_path / "uv.lock" + if not uv_lock_path.exists(): + raise DgError(f"uv.lock file not found in {code_location_root_path}") + local_components_path = ( + code_location_root_path / code_location_root_path.name / _CODE_LOCATION_COMPONENTS_LIB_DIR + ) + if not local_components_path.exists(): + raise DgError(f"Local components directory not found in {code_location_root_path}") + hasher = hashlib.md5() + hash_file_metadata(hasher, uv_lock_path) + hash_directory_metadata(hasher, local_components_path) + return hasher.hexdigest() + + +def make_cache_key(code_location_path: Path, data_type: CachableDataType) -> Tuple[str, str, str]: + path_parts = [str(part) for part in code_location_path.parts if part != "/"] + env_hash = get_code_location_env_hash(code_location_path) + return ("_".join(path_parts), env_hash, data_type) + + @dataclass class CodeLocationDirectoryContext: """Class encapsulating contextual information about a components code location directory. @@ -127,10 +156,21 @@ class CodeLocationDirectoryContext: @classmethod def from_path(cls, path: Path, dg_context: DgContext) -> Self: root_path = _resolve_code_location_root_directory(path) - component_registry_data = execute_code_location_command( - root_path, ["list", "component-types"], dg_context - ) - component_registry = RemoteComponentRegistry.from_dict(json.loads(component_registry_data)) + + cache = dg_context.cache + if cache: + cache_key = make_cache_key(root_path, "component_registry_data") + + raw_registry_data = cache.get(cache_key) if cache else None + if not raw_registry_data: + raw_registry_data = execute_code_location_command( + root_path, ["list", "component-types"], dg_context + ) + if cache: + cache.set(cache_key, raw_registry_data) + + registry_data = json.loads(raw_registry_data) + component_registry = RemoteComponentRegistry.from_dict(registry_data) return cls( root_path=root_path, @@ -148,11 +188,11 @@ def config(self) -> DgConfig: @property def local_component_types_root_path(self) -> str: - return os.path.join(self.root_path, self.name, _CODE_LOCATION_CUSTOM_COMPONENTS_DIR) + return os.path.join(self.root_path, self.name, _CODE_LOCATION_COMPONENTS_LIB_DIR) @property def local_component_types_root_module_name(self) -> str: - return f"{self.name}.{_CODE_LOCATION_CUSTOM_COMPONENTS_DIR}" + return f"{self.name}.{_CODE_LOCATION_COMPONENTS_LIB_DIR}" def iter_component_types(self) -> Iterable[Tuple[str, RemoteComponentType]]: for key in sorted(self.component_registry.keys()): diff --git a/python_modules/libraries/dagster-dg/dagster_dg/utils.py b/python_modules/libraries/dagster-dg/dagster_dg/utils.py index 79f69e95ffacd..57be0cc34e53f 100644 --- a/python_modules/libraries/dagster-dg/dagster_dg/utils.py +++ b/python_modules/libraries/dagster-dg/dagster_dg/utils.py @@ -4,14 +4,20 @@ import re import subprocess import sys +from fnmatch import fnmatch from pathlib import Path from typing import TYPE_CHECKING, Any, Final, Iterator, List, Mapping, Optional, Sequence, Union import click import jinja2 +from typing_extensions import TypeAlias from dagster_dg.version import __version__ as dagster_version +# There is some weirdness concerning the availabilty of hashlib.HASH between different Python +# versions, so for nowe we avoid trying to import it and just alias the type to Any. +Hash: TypeAlias = Any + if TYPE_CHECKING: from dagster_dg.context import DgContext @@ -84,6 +90,7 @@ def snakecase(string: str) -> str: "__pycache__", ".pytest_cache", "*.egg-info", + "*.cpython-*", ".DS_Store", ".ruff_cache", "tox.ini", @@ -188,3 +195,19 @@ def ensure_dagster_dg_tests_import() -> None: dagster_dg_package_root / "dagster_dg_tests" ).exists(), "Could not find dagster_dg_tests where expected" sys.path.append(dagster_dg_package_root.as_posix()) + + +def hash_directory_metadata(hasher: Hash, path: Union[str, Path]) -> None: + for root, dirs, files in os.walk(path): + for name in dirs + files: + if any(fnmatch(name, pattern) for pattern in _DEFAULT_EXCLUDES): + continue + filepath = os.path.join(root, name) + hash_file_metadata(hasher, filepath) + + +def hash_file_metadata(hasher: Hash, path: Union[str, Path]) -> None: + stat = os.stat(path=path) + hasher.update(str(path).encode()) + hasher.update(str(stat.st_mtime).encode()) # Last modified time + hasher.update(str(stat.st_size).encode()) # File size diff --git a/python_modules/libraries/dagster-dg/dagster_dg_tests/cli_tests/test_generate_commands.py b/python_modules/libraries/dagster-dg/dagster_dg_tests/cli_tests/test_generate_commands.py index 3e149a59abe38..7db275b8b0b92 100644 --- a/python_modules/libraries/dagster-dg/dagster_dg_tests/cli_tests/test_generate_commands.py +++ b/python_modules/libraries/dagster-dg/dagster_dg_tests/cli_tests/test_generate_commands.py @@ -5,8 +5,6 @@ import pytest import tomli -from click.testing import CliRunner -from dagster_dg.cli import cli as dg_cli from dagster_dg.context import CodeLocationDirectoryContext, DgContext from dagster_dg.utils import discover_git_root, ensure_dagster_dg_tests_import @@ -21,8 +19,7 @@ def test_generate_deployment_command_success() -> None: - runner = ProxyRunner.test() - with runner.isolated_filesystem(): + with ProxyRunner.test() as runner, runner.isolated_filesystem(): result = runner.invoke("generate", "deployment", "foo") assert_runner_result(result) assert Path("foo").exists() @@ -34,8 +31,7 @@ def test_generate_deployment_command_success() -> None: def test_generate_deployment_command_already_exists_fails() -> None: - runner = ProxyRunner.test() - with runner.isolated_filesystem(): + with ProxyRunner.test() as runner, runner.isolated_filesystem(): os.mkdir("foo") result = runner.invoke("generate", "deployment", "foo") assert_runner_result(result, exit_0=False) @@ -43,8 +39,7 @@ def test_generate_deployment_command_already_exists_fails() -> None: def test_generate_code_location_inside_deployment_success() -> None: - runner = ProxyRunner.test() - with isolated_example_deployment_foo(runner): + with ProxyRunner.test() as runner, isolated_example_deployment_foo(runner): result = runner.invoke("generate", "code-location", "bar") assert_runner_result(result) assert Path("code_locations/bar").exists() @@ -66,8 +61,7 @@ def test_generate_code_location_inside_deployment_success() -> None: def test_generate_code_location_outside_deployment_success() -> None: - runner = ProxyRunner.test() - with runner.isolated_filesystem(): + with ProxyRunner.test() as runner, runner.isolated_filesystem(): result = runner.invoke("generate", "code-location", "bar") assert_runner_result(result) assert Path("bar").exists() @@ -84,14 +78,13 @@ def test_generate_code_location_outside_deployment_success() -> None: @pytest.mark.parametrize("mode", ["env_var", "arg"]) def test_generate_code_location_editable_dagster_success(mode: str, monkeypatch) -> None: - runner = ProxyRunner.test() dagster_git_repo_dir = discover_git_root(Path(__file__)) if mode == "env_var": monkeypatch.setenv("DAGSTER_GIT_REPO_DIR", str(dagster_git_repo_dir)) editable_args = ["--use-editable-dagster", "--"] else: editable_args = ["--use-editable-dagster", str(dagster_git_repo_dir)] - with isolated_example_deployment_foo(runner): + with ProxyRunner.test() as runner, isolated_example_deployment_foo(runner): result = runner.invoke("generate", "code-location", *editable_args, "bar") assert_runner_result(result) assert Path("code_locations/bar").exists() @@ -117,17 +110,15 @@ def test_generate_code_location_editable_dagster_success(mode: str, monkeypatch) def test_generate_code_location_editable_dagster_no_env_var_no_value_fails(monkeypatch) -> None: - runner = ProxyRunner.test() monkeypatch.setenv("DAGSTER_GIT_REPO_DIR", "") - with isolated_example_deployment_foo(runner): + with ProxyRunner.test() as runner, isolated_example_deployment_foo(runner): result = runner.invoke("generate", "code-location", "--use-editable-dagster", "--", "bar") assert_runner_result(result, exit_0=False) assert "requires the `DAGSTER_GIT_REPO_DIR`" in result.output def test_generate_code_location_already_exists_fails() -> None: - runner = ProxyRunner.test() - with isolated_example_deployment_foo(runner): + with ProxyRunner.test() as runner, isolated_example_deployment_foo(runner): result = runner.invoke("generate", "code-location", "bar") assert_runner_result(result) result = runner.invoke("generate", "code-location", "bar") @@ -137,8 +128,7 @@ def test_generate_code_location_already_exists_fails() -> None: @pytest.mark.parametrize("in_deployment", [True, False]) def test_generate_component_type_success(in_deployment: bool) -> None: - runner = ProxyRunner.test() - with isolated_example_code_location_bar(runner, in_deployment): + with ProxyRunner.test() as runner, isolated_example_code_location_bar(runner, in_deployment): result = runner.invoke("generate", "component-type", "baz") assert_runner_result(result) assert Path("bar/lib/baz.py").exists() @@ -147,8 +137,7 @@ def test_generate_component_type_success(in_deployment: bool) -> None: def test_generate_component_type_outside_code_location_fails() -> None: - runner = ProxyRunner.test() - with isolated_example_deployment_foo(runner): + with ProxyRunner.test() as runner, isolated_example_deployment_foo(runner): result = runner.invoke("generate", "component-type", "baz") assert_runner_result(result, exit_0=False) assert "must be run inside a Dagster code location directory" in result.output @@ -156,8 +145,7 @@ def test_generate_component_type_outside_code_location_fails() -> None: @pytest.mark.parametrize("in_deployment", [True, False]) def test_generate_component_type_already_exists_fails(in_deployment: bool) -> None: - runner = ProxyRunner.test() - with isolated_example_code_location_bar(runner, in_deployment): + with ProxyRunner.test() as runner, isolated_example_code_location_bar(runner, in_deployment): result = runner.invoke("generate", "component-type", "baz") assert_runner_result(result) result = runner.invoke("generate", "component-type", "baz") @@ -167,8 +155,7 @@ def test_generate_component_type_already_exists_fails(in_deployment: bool) -> No @pytest.mark.parametrize("in_deployment", [True, False]) def test_generate_component_no_params_success(in_deployment: bool) -> None: - runner = ProxyRunner.test() - with isolated_example_code_location_bar(runner, in_deployment): + with ProxyRunner.test() as runner, isolated_example_code_location_bar(runner, in_deployment): result = runner.invoke( "generate", "component", @@ -187,8 +174,7 @@ def test_generate_component_no_params_success(in_deployment: bool) -> None: @pytest.mark.parametrize("in_deployment", [True, False]) def test_generate_component_json_params_success(in_deployment: bool) -> None: - runner = ProxyRunner.test() - with isolated_example_code_location_bar(runner, in_deployment): + with ProxyRunner.test() as runner, isolated_example_code_location_bar(runner, in_deployment): result = runner.invoke( "generate", "component", @@ -210,8 +196,7 @@ def test_generate_component_json_params_success(in_deployment: bool) -> None: @pytest.mark.parametrize("in_deployment", [True, False]) def test_generate_component_extra_args_success(in_deployment: bool) -> None: - runner = ProxyRunner.test() - with isolated_example_code_location_bar(runner, in_deployment): + with ProxyRunner.test() as runner, isolated_example_code_location_bar(runner, in_deployment): result = runner.invoke( "generate", "component", @@ -233,8 +218,7 @@ def test_generate_component_extra_args_success(in_deployment: bool) -> None: def test_generate_component_json_params_and_extra_args_fails() -> None: - runner = ProxyRunner.test() - with isolated_example_code_location_bar(runner): + with ProxyRunner.test() as runner, isolated_example_code_location_bar(runner): result = runner.invoke( "generate", "component", @@ -250,8 +234,7 @@ def test_generate_component_json_params_and_extra_args_fails() -> None: def test_generate_component_outside_code_location_fails() -> None: - runner = ProxyRunner.test() - with isolated_example_deployment_foo(runner): + with ProxyRunner.test() as runner, isolated_example_deployment_foo(runner): result = runner.invoke("generate", "component", "bar.baz", "qux") assert_runner_result(result, exit_0=False) assert "must be run inside a Dagster code location directory" in result.output @@ -259,8 +242,7 @@ def test_generate_component_outside_code_location_fails() -> None: @pytest.mark.parametrize("in_deployment", [True, False]) def test_generate_component_already_exists_fails(in_deployment: bool) -> None: - runner = ProxyRunner.test() - with isolated_example_code_location_bar(runner, in_deployment): + with ProxyRunner.test() as runner, isolated_example_code_location_bar(runner, in_deployment): result = runner.invoke( "generate", "component", @@ -284,15 +266,17 @@ def test_generate_component_already_exists_fails(in_deployment: bool) -> None: def test_generate_sling_replication_instance() -> None: - runner = CliRunner() - with isolated_example_code_location_bar(runner): + with ( + ProxyRunner.test(use_test_component_lib=False) as runner, + isolated_example_code_location_bar(runner), + ): # We need to add dagster-embedded-elt also because we are using editable installs. Only # direct dependencies will be resolved by uv.tool.sources. subprocess.run( ["uv", "add", "dagster-components[sling]", "dagster-embedded-elt"], check=True ) result = runner.invoke( - dg_cli, ["generate", "component", "dagster_components.sling_replication", "file_ingest"] + "generate", "component", "dagster_components.sling_replication", "file_ingest" ) assert_runner_result(result) assert Path("bar/components/file_ingest").exists() @@ -317,14 +301,19 @@ def test_generate_sling_replication_instance() -> None: ], ) def test_generate_dbt_project_instance(params) -> None: - runner = CliRunner() - with isolated_example_code_location_bar(runner): + with ( + ProxyRunner.test(use_test_component_lib=False) as runner, + isolated_example_code_location_bar(runner), + ): # We need to add dagster-dbt also because we are using editable installs. Only # direct dependencies will be resolved by uv.tool.sources. subprocess.run(["uv", "add", "dagster-components[dbt]", "dagster-dbt"], check=True) result = runner.invoke( - dg_cli, - ["generate", "component", "dagster_components.dbt_project", "my_project", *params], + "generate", + "component", + "dagster_components.dbt_project", + "my_project", + *params, ) assert_runner_result(result) assert Path("bar/components/my_project").exists() diff --git a/python_modules/libraries/dagster-dg/dagster_dg_tests/cli_tests/test_info_commands.py b/python_modules/libraries/dagster-dg/dagster_dg_tests/cli_tests/test_info_commands.py index eb4305fae02ca..27d9005156ccb 100644 --- a/python_modules/libraries/dagster-dg/dagster_dg_tests/cli_tests/test_info_commands.py +++ b/python_modules/libraries/dagster-dg/dagster_dg_tests/cli_tests/test_info_commands.py @@ -12,8 +12,7 @@ def test_info_component_type_all_metadata_success(): - runner = ProxyRunner.test() - with isolated_example_code_location_bar(runner): + with ProxyRunner.test() as runner, isolated_example_code_location_bar(runner): result = runner.invoke( "info", "component-type", @@ -77,8 +76,7 @@ def test_info_component_type_all_metadata_success(): def test_info_component_type_all_metadata_empty_success(): - runner = ProxyRunner.test() - with isolated_example_code_location_bar(runner): + with ProxyRunner.test() as runner, isolated_example_code_location_bar(runner): result = runner.invoke( "info", "component-type", @@ -94,8 +92,7 @@ def test_info_component_type_all_metadata_empty_success(): def test_info_component_type_flag_fields_success(): - runner = ProxyRunner.test() - with isolated_example_code_location_bar(runner): + with ProxyRunner.test() as runner, isolated_example_code_location_bar(runner): result = runner.invoke( "info", "component-type", @@ -176,8 +173,7 @@ def test_info_component_type_flag_fields_success(): def test_info_component_type_outside_code_location_fails() -> None: - runner = ProxyRunner.test() - with runner.isolated_filesystem(): + with ProxyRunner.test() as runner, runner.isolated_filesystem(): result = runner.invoke( "info", "component-type", @@ -189,8 +185,7 @@ def test_info_component_type_outside_code_location_fails() -> None: def test_info_component_type_multiple_flags_fails() -> None: - runner = ProxyRunner.test() - with isolated_example_code_location_bar(runner): + with ProxyRunner.test() as runner, isolated_example_code_location_bar(runner): result = runner.invoke( "info", "component-type", diff --git a/python_modules/libraries/dagster-dg/dagster_dg_tests/cli_tests/test_list_commands.py b/python_modules/libraries/dagster-dg/dagster_dg_tests/cli_tests/test_list_commands.py index 5d9adb3a17d15..abb22d8934b93 100644 --- a/python_modules/libraries/dagster-dg/dagster_dg_tests/cli_tests/test_list_commands.py +++ b/python_modules/libraries/dagster-dg/dagster_dg_tests/cli_tests/test_list_commands.py @@ -13,8 +13,7 @@ def test_list_code_locations_success(): - runner = ProxyRunner.test() - with isolated_example_deployment_foo(runner): + with ProxyRunner.test() as runner, isolated_example_deployment_foo(runner): runner.invoke("generate", "code-location", "foo") runner.invoke("generate", "code-location", "bar") result = runner.invoke("list", "code-locations") @@ -29,16 +28,14 @@ def test_list_code_locations_success(): def test_list_code_locations_outside_deployment_fails() -> None: - runner = ProxyRunner.test() - with runner.isolated_filesystem(): + with ProxyRunner.test() as runner, runner.isolated_filesystem(): result = runner.invoke("list", "code-locations") assert_runner_result(result, exit_0=False) assert "must be run inside a Dagster deployment directory" in result.output def test_list_component_types_success(): - runner = ProxyRunner.test() - with isolated_example_code_location_bar(runner): + with ProxyRunner.test() as runner, isolated_example_code_location_bar(runner): result = runner.invoke("list", "component-types") assert_runner_result(result) assert ( @@ -54,16 +51,14 @@ def test_list_component_types_success(): def test_list_component_types_outside_code_location_fails() -> None: - runner = ProxyRunner.test() - with runner.isolated_filesystem(): + with ProxyRunner.test() as runner, runner.isolated_filesystem(): result = runner.invoke("list", "component-types") assert_runner_result(result, exit_0=False) assert "must be run inside a Dagster code location directory" in result.output def test_list_components_succeeds(): - runner = ProxyRunner.test() - with isolated_example_code_location_bar(runner): + with ProxyRunner.test() as runner, isolated_example_code_location_bar(runner): result = runner.invoke( "generate", "component", @@ -82,8 +77,7 @@ def test_list_components_succeeds(): def test_list_components_command_outside_code_location_fails() -> None: - runner = ProxyRunner.test() - with runner.isolated_filesystem(): + with ProxyRunner.test() as runner, runner.isolated_filesystem(): result = runner.invoke("list", "components") assert_runner_result(result, exit_0=False) assert "must be run inside a Dagster code location directory" in result.output diff --git a/python_modules/libraries/dagster-dg/dagster_dg_tests/test_cache.py b/python_modules/libraries/dagster-dg/dagster_dg_tests/test_cache.py new file mode 100644 index 0000000000000..4636259c921fc --- /dev/null +++ b/python_modules/libraries/dagster-dg/dagster_dg_tests/test_cache.py @@ -0,0 +1,93 @@ +import subprocess +from pathlib import Path + +import pytest + +from dagster_dg_tests.utils import ( + ProxyRunner, + assert_runner_result, + isolated_example_code_location_bar, +) + + +def test_load_from_cache(): + with ProxyRunner.test(verbose=True) as runner, isolated_example_code_location_bar(runner): + result = runner.invoke("list", "component-types") + assert_runner_result(result) + assert "CACHE [miss]" in result.output + assert "CACHE [write]" in result.output + result = runner.invoke("list", "component-types") + assert_runner_result(result) + assert "CACHE [hit]" in result.output + + +def test_cache_invalidation_uv_lock(): + with ProxyRunner.test(verbose=True) as runner, isolated_example_code_location_bar(runner): + result = runner.invoke("list", "component-types") + assert_runner_result(result) + assert "CACHE [miss]" in result.output + assert "CACHE [write]" in result.output + + subprocess.run(["uv", "add", "dagster-components[dbt]"], check=True) + + result = runner.invoke("list", "component-types") + assert_runner_result(result) + assert "CACHE [miss]" in result.output + + +def test_cache_invalidation_modified_lib(): + with ProxyRunner.test(verbose=True) as runner, isolated_example_code_location_bar(runner): + result = runner.invoke("list", "component-types") + assert_runner_result(result) + assert "CACHE [miss]" in result.output + assert "CACHE [write]" in result.output + + result = runner.invoke("generate", "component-type", "my_component") + assert_runner_result(result) + + result = runner.invoke("list", "component-types") + assert_runner_result(result) + assert "CACHE [miss]" in result.output + + +def test_cache_no_invalidation_modified_pkg(): + with ProxyRunner.test(verbose=True) as runner, isolated_example_code_location_bar(runner): + result = runner.invoke("list", "component-types") + assert_runner_result(result) + assert "CACHE [miss]" in result.output + assert "CACHE [write]" in result.output + + Path("bar/submodule.py").write_text("print('hello')") + + result = runner.invoke("list", "component-types") + assert_runner_result(result) + assert "CACHE [hit]" in result.output + + +@pytest.mark.parametrize("with_command", [True, False]) +def test_cache_clear(with_command: bool): + with ProxyRunner.test(verbose=True) as runner, isolated_example_code_location_bar(runner): + result = runner.invoke("list", "component-types") + assert_runner_result(result) + assert "CACHE [miss]" in result.output + assert "CACHE [write]" in result.output + + if with_command: + result = runner.invoke("--clear-cache", "list", "component-types") + else: + result = runner.invoke("--clear-cache") + assert_runner_result(result) + result = runner.invoke("list", "component-types") + + assert_runner_result(result) + assert "CACHE [miss]" in result.output + + +def test_cache_disabled(): + with ( + ProxyRunner.test(verbose=True, disable_cache=True) as runner, + isolated_example_code_location_bar(runner), + ): + result = runner.invoke("list", "component-types") + assert_runner_result(result) + assert "CACHE" not in result.output diff --git a/python_modules/libraries/dagster-dg/dagster_dg_tests/utils.py b/python_modules/libraries/dagster-dg/dagster_dg_tests/utils.py index 0d32afe12494c..d3628b5e229ed 100644 --- a/python_modules/libraries/dagster-dg/dagster_dg_tests/utils.py +++ b/python_modules/libraries/dagster-dg/dagster_dg_tests/utils.py @@ -2,12 +2,14 @@ from contextlib import contextmanager from dataclasses import dataclass from pathlib import Path +from tempfile import TemporaryDirectory from types import TracebackType from typing import Iterator, Optional, Sequence, Tuple, Type, Union from click.testing import CliRunner, Result from dagster_dg.cli import cli as dg_cli from dagster_dg.utils import discover_git_root, pushd +from typing_extensions import Self @contextmanager @@ -55,8 +57,23 @@ class ProxyRunner: prepend_args: Optional[Sequence[str]] = None @classmethod - def test(cls): - return cls(CliRunner(), ["--builtin-component-lib", "dagster_components.test"]) + @contextmanager + def test( + cls, use_test_component_lib: bool = True, verbose: bool = False, disable_cache: bool = False + ) -> Iterator[Self]: + with TemporaryDirectory() as cache_dir: + prepend_args = [ + *( + ["--builtin-component-lib", "dagster_components.test"] + if use_test_component_lib + else [] + ), + "--cache-dir", + str(cache_dir), + *(["--verbose"] if verbose else []), + *(["--disable-cache"] if disable_cache else []), + ] + yield cls(CliRunner(), prepend_args=prepend_args) def invoke(self, *args: str): all_args = [*(self.prepend_args or []), *args]