Skip to content

Commit e33af3c

Browse files
author
jjyeo
committed
[docs] fix dagster-dbt tutorial
- change variable name `dbt_project` to `jaffle_shop_project` in code snippets - add `packaged_project_dir` parameter to `DbtProject` - this parameter is automatically added by the `dagster-dbt project scaffold` command - update `downstream-assets.md` and `load-dbt-models.md`
1 parent a28a23f commit e33af3c

File tree

10 files changed

+26
-23
lines changed

10 files changed

+26
-23
lines changed

docs/docs/integrations/libraries/dbt/using-dbt-with-dagster/downstream-assets.md

+2-2
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,7 @@ pip install plotly
2222

2323
You've added upstream assets to your data pipeline, but nothing downstream - until now. In this step, you'll define a Dagster asset called `order_count_chart` that uses the data in the `customers` dbt model to computes a plotly chart of the number of orders per customer.
2424

25-
Like the `raw_customers` asset that we added in the [previous section](upstream-assets#step-2-define-an-upstream-dagster-asset), we'll put this asset in our `definitions.py` file, inside the `jaffle_dagster` directory.
25+
Like the `raw_customers` asset that we added in the [previous section](upstream-assets#step-2-define-an-upstream-dagster-asset), we'll put this asset in our `assets.py` file, inside the `jaffle_dagster` directory.
2626

2727
To add the `order_count_chart` asset:
2828

@@ -38,7 +38,7 @@ To add the `order_count_chart` asset:
3838

3939
This asset definition looks similar the asset we defined in the previous section. In this case, instead of fetching data from an external source and writing it to DuckDB, it reads data from DuckDB, and then uses it to make a plot.
4040

41-
The line `deps=get_asset_key_for_model([jaffle_shop_dbt_assets], "customers")` tells Dagster that this asset is downstream of the `customers` dbt model. This dependency will be displayed as such in Dagster's UI. If you launch a run to materialize both of them, Dagster won't run `order_count_chart` until `customers` completes.
41+
The line `deps=[get_asset_key_for_model([jaffle_shop_dbt_assets], "customers")]` tells Dagster that this asset is downstream of the `customers` dbt model. This dependency will be displayed as such in Dagster's UI. If you launch a run to materialize both of them, Dagster won't run `order_count_chart` until `customers` completes.
4242

4343
3. Add the `order_count_chart` to the `Definitions`:
4444

docs/docs/integrations/libraries/dbt/using-dbt-with-dagster/load-dbt-models.md

+4-4
Original file line numberDiff line numberDiff line change
@@ -74,19 +74,19 @@ The most important file is the Python file that contains the set of definitions
7474

7575
In our `definitions.py` Python file, we import from `assets.py`, which contains the code to model our dbt models as Dagster assets. To return a Dagster asset for each dbt model, the code in this `assets.py` file needs to know what dbt models you have. It finds out what models you have by reading a file called a `manifest.json`, which is a file that dbt can generate for any dbt project and contains information about every model, seed, snapshot, test, etc. in the project.
7676

77-
To retrieve the `manifest.json`, `assets.py` imports from `project.py`, which defines an internal representation of your dbt project. Then, in `assets.py`, the path to the `manifest.json` file can be accessed with `dbt_project.manifest_path`:
77+
To retrieve the `manifest.json`, `assets.py` imports from `project.py`, which defines an internal representation of your dbt project. Then, in `assets.py`, the path to the `manifest.json` file can be accessed with `jaffle_shop_project.manifest_path`:
7878

79-
<CodeExample path="docs_snippets/docs_snippets/integrations/dbt/tutorial/load_dbt_models/project.py" startAfter="start_load_project" endBefore="=end_load_project" />
79+
<CodeExample path="docs_snippets/docs_snippets/integrations/dbt/tutorial/load_dbt_models/project.py" startAfter="start_load_project" endBefore="end_load_project" />
8080

8181
Generating the `manifest.json` file for a dbt project is time-consuming, so it's best to avoid doing so every time this Python module is imported. Thus, in production deployments of Dagster, you'll typically have the CI/CD system that packages up your code generate your `manifest.json`.
8282

8383
However, in development, you typically want changes made to files in your dbt project to be immediately reflected in the Dagster UI without needing to regenerate the manifest.
8484

85-
`dbt_project.prepare_if_dev()` helps with this – it re-generates your `manifest.json` at the time Dagster imports your code, _but_ only if it's being imported by the `dagster dev` command.
85+
`jaffle_shop_project.prepare_if_dev()` helps with this – it re-generates your `manifest.json` at the time Dagster imports your code, _but_ only if it's being imported by the `dagster dev` command.
8686

8787
Once you've got a `manifest.json` file, it's time to define your Dagster assets using it. The following code, in your project's `assets.py`, does this:
8888

89-
<CodeExample path="docs_snippets/docs_snippets/integrations/dbt/tutorial/load_dbt_models/assets.py" startAfter="start_dbt_assets" endBefore="=end_dbt_assets" />
89+
<CodeExample path="docs_snippets/docs_snippets/integrations/dbt/tutorial/load_dbt_models/assets.py" startAfter="start_dbt_assets" endBefore="end_dbt_assets" />
9090

9191
This code might look a bit fancy, because it uses a decorator. Here's a breakdown of what's going on:
9292

examples/docs_snippets/docs_snippets/integrations/dbt/tutorial/downstream_assets/assets.py

+2-2
Original file line numberDiff line numberDiff line change
@@ -8,11 +8,11 @@
88
from dagster import MetadataValue, AssetExecutionContext, asset
99
from dagster_dbt import DbtCliResource, dbt_assets, get_asset_key_for_model
1010

11-
from .project import dbt_project
11+
from .project import jaffle_shop_project
1212

1313
# end_imports
1414

15-
duckdb_database_path = dbt_project.project_dir.joinpath("tutorial.duckdb")
15+
duckdb_database_path = jaffle_shop_project.project_dir.joinpath("tutorial.duckdb")
1616

1717

1818
@asset(compute_kind="python")

examples/docs_snippets/docs_snippets/integrations/dbt/tutorial/downstream_assets/definitions.py

+2-2
Original file line numberDiff line numberDiff line change
@@ -5,14 +5,14 @@
55
from dagster_dbt import DbtCliResource
66

77
from .assets import jaffle_shop_dbt_assets, order_count_chart, raw_customers
8-
from .project import dbt_project
8+
from .project import jaffle_shop_project
99
from .schedules import schedules
1010

1111
defs = Definitions(
1212
assets=[raw_customers, jaffle_shop_dbt_assets, order_count_chart],
1313
schedules=schedules,
1414
resources={
15-
"dbt": DbtCliResource(project_dir=dbt_project),
15+
"dbt": DbtCliResource(project_dir=jaffle_shop_project),
1616
},
1717
)
1818
# end_defs

examples/docs_snippets/docs_snippets/integrations/dbt/tutorial/downstream_assets/project.py

+3-2
Original file line numberDiff line numberDiff line change
@@ -3,7 +3,8 @@
33

44
from dagster_dbt import DbtProject
55

6-
dbt_project = DbtProject(
6+
jaffle_shop_project = DbtProject(
77
project_dir=Path(__file__).joinpath("..", "..", "..").resolve(),
8+
packaged_project_dir=Path(__file__).joinpath("..", "..", "dbt-project").resolve(),
89
)
9-
dbt_project.prepare_if_dev()
10+
jaffle_shop_project.prepare_if_dev()

examples/docs_snippets/docs_snippets/integrations/dbt/tutorial/load_dbt_models/assets.py

+2-2
Original file line numberDiff line numberDiff line change
@@ -3,10 +3,10 @@
33
from dagster import AssetExecutionContext
44
from dagster_dbt import DbtCliResource, dbt_assets
55

6-
from .project import dbt_project
6+
from .project import jaffle_shop_project
77

88

9-
@dbt_assets(manifest=dbt_project.manifest_path)
9+
@dbt_assets(manifest=jaffle_shop_project.manifest_path)
1010
def jaffle_shop_dbt_assets(context: AssetExecutionContext, dbt: DbtCliResource):
1111
yield from dbt.cli(["build"], context=context).stream()
1212

examples/docs_snippets/docs_snippets/integrations/dbt/tutorial/load_dbt_models/project.py

+3-2
Original file line numberDiff line numberDiff line change
@@ -4,11 +4,12 @@
44

55
from dagster_dbt import DbtProject
66

7-
dbt_project = DbtProject(
7+
jaffle_shop_project = DbtProject(
88
project_dir=Path(__file__).joinpath("..", "..", "..").resolve(),
9+
packaged_project_dir=Path(__file__).joinpath("..", "..", "dbt-project").resolve(),
910
)
1011
# If `dagster dev` is used, the dbt project will be prepared to create the manifest at run time.
1112
# Otherwise, we expect a manifest to be present in the project's target directory.
12-
dbt_project.prepare_if_dev()
13+
jaffle_shop_project.prepare_if_dev()
1314

1415
# end_load_project

examples/docs_snippets/docs_snippets/integrations/dbt/tutorial/upstream_assets/assets.py

+3-3
Original file line numberDiff line numberDiff line change
@@ -7,9 +7,9 @@
77
from dagster import AssetExecutionContext, asset
88
from dagster_dbt import DbtCliResource, dbt_assets
99

10-
from .project import dbt_project
10+
from .project import jaffle_shop_project
1111

12-
duckdb_database_path = dbt_project.project_dir.joinpath("tutorial.duckdb")
12+
duckdb_database_path = jaffle_shop_project.project_dir.joinpath("tutorial.duckdb")
1313

1414

1515
@asset(compute_kind="python")
@@ -25,7 +25,7 @@ def raw_customers(context: AssetExecutionContext) -> None:
2525
context.add_output_metadata({"num_rows": data.shape[0]})
2626

2727

28-
@dbt_assets(manifest=dbt_project.manifest_path)
28+
@dbt_assets(manifest=jaffle_shop_project.manifest_path)
2929
def jaffle_shop_dbt_assets(context: AssetExecutionContext, dbt: DbtCliResource):
3030
yield from dbt.cli(["build"], context=context).stream()
3131

examples/docs_snippets/docs_snippets/integrations/dbt/tutorial/upstream_assets/definitions.py

+2-2
Original file line numberDiff line numberDiff line change
@@ -6,14 +6,14 @@
66
from dagster_dbt import DbtCliResource
77

88
from .assets import jaffle_shop_dbt_assets, raw_customers
9-
from .project import dbt_project
9+
from .project import jaffle_shop_project
1010
from .schedules import schedules
1111

1212
defs = Definitions(
1313
assets=[raw_customers, jaffle_shop_dbt_assets],
1414
schedules=schedules,
1515
resources={
16-
"dbt": DbtCliResource(project_dir=dbt_project),
16+
"dbt": DbtCliResource(project_dir=jaffle_shop_project),
1717
},
1818
)
1919

examples/docs_snippets/docs_snippets/integrations/dbt/tutorial/upstream_assets/project.py

+3-2
Original file line numberDiff line numberDiff line change
@@ -3,7 +3,8 @@
33

44
from dagster_dbt import DbtProject
55

6-
dbt_project = DbtProject(
6+
jaffle_shop_project = DbtProject(
77
project_dir=Path(__file__).joinpath("..", "..", "..").resolve(),
8+
packaged_project_dir=Path(__file__).joinpath("..", "..", "dbt-project").resolve(),
89
)
9-
dbt_project.prepare_if_dev()
10+
jaffle_shop_project.prepare_if_dev()

0 commit comments

Comments
 (0)