Skip to content

Commit

Permalink
[DbtProject] Rename --use-experimental-dbt-project flag to `--use-d…
Browse files Browse the repository at this point in the history
…bt-project` in dbt scaffold CLI (#22794)

## Summary & Motivation

This PR renames the `--use-experimental-dbt-project` flag to
`--use-dbt-project` in the dbt scaffold CLI, now that [DbtProject is no
more experimental](#22578).

This is the first step in making DbtProject the default and only
strategy when scaffolding a dbt project. The goal is to facilitate the
transition from the previous default strategy to that of DbtProject in
our tutorials, courses and docs.

Once this is merged, next steps will be to
- update tutorials, courses and docs to use DbtProject.
- Since our examples heavily rely on the scaffold command, we'll be able
to use `dagster-dbt project scaffold --use-dbt-project` with the updated
flag
- make the DbtProject strategy the default and only one when using
`dagster-dbt project scaffold`
- update tutorials, courses and docs to remove the flag

Tickets are listed
[here](https://linear.app/dagster-labs/issue/DS-337/make-dbtproject-the-default-scaffold-strategy).

## How I Tested These Changes

BK with updated tests
  • Loading branch information
maximearmstrong authored Jul 2, 2024
1 parent b423978 commit 8116206
Show file tree
Hide file tree
Showing 6 changed files with 37 additions and 36 deletions.
15 changes: 8 additions & 7 deletions python_modules/libraries/dagster-dbt/dagster_dbt/cli/app.py
Original file line number Diff line number Diff line change
Expand Up @@ -108,7 +108,7 @@ def copy_scaffold(
dagster_project_dir: Path,
dbt_project_dir: Path,
use_experimental_dbt_state: bool,
use_experimental_dbt_project: bool,
use_dbt_project: bool,
) -> None:
dbt_project_yaml_path = dbt_project_dir.joinpath(DBT_PROJECT_YML_NAME)
dbt_project_yaml: Dict[str, Any] = yaml.safe_load(dbt_project_yaml_path.read_bytes())
Expand Down Expand Up @@ -161,14 +161,14 @@ def copy_scaffold(
dbt_core_version_upper_bound=DBT_CORE_VERSION_UPPER_BOUND,
project_name=project_name,
use_experimental_dbt_state=use_experimental_dbt_state,
use_experimental_dbt_project=use_experimental_dbt_project,
use_dbt_project=use_dbt_project,
).dump(destination_path)

path.unlink()

dagster_project_dir.joinpath("scaffold").rename(dagster_project_dir.joinpath(project_name))

if use_experimental_dbt_project:
if use_dbt_project:
dagster_project_dir.joinpath(project_name, "constants.py").unlink()
else:
dagster_project_dir.joinpath(project_name, "project.py").unlink()
Expand Down Expand Up @@ -246,10 +246,11 @@ def project_scaffold_command(
hidden=True,
),
] = False,
use_experimental_dbt_project: Annotated[
use_dbt_project: Annotated[
bool,
typer.Option(
...,
"--use-dbt-project",
"--use-experimental-dbt-project",
"--use-dbt-project-package-data-dir",
help="Controls whether `DbtProject` is used.",
Expand All @@ -273,18 +274,18 @@ def project_scaffold_command(
)

dagster_project_dir = Path.cwd().joinpath(project_name)
use_experimental_dbt_project = use_experimental_dbt_project or use_experimental_dbt_state
use_dbt_project = use_dbt_project or use_experimental_dbt_state

copy_scaffold(
project_name=project_name,
dagster_project_dir=dagster_project_dir,
dbt_project_dir=dbt_project_dir,
use_experimental_dbt_state=use_experimental_dbt_state,
use_experimental_dbt_project=use_experimental_dbt_project,
use_dbt_project=use_dbt_project,
)

dagster_dev_command = "DAGSTER_DBT_PARSE_PROJECT_ON_LOAD=1 dagster dev"
if use_experimental_dbt_project:
if use_dbt_project:
dagster_dev_command = "dagster dev"

console.print(
Expand Down
Original file line number Diff line number Diff line change
@@ -1,13 +1,13 @@
from dagster import AssetExecutionContext
from dagster_dbt import DbtCliResource, dbt_assets

{% if use_experimental_dbt_project -%}
{% if use_dbt_project -%}
from .project import {{ dbt_project_name }}
{% else -%}
from .constants import dbt_manifest_path
{% endif %}

{% if use_experimental_dbt_project -%}
{% if use_dbt_project -%}
@dbt_assets(manifest={{ dbt_project_name }}.manifest_path)
{% else -%}
@dbt_assets(manifest=dbt_manifest_path)
Expand Down
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
{% if use_experimental_dbt_project -%}
{% if use_dbt_project -%}
from dagster import Definitions
from dagster_dbt import DbtCliResource
{% else -%}
Expand All @@ -8,7 +8,7 @@ from dagster import Definitions
from dagster_dbt import DbtCliResource
{% endif %}
from .assets import {{ dbt_assets_name }}
{% if use_experimental_dbt_project -%}
{% if use_dbt_project -%}
from .project import {{ dbt_project_name }}
{% else -%}
from .constants import dbt_project_dir
Expand All @@ -19,7 +19,7 @@ defs = Definitions(
assets=[{{ dbt_assets_name }}],
schedules=schedules,
resources={
{% if use_experimental_dbt_project -%}
{% if use_dbt_project -%}
"dbt": DbtCliResource(project_dir={{ dbt_project_name }}),
{%- else -%}
"dbt": DbtCliResource(project_dir=os.fspath(dbt_project_dir)),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -4,7 +4,7 @@ setup(
name="{{ project_name }}",
version="0.0.1",
packages=find_packages(),
{%- if use_experimental_dbt_project %}
{%- if use_dbt_project %}
package_data={
"{{ project_name }}": [
"dbt-project/**/*",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ def test_prepare_and_package(monkeypatch: pytest.MonkeyPatch, dbt_project_dir: P
project_name,
"--dbt-project-dir",
os.fspath(dbt_project_dir),
"--use-experimental-dbt-project",
"--use-dbt-project",
],
)

Expand Down Expand Up @@ -75,7 +75,7 @@ def test_prepare_and_package_with_dependencies(
project_name,
"--dbt-project-dir",
os.fspath(dbt_project_dir),
"--use-experimental-dbt-project",
"--use-dbt-project",
],
)

Expand Down Expand Up @@ -133,7 +133,7 @@ def test_prepare_and_package_with_packages(
project_name,
"--dbt-project-dir",
os.fspath(dbt_project_dir),
"--use-experimental-dbt-project",
"--use-dbt-project",
],
)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ def _assert_scaffold_invocation(
dbt_project_dir: Path,
dagster_project_dir: Path,
use_dbt_project_package_data_dir: bool,
use_experimental_dbt_project: bool,
use_dbt_project: bool,
) -> None:
result = runner.invoke(
app,
Expand All @@ -38,7 +38,7 @@ def _assert_scaffold_invocation(
"--dbt-project-dir",
os.fspath(dbt_project_dir),
*(["--use-dbt-project-package-data-dir"] if use_dbt_project_package_data_dir else []),
*(["--use-experimental-dbt-project"] if use_experimental_dbt_project else []),
*(["--use-dbt-project"] if use_dbt_project else []),
],
)

Expand All @@ -53,7 +53,7 @@ def _assert_scaffold_invocation(
in dagster_project_dir.joinpath("setup.py").read_text()
)

if use_dbt_project_package_data_dir or use_experimental_dbt_project:
if use_dbt_project_package_data_dir or use_dbt_project:
assert dagster_project_dir.joinpath(project_name, "project.py").exists()
assert not dagster_project_dir.joinpath(project_name, "constants.py").exists()
else:
Expand Down Expand Up @@ -89,26 +89,26 @@ def _update_dbt_project_path(
return dbt_project_dir


@pytest.mark.parametrize("use_experimental_dbt_project", [True, False])
@pytest.mark.parametrize("use_dbt_project", [True, False])
@pytest.mark.parametrize("use_dbt_project_package_data_dir", [True, False])
def test_project_scaffold_command_with_precompiled_manifest(
monkeypatch: pytest.MonkeyPatch,
tmp_path: Path,
dbt_project_dir: Path,
use_dbt_project_package_data_dir: bool,
use_experimental_dbt_project: bool,
use_dbt_project: bool,
) -> None:
monkeypatch.chdir(tmp_path)

project_name = f"test_dagster_scaffold_precompiled_manifest_{use_experimental_dbt_project}"
project_name = f"test_dagster_scaffold_precompiled_manifest_{use_dbt_project}"
dagster_project_dir = tmp_path.joinpath(project_name)

_assert_scaffold_invocation(
project_name=project_name,
dbt_project_dir=dbt_project_dir,
dagster_project_dir=dagster_project_dir,
use_dbt_project_package_data_dir=use_dbt_project_package_data_dir,
use_experimental_dbt_project=use_experimental_dbt_project,
use_dbt_project=use_dbt_project,
)

dbt_project_dir = _update_dbt_project_path(
Expand All @@ -127,26 +127,26 @@ def test_project_scaffold_command_with_precompiled_manifest(
_assert_scaffold_defs(project_name=project_name, dagster_project_dir=dagster_project_dir)


@pytest.mark.parametrize("use_experimental_dbt_project", [False])
@pytest.mark.parametrize("use_dbt_project", [False])
@pytest.mark.parametrize("use_dbt_project_package_data_dir", [False])
def test_project_scaffold_command_without_dbt_project_with_runtime_manifest(
monkeypatch: pytest.MonkeyPatch,
tmp_path: Path,
dbt_project_dir: Path,
use_dbt_project_package_data_dir: bool,
use_experimental_dbt_project: bool,
use_dbt_project: bool,
) -> None:
monkeypatch.chdir(tmp_path)

project_name = f"test_dagster_scaffold_runtime_manifest_{use_experimental_dbt_project}"
project_name = f"test_dagster_scaffold_runtime_manifest_{use_dbt_project}"
dagster_project_dir = tmp_path.joinpath(project_name)

_assert_scaffold_invocation(
project_name=project_name,
dbt_project_dir=dbt_project_dir,
dagster_project_dir=dagster_project_dir,
use_dbt_project_package_data_dir=use_dbt_project_package_data_dir,
use_experimental_dbt_project=use_experimental_dbt_project,
use_dbt_project=use_dbt_project,
)

dbt_project_dir = _update_dbt_project_path(
Expand All @@ -162,30 +162,30 @@ def test_project_scaffold_command_without_dbt_project_with_runtime_manifest(
_assert_scaffold_defs(project_name=project_name, dagster_project_dir=dagster_project_dir)


@pytest.mark.parametrize("use_experimental_dbt_project", [True, False])
@pytest.mark.parametrize("use_dbt_project", [True, False])
@pytest.mark.parametrize("use_dbt_project_package_data_dir", [True, False])
def test_project_scaffold_command_with_dbt_project_with_runtime_manifest(
monkeypatch: pytest.MonkeyPatch,
tmp_path: Path,
dbt_project_dir: Path,
use_dbt_project_package_data_dir: bool,
use_experimental_dbt_project: bool,
use_dbt_project: bool,
) -> None:
# Skip the case where we don't use DbtProject
if not use_dbt_project_package_data_dir and not use_experimental_dbt_project:
if not use_dbt_project_package_data_dir and not use_dbt_project:
pytest.skip()

monkeypatch.chdir(tmp_path)

project_name = f"test_dagster_scaffold_runtime_manifest_{use_experimental_dbt_project}"
project_name = f"test_dagster_scaffold_runtime_manifest_{use_dbt_project}"
dagster_project_dir = tmp_path.joinpath(project_name)

_assert_scaffold_invocation(
project_name=project_name,
dbt_project_dir=dbt_project_dir,
dagster_project_dir=dagster_project_dir,
use_dbt_project_package_data_dir=use_dbt_project_package_data_dir,
use_experimental_dbt_project=use_experimental_dbt_project,
use_dbt_project=use_dbt_project,
)

dbt_project_dir = _update_dbt_project_path(
Expand All @@ -202,26 +202,26 @@ def test_project_scaffold_command_with_dbt_project_with_runtime_manifest(
_assert_scaffold_defs(project_name=project_name, dagster_project_dir=dagster_project_dir)


@pytest.mark.parametrize("use_experimental_dbt_project", [True, False])
@pytest.mark.parametrize("use_dbt_project", [True, False])
@pytest.mark.parametrize("use_dbt_project_package_data_dir", [True, False])
def test_project_scaffold_command_with_runtime_manifest_without_env_var(
monkeypatch: pytest.MonkeyPatch,
tmp_path: Path,
dbt_project_dir: Path,
use_dbt_project_package_data_dir: bool,
use_experimental_dbt_project: bool,
use_dbt_project: bool,
) -> None:
monkeypatch.chdir(tmp_path)

project_name = f"test_scaffold_runtime_without_env_var_{use_experimental_dbt_project}"
project_name = f"test_scaffold_runtime_without_env_var_{use_dbt_project}"
dagster_project_dir = tmp_path.joinpath(project_name)

_assert_scaffold_invocation(
project_name=project_name,
dbt_project_dir=dbt_project_dir,
dagster_project_dir=dagster_project_dir,
use_dbt_project_package_data_dir=use_dbt_project_package_data_dir,
use_experimental_dbt_project=use_experimental_dbt_project,
use_dbt_project=use_dbt_project,
)

dbt_project_dir = _update_dbt_project_path(
Expand Down

0 comments on commit 8116206

Please sign in to comment.