Skip to main content

Migrating to dbt components

Dagster supports two ways to integrate with dbt: the dbt component (recommended) and the Pythonic integration library. If you built your Dagster and dbt project with the Pythonic integration, you can migrate to the dbt component and get the same result.

1. Scaffold the dbt component

The first step is to scaffold a dbt component definition. This will generate the defs.yaml configuration file with a path to your dbt project:

my_project/defs/dbt_ingest/defs.yaml
type: dagster_dbt.DbtProjectComponent

attributes:
project: '{{ context.project_root }}/dbt'

2. Remove Pythonic definitions

Since the component handles the creation of any dbt assets in your Dagster project, as well as the configuration of the underlying resource, you can remove the explicit dbt resource creation code:

my_project/defs/resources.py
# These lines can be removed
from pathlib import Path

from dagster_dbt import DbtCliResource, DbtProject

import dagster as dg

dbt_project_directory = Path(__file__).absolute().parent / "dbt"
dbt_project = DbtProject(project_dir=dbt_project_directory)

dbt_resource = DbtCliResource(project_dir=dbt_project)


@dg.definitions
def resources():
return dg.Definitions(
resources={
"dbt": dbt_resource,
}
)


You can also remove any @dbt_assets assets from your code:

my_project/defs/assets.py
# These lines can be removed
from dagster_dbt import DbtCliResource, dbt_assets

import dagster as dg

from .resources import dbt_project


@dbt_assets(manifest=dbt_project.manifest_path)
def dbt_models(context: dg.AssetExecutionContext, dbt: DbtCliResource):
yield from dbt.cli(["build"], context=context).stream()


To ensure that the dbt assets have been replaced correctly, you can execute:

dg check defs

If there are still dbt assets defined via the Pythonic API, or the dbt resource is still present, you will receive a validation error due to duplication of definitions.

Assuming the check passes, you can also execute:

dg list defs

This will list all the assets in your project and allow you to see that the expected dbt assets are present.

3. Migrating custom translators (Optional)

If you had defined a custom DagsterDbtTranslator for your dbt project, the recommended approach is to create a custom subclass of DbtProjectComponent and override the get_asset_spec method. This provides more flexibility and type safety than YAML configuration.

For example, the custom translator:

my_project/defs/assets.py
class CustomDagsterDbtTranslator(DagsterDbtTranslator):
def get_asset_key(self, dbt_resource_props: Mapping[str, Any]) -> dg.AssetKey:
asset_key = super().get_asset_key(dbt_resource_props)
return asset_key.with_prefix("my_prefix_")

def get_group_name(self, dbt_resource_props: Mapping[str, Any]) -> str:
# Customize group names
return "my_dbt_group"

Can be migrated to a custom component by creating a new Python file:

my_project/lib/custom_dbt_component.py
class CustomDbtProjectComponent(DbtProjectComponent):
"""Custom DbtProjectComponent that customizes asset metadata.

This is the recommended approach for migrating custom DagsterDbtTranslator logic.
"""

def get_asset_spec(
self, manifest: Mapping[str, Any], unique_id: str, project: Optional[DbtProject]
) -> dg.AssetSpec:
base_spec = super().get_asset_spec(manifest, unique_id, project)
dbt_props = self.get_resource_props(manifest, unique_id)

# Customize the asset key with a prefix
new_key = dg.AssetKey(["my_prefix_"] + list(base_spec.key.path))

# Customize group name and add custom metadata
return base_spec.replace_attributes(
key=new_key,
group_name="my_dbt_group",
).merge_attributes(
metadata={
"dbt_model_name": dbt_props["name"],
},
)

Then reference this custom component in your defs.yaml:

my_project/defs/dbt_ingest/defs.yaml
type: my_project.lib.custom_dbt_component.CustomDbtProjectComponent

attributes:
project: '{{ context.project_root }}/dbt'

This approach maps translator methods to component methods:

  • get_asset_key() → Override get_asset_spec() and customize the key attribute
  • get_group_name() → Override get_asset_spec() and customize the group_name attribute
  • get_description() → Override get_asset_spec() and customize the description attribute
  • get_metadata() → Override get_asset_spec() and customize the metadata attribute

Alternative: YAML configuration

For simpler customizations, you can also use YAML configuration with the translation field:

my_project/defs/dbt_ingest/defs.yaml
type: dagster_dbt.DbtProjectComponent

attributes:
project: '{{ context.project_root }}/dbt'
select: "customers"
translation:
group_name: dbt_models
description: "Transforms data using dbt model {{ node.name }}"

4. Migrating incremental models (Optional)

If you had incremental models defined in your dbt project, the recommended approach is to create a custom subclass of DbtProjectComponent and override the execute method. This allows you to customize the dbt CLI arguments based on partition information.

For example, the partitioned incremental models:

my_project/defs/assets.py
@dbt_assets(
manifest=dbt_project.manifest_path,
select=INCREMENTAL_SELECTOR,
partitions_def=daily_partition,
)
def incremental_dbt_models(context: dg.AssetExecutionContext, dbt: DbtCliResource):
time_window = context.partition_time_window
dbt_vars = {
"start_date": time_window.start.strftime("%Y-%m-%d"),
"end_date": time_window.end.strftime("%Y-%m-%d"),
}

yield from dbt.cli(
["build", "--vars", json.dumps(dbt_vars)], context=context
).stream()

Can be migrated to a custom component by creating a new Python file:

my_project/lib/incremental_dbt_component.py
class IncrementalDbtProjectComponent(DbtProjectComponent):
"""Custom DbtProjectComponent that handles incremental models with partitions.

This is the recommended approach for migrating incremental model logic.
"""

def execute(
self, context: dg.AssetExecutionContext, dbt: DbtCliResource
) -> Iterator:
time_window = context.partition_time_window
dbt_vars = {
"start_date": time_window.start.strftime("%Y-%m-%d"),
"end_date": time_window.end.strftime("%Y-%m-%d"),
}

yield from dbt.cli(
["build", "--vars", json.dumps(dbt_vars)], context=context
).stream()

To use this custom component, you'll also need to define the partition definition. First, add a new template var to define the partitions:

my_project/defs/dbt_ingest/template_vars.py
import dagster as dg


@dg.template_var
def daily_partitions_def() -> dg.DailyPartitionsDefinition:
return dg.DailyPartitionsDefinition(start_date="2023-01-01")

Then reference the custom component and apply the partition in your defs.yaml:

my_project/defs/dbt_ingest/defs.yaml
type: my_project.lib.incremental_dbt_component.IncrementalDbtProjectComponent

template_vars_module: .template_vars
attributes:
project: '{{ context.project_root }}/dbt'
select: "config.materialized:incremental"
post_processing:
assets:
- target: "*"
attributes:
partitions_def: "{{ daily_partitions_def }}"

Alternative: YAML configuration with cli_args

For simpler cases where you don't need custom logic, you can use YAML configuration with the cli_args field:

my_project/defs/dbt_ingest/defs.yaml
type: dagster_dbt.DbtProjectComponent

template_vars_module: .template_vars
attributes:
project: '{{ context.project_root }}/dbt'
select: "customers"
translation:
group_name: dbt_models
description: "Transforms data using dbt model {{ node.name }}"
cli_args:
- build
- --vars:
start_date: "{{ partition_time_window.start.strftime('%Y-%m-%d') }}"
end_date: "{{ partition_time_window.end.strftime('%Y-%m-%d') }}"
post_processing:
assets:
- target: "*"
attributes:
partitions_def: "{{ daily_partitions_def }}"