Skip to content

Commit

Permalink
[components] Set up test components (#26512)
Browse files Browse the repository at this point in the history
## Summary & Motivation

This PR is an overhaul of testing in `dagster-dg` with the aim of
decoupling development of "real" components from the tests of `dg`.

- Introduces an alternative set of test components in
`dagster-components`. `dagster-components` will only ever load one of
its builtin component libraries-- the test or the published components.
Test components allow us to ensure coverage over cases like components
with no summary/description/params.
- Which library gets loaded is controlled via the
`use_test_component_lib` to
`ComponentRegistry.from_entry_point_discovery`. This in turn can be
controlled via the `--use-test-component-lib` CLI param on both the
`dagster-dg` and `dagster-components` CLIs.
- Many tests are rewritten against the set of test components and are
now more precise (for example, check the exact format of the output of
the `dg info component-type` command output)
- Introduced some new test utilities introduced in `dagster-dg` to make
all this easier.

## How I Tested These Changes

Unit tests
  • Loading branch information
smackesey authored Dec 17, 2024
1 parent 982b69a commit 3c9b9ac
Show file tree
Hide file tree
Showing 25 changed files with 929 additions and 314 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,8 @@

from dagster_components.cli.generate import generate_cli
from dagster_components.cli.list import list_cli
from dagster_components.core.component import BUILTIN_PUBLISHED_COMPONENT_ENTRY_POINT
from dagster_components.utils import CLI_BUILTIN_COMPONENT_LIB_KEY


def create_dagster_components_cli():
Expand All @@ -15,9 +17,18 @@ def create_dagster_components_cli():
commands=commands,
context_settings={"max_content_width": 120, "help_option_names": ["-h", "--help"]},
)
@click.option(
"--builtin-component-lib",
type=str,
default=BUILTIN_PUBLISHED_COMPONENT_ENTRY_POINT,
help="Specify the builitin component library to load.",
)
@click.version_option(__version__, "--version", "-v")
def group():
@click.pass_context
def group(ctx: click.Context, builtin_component_lib: str):
"""CLI tools for working with Dagster."""
ctx.ensure_object(dict)
ctx.obj[CLI_BUILTIN_COMPONENT_LIB_KEY] = builtin_component_lib

return group

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@
is_inside_code_location_project,
)
from dagster_components.generate import generate_component_instance
from dagster_components.utils import CLI_BUILTIN_COMPONENT_LIB_KEY


@click.group(name="generate")
Expand All @@ -23,12 +24,15 @@ def generate_cli() -> None:
@click.argument("component_name", type=str)
@click.option("--json-params", type=str, default=None)
@click.argument("extra_args", nargs=-1, type=str)
@click.pass_context
def generate_component_command(
ctx: click.Context,
component_type: str,
component_name: str,
json_params: Optional[str],
extra_args: Tuple[str, ...],
) -> None:
builtin_component_lib = ctx.obj.get(CLI_BUILTIN_COMPONENT_LIB_KEY, False)
if not is_inside_code_location_project(Path.cwd()):
click.echo(
click.style(
Expand All @@ -38,7 +42,8 @@ def generate_component_command(
sys.exit(1)

context = CodeLocationProjectContext.from_path(
Path.cwd(), ComponentRegistry.from_entry_point_discovery()
Path.cwd(),
ComponentRegistry.from_entry_point_discovery(builtin_component_lib=builtin_component_lib),
)
if not context.has_component_type(component_type):
click.echo(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@
CodeLocationProjectContext,
is_inside_code_location_project,
)
from dagster_components.utils import CLI_BUILTIN_COMPONENT_LIB_KEY


@click.group(name="generate")
Expand All @@ -18,8 +19,10 @@ def list_cli():


@list_cli.command(name="component-types")
def list_component_types_command() -> None:
@click.pass_context
def list_component_types_command(ctx: click.Context) -> None:
"""List registered Dagster components."""
builtin_component_lib = ctx.obj.get(CLI_BUILTIN_COMPONENT_LIB_KEY, False)
if not is_inside_code_location_project(Path.cwd()):
click.echo(
click.style(
Expand All @@ -29,7 +32,8 @@ def list_component_types_command() -> None:
sys.exit(1)

context = CodeLocationProjectContext.from_path(
Path.cwd(), ComponentRegistry.from_entry_point_discovery()
Path.cwd(),
ComponentRegistry.from_entry_point_discovery(builtin_component_lib=builtin_component_lib),
)
output: Dict[str, Any] = {}
for key, component_type in context.list_component_types():
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,7 @@
import importlib.metadata
import inspect
import sys
import textwrap
from abc import ABC, abstractmethod
from pathlib import Path
from types import ModuleType
Expand All @@ -25,6 +26,8 @@
from dagster._utils import snakecase
from typing_extensions import Self

from dagster_components.utils import ensure_dagster_components_tests_import

if TYPE_CHECKING:
from dagster._core.definitions.definitions_class import Definitions

Expand All @@ -40,7 +43,7 @@ class ComponentGenerateRequest:

class Component(ABC):
name: ClassVar[Optional[str]] = None
component_params_schema: ClassVar = None
params_schema: ClassVar = None
generate_params_schema: ClassVar = None

@classmethod
Expand All @@ -58,22 +61,32 @@ def from_decl_node(
@classmethod
def get_metadata(cls) -> "ComponentInternalMetadata":
docstring = cls.__doc__
clean_docstring = _clean_docstring(docstring) if docstring else None

return {
"summary": docstring.split("\n\n")[0] if docstring else None,
"description": docstring,
"summary": clean_docstring.split("\n\n")[0] if clean_docstring else None,
"description": clean_docstring if clean_docstring else None,
"generate_params_schema": cls.generate_params_schema.schema()
if cls.generate_params_schema
else None,
"component_params_schema": cls.component_params_schema.schema()
if cls.component_params_schema
else None,
"component_params_schema": cls.params_schema.schema() if cls.params_schema else None,
}

@classmethod
def get_description(cls) -> Optional[str]:
return inspect.getdoc(cls)


def _clean_docstring(docstring: str) -> str:
lines = docstring.strip().splitlines()
first_line = lines[0]
if len(lines) == 1:
return first_line
else:
rest = textwrap.dedent("\n".join(lines[1:]))
return f"{first_line}\n{rest}"


class ComponentInternalMetadata(TypedDict):
summary: Optional[str]
description: Optional[str]
Expand All @@ -94,13 +107,43 @@ def get_entry_points_from_python_environment(group: str) -> Sequence[importlib.m


COMPONENTS_ENTRY_POINT_GROUP = "dagster.components"
BUILTIN_COMPONENTS_ENTRY_POINT_BASE = "dagster_components"
BUILTIN_PUBLISHED_COMPONENT_ENTRY_POINT = BUILTIN_COMPONENTS_ENTRY_POINT_BASE
BUILTIN_TEST_COMPONENT_ENTRY_POINT = ".".join([BUILTIN_COMPONENTS_ENTRY_POINT_BASE, "test"])


class ComponentRegistry:
@classmethod
def from_entry_point_discovery(cls) -> "ComponentRegistry":
def from_entry_point_discovery(
cls, builtin_component_lib: str = BUILTIN_PUBLISHED_COMPONENT_ENTRY_POINT
) -> "ComponentRegistry":
"""Discover components registered in the Python environment via the `dagster_components` entry point group.
`dagster-components` itself registers multiple component entry points. We call these
"builtin" component libraries. The `dagster_components` entry point resolves to published
components and is loaded by default. Other entry points resolve to various sets of test
components. This method will only ever load one builtin component library.
Args:
builtin-component-lib (str): Specifies the builtin components library to load. Builtin
copmonents libraries are defined under entry points with names matching the pattern
`dagster_components*`. Only one builtin component library can be loaded at a time.
Defaults to `dagster_components`, the standard set of published components.
"""
components: Dict[str, Type[Component]] = {}
for entry_point in get_entry_points_from_python_environment(COMPONENTS_ENTRY_POINT_GROUP):
# Skip builtin entry points that are not the specified builtin component library.
if (
entry_point.name.startswith(BUILTIN_COMPONENTS_ENTRY_POINT_BASE)
and not entry_point.name == builtin_component_lib
):
continue
elif entry_point.name == BUILTIN_TEST_COMPONENT_ENTRY_POINT:
if builtin_component_lib:
ensure_dagster_components_tests_import()
else:
continue

root_module = entry_point.load()
if not isinstance(root_module, ModuleType):
raise DagsterError(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -77,21 +77,21 @@ def introspect_from_path(path: Path) -> "PipesSubprocessScriptCollection":

@classmethod
def from_decl_node(
cls, load_context: ComponentLoadContext, component_decl: ComponentDeclNode
cls, load_context: ComponentLoadContext, decl_node: ComponentDeclNode
) -> "PipesSubprocessScriptCollection":
assert isinstance(component_decl, YamlComponentDecl)
assert isinstance(decl_node, YamlComponentDecl)
loaded_params = TypeAdapter(cls.params_schema).validate_python(
component_decl.component_file_model.params
decl_node.component_file_model.params
)

path_specs = {}
for script in loaded_params.scripts:
script_path = component_decl.path / script.path
script_path = decl_node.path / script.path
if not script_path.exists():
raise FileNotFoundError(f"Script {script_path} does not exist")
path_specs[script_path] = [spec.to_asset_spec() for spec in script.assets]

return cls(dirpath=component_decl.path, path_specs=path_specs)
return cls(dirpath=decl_node.path, path_specs=path_specs)

def build_defs(self, load_context: "ComponentLoadContext") -> "Definitions":
from dagster._core.definitions.definitions_class import Definitions
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,14 @@
import sys
from pathlib import Path

CLI_BUILTIN_COMPONENT_LIB_KEY = "builtin_component_lib"


def ensure_dagster_components_tests_import() -> None:
from dagster_components import __file__ as dagster_components_init_py

dagster_components_package_root = (Path(dagster_components_init_py) / ".." / "..").resolve()
assert (
dagster_components_package_root / "dagster_components_tests"
).exists(), "Could not find dagster_components_tests where expected"
sys.path.append(dagster_components_package_root.as_posix())
Original file line number Diff line number Diff line change
@@ -0,0 +1,103 @@
import json
from pathlib import Path

from click.testing import CliRunner
from dagster_components.cli import cli
from dagster_components.utils import ensure_dagster_components_tests_import

ensure_dagster_components_tests_import()

from dagster_components_tests.utils import temp_code_location_bar


# Test that the global --use-test-component-lib flag changes the registered components
def test_global_test_flag():
runner: CliRunner = CliRunner()

# standard
result = runner.invoke(cli, ["list", "component-types"])
assert result.exit_code == 0
default_result_keys = list(json.loads(result.output).keys())
assert len(default_result_keys) > 0

result = runner.invoke(
cli, ["--builtin-component-lib", "dagster_components.test", "list", "component-types"]
)
assert result.exit_code == 0
test_result_keys = list(json.loads(result.output).keys())
assert len(default_result_keys) > 0

assert default_result_keys != test_result_keys


def test_list_component_types_command():
runner = CliRunner()

result = runner.invoke(
cli, ["--builtin-component-lib", "dagster_components.test", "list", "component-types"]
)
assert result.exit_code == 0
result = json.loads(result.output)

assert list(result.keys()) == [
"dagster_components.test.all_metadata_empty_asset",
"dagster_components.test.simple_asset",
"dagster_components.test.simple_pipes_script_asset",
]

assert result["dagster_components.test.simple_asset"] == {
"name": "simple_asset",
"package": "dagster_components.test",
"summary": "A simple asset that returns a constant string value.",
"description": "A simple asset that returns a constant string value.",
"generate_params_schema": None,
"component_params_schema": {
"properties": {
"asset_key": {"title": "Asset Key", "type": "string"},
"value": {"title": "Value", "type": "string"},
},
"required": ["asset_key", "value"],
"title": "SimpleAssetParams",
"type": "object",
},
}

pipes_script_params_schema = {
"properties": {
"asset_key": {"title": "Asset Key", "type": "string"},
"filename": {"title": "Filename", "type": "string"},
},
"required": ["asset_key", "filename"],
"title": "SimplePipesScriptAssetParams",
"type": "object",
}

assert result["dagster_components.test.simple_pipes_script_asset"] == {
"name": "simple_pipes_script_asset",
"package": "dagster_components.test",
"summary": "A simple asset that runs a Python script with the Pipes subprocess client.",
"description": "A simple asset that runs a Python script with the Pipes subprocess client.\n\nBecause it is a pipes asset, no value is returned.",
"generate_params_schema": pipes_script_params_schema,
"component_params_schema": pipes_script_params_schema,
}


def test_generate_component_command():
runner = CliRunner()

with temp_code_location_bar():
result = runner.invoke(
cli,
[
"--builtin-component-lib",
"dagster_components.test",
"generate",
"component",
"dagster_components.test.simple_pipes_script_asset",
"qux",
"--json-params",
'{"asset_key": "my_asset", "filename": "my_asset.py"}',
],
)
assert result.exit_code == 0
assert Path("bar/components/qux/my_asset.py").exists()
Original file line number Diff line number Diff line change
@@ -0,0 +1,7 @@
from dagster_components_tests.lib.all_metadata_empty_asset import (
AllMetadataEmptyAsset as AllMetadataEmptyAsset,
)
from dagster_components_tests.lib.simple_asset import SimpleAsset as SimpleAsset
from dagster_components_tests.lib.simple_pipes_script_asset import (
SimplePipesScriptAsset as SimplePipesScriptAsset,
)
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
from typing import TYPE_CHECKING, Any

from dagster._core.definitions.decorators.asset_decorator import asset
from dagster._core.definitions.definitions_class import Definitions
from dagster._core.execution.context.asset_execution_context import AssetExecutionContext
from dagster_components import Component, ComponentLoadContext, component
from dagster_components.core.component import ComponentGenerateRequest
from dagster_components.core.component_decl_builder import YamlComponentDecl
from dagster_components.generate import generate_component_yaml
from typing_extensions import Self

if TYPE_CHECKING:
from dagster_components.core.component import ComponentDeclNode


@component(name="all_metadata_empty_asset")
class AllMetadataEmptyAsset(Component):
@classmethod
def from_decl_node(
cls, context: "ComponentLoadContext", decl_node: "ComponentDeclNode"
) -> Self:
assert isinstance(decl_node, YamlComponentDecl)
return cls()

@classmethod
def generate_files(cls, request: ComponentGenerateRequest, params: Any) -> None:
generate_component_yaml(request, params)

def build_defs(self, context: ComponentLoadContext) -> Definitions:
@asset
def hardcoded_asset(context: AssetExecutionContext):
return 1

return Definitions(assets=[hardcoded_asset])
Loading

0 comments on commit 3c9b9ac

Please sign in to comment.