Skip to content

Commit

Permalink
[components] Remove extend key from config (BUILD-635) (#27998)
Browse files Browse the repository at this point in the history
## Summary & Motivation

Now that we converged on the concept of workspaces, we no longer need
the `extend` key in config for inheritance. This PR simplifies config
merging:

- If the closest config file to CWD is a workspace config file, that's
our only config
- If the closest config file to CWD is not a workspace config, then look
for the workspace config and merge the two files if you find one

Also adds some previously missing tests for config inheritance and
invalid config keys.

## How I Tested These Changes

Unit tests.
  • Loading branch information
smackesey authored Feb 23, 2025
1 parent e3ce424 commit b417349
Show file tree
Hide file tree
Showing 4 changed files with 114 additions and 45 deletions.
49 changes: 9 additions & 40 deletions python_modules/libraries/dagster-dg/dagster_dg/config.py
Original file line number Diff line number Diff line change
@@ -1,10 +1,11 @@
from collections.abc import Mapping
from dataclasses import dataclass, replace
from dataclasses import dataclass
from pathlib import Path
from typing import Any, Callable, Optional, TypedDict, TypeVar, cast

import click
import tomlkit
import tomlkit.items
from click.core import ParameterSource

from dagster_dg.error import DgError, DgValidationError
Expand Down Expand Up @@ -73,21 +74,6 @@ def discover_config_file(
return
current_path = current_path.parent

@classmethod
def from_config_file(cls, path: Path) -> "DgConfig":
current_config = load_dg_config_file(path)
current_directory_path = path.parent
while "extend" in current_config:
extend_path = current_directory_path / current_config["extend"]
if not is_dg_config_file(extend_path):
raise DgValidationError(
"Config file {extend_path} was specified in `extend` field but does not contain a `tool.dg` section."
)
extend_config = load_dg_config_file(extend_path)
current_config = cast(DgPartialFileConfig, {**extend_config, **current_config})
current_directory_path = extend_path.parent
return replace(DgConfig.default(), **current_config)

@classmethod
def default(cls) -> "DgConfig":
return cls()
Expand Down Expand Up @@ -182,28 +168,6 @@ def get_config_from_cli_context(cli_context: click.Context) -> DgPartialConfig:
# ########################


class DgPartialFileConfig(DgPartialConfig, total=False):
extend: str


def _validate_dg_partial_file_config(
raw_dict: Mapping[str, object], file_path: Path
) -> DgPartialFileConfig:
if "extend" in raw_dict:
if not isinstance(raw_dict["extend"], str):
_raise_file_config_validation_error("`extend` must be a string.", file_path)
elif not (file_path.parent / raw_dict["extend"]).exists():
_raise_file_config_validation_error(
"Config specifies `extend` setting to non-existent file: {raw_dict['extend']}",
file_path,
)
try:
_normalize_dg_partial_config({k: v for k, v in raw_dict.items() if k not in ["extend"]})
except DgValidationError as e:
_raise_file_config_validation_error(str(e), file_path)
return cast(DgPartialFileConfig, raw_dict)


def is_dg_config_file(
path: Path, predicate: Optional[Callable[[Mapping[str, Any]], bool]] = None
) -> bool:
Expand All @@ -213,9 +177,14 @@ def is_dg_config_file(
)


def load_dg_config_file(path: Path) -> DgPartialFileConfig:
def load_dg_config_file(path: Path) -> DgPartialConfig:
toml = tomlkit.parse(path.read_text())
return _validate_dg_partial_file_config(get_toml_value(toml, ["tool", "dg"], dict), path)
raw_dict = get_toml_value(toml, ["tool", "dg"], tomlkit.items.Table).unwrap()
try:
_normalize_dg_partial_config({k: v for k, v in raw_dict.items()})
except DgValidationError as e:
_raise_file_config_validation_error(str(e), path)
return cast(DgPartialConfig, raw_dict)


def _raise_file_config_validation_error(message: str, file_path: Path) -> None:
Expand Down
19 changes: 16 additions & 3 deletions python_modules/libraries/dagster-dg/dagster_dg/context.py
Original file line number Diff line number Diff line change
Expand Up @@ -111,9 +111,22 @@ def from_config_file_discovery_and_cli_config(
cls, path: Path, cli_config: DgPartialConfig
) -> Self:
config_path = DgConfig.discover_config_file(path)
root_path = config_path.parent if config_path else path
base_config = DgConfig.from_config_file(config_path) if config_path else DgConfig.default()
config = replace(base_config, **cli_config)
workspace_config_path = DgConfig.discover_config_file(
path, lambda x: bool(x.get("is_workspace"))
)

# Build the config in the following order: defaults, workspace, project, CLI
config = DgConfig.default()
if config_path:
# Add workspace config only if it's different from the project config
if workspace_config_path and config_path != workspace_config_path:
config = replace(config, **load_dg_config_file(workspace_config_path))
config = replace(config, **load_dg_config_file(config_path))
root_path = config_path.parent
else:
root_path = path
config = replace(config, **cli_config)

return cls(config=config, root_path=root_path)

@classmethod
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,85 @@
from pathlib import Path

import pytest
from dagster_dg.context import DgContext
from dagster_dg.error import DgError
from dagster_dg.utils import pushd, set_toml_value

from dagster_dg_tests.utils import (
ProxyRunner,
isolated_components_venv,
isolated_example_project_foo_bar,
isolated_example_workspace,
modify_pyproject_toml,
)


def test_context_in_workspace():
with ProxyRunner.test() as runner, isolated_example_workspace(runner):
# go into a subdirectory to make sure root resolution works
path_arg = Path.cwd() / "libraries"

context = DgContext.for_workspace_environment(path_arg, {})
assert context.root_path == Path.cwd()

# Test config properly set
with modify_pyproject_toml() as pyproject_toml:
set_toml_value(pyproject_toml, ("tool", "dg", "verbose"), True)
context = DgContext.for_workspace_environment(path_arg, {})
assert context.config.verbose is True


def test_context_in_project_in_workspace():
with ProxyRunner.test() as runner, isolated_example_workspace(runner, project_name="foo"):
project_path = Path.cwd() / "projects" / "foo"
# go into a project subdirectory to make sure root resolution works
path_arg = project_path / "foo_tests"

context = DgContext.for_project_environment(path_arg, {})
assert context.root_path == project_path
assert context.config.verbose is False # default

# Test config inheritance from workspace
with modify_pyproject_toml() as pyproject_toml:
set_toml_value(pyproject_toml, ("tool", "dg", "verbose"), True)
context = DgContext.for_project_environment(path_arg, {})
assert context.config.verbose is True

# Test config from project overrides workspace
with pushd(project_path), modify_pyproject_toml() as pyproject_toml:
set_toml_value(pyproject_toml, ("tool", "dg", "verbose"), False)
context = DgContext.for_project_environment(path_arg, {})
assert context.config.verbose is False


def test_context_in_project_outside_workspace():
with ProxyRunner.test() as runner, isolated_example_project_foo_bar(runner):
project_path = Path.cwd()
# go into a project subdirectory to make sure root resolution works
path_arg = project_path / "foo_tests"

context = DgContext.for_project_environment(path_arg, {})
assert context.root_path == project_path
assert context.config.verbose is False

with modify_pyproject_toml() as pyproject_toml:
set_toml_value(pyproject_toml, ("tool", "dg", "verbose"), True)
context = DgContext.for_project_environment(path_arg, {})
assert context.config.verbose is True


def test_context_outside_project_or_workspace():
with ProxyRunner.test() as runner, isolated_components_venv(runner):
context = DgContext.from_config_file_discovery_and_cli_config(Path.cwd(), {})
assert context.root_path == Path.cwd()
assert context.config.verbose is False


def test_invalid_key_in_config():
with ProxyRunner.test() as runner, isolated_example_workspace(runner):
with modify_pyproject_toml() as pyproject_toml:
set_toml_value(pyproject_toml, ("tool", "dg", "invalid_key"), True)
with pytest.raises(
DgError, match=r"Unrecognized fields in configuration: \['invalid_key'\]"
):
DgContext.from_config_file_discovery_and_cli_config(Path.cwd(), {})
6 changes: 4 additions & 2 deletions python_modules/libraries/dagster-dg/dagster_dg_tests/utils.py
Original file line number Diff line number Diff line change
Expand Up @@ -56,11 +56,13 @@ def isolated_components_venv(runner: Union[CliRunner, "ProxyRunner"]) -> Iterato

@contextmanager
def isolated_example_workspace(
runner: Union[CliRunner, "ProxyRunner"], create_venv: bool = False
runner: Union[CliRunner, "ProxyRunner"],
project_name: Optional[str] = None,
create_venv: bool = False,
) -> Iterator[None]:
runner = ProxyRunner(runner) if isinstance(runner, CliRunner) else runner
with runner.isolated_filesystem(), clear_module_from_cache("foo_bar"):
runner.invoke("init", input=" \n")
runner.invoke("init", input=f" {project_name or ''}\n")
with pushd("workspace"):
# Create a venv capable of running dagster dev
if create_venv:
Expand Down

0 comments on commit b417349

Please sign in to comment.