Skip to content

Commit

Permalink
[components] Add windows testing to dagster-dg
Browse files Browse the repository at this point in the history
  • Loading branch information
smackesey committed Feb 4, 2025
1 parent 594bc15 commit f13115a
Show file tree
Hide file tree
Showing 11 changed files with 68 additions and 31 deletions.
28 changes: 23 additions & 5 deletions azure-pipelines.yml
Original file line number Diff line number Diff line change
Expand Up @@ -28,21 +28,39 @@ jobs:
${{ each py_version in parameters.py3_versions }}:
${{ each env_suffix in parameters.py3_env_suffixes }}:
${{ replace(py_version, '.', '') }}-windows-${{ env_suffix }}:
TOXENV: "py${{ replace(py_version, '.', '') }}-windows-${{ env_suffix }}"
python.version: "${{ py_version }}"
PACKAGE_ROOT: "python_modules\\dagster"
PATH_BACK_TO_REPO_ROOT: "..\\.."
TOX_ENV: "py${{ replace(py_version, '.', '') }}-windows-${{ env_suffix }}"
PYTHON_VERSION: "${{ py_version }}"

${{ each py_version in parameters.py3_versions }}:
dagster-dg-${{ replace(py_version, '.', '') }}:
PACKAGE_ROOT: "python_modules\\libraries\\dagster-dg"
TOX_ENV: windows
PYTHON_VERSION: "${{ py_version }}"
PATH_BACK_TO_REPO_ROOT: "..\\..\\.."
variables:
PYTHONLEGACYWINDOWSSTDIO: "1"
PYTHONUTF8: "1"
steps:
- task: UsePythonVersion@0
inputs:
versionSpec: "$(python.version)"
versionSpec: "$(PYTHON_VERSION)"
architecture: "x64"
- script: pip install "tox<4.0.0" uv
displayName: "Install tox & uv"
- script: cd python_modules\dagster && tox -e %TOXENV% && cd ..\..
- script: |
[Console]::OutputEncoding = [System.Text.Encoding]::UTF8
cd %PACKAGE_ROOT%
tox -e %TOX_ENV%
cd %PATH_BACK_TO_REPO_ROOT%
displayName: "Run tests"
# Use PowerShell for UTF-8 support. If we do not have proper UTF-8 support, there can be
# anomalies in the rendering of certain characters in command output, which can break tests
# that look at e.g. fancy box characters in CLI --help output.
shell: pwsh
- task: PublishTestResults@2
inputs:
testResultsFiles: "**/test_results.xml"
testRunTitle: "dagster $(TOXENV)"
testRunTitle: "$(PACKAGE_ROOT) $(TOX_ENV)"
condition: succeededOrFailed()
7 changes: 3 additions & 4 deletions python_modules/libraries/dagster-dg/dagster_dg/cache.py
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,7 @@
import sys
from collections.abc import Sequence
from pathlib import Path
from typing import Final, Literal, Optional, Union
from typing import Final, Literal, Optional

from typing_extensions import Self, TypeAlias

Expand All @@ -16,9 +16,7 @@

_CACHE_CONTAINER_DIR_NAME: Final = "dg-cache"

CachableDataType: TypeAlias = Union[
Literal["component_registry_data"], Literal["all_components_schema"]
]
CachableDataType: TypeAlias = Literal["component_registry_data", "all_components_schema"]


def get_default_cache_dir() -> Path:
Expand Down Expand Up @@ -78,6 +76,7 @@ def clear_key(self, key: tuple[str, ...]) -> None:

def clear_all(self) -> None:
shutil.rmtree(self._root_path)
assert not self._root_path.exists()
self.log(f"CACHE [clear-all]: {self._root_path}")

def get(self, key: tuple[str, ...]) -> Optional[str]:
Expand Down
3 changes: 2 additions & 1 deletion python_modules/libraries/dagster-dg/dagster_dg/cli/dev.py
Original file line number Diff line number Diff line change
Expand Up @@ -185,7 +185,8 @@ def _format_forwarded_option(option: str, value: object) -> list[str]:


def _get_child_process_pid(proc: "subprocess.Popen") -> int:
children = psutil.Process(proc.pid).children(recursive=False)
# Windows will sometimes return the parent process as its own child. Filter this out.
children = [p for p in psutil.Process(proc.pid).children(recursive=False) if p.pid != proc.pid]
if len(children) != 1:
raise ValueError(f"Expected exactly one child process, but found {len(children)}")
return children[0].pid
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ def install_completion(context: click.Context):
shell, _ = shellingham.detect_shell()
comp_class = click.shell_completion.get_completion_class(shell)
if comp_class is None:
exit_with_error(f"Shell {shell} is not supported.")
exit_with_error(f"Shell `{shell}` is not supported.")
else:
comp_inst = comp_class(
cli=context.command, ctx_args={}, prog_name="dg", complete_var="_DG_COMPLETE"
Expand Down
4 changes: 2 additions & 2 deletions python_modules/libraries/dagster-dg/dagster_dg/scaffold.py
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,7 @@ def get_pyproject_toml_dev_dependencies(use_editable_dagster: bool) -> str:

def get_pyproject_toml_uv_sources(editable_dagster_root: Path) -> str:
lib_lines = [
f'{path.name} = {{ path = "{path}", editable = true }}'
f"{path.name} = {{ path = '{path}', editable = true }}"
for path in _gather_dagster_packages(editable_dagster_root)
]
return "\n".join(
Expand Down Expand Up @@ -140,7 +140,7 @@ def scaffold_component_type(dg_context: DgContext, name: str) -> None:
scaffold_subtree(
path=root_path,
name_placeholder="COMPONENT_TYPE_NAME_PLACEHOLDER",
templates_path=os.path.join(os.path.dirname(__file__), "templates", "COMPONENT_TYPE"),
templates_path=str(Path(__file__).parent / "templates" / "COMPONENT_TYPE"),
project_name=name,
component_type_class_name=camelcase(name),
name=name,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -82,6 +82,12 @@ def is_executable_available(command: str) -> bool:
return bool(shutil.which(command))


# Short for "normalize path"-- use this to get the platform-correct string representation of an
# existing string path.
def npath(path: str):
return str(Path(path))


# uv commands should be executed in an environment with no pre-existing VIRTUAL_ENV set. If this
# variable is set (common during development) and does not match the venv resolved by uv, it prints
# undesireable warnings.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -4,7 +4,7 @@
from pathlib import Path

import pytest
from dagster_dg.utils import ensure_dagster_dg_tests_import
from dagster_dg.utils import ensure_dagster_dg_tests_import, npath

ensure_dagster_dg_tests_import()

Expand Down Expand Up @@ -222,14 +222,14 @@ def test_component_scaffold_succeeds_scaffolded_component_type() -> None:
# ##### REAL COMPONENTS


dbt_project_path = "../stub_code_locations/dbt_project_location/components/jaffle_shop"
dbt_project_path = Path("../stub_code_locations/dbt_project_location/components/jaffle_shop")


@pytest.mark.parametrize(
"params",
[
["--json-params", json.dumps({"project_path": str(dbt_project_path)})],
["--project-path", dbt_project_path],
["--project-path", str(dbt_project_path)],
],
)
def test_scaffold_dbt_project_instance(params) -> None:
Expand All @@ -254,7 +254,7 @@ def test_scaffold_dbt_project_instance(params) -> None:
assert component_yaml_path.exists()
assert "type: dagster_components.dbt_project" in component_yaml_path.read_text()
assert (
"stub_code_locations/dbt_project_location/components/jaffle_shop"
npath("stub_code_locations/dbt_project_location/components/jaffle_shop")
in component_yaml_path.read_text()
)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -102,25 +102,27 @@ def test_code_location_scaffold_editable_dagster_success(mode: str, monkeypatch)
with open("code_locations/foo-bar/pyproject.toml") as f:
toml = tomli.loads(f.read())
assert toml["tool"]["uv"]["sources"]["dagster"] == {
"path": f"{dagster_git_repo_dir}/python_modules/dagster",
"path": f"{dagster_git_repo_dir / 'python_modules' / 'dagster'}",
"editable": True,
}
assert toml["tool"]["uv"]["sources"]["dagster-pipes"] == {
"path": f"{dagster_git_repo_dir}/python_modules/dagster-pipes",
"path": str(dagster_git_repo_dir / "python_modules" / "dagster-pipes"),
"editable": True,
}
assert toml["tool"]["uv"]["sources"]["dagster-webserver"] == {
"path": f"{dagster_git_repo_dir}/python_modules/dagster-webserver",
"path": str(dagster_git_repo_dir / "python_modules" / "dagster-webserver"),
"editable": True,
}
assert toml["tool"]["uv"]["sources"]["dagster-components"] == {
"path": f"{dagster_git_repo_dir}/python_modules/libraries/dagster-components",
"path": str(
dagster_git_repo_dir / "python_modules" / "libraries" / "dagster-components"
),
"editable": True,
}
# Check for presence of one random package with no component to ensure we are
# preemptively adding all packages
assert toml["tool"]["uv"]["sources"]["dagstermill"] == {
"path": f"{dagster_git_repo_dir}/python_modules/libraries/dagstermill",
"path": str(dagster_git_repo_dir / "python_modules" / "libraries" / "dagstermill"),
"editable": True,
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -151,7 +151,8 @@ def _assert_no_child_processes_running(child_procs: list[psutil.Process]) -> Non

def _get_child_processes(pid) -> list[psutil.Process]:
parent = psutil.Process(pid)
return parent.children(recursive=True)
# Windows will sometimes return the parent process as its own child. Filter this out.
return [p for p in parent.children(recursive=True) if p.pid != pid]


def _find_free_port() -> int:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,5 +13,9 @@ def test_install_completion():
with patch("typer._completion_shared.install") as mock_install:
mock_install.return_value = shell, "/some/path"
result = runner.invoke("--install-completion")
assert_runner_result(result)
assert f"{shell} completion installed in /some/path" in result.output
if shell == "cmd": # windows command shell is not supported
assert_runner_result(result, exit_0=False)
assert "Shell `cmd` is not supported" in result.output
else:
assert_runner_result(result)
assert f"{shell} completion installed in /some/path" in result.output
18 changes: 12 additions & 6 deletions python_modules/libraries/dagster-dg/dagster_dg_tests/utils.py
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
DG_CLI_MAX_OUTPUT_WIDTH,
cli as dg_cli,
)
from dagster_dg.utils import discover_git_root, pushd
from dagster_dg.utils import discover_git_root, npath, pushd
from typing_extensions import Self


Expand All @@ -35,9 +35,14 @@ def isolated_components_venv(runner: Union[CliRunner, "ProxyRunner"]) -> Iterato
for path in libraries_paths:
install_args.extend(["-e", str(path)])
subprocess.run(
["uv", "pip", "install", "--python", ".venv/bin/python", *install_args], check=True
["uv", "pip", "install", "--python", npath(".venv/bin/python"), *install_args],
check=True,
)
with modify_environment_variable("PATH", f"{Path.cwd()}/.venv/bin:{os.environ['PATH']}"):

venv_bin = Path.cwd() / ".venv" / "bin"
with modify_environment_variable(
"PATH", os.pathsep.join([str(venv_bin), os.environ["PATH"]])
):
yield


Expand Down Expand Up @@ -71,7 +76,7 @@ def isolated_example_code_location_foo_bar(
dagster_git_repo_dir = str(discover_git_root(Path(__file__)))
if in_deployment:
fs_context = isolated_example_deployment_foo(runner)
code_loc_path = "code_locations/foo-bar"
code_loc_path = npath("code_locations/foo-bar")
else:
fs_context = runner.isolated_filesystem()
code_loc_path = "foo-bar"
Expand Down Expand Up @@ -112,7 +117,7 @@ def isolated_example_component_library_foo_bar(
"foo-bar",
)
with clear_module_from_cache("foo_bar"), pushd("foo-bar"):
shutil.rmtree("foo_bar/components")
shutil.rmtree(npath("foo_bar/components"))

# Make it not a code location
with modify_pyproject_toml() as pyproject_toml:
Expand All @@ -129,7 +134,8 @@ def isolated_example_component_library_foo_bar(

# Install the component library into our venv
subprocess.run(
["uv", "pip", "install", "--python", "../.venv/bin/python", "-e", "."], check=True
["uv", "pip", "install", "--python", npath("../.venv/bin/python"), "-e", "."],
check=True,
)
yield

Expand Down

0 comments on commit f13115a

Please sign in to comment.