[RFC] Ergonomic improvements for Dagster’s dbt integration #14477
Replies: 13 comments 54 replies
-
The @dbt_asset requires the manifest. Does this mean you will no longer support the dynamic generation of the manifest like: load_assets_from_dbt_project? Where is the io_manager_key specified in the new approach? In case you want to get pre-created manifests:
In the past, the target was branch aware. Will this still be true? Or would you like me to take care myself when generating the manifest file? |
Beta Was this translation helpful? Give feedback.
-
Previously I had a function like:
which ran outside of the normal Now, I see that the |
Beta Was this translation helpful? Give feedback.
-
Have you folks considered using the new interfaces in Similar in mechanics to what you have built but native to dbt. Disclaimer: still early days, so it's subject to change. https://docs.getdbt.com/reference/programmatic-invocations Simple run example from dbt.cli.main import dbtRunner, dbtRunnerResult
# initialize
dbt = dbtRunner()
# create CLI args as a list of strings
cli_args = ["run", "--select", "tag:my_tag"]
# run the command
res: dbtRunnerResult = dbt.invoke(cli_args)
# inspect the results
for r in res.result:
print(f"{r.node.name}: {r.status}") Do fancier things with the from dbt.cli.main import dbtRunner, dbtRunnerResult
from dbt.contracts.graph.manifest import Manifest
# use 'parse' command to load a Manifest
res: dbtRunnerResult = dbtRunner().invoke(["parse"])
manifest: Manifest = res.result
# introspect manifest
# e.g. assert every public model has a description
for node in manifest.nodes.values():
if node.resource_type == "model" and node.access == "public":
assert node.description != "", f"{node.name} is missing a description"
# reuse this manifest in subsequent commands to skip parsing
dbt = dbtRunner(manifest=manifest)
cli_args = ["run", "--select", "tag:my_tag"]
res = dbt.invoke(cli_args) |
Beta Was this translation helpful? Give feedback.
-
I am currently using the This is an example of my current configuration: dbt_assets = load_assets_from_dbt_manifest(
manifest_file,
io_manager_key="io_manager",
key_prefix=["analytics"],
source_key_prefix=["analytics"],
node_info_to_group_fn=node_info_to_group_fn,
)
antifraud_dbt_assets = DbtManifestAssetSelection(
manifest_json=manifest_file,
select="tag:antifraud",
node_info_to_asset_key=node_info_to_asset_key,
)
antifraud_assets_job = define_asset_job(
name="antifraud_assets_job",
executor_def=in_process_executor,
selection=antifraud_dbt_assets,
) I then declare |
Beta Was this translation helpful? Give feedback.
-
Big fan of these changes, feel like I have a lot of control over my dbt run configs now :) It would be great if the Also is there a way to access the dbt exit code? I had a situation where the dagster run status was Success even though the dbt run failed because of bad credentials, so I've been using the run artifacts to confirm the run was successful: def dbt_post_run_analysis(dbt_cli_task: DbtCliTask):
try:
run_results = dbt_cli_task.get_artifact("run_results.json")
sys.stdout.write(pprint.pformat(run_results))
except FileNotFoundError:
raise Exception("DBT run results not found")
for asset_result in run_results['results']:
if asset_result['status'] != 'success':
raise Exception("DBT run failed") |
Beta Was this translation helpful? Give feedback.
-
I noticed you mentioned that as a dbt user I "want to declaratively schedule my assets (node_info_to_freshness_policy_fn, node_info_to_auto_materialize_policy_fn)". How does this play into the new @dbt_assets decorator? Do we define auto materialize there? I'm hoping this makes it easier to declaratively schedule my entire dbt graph, and preferably, also be able to access the dbt sources and set up source observations. |
Beta Was this translation helpful? Give feedback.
-
Hi! I'm trying to adapt our project to use the new dbt API but I'm having some issues when trying to materialize the events generated by DbtCli. @op(
required_resource_keys={"dbt"},
)
def custom_dbt_build_op(context):
# run the command
output = context.resources.dbt.cli(command="build")
manifest = context.resources.dbt.get_manifest_json()
# logs all materializations to Dagster
events = generate_events(
dbt_output=output,
manifest_json=manifest,
)
for event in events:
context.log_event(event)
return output My question is how to log the materializations events using the new |
Beta Was this translation helpful? Give feedback.
-
Hey! I started using the new API and noticed a couple of things missing (or I did not understood fully):
|
Beta Was this translation helpful? Give feedback.
-
An end-to-end example of using the new decorator is now available in our demo project: https://github.com/dagster-io/hooli-data-eng-pipelines Take a look specifically at: https://github.com/dagster-io/hooli-data-eng-pipelines/blob/master/hooli_data_eng/assets/dbt_assets.py A few notes:
|
Beta Was this translation helpful? Give feedback.
-
Hello, here is an other feature request for the new API. We are using state selector, specifically the source_status:fresher+. The issue that we encounter with the new API is the ability to execute multiple dbt commands in the same target folder or to be able to set the target folder. In order to use source_status:fresher+ selector, See the following dbt documentation for reference : Thanks |
Beta Was this translation helpful? Give feedback.
-
Hi all, I am wondering if future releases would include creating an upstream dependency between a DBT model and any arbitrary asset in Dagster. For example, a dummy function like
and having a DBT model have an upstream dependency on this hello asset. If this is not supported, does anyone have some suggestions on a potential work around? Thank you! |
Beta Was this translation helpful? Give feedback.
-
It would be really nice to be able to pass dbt tags to dagster tags. My main problem is that I need to limit the number of concurrent dbt runs to 1. But I only want to limit dbt assets and leave the python dagster assets unlimited. I found about the limits with tags but I'm not able to set the tags. I'm using the latest version 1.4.2 and importing assets with @dbt_assets syntax. |
Beta Was this translation helpful? Give feedback.
-
Tried using this out and I'm getting an import error, is there anything that I missed?
I am using poetry, and these are the dependencies.
|
Beta Was this translation helpful? Give feedback.
-
This discussion is now locked, as the new
dagster-dbt
API's have been released. Thank you to the community for feedback on the early release of these API's.In our
dagster>1.4.0
anddagster-dbt>0.20.0
release, these new API's were unmarked as experimental.You can check the following resources for information on the new API's:
dagster-dbt
API docsFor a previous history on the RFC, see below.
[RFC] Ergonomic improvements for Dagster’s dbt integration
Summary
This discussion will provide an overview of the API changes to the
dagster-dbt
integration which will launch as experimental with the integration’s0.19.6
release, scheduled for May 25, 2023.We would like to use this document to gather feedback and answer questions about the new APIs. If you have any comments or use cases that are not addressed, please participate in the discussion!
There are three main new interfaces that we plan to release as part of this release:
@dbt_assets
, a decorator-based interface for users to define software-defined assets from their dbt resources.DbtCli
, an improved and lighter interface for invoking dbt commands within the Dagster framework.DbtManifest
, a representation of your dbt project’s resources (models, tests, macros, etc), which will be used as arguments when invoking (1) and (2).Our goal is that this combination of new tools will replace usage of the existing interfaces in the future, as the new tools will provide equivalent and improved functionality, with better ergonomics in composability. Explicitly, the existing interfaces we plan to replace are:
load_assets_from_dbt_manifest
load_assets_from_dbt_project
DbtCliClient
Requirements
dagster-dbt>=0.19.6
, which is scheduled to be released on May 25, 2023.dbt-core>=1.4.0
, which was released on January 25, 2023. We include this constraint to make first-class use of dbt’s new consistency in their events and structured logging system.Notable Changes
Using a decorator to define software-defined assets from dbt
The current
dagster-dbt
APIs for creating software-defined assets were sufficient for common use cases like invokingdbt run
anddbt build
on a set of dbt resources. However, as use cases expanded and customization became a first-class requirement, the current interface needs to be more robust to accomplish complex use cases. It is not sustainable to continue adding a function argument to our APIs every time a new use case emerges.We classified customization concerns into two classes of problems. Included are the function arguments that were used to allow for this behavior. We analyze
load_assets_from_dbt_manifest
as an example.AssetsDefinition
.manifest_json
,select
,exclude
,selected_unique_ids
).op_name
).key_prefix
,source_key_prefix
,node_info_to_asset_key
).partitions_def
).node_info_to_group_fn
).node_info_to_freshness_policy_fn
,node_info_to_auto_materialize_policy_fn
).node_info_to_definition_metadata_fn
,display_raw_sql
).AssetsDefinition
using an arbitrary compute function.runtime_metadata_fn
).io_manager_key
).dbt run
ordbt build
(use_build_command
).partition_key_to_vars_fn
).dbt_resource_key
).Our major insight was that the
dagster-dbt
integration can focus on (1), which is providing utilities for ingesting software-defined assets from dbt. Meanwhile, (2) was essentially giving the user reign about defining their computation, using the Dagster framework, for the software-defined assets they have selected in (1).By leaning into this insight, we essentially a receive similar interface to what we have for our
@asset
and@multi_asset
decorators in the core Dagster framework! The utilities for (1) essentially give sensible defaults for the arguments to the@asset
and@multi_asset
decorator. Then, (2) gives the user the freedom within the bounds of the Dagster framework to define their computation.New Proposed APIs
We introduce three new interfaces to revamp the integration.
@dbt_assets
, a decorator-based interface for users to define software-defined assets from their dbt resources.DbtManifest
, and dbt selection methods (select
,exclude
) to choose a set of dbt resources to model as software-defined assets.DbtCli
, an improved and lighter interface for invoking dbt commands within the Dagster framework.DbtManifest
, a representation of your dbt project’s resources (models, tests, macros, etc), which will be used as arguments when invoking (1) and (2).Examples
We have released an example repository that shows off these new APIs in action against a standard
jaffle_shop
dbt project.Later, we will release a dedicated tutorial and additional documentation to explain how to get started with these APIs in an existing dbt project.
Here, we highlight some of the main examples.
1. Invoking
dbt run
on my entire dbt projectBefore 0.19.6
After 0.19.6
2. Invoking
dbt build
on my entire dbt projectBefore 0.19.6
After 0.19.6
3. Customizing the asset keys for my dbt software-defined assets
Before 0.19.6
After 0.19.6
4. Invoking
dbt seed
,dbt run
,dbt test
as separate commandsBefore 0.19.6
Not possible without forking the
load_assets_from_dbt_manifest
implementation.After 0.19.6
5. Accessing dbt artifacts after invoking your computation
Before 0.19.6
Not possible without forking the
load_assets_from_dbt_manifest
implementation.After 0.19.6
Next Steps
Again, we would like to use this document to gather feedback and answer questions about the new APIs. If you have any comments or use cases that are not addressed, please participate in the discussion!
Beta Was this translation helpful? Give feedback.
All reactions