Skip to content

Commit

Permalink
[components] Better error message for failing entry points
Browse files Browse the repository at this point in the history
  • Loading branch information
smackesey committed Feb 13, 2025
1 parent ccae30f commit a289baa
Show file tree
Hide file tree
Showing 4 changed files with 112 additions and 43 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@
import inspect
import sys
import textwrap
import warnings
from abc import ABC, abstractmethod
from collections.abc import Iterable, Mapping, Sequence
from dataclasses import dataclass
Expand All @@ -30,7 +31,11 @@
)
from dagster_components.core.schema.base import ResolvableSchema
from dagster_components.core.schema.context import ResolutionContext
from dagster_components.utils import load_module_from_path
from dagster_components.utils import format_error_message, load_module_from_path


class ComponentsEntryPointLoadError(DagsterError):
pass


class ComponentDeclNode(ABC):
Expand Down Expand Up @@ -158,7 +163,19 @@ def from_entry_point_discovery(
):
continue

root_module = entry_point.load()
try:
root_module = entry_point.load()
except Exception as e:
warnings.warn(
format_error_message(f"""
Error loading entry point {entry_point.name} in group
{COMPONENTS_ENTRY_POINT_GROUP}. Any components defined by this entry point
will note be available. Error:
""")
+ "\n"
+ f" {(e)}"
)

if not isinstance(root_module, ModuleType):
raise DagsterError(
f"Invalid entry point {entry_point.name} in group {COMPONENTS_ENTRY_POINT_GROUP}. "
Expand Down
Original file line number Diff line number Diff line change
@@ -1,5 +1,6 @@
import importlib.util
import sys
import textwrap
from collections.abc import Iterator, Mapping
from contextlib import contextmanager
from dataclasses import dataclass
Expand Down Expand Up @@ -36,6 +37,11 @@ def exit_with_error(error_message: str) -> None:
sys.exit(1)


def format_error_message(message: str) -> str:
# width=10000 unwraps any hardwrapping
return textwrap.dedent(textwrap.fill(message, width=10000))


# Temporarily places a path at the front of sys.path, ensuring that any modules in that path are
# importable.
@contextmanager
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,11 @@
from collections.abc import Iterator, Sequence
from contextlib import contextmanager
from pathlib import Path
from typing import Any, Callable, Optional

import tomli
import tomli_w
from dagster._utils import pushd

from dagster_dg.component import GlobalComponentKey

Expand Down Expand Up @@ -36,14 +41,18 @@ def _temp_venv(install_args: Sequence[str]) -> Iterator[Path]:
"""


def _get_component_types_in_python_environment(python_executable: Path) -> Sequence[str]:
def _get_component_print_script_output(python_executable: Path) -> str:
with tempfile.NamedTemporaryFile(mode="w") as f:
f.write(COMPONENT_PRINT_SCRIPT)
f.flush()
result = subprocess.run(
[str(python_executable), f.name], capture_output=True, text=True, check=False
[str(python_executable), f.name], capture_output=True, text=True, check=True
)
return result.stdout.strip().split("\n")
return result.stdout.strip()


def _get_component_types_in_python_environment(python_executable: Path) -> Sequence[str]:
return _get_component_print_script_output(python_executable).split("\n")


def _find_repo_root():
Expand Down Expand Up @@ -151,40 +160,77 @@ def test_all_dagster_components_have_defined_summary():
"""


def test_components_from_third_party_lib(tmpdir):
with tmpdir.as_cwd():
# Create test package that defines some components
os.makedirs("dagster-foo")
with open("dagster-foo/pyproject.toml", "w") as f:
f.write(DAGSTER_FOO_PYPROJECT_TOML)

os.makedirs("dagster-foo/dagster_foo/lib/sub")

with open("dagster-foo/dagster_foo/lib/__init__.py", "w") as f:
f.write(DAGSTER_FOO_LIB_ROOT)

with open("dagster-foo/dagster_foo/lib/sub/__init__.py", "w") as f:
f.write(_generate_test_component_source(2))

# Need pipes because dependency of dagster
deps = [
"-e",
_get_editable_package_root("dagster"),
"-e",
_get_editable_package_root("dagster-components"),
"-e",
_get_editable_package_root("dagster-pipes"),
"-e",
"dagster-foo",
]

with _temp_venv(deps) as python_executable:
component_types = _get_component_types_in_python_environment(python_executable)
assert (
GlobalComponentKey(name="test_component_1", namespace="dagster_foo").to_typename()
in component_types
)
assert (
GlobalComponentKey(name="test_component_2", namespace="dagster_foo").to_typename()
in component_types
)
@contextmanager
def isolated_venv_with_component_lib_dagster_foo(
pre_install_hook: Optional[Callable[[], None]] = None,
):
with tempfile.TemporaryDirectory() as tmpdir:
with pushd(tmpdir):
# Create test package that defines some components
os.makedirs("dagster-foo")
with open("dagster-foo/pyproject.toml", "w") as f:
f.write(DAGSTER_FOO_PYPROJECT_TOML)

os.makedirs("dagster-foo/dagster_foo/lib/sub")

with open("dagster-foo/dagster_foo/lib/__init__.py", "w") as f:
f.write(DAGSTER_FOO_LIB_ROOT)

with open("dagster-foo/dagster_foo/lib/sub/__init__.py", "w") as f:
f.write(_generate_test_component_source(2))

if pre_install_hook:
pre_install_hook()

# Need pipes because dependency of dagster
deps = [
"-e",
_get_editable_package_root("dagster"),
"-e",
_get_editable_package_root("dagster-components"),
"-e",
_get_editable_package_root("dagster-pipes"),
"-e",
"dagster-foo",
]

with _temp_venv(deps) as python_executable:
component_types = _get_component_types_in_python_environment(python_executable)
assert (
GlobalComponentKey(name="test_component_1", namespace="dagster_foo").to_typename()
in component_types
)
assert (
GlobalComponentKey(name="test_component_2", namespace="dagster_foo").to_typename()
in component_types
)
yield python_executable


@contextmanager
def modify_toml(path: Path) -> Iterator[dict[str, Any]]:
toml = tomli.loads(path.read_text())
yield toml
path.write_text(tomli_w.dumps(toml))


def test_components_from_third_party_lib():
with isolated_venv_with_component_lib_dagster_foo() as python_executable:
component_types = _get_component_types_in_python_environment(python_executable)
assert "dagster_foo.test_component_1" in component_types
assert "dagster_foo.test_component_2" in component_types


# def test_entry_point_null_reference():
#
# # Modify the entry point to point to a non-existent module
# def pre_install_hook():
# with modify_toml(Path("dagster-foo/pyproject.toml")) as toml:
# toml["project"]["entry-points"]["dagster.components"]["dagster_foo"] = "fake.module"
#
# with isolated_venv_with_component_lib_dagster_foo(pre_install_hook) as python_executable:
# script_output = _get_component_print_script_output(python_executable)
# # assert
# component_types = _get_component_types_in_python_environment(python_executable)
# assert "dagster_foo.test_component_1" in component_types
# assert "dagster_foo.test_component_2" in component_types
2 changes: 1 addition & 1 deletion python_modules/libraries/dagster-components/setup.py
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,6 @@ def get_version() -> str:
extras_require={
"sling": ["dagster-sling"],
"dbt": ["dagster-dbt"],
"test": ["dbt-duckdb", "dagster-dg"],
"test": ["dbt-duckdb", "dagster-dg", "tomli", "tomli_w"],
},
)

0 comments on commit a289baa

Please sign in to comment.