This library provides a Dagster integration with dbt (data build tool), created by dbt Labs.
dagster_dbt
provides a set of pre-built ops that work with either the CLI or RPC interfaces. For
more advanced use cases, we suggest building your own ops which directly interact with these resources.
dagster_dbt.
dbt_run_op
= <dagster.core.definitions.op_definition.OpDefinition object>[source]¶If True, materializations corresponding to the results of the dbt operation will be yielded when the op executes. Default: True
Default Value: True
If provided and yield_materializations is True, these components will be used to prefix the generated asset keys.
Default Value: [‘dbt’]
This op executes a dbt run
command. It requires the use of a dbt resource, which can be
set to execute this command through the CLI (using the dbt_cli_resource
) or
over RPC (using the dbt_rpc_sync_resource
).
Examples:
from dagster import job
from dagster_dbt import dbt_run_op, dbt_cli_resource, dbt_rpc_sync_resource
@job(resource_defs={"dbt":dbt_cli_resource})
def my_dbt_cli_job():
dbt_run_op()
@job(resource_defs={"dbt":dbt_rpc_sync_resource})
def my_dbt_rpc_job():
dbt_run_op()
dagster_dbt.
dbt_compile_op
(context)[source]¶This op executes a dbt compile
command. It requires the use of a dbt resource, which can be
set to execute this command through the CLI (using the dbt_cli_resource
) or
over RPC (using the dbt_rpc_sync_resource
).
Examples:
from dagster import job
from dagster_dbt import dbt_compile_op, dbt_cli_resource, dbt_rpc_sync_resource
@job(resource_defs={"dbt":dbt_cli_resource})
def my_dbt_cli_job():
dbt_compile_op()
@job(resource_defs={"dbt":dbt_rpc_sync_resource})
def my_dbt_rpc_job():
dbt_compile_op()
dagster_dbt.
dbt_ls_op
(context)[source]¶This op executes a dbt ls
command. It requires the use of a dbt resource, which can be
set to execute this command through the CLI (using the dbt_cli_resource
) or
over RPC (using the dbt_rpc_sync_resource
).
Examples:
from dagster import job
from dagster_dbt import dbt_ls_op, dbt_cli_resource, dbt_rpc_sync_resource
@job(resource_defs={"dbt":dbt_cli_resource})
def my_dbt_cli_job():
dbt_ls_op()
@job(resource_defs={"dbt":dbt_rpc_sync_resource})
def my_dbt_rpc_job():
dbt_ls_op()
dagster_dbt.
dbt_test_op
(context)[source]¶This op executes a dbt test
command. It requires the use of a dbt resource, which can be
set to execute this command through the CLI (using the dbt_cli_resource
) or
over RPC (using the dbt_rpc_sync_resource
).
Examples:
from dagster import job
from dagster_dbt import dbt_test_op, dbt_cli_resource, dbt_rpc_sync_resource
@job(resource_defs={"dbt":dbt_cli_resource})
def my_dbt_cli_job():
dbt_test_op()
@job(resource_defs={"dbt":dbt_rpc_sync_resource})
def my_dbt_rpc_job():
dbt_test_op()
dagster_dbt.
dbt_snapshot_op
(context)[source]¶This op executes a dbt snapshot
command. It requires the use of a dbt resource, which can be
set to execute this command through the CLI (using the dbt_cli_resource
) or
over RPC (using the dbt_rpc_sync_resource
).
Examples:
from dagster import job
from dagster_dbt import dbt_snapshot_op, dbt_cli_resource, dbt_rpc_sync_resource
@job(resource_defs={"dbt":dbt_cli_resource})
def my_dbt_cli_job():
dbt_snapshot_op()
@job(resource_defs={"dbt":dbt_rpc_sync_resource})
def my_dbt_rpc_job():
dbt_snapshot_op()
dagster_dbt.
dbt_seed_op
(context)[source]¶This op executes a dbt seed
command. It requires the use of a dbt resource, which can be
set to execute this command through the CLI (using the dbt_cli_resource
) or
over RPC (using the dbt_rpc_sync_resource
).
Examples:
from dagster import job
from dagster_dbt import dbt_seed_op, dbt_cli_resource, dbt_rpc_sync_resource
@job(resource_defs={"dbt":dbt_cli_resource})
def my_dbt_cli_job():
dbt_seed_op()
@job(resource_defs={"dbt":dbt_rpc_sync_resource})
def my_dbt_rpc_job():
dbt_seed_op()
dagster_dbt.
dbt_docs_generate_op
(context)[source]¶This op executes a dbt docs generate
command. It requires the use of a dbt resource, which can be
set to execute this command through the CLI (using the dbt_cli_resource
) or
over RPC (using the dbt_rpc_sync_resource
).
Examples:
from dagster import job
from dagster_dbt import dbt_docs_generate_op, dbt_cli_resource, dbt_rpc_sync_resource
@job(resource_defs={"dbt":dbt_cli_resource})
def my_dbt_cli_job():
dbt_docs_generate_op()
@job(resource_defs={"dbt":dbt_rpc_sync_resource})
def my_dbt_rpc_job():
dbt_docs_generate_op()
dagster_dbt.
dbt_cloud_run_op
= <dagster.core.definitions.op_definition.OpDefinition object>[source]¶The integer ID of the relevant dbt Cloud job. You can find this value by going to the details page of your job in the dbt Cloud UI. It will be the final number in the url, e.g.: https://cloud.getdbt.com/#/accounts/{account_id}/projects/{project_id}/jobs/{job_id}/
The time (in seconds) that will be waited between successive polls.
Default Value: 10
The maximum time that will waited before this operation is timed out. By default, this will never time out.
Default Value: None
If True, materializations corresponding to the results of the dbt operation will be yielded when the op executes.
Default Value: True
If provided and yield_materializations is True, these components will be used to prefix the generated asset keys.
Default Value: [‘dbt’]
Initiates a run for a dbt Cloud job, then polls until the run completes. If the job fails or is otherwised stopped before succeeding, a dagster.Failure exception will be raised, and this op will fail.
It requires the use of a ‘dbt_cloud’ resource, which is used to connect to the dbt Cloud API.
Config Options:
The integer ID of the relevant dbt Cloud job. You can find this value by going to the details
page of your job in the dbt Cloud UI. It will be the final number in the url, e.g.:
https://cloud.getdbt.com/#/accounts/{account_id}/projects/{project_id}/jobs/{job_id}/
The time (in seconds) that will be waited between successive polls. Defaults to 10
.
The maximum time (in seconds) that will waited before this operation is timed out. By default, this will never time out.
If True, materializations corresponding to the results of the dbt operation will be
yielded when the solid executes. Defaults to True
.
If provided and yield_materializations is True, these components will be used to ” prefix the generated asset keys. Defaults to [“dbt”].
Examples:
from dagster import job
from dagster_dbt import dbt_cloud_resource, run_dbt_cloud_op
my_dbt_cloud_resource = dbt_cloud_resource.configured(
{"auth_token": {"env": "DBT_CLOUD_AUTH_TOKEN"}, "account_id": 77777}
)
run_dbt_nightly_sync = run_dbt_cloud_op.configured(
{"job_id": 54321}, name="run_dbt_nightly_sync"
)
@job(resource_defs={"dbt_cloud": my_dbt_cloud_resource})
def dbt_cloud():
run_dbt_nightly_sync()
dagster_dbt.
DbtCliResource
(executable, default_flags, warn_error, ignore_handled_error, target_path, logger=None)[source]¶A resource that allows you to execute dbt cli commands. For the most up-to-date documentation on the specific parameters available to you for each command, check out the dbt docs:
https://docs.getdbt.com/reference/commands/run
To use this as a dagster resource, we recommend using
dbt_cli_resource
.
cli
(command, **kwargs)[source]¶default flags that were configured on resource initialization (if any) overriding the default values if necessary.
command (str) – The command you wish to run (e.g. ‘run’, ‘test’, ‘docs generate’, etc.)
DbtCliOutput
containingparsed log output as well as the contents of run_results.json (if applicable).
compile
(models=None, exclude=None, **kwargs)[source]¶Run the compile
command on a dbt project. kwargs are passed in as additional parameters.
DbtCliOutput
containingparsed log output as well as the contents of run_results.json (if applicable).
default_flags
¶A set of params populated from resource config that are passed as flags to each dbt CLI command.
freshness
(select=None, **kwargs)[source]¶Run the source snapshot-freshness
command on a dbt project. kwargs are passed in as additional parameters.
select (List[str], optional) – the sources to include in the run.
DbtCliOutput
containingparsed log output as well as the contents of run_results.json (if applicable).
generate_docs
(compile_project=False, **kwargs)[source]¶Run the docs generate
command on a dbt project. kwargs are passed in as additional parameters.
compile_project (bool, optional) – If true, compile the project before generating a catalog.
DbtCliOutput
containingparsed log output as well as the contents of run_results.json (if applicable).
ls
(select=None, models=None, exclude=None, **kwargs)[source]¶Run the ls
command on a dbt project. kwargs are passed in as additional parameters.
DbtCliOutput
containingparsed log output as well as the contents of run_results.json (if applicable).
run
(models=None, exclude=None, **kwargs)[source]¶Run the run
command on a dbt project. kwargs are passed in as additional parameters.
DbtCliOutput
containingparsed log output as well as the contents of run_results.json (if applicable).
run_operation
(macro, args=None, **kwargs)[source]¶Run the run-operation
command on a dbt project. kwargs are passed in as additional parameters.
DbtCliOutput
containingparsed log output as well as the contents of run_results.json (if applicable).
seed
(show=False, select=None, exclude=None, **kwargs)[source]¶Run the seed
command on a dbt project. kwargs are passed in as additional parameters.
DbtCliOutput
containingparsed log output as well as the contents of run_results.json (if applicable).
snapshot
(select=None, exclude=None, **kwargs)[source]¶Run the snapshot
command on a dbt project. kwargs are passed in as additional parameters.
DbtCliOutput
containingparsed log output as well as the contents of run_results.json (if applicable).
strict_flags
¶A set of flags that should not be auto-populated from the default flags unless they are arguments to the associated function.
test
(models=None, exclude=None, data=True, schema=True, **kwargs)[source]¶Run the test
command on a dbt project. kwargs are passed in as additional parameters.
DbtCliOutput
containingparsed log output as well as the contents of run_results.json (if applicable).
dagster_dbt.
DbtCliOutput
(command, return_code, raw_output, logs, result)[source]¶The results of executing a dbt command, along with additional metadata about the dbt CLI process that was run.
Note that users should not construct instances of this class directly. This class is intended to be constructed from the JSON output of dbt commands.
dagster_dbt.
dbt_cli_resource
ResourceDefinition[source]¶Which directory to look in for the dbt_project.yml file. Default is the current working directory and its parents.
Default Value: ‘.’
Which directory to look in for the profiles.yml file. Default = $DBT_PROFILES_DIR or $HOME/.dbt
Which profile to load. Overrides setting in dbt_project.yml.
Which target to load for the given profile.
Supply variables to the project. This argument overrides variables defined in your dbt_project.yml file. This argument should be a dictionary, eg. {‘my_variable’: ‘my_value’}
If set, bypass the adapter-level cache of database state
Default Value: False
If dbt would normally warn, instead raise an exception. Examples include –models that selects nothing, deprecations, configurations with no associated models, invalid test configurations, and missing sources/refs in tests.
Default Value: False
Path to the dbt executable. Default is dbt
Default Value: ‘dbt’
When True, will not raise an exception when the dbt CLI returns error code 1. Default is False.
Default Value: False
The directory path for target if different from the default target-path in your dbt project configuration file.
Default Value: ‘target’
This resource defines a dbt CLI interface.
To configure this resource, we recommend using the configured method.
Examples:
custom_dbt_cli_resource = dbt_cli_resource.configured({"project-dir": "path/to/my/dbt_project"})
@pipeline(mode_defs=[ModeDefinition(resource_defs={"dbt": custom_dbt_cli_resource})])
def dbt_cli_pipeline():
# Run solids with `required_resource_keys={"dbt", ...}`.
You may configure this resource as follows:
resources:
dbt_cli_resource:
config:
project_dir: "."
# Optional[str]: Which directory to look in for the dbt_project.yml file. Default is
# the current working directory and its parents.
profiles_dir: $DBT_PROFILES_DIR or $HOME/.dbt
# Optional[str]: Which directory to look in for the profiles.yml file.
profile: ""
# Optional[str]: Which profile to load. Overrides setting in dbt_project.yml.
target: ""
# Optional[str]: Which target to load for the given profile.
vars: {}
# Optional[Permissive]: Supply variables to the project. This argument overrides
# variables defined in your dbt_project.yml file. This argument should be a
# dictionary, eg. "{'my_variable': 'my_value'}"
bypass_cache: False
# Optional[bool]: If set, bypass the adapter-level cache of database state.
dagster_dbt.
DbtRpcResource
(host='0.0.0.0', port=8580, jsonrpc_version='2.0', logger=None, **_)[source]¶A client for a dbt RPC server.
To use this as a dagster resource, we recommend using
dbt_rpc_resource
.
cli
(command, **kwargs)[source]¶Sends a request with CLI syntax to the dbt RPC server, and returns the response. For more details, see the dbt docs for running CLI commands via RPC.
cli (str) – a dbt command in CLI syntax.
the HTTP response from the dbt RPC server.
Response
compile
(models=None, exclude=None, **kwargs)[source]¶Sends a request with the method compile
to the dbt RPC server, and returns the
response. For more details, see the dbt docs for compiling projects via RPC.
compile_sql
(sql, name)[source]¶Sends a request with the method compile_sql
to the dbt RPC server, and returns the
response. For more details, see the dbt docs for compiling SQL via RPC.
generate_docs
(compile_project=False, **kwargs)[source]¶Sends a request with the method docs.generate
to the dbt RPC server, and returns the
response. For more details, see the dbt docs for the RPC method docs.generate.
compile_project (bool, optional) – If true, compile the project before generating a catalog.
kill
(task_id)[source]¶Sends a request with the method kill
to the dbt RPC server, and returns the response.
For more details, see the dbt docs for the RPC method kill.
task_id (str) – the ID of the task to terminate.
the HTTP response from the dbt RPC server.
Response
logger
¶A property for injecting a logger dependency.
Optional[Any]
ls
(select=None, models=None, exclude=None, **kwargs)[source]¶Sends a request with the method list
to the dbt RPC server, and returns the
response. For more details, see the dbt docs for list.
poll
(request_token, logs=False, logs_start=0)[source]¶Sends a request with the method poll
to the dbt RPC server, and returns the response.
For more details, see the dbt docs for the RPC method poll.
ps
(completed=False)[source]¶Sends a request with the method ps
to the dbt RPC server, and returns the response.
For more details, see the dbt docs for the RPC method ps.
compelted (bool) – If True
, then also return completed tasks. Defaults to False
.
the HTTP response from the dbt RPC server.
Response
run
(models=None, exclude=None, **kwargs)[source]¶Sends a request with the method run
to the dbt RPC server, and returns the response.
For more details, see the dbt docs for the RPC method run.
run_operation
(macro, args=None, **kwargs)[source]¶Sends a request with the method run-operation
to the dbt RPC server, and returns the
response. For more details, see the dbt docs for the command run-operation.
run_sql
(sql, name)[source]¶Sends a request with the method run_sql
to the dbt RPC server, and returns the
response. For more details, see the dbt docs for running SQL via RPC.
seed
(show=False, select=None, exclude=None, **kwargs)[source]¶Sends a request with the method seed
to the dbt RPC server, and returns the response.
For more details, see the dbt docs for the RPC method seed.
the HTTP response from the dbt RPC server.
Response
snapshot
(select=None, exclude=None, **kwargs)[source]¶Sends a request with the method snapshot
to the dbt RPC server, and returns the
response. For more details, see the dbt docs for the command snapshot.
snapshot_freshness
(select=None, **kwargs)[source]¶Sends a request with the method snapshot-freshness
to the dbt RPC server, and returns
the response. For more details, see the dbt docs for the command source snapshot-freshness.
select (List[str], optional) – the models to include in calculating snapshot freshness.
the HTTP response from the dbt RPC server.
Response
status
()[source]¶Sends a request with the method status
to the dbt RPC server, and returns the
response. For more details, see the dbt docs for the RPC method status.
the HTTP response from the dbt RPC server.
Response
test
(models=None, exclude=None, data=True, schema=True, **kwargs)[source]¶Sends a request with the method test
to the dbt RPC server, and returns the response.
For more details, see the dbt docs for the RPC method test.
the HTTP response from the dbt RPC server.
Response
dagster_dbt.
DbtRpcSyncResource
(host='0.0.0.0', port=8580, jsonrpc_version='2.0', logger=None, poll_interval=1, **_)[source]¶dagster_dbt.
DbtRpcOutput
(response)[source]¶The output from executing a dbt command via the dbt RPC server.
result
¶The parsed contents of the “result” field of the JSON response from the rpc server (if any).
Dict[str, Any]
response
¶The original Response from which this output was generated.
requests.Response
dagster_dbt.
local_dbt_rpc_resource
ResourceDefinition¶This resource defines a dbt RPC client for an RPC server running on 0.0.0.0:8580.
dagster_dbt.
dbt_rpc_resource
ResourceDefinition[source]¶Default Value: 8580
This resource defines a dbt RPC client.
To configure this resource, we recommend using the configured method.
Examples:
Examples:
from dagster_dbt import dbt_rpc_resource
custom_dbt_rpc_resource = dbt_rpc_resource.configured({"host": "80.80.80.80","port": 8080,})
@job(resource_defs={"dbt_rpc": custom_dbt_rpc_sync_resource})
def dbt_rpc_job():
# Run ops with `required_resource_keys={"dbt_rpc", ...}`.
dagster_dbt.
dbt_rpc_sync_resource
ResourceDefinition[source]¶Default Value: 8580
Default Value: 1
This resource defines a synchronous dbt RPC client, which sends requests to a dbt RPC server, and waits for the request to complete before returning.
To configure this resource, we recommend using the configured method.
Examples:
from dagster_dbt import dbt_rpc_sync_resource
custom_sync_dbt_rpc_resource = dbt_rpc_sync_resource.configured({"host": "80.80.80.80","port": 8080,})
@job(resource_defs={"dbt_rpc": custom_dbt_rpc_sync_resource})
def dbt_rpc_sync_job():
# Run ops with `required_resource_keys={"dbt_rpc", ...}`.
dagster_dbt.
DbtCloudResourceV2
(auth_token, account_id, disable_schedule_on_trigger=True, request_max_retries=3, request_retry_delay=0.25, dbt_cloud_host='https://cloud.getdbt.com/', log=<Logger dagster.builtin (DEBUG)>, log_requests=False)[source]¶This class exposes methods on top of the dbt Cloud REST API v2.
For a complete set of documentation on the dbt Cloud Administrative REST API, including expected response JSON schemae, see the dbt Cloud API Docs.
cancel_run
(run_id)[source]¶Cancels a dbt Cloud run.
run_id (int) – The ID of the relevant dbt Cloud run. You can find this value by going to
the details page of your run in the dbt Cloud UI. It will be the final number in the
url, e.g.: https://cloud.getdbt.com/#/accounts/{account_id}/projects/{project_id}/runs/{run_id}/
See: https://docs.getdbt.com/dbt-cloud/api-v2#operation/getRunById for schema.
Dict[str, Any]
get_job
(job_id)[source]¶Gets details about a given dbt job from the dbt Cloud API.
job_id (int) – The ID of the relevant dbt Cloud job. You can find this value by going to
the details page of your job in the dbt Cloud UI. It will be the final number in the
url, e.g.: https://cloud.getdbt.com/#/accounts/{account_id}/projects/{project_id}/jobs/{job_id}/
Parsed json data from the response to this request
Dict[str, Any]
get_manifest
(run_id, step=None)[source]¶The parsed contents of a manifest.json file created by a completed run.
run_id (int) – The ID of the relevant dbt Cloud run. You can find this value by going to
the details page of your run in the dbt Cloud UI. It will be the final number in the
url, e.g.: https://cloud.getdbt.com/#/accounts/{account_id}/projects/{project_id}/runs/{run_id}/
step (int) – The index of the step in the run to query for artifacts. The first step in the run has the index 1. If the step parameter is omitted, then this endpoint will return the artifacts compiled for the last step in the run.
Parsed contents of the manifest.json file
Dict[str, Any]
get_run
(run_id, include_related=None)[source]¶Gets details about a specific job run.
run_id (int) – The ID of the relevant dbt Cloud run. You can find this value by going to
the details page of your run in the dbt Cloud UI. It will be the final number in the
url, e.g.: https://cloud.getdbt.com/#/accounts/{account_id}/projects/{project_id}/runs/{run_id}/
include_related (List[str]) – List of related fields to pull with the run. Valid values are “trigger”, “job”, and “debug_logs”.
See: https://docs.getdbt.com/dbt-cloud/api-v2#operation/getRunById for schema.
Dict[str, Any]
get_run_artifact
(run_id, path, step=None)[source]¶The string contents of a run artifact from a dbt Cloud run.
run_id (int) – The ID of the relevant dbt Cloud run. You can find this value by going to
the details page of your run in the dbt Cloud UI. It will be the final number in the
url, e.g.: https://cloud.getdbt.com/#/accounts/{account_id}/projects/{project_id}/runs/{run_id}/
path (str) – The path to this run artifact (e.g. ‘run/my_new_project/models/example/my_first_dbt_model.sql’)
step (int) – The index of the step in the run to query for artifacts. The first step in the run has the index 1. If the step parameter is omitted, then this endpoint will return the artifacts compiled for the last step in the run.
List of the names of the available run artifacts
List[str]
get_run_results
(run_id, step=None)[source]¶The parsed contents of a run_results.json file created by a completed run.
run_id (int) – The ID of the relevant dbt Cloud run. You can find this value by going to
the details page of your run in the dbt Cloud UI. It will be the final number in the
url, e.g.: https://cloud.getdbt.com/#/accounts/{account_id}/projects/{project_id}/runs/{run_id}/
step (int) – The index of the step in the run to query for artifacts. The first step in the run has the index 1. If the step parameter is omitted, then this endpoint will return the artifacts compiled for the last step in the run.
Parsed contents of the run_results.json file
Dict[str, Any]
get_run_steps
(run_id)[source]¶Gets the steps of an initialized dbt Cloud run.
run_id (int) – The ID of the relevant dbt Cloud run. You can find this value by going to
the details page of your run in the dbt Cloud UI. It will be the final number in the
url, e.g.: https://cloud.getdbt.com/#/accounts/{account_id}/projects/{project_id}/runs/{run_id}/
List of commands for each step of the run.
List[str, Any]
list_run_artifacts
(run_id, step=None)[source]¶Lists the paths of the available run artifacts from a completed dbt Cloud run.
run_id (int) – The ID of the relevant dbt Cloud run. You can find this value by going to
the details page of your run in the dbt Cloud UI. It will be the final number in the
url, e.g.: https://cloud.getdbt.com/#/accounts/{account_id}/projects/{project_id}/runs/{run_id}/
step (int) – The index of the step in the run to query for artifacts. The first step in the run has the index 1. If the step parameter is omitted, then this endpoint will return the artifacts compiled for the last step in the run
List of the paths of the available run artifacts
List[str]
make_request
(method, endpoint, data=None, return_text=False)[source]¶Creates and sends a request to the desired dbt Cloud API endpoint.
method (str) – The http method to use for this request (e.g. “POST”, “GET”, “PATCH”).
endpoint (str) – The dbt Cloud API endpoint to send this request to.
data (Optional[str]) – JSON-formatted data string to be included in the request.
return_text (bool) – Override default behavior and return unparsed {“text”: response.text} blob instead of json.
Parsed json data from the response to this request
Dict[str, Any]
poll_run
(run_id, poll_interval=10, poll_timeout=None, href=None)[source]¶Polls a dbt Cloud job run until it completes. Will raise a dagster.Failure exception if the run does not complete successfully.
run_id (int) – The ID of the relevant dbt Cloud run. You can find this value by going to
the details page of your run in the dbt Cloud UI. It will be the final number in the
url, e.g.: https://cloud.getdbt.com/#/accounts/{account_id}/projects/{project_id}/runs/{run_id}/
poll_interval (float) – The time (in seconds) that should be waited between successive polls of the dbt Cloud API.
poll_timeout (float) – The maximum time (in seconds) that should be waited for this run to complete. If this threshold is exceeded, the run will be cancelled and an exception will be thrown. By default, this will poll forver.
href (str) – For internal use, generally should not be set manually.
See: https://docs.getdbt.com/dbt-cloud/api-v2#operation/getRunById for schema.
Dict[str, Any]
run_job
(job_id, **kwargs)[source]¶Initializes a run for a job. Overrides for specific properties can be set by passing in values to the kwargs. A full list of overridable properties can be found here: https://docs.getdbt.com/dbt-cloud/api-v2#operation/triggerRun
job_id (int) – The ID of the relevant dbt Cloud job. You can find this value by going to
the details page of your job in the dbt Cloud UI. It will be the final number in the
url, e.g.: https://cloud.getdbt.com/#/accounts/{account_id}/projects/{project_id}/jobs/{job_id}/
kwargs – Passed in as the properties to be overridden.
Parsed json data from the response to this request
Dict[str, Any]
run_job_and_poll
(job_id, poll_interval=10, poll_timeout=None)[source]¶Runs a dbt Cloud job and polls until it completes. Will raise a dagster.Failure exception if the run does not complete successfully.
job_id (int) – The ID of the relevant dbt Cloud job. You can find this value by going to
the details page of your job in the dbt Cloud UI. It will be the final number in the
url, e.g.: https://cloud.getdbt.com/#/accounts/{account_id}/projects/{project_id}/jobs/{job_id}/
poll_interval (float) – The time (in seconds) that should be waited between successive polls of the dbt Cloud API.
poll_timeout (float) – The maximum time (in seconds) that should be waited for this run to complete. If this threshold is exceeded, the run will be cancelled and an exception will be thrown. By default, this will poll forver.
parsed run results.
DbtCloudOutput
update_job
(job_id, **kwargs)[source]¶Updates specific properties of a dbt job. Documentation on the full set of potential parameters can be found here: https://docs.getdbt.com/dbt-cloud/api-v2#operation/updateJobById
job_id (int) – The ID of the relevant dbt Cloud job. You can find this value by going to
the details page of your job in the dbt Cloud UI. It will be the final number in the
url, e.g.: https://cloud.getdbt.com/#/accounts/{account_id}/projects/{project_id}/jobs/{job_id}/
kwargs – Passed in as the properties to be changed.
Parsed json data from the response to this request
Dict[str, Any]
Examples:
# disable schedule for job with id=12345
my_dbt_cloud_resource.update_job(12345, triggers={"schedule": False})
dagster_dbt.
dbt_cloud_resource
ResourceDefinition[source]¶dbt Cloud API Token. User tokens can be found in the [dbt Cloud UI](https://cloud.getdbt.com/#/profile/api/), or see the [dbt Cloud Docs](https://docs.getdbt.com/docs/dbt-cloud/dbt-cloud-api/service-tokens) for instructions on creating a Service Account token.
dbt Cloud Account ID. This value can be found in the url of a variety of views in the dbt Cloud UI, e.g. https://cloud.getdbt.com/#/accounts/{account_id}/settings/.
Specifies if you would like any job that is triggered using this resource to automatically disable its schedule.
Default Value: True
The maximum number of times requests to the dbt Cloud API should be retried before failing.
Default Value: 3
Time (in seconds) to wait between each request retry.
Default Value: 0.25
This resource allows users to programatically interface with the dbt Cloud Administrative REST API (v2) to launch jobs and monitor their progress. This currently implements only a subset of the functionality exposed by the API.
For a complete set of documentation on the dbt Cloud Administrative REST API, including expected response JSON schemae, see the dbt Cloud API Docs.
To configure this resource, we recommend using the configured method.
Examples:
from dagster import job
from dagster_dbt import dbt_cloud_resource
my_dbt_cloud_resource = dbt_cloud_resource.configured(
{
"auth_token": {"env": "DBT_CLOUD_AUTH_TOKEN"},
"account_id": 30000,
}
)
@job(resource_defs={"dbt_cloud":my_dbt_cloud_resource})
def my_dbt_cloud_job():
...
dagster_dbt.
load_assets_from_dbt_project
(project_dir, profiles_dir=None, target_dir=None, select=None, runtime_metadata_fn=None, io_manager_key=None, node_info_to_asset_key=<function _get_node_asset_key>)[source]¶Loads a set of DBT models from a DBT project into Dagster assets.
Creates one Dagster asset for each dbt model. All assets will be re-materialized using a single dbt run command.
project_dir (Optional[str]) – The directory containing the DBT project to load.
profiles_dir (Optional[str]) – The profiles directory to use for loading the DBT project. Defaults to a directory called “config” inside the project_dir.
target_dir (Optional[str]) – The target directory where DBT will place compiled artifacts. Defaults to “target” underneath the project_dir.
select (str) – A DBT selection string for the models in a project that you want to include. Defaults to “*”.
runtime_metadata_fn – (Optional[Callable[[SolidExecutionContext, Mapping[str, Any]], Mapping[str, Any]]]): A function that will be run after any of the assets are materialized and returns metadata entries for the asset, to be displayed in the asset catalog for that run.
io_manager_key (Optional[str]) – The IO manager key that will be set on each of the returned assets. When other ops are downstream of the loaded assets, the IOManager specified here determines how the inputs to those ops are loaded. Defaults to “io_manager”.
node_info_to_asset_key – (Mapping[str, Any] -> AssetKey): A function that takes a dictionary of dbt node info and returns the AssetKey that you want to represent that node. By default, the asset key will simply be the name of the dbt model.
dagster_dbt.
load_assets_from_dbt_manifest
(manifest_json, runtime_metadata_fn=None, io_manager_key=None, selected_unique_ids=None, node_info_to_asset_key=<function _get_node_asset_key>)[source]¶Loads a set of dbt models, described in a manifest.json, into Dagster assets.
Creates one Dagster asset for each dbt model. All assets will be re-materialized using a single dbt run command.
manifest_json (Optional[Mapping[str, Any]]) – The contents of a DBT manifest.json, which contains a set of models to load into assets.
runtime_metadata_fn – (Optional[Callable[[SolidExecutionContext, Mapping[str, Any]], Mapping[str, Any]]]): A function that will be run after any of the assets are materialized and returns metadata entries for the asset, to be displayed in the asset catalog for that run.
io_manager_key (Optional[str]) – The IO manager key that will be set on each of the returned assets. When other ops are downstream of the loaded assets, the IOManager specified here determines how the inputs to those ops are loaded. Defaults to “io_manager”.
selected_unique_ids (Optional[Set[str]]) – The set of dbt unique_ids that you want to load as assets.
node_info_to_asset_key – (Mapping[str, Any] -> AssetKey): A function that takes a dictionary of dbt node info and returns the AssetKey that you want to represent that node. By default, the asset key will simply be the name of the dbt model.
dagster_dbt.
DbtOutput
(result)[source]¶Base class for both DbtCliOutput and DbtRPCOutput. Contains a single field, result, which represents the dbt-formatted result of the command that was run (if any).
Used internally, should not be instantiated directly by the user.
dagster_dbt.
DbtResource
(logger=None)[source]¶Base class for a resource allowing users to interface with dbt
compile
(models=None, exclude=None, **kwargs)[source]¶Run the compile
command on a dbt project. kwargs are passed in as additional parameters.
generate_docs
(compile_project=False, **kwargs)[source]¶Run the docs generate
command on a dbt project. kwargs are passed in as additional parameters.
logger
¶A property for injecting a logger dependency.
ls
(select=None, models=None, exclude=None, **kwargs)[source]¶Run the ls
command on a dbt project. kwargs are passed in as additional parameters.
run
(models=None, exclude=None, **kwargs)[source]¶Run the run
command on a dbt project. kwargs are passed in as additional parameters.
run_operation
(macro, args=None, **kwargs)[source]¶Run the run-operation
command on a dbt project. kwargs are passed in as additional parameters.
seed
(show=False, select=None, exclude=None, **kwargs)[source]¶Run the seed
command on a dbt project. kwargs are passed in as additional parameters.
object containing parsed output from dbt
snapshot
(select=None, exclude=None, **kwargs)[source]¶Run the snapshot
command on a dbt project. kwargs are passed in as additional parameters.
dagster_dbt.
DagsterDbtError
(description=None, metadata_entries=None, metadata=None)[source]¶The base exception of the dagster-dbt
library.
dagster_dbt.
DagsterDbtCliRuntimeError
(description, logs, raw_output)[source]¶Represents an error while executing a dbt CLI command.
dagster_dbt.
DagsterDbtCliFatalRuntimeError
(logs, raw_output)[source]¶Represents a fatal error in the dbt CLI (return code 2).
dagster_dbt.
DagsterDbtCliHandledRuntimeError
(logs, raw_output)[source]¶Represents a model error reported by the dbt CLI at runtime (return code 1).
dagster_dbt.
DagsterDbtCliOutputsNotFoundError
(path)[source]¶Represents a problem in finding the target/run_results.json
artifact when executing a dbt
CLI command.
For more details on target/run_results.json
, see
https://docs.getdbt.com/reference/dbt-artifacts#run_resultsjson.
dagster_dbt.utils.
generate_materializations
(dbt_output, asset_key_prefix=None)[source]¶This function yields dagster.AssetMaterialization
events for each model created by
a dbt run command (with information parsed from a DbtOutput
object).
Note that this will not work with output from the dbt_rpc_resource, because this resource does not wait for a response from the RPC server before returning. Instead, use the dbt_rpc_sync_resource, which will wait for execution to complete.
Examples:
from dagster import op, Output
from dagster_dbt.utils import generate_materializations
from dagster_dbt import dbt_cli_resource, dbt_rpc_sync_resource
@op(required_resource_keys={"dbt"})
def my_custom_dbt_run(context):
dbt_output = context.resources.dbt.run()
for materialization in generate_materializations(dbt_output):
# you can modify the materialization object to add extra metadata, if desired
yield materialization
yield Output(my_dbt_output)
@job(resource_defs={{"dbt":dbt_cli_resource}})
def my_dbt_cli_job():
my_custom_dbt_run()
@job(resource_defs={{"dbt":dbt_rpc_sync_resource}})
def my_dbt_rpc_job():
my_custom_dbt_run()
dagster_dbt provides a set of solids that may be used in legacy pipelines.
dagster_dbt.
dbt_cli_compile
= <dagster.core.definitions.solid_definition.SolidDefinition object>[source]¶Which directory to look in for the dbt_project.yml file. Default is the current working directory and its parents.
Default Value: ‘.’
Which directory to look in for the profiles.yml file. Default = $DBT_PROFILES_DIR or $HOME/.dbt
Which profile to load. Overrides setting in dbt_project.yml.
Which target to load for the given profile.
Supply variables to the project. This argument overrides variables defined in your dbt_project.yml file. This argument should be a dictionary, eg. {‘my_variable’: ‘my_value’}
If set, bypass the adapter-level cache of database state
Default Value: False
If dbt would normally warn, instead raise an exception. Examples include –models that selects nothing, deprecations, configurations with no associated models, invalid test configurations, and missing sources/refs in tests.
Default Value: False
Path to the dbt executable. Default is dbt
Default Value: ‘dbt’
When True, will not raise an exception when the dbt CLI returns error code 1. Default is False.
Default Value: False
The directory path for target if different from the default target-path in your dbt project configuration file.
Default Value: ‘target’
Default Value: False
Specify number of threads to use while executing models. Overrides settings in profiles.yml.
Default Value: None
Skip the check that dbt’s version matches the one specified in the dbt_project.yml file (‘require-dbt-version’)
Default Value: False
The dbt models to run.
Default Value: None
The dbt models to exclude.
Default Value: None
The selector name to use, as defined in your selectors.yml
Default Value: None
If set, use the given directory as the source for json files to compare with this project.
Default Value: None
If specified, DBT will drop incremental models and fully-recalculate the incremental table from the model definition. (–full-refresh)
Default Value: False
This solid executes dbt compile
via the dbt CLI.
dagster_dbt.
dbt_cli_run
= <dagster.core.definitions.solid_definition.SolidDefinition object>[source]¶Which directory to look in for the dbt_project.yml file. Default is the current working directory and its parents.
Default Value: ‘.’
Which directory to look in for the profiles.yml file. Default = $DBT_PROFILES_DIR or $HOME/.dbt
Which profile to load. Overrides setting in dbt_project.yml.
Which target to load for the given profile.
Supply variables to the project. This argument overrides variables defined in your dbt_project.yml file. This argument should be a dictionary, eg. {‘my_variable’: ‘my_value’}
If set, bypass the adapter-level cache of database state
Default Value: False
If dbt would normally warn, instead raise an exception. Examples include –models that selects nothing, deprecations, configurations with no associated models, invalid test configurations, and missing sources/refs in tests.
Default Value: False
Path to the dbt executable. Default is dbt
Default Value: ‘dbt’
When True, will not raise an exception when the dbt CLI returns error code 1. Default is False.
Default Value: False
The directory path for target if different from the default target-path in your dbt project configuration file.
Default Value: ‘target’
Specify number of threads to use while executing models. Overrides settings in profiles.yml.
Default Value: None
The dbt models to run.
Default Value: None
The dbt models to exclude.
Default Value: None
If specified, DBT will drop incremental models and fully-recalculate the incremental table from the model definition. (–full-refresh)
Default Value: False
Stop execution upon a first failure. (–fail-fast)
Default Value: False
If True, materializations corresponding to the results of the dbt operation will be yielded when the solid executes. Default: True
Default Value: True
If provided and yield_materializations is True, these components will be used to prefix the generated asset keys.
Default Value: []
This solid executes dbt run
via the dbt CLI. See the solid definition for available
parameters.
dagster_dbt.
dbt_cli_run_operation
= <dagster.core.definitions.solid_definition.SolidDefinition object>[source]¶Which directory to look in for the dbt_project.yml file. Default is the current working directory and its parents.
Default Value: ‘.’
Which directory to look in for the profiles.yml file. Default = $DBT_PROFILES_DIR or $HOME/.dbt
Which profile to load. Overrides setting in dbt_project.yml.
Which target to load for the given profile.
Supply variables to the project. This argument overrides variables defined in your dbt_project.yml file. This argument should be a dictionary, eg. {‘my_variable’: ‘my_value’}
If set, bypass the adapter-level cache of database state
Default Value: False
If dbt would normally warn, instead raise an exception. Examples include –models that selects nothing, deprecations, configurations with no associated models, invalid test configurations, and missing sources/refs in tests.
Default Value: False
Path to the dbt executable. Default is dbt
Default Value: ‘dbt’
When True, will not raise an exception when the dbt CLI returns error code 1. Default is False.
Default Value: False
The directory path for target if different from the default target-path in your dbt project configuration file.
Default Value: ‘target’
Specify the macro to invoke. dbt will call this macro with the supplied arguments and then exit.
Supply arguments to the macro. This dictionary will be mapped to the keyword arguments defined in the selected macro. This argument should be a dictionary, eg. {‘my_variable’: ‘my_value’}
This solid executes dbt run-operation
via the dbt CLI.
dagster_dbt.
dbt_cli_snapshot
= <dagster.core.definitions.solid_definition.SolidDefinition object>[source]¶Which directory to look in for the dbt_project.yml file. Default is the current working directory and its parents.
Default Value: ‘.’
Which directory to look in for the profiles.yml file. Default = $DBT_PROFILES_DIR or $HOME/.dbt
Which profile to load. Overrides setting in dbt_project.yml.
Which target to load for the given profile.
Supply variables to the project. This argument overrides variables defined in your dbt_project.yml file. This argument should be a dictionary, eg. {‘my_variable’: ‘my_value’}
If set, bypass the adapter-level cache of database state
Default Value: False
If dbt would normally warn, instead raise an exception. Examples include –models that selects nothing, deprecations, configurations with no associated models, invalid test configurations, and missing sources/refs in tests.
Default Value: False
Path to the dbt executable. Default is dbt
Default Value: ‘dbt’
When True, will not raise an exception when the dbt CLI returns error code 1. Default is False.
Default Value: False
The directory path for target if different from the default target-path in your dbt project configuration file.
Default Value: ‘target’
Specify number of threads to use while executing models. Overrides settings in profiles.yml.
Default Value: None
The dbt models to include.
Default Value: None
The dbt models to exclude.
Default Value: None
This solid executes dbt snapshot
via the dbt CLI.
dagster_dbt.
dbt_cli_snapshot_freshness
= <dagster.core.definitions.solid_definition.SolidDefinition object>[source]¶Which directory to look in for the dbt_project.yml file. Default is the current working directory and its parents.
Default Value: ‘.’
Which directory to look in for the profiles.yml file. Default = $DBT_PROFILES_DIR or $HOME/.dbt
Which profile to load. Overrides setting in dbt_project.yml.
Which target to load for the given profile.
Supply variables to the project. This argument overrides variables defined in your dbt_project.yml file. This argument should be a dictionary, eg. {‘my_variable’: ‘my_value’}
If set, bypass the adapter-level cache of database state
Default Value: False
If dbt would normally warn, instead raise an exception. Examples include –models that selects nothing, deprecations, configurations with no associated models, invalid test configurations, and missing sources/refs in tests.
Default Value: False
Path to the dbt executable. Default is dbt
Default Value: ‘dbt’
When True, will not raise an exception when the dbt CLI returns error code 1. Default is False.
Default Value: False
The directory path for target if different from the default target-path in your dbt project configuration file.
Default Value: ‘target’
Specify the sources to snapshot freshness.
Default Value: None
Specify the output path for the json report. By default, outputs to target/sources.json
Specify number of threads to use while executing models. Overrides settings in profiles.yml.
Default Value: None
This solid executes dbt source snapshot-freshness
via the dbt CLI.
dagster_dbt.
dbt_cli_test
= <dagster.core.definitions.solid_definition.SolidDefinition object>[source]¶Which directory to look in for the dbt_project.yml file. Default is the current working directory and its parents.
Default Value: ‘.’
Which directory to look in for the profiles.yml file. Default = $DBT_PROFILES_DIR or $HOME/.dbt
Which profile to load. Overrides setting in dbt_project.yml.
Which target to load for the given profile.
Supply variables to the project. This argument overrides variables defined in your dbt_project.yml file. This argument should be a dictionary, eg. {‘my_variable’: ‘my_value’}
If set, bypass the adapter-level cache of database state
Default Value: False
If dbt would normally warn, instead raise an exception. Examples include –models that selects nothing, deprecations, configurations with no associated models, invalid test configurations, and missing sources/refs in tests.
Default Value: False
Path to the dbt executable. Default is dbt
Default Value: ‘dbt’
When True, will not raise an exception when the dbt CLI returns error code 1. Default is False.
Default Value: False
The directory path for target if different from the default target-path in your dbt project configuration file.
Default Value: ‘target’
Run data tests defined in “tests” directory.
Default Value: False
Run constraint validations from schema.yml files.
Default Value: False
Stop execution upon a first test failure.
Default Value: False
Specify number of threads to use while executing models. Overrides settings in profiles.yml.
Default Value: None
The dbt models to run.
Default Value: None
The dbt models to exclude.
Default Value: None
This solid executes dbt test
via the dbt CLI. See the solid definition for available
parameters.
dagster_dbt.
create_dbt_rpc_run_sql_solid
(name, output_def=None, **kwargs)[source]¶This function is a factory which constructs a solid that will copy the results of a SQL query
run within the context of a dbt project to a pandas DataFrame
.
Any kwargs passed to this function will be passed along to the underlying @solid
decorator. However, note that overriding config_schema
, input_defs
, and
required_resource_keys
is not allowed and will throw a DagsterInvalidDefinitionError
.
If you would like to configure this solid with different config fields, you could consider using
@composite_solid
to wrap this solid.
name (str) – The name of this solid.
output_def (OutputDefinition, optional) – The OutputDefinition
for the solid. This value should always be a representation
of a pandas DataFrame
. If not specified, the solid will default to an
OutputDefinition
named “df” with a DataFrame
dagster type.
Returns the constructed solid definition.
dagster_dbt.
dbt_rpc_compile_sql
= <dagster.core.definitions.solid_definition.SolidDefinition object>[source]¶The interval (in seconds) at which to poll the dbt rpc process.
Default Value: 10
Whether or not to return logs from the process.
Default Value: True
If True, materializations corresponding to the results of the dbt operation will be yielded when the solid executes. Default: True
Default Value: True
This solid sends the dbt compile
command to a dbt RPC server and returns the request
token.
This dbt RPC solid is asynchronous. The request token can be used in subsequent RPC requests to poll the progress of the running dbt process.
dagster_dbt.
dbt_rpc_run
= <dagster.core.definitions.solid_definition.SolidDefinition object>[source]¶The dbt models to run.
Default Value: None
The dbt models to exclude.
Default Value: None
This solid sends the dbt run
command to a dbt RPC server and returns the request token.
This dbt RPC solid is asynchronous. The request token can be used in subsequent RPC requests to poll the progress of the running dbt process.
dagster_dbt.
dbt_rpc_run_and_wait
= <dagster.core.definitions.solid_definition.SolidDefinition object>[source]¶The dbt models to run.
Default Value: None
The dbt models to exclude.
Default Value: None
Whether or not to perform a –full-refresh.
Default Value: False
Whether or not to –fail-fast.
Default Value: False
Whether or not to –warn-error.
Default Value: False
The interval (in seconds) at which to poll the dbt rpc process.
Default Value: 10
Whether or not to return logs from the process.
Default Value: True
{}
Default Value: 5
Default Value: 120
If True, materializations corresponding to the results of the dbt operation will be yielded when the solid executes. Default: True
Default Value: True
This solid sends the dbt run
command to a dbt RPC server and returns the result of the
executed dbt process.
This dbt RPC solid is synchronous, and will periodically poll the dbt RPC server until the dbt process is completed.
dagster_dbt.
dbt_rpc_run_operation
= <dagster.core.definitions.solid_definition.SolidDefinition object>[source]¶The dbt macro to invoke as a run operation
Arguments to supply to the invoked macro.
Default Value: None
This solid sends the dbt run-operation
command to a dbt RPC server and returns the
request token.
This dbt RPC solid is asynchronous. The request token can be used in subsequent RPC requests to poll the progress of the running dbt process.
dagster_dbt.
dbt_rpc_run_operation_and_wait
= <dagster.core.definitions.solid_definition.SolidDefinition object>[source]¶The dbt macro to invoke as a run operation
Arguments to supply to the invoked macro.
Default Value: None
The interval (in seconds) at which to poll the dbt rpc process.
Default Value: 10
Whether or not to return logs from the process.
Default Value: True
If True, materializations corresponding to the results of the dbt operation will be yielded when the solid executes. Default: True
Default Value: True
This solid sends the dbt run-operation
command to a dbt RPC server and returns the
result of the executed dbt process.
This dbt RPC solid is synchronous, and will periodically poll the dbt RPC server until the dbt process is completed.
dagster_dbt.
dbt_rpc_snapshot
= <dagster.core.definitions.solid_definition.SolidDefinition object>[source]¶The dbt snapshot files to snapshot.
Default Value: None
The dbt snapshot files to exclude from the snapshot.
Default Value: None
This solid sends the dbt snapshot
command to a dbt RPC server and returns the
request token.
This dbt RPC solid is asynchronous. The request token can be used in subsequent RPC requests to poll the progress of the running dbt process.
dagster_dbt.
dbt_rpc_snapshot_and_wait
= <dagster.core.definitions.solid_definition.SolidDefinition object>[source]¶The dbt snapshot files to snapshot.
Default Value: None
The dbt snapshot files to exclude from the snapshot.
Default Value: None
The interval (in seconds) at which to poll the dbt rpc process.
Default Value: 10
Whether or not to return logs from the process.
Default Value: True
{}
Default Value: 5
Default Value: 120
If True, materializations corresponding to the results of the dbt operation will be yielded when the solid executes. Default: True
Default Value: True
This solid sends the dbt snapshot
command to a dbt RPC server and returns the result of
the executed dbt process.
This dbt RPC solid is synchronous, and will periodically poll the dbt RPC server until the dbt process is completed.
dagster_dbt.
dbt_rpc_snapshot_freshness
= <dagster.core.definitions.solid_definition.SolidDefinition object>[source]¶The dbt sources to snapshot-freshness for.
Default Value: None
Whether or not to –warn-error.
Default Value: False
This solid sends the dbt source snapshot-freshness
command to a dbt RPC server and
returns the request token.
This dbt RPC solid is asynchronous. The request token can be used in subsequent RPC requests to poll the progress of the running dbt process.
dagster_dbt.
dbt_rpc_snapshot_freshness_and_wait
= <dagster.core.definitions.solid_definition.SolidDefinition object>[source]¶The dbt sources to snapshot-freshness for.
Default Value: None
Whether or not to –warn-error.
Default Value: False
The interval (in seconds) at which to poll the dbt rpc process.
Default Value: 10
Whether or not to return logs from the process.
Default Value: True
If True, materializations corresponding to the results of the dbt operation will be yielded when the solid executes. Default: True
Default Value: True
This solid sends the dbt source snapshot
command to a dbt RPC server and returns the
result of the executed dbt process.
This dbt RPC solid is synchronous, and will periodically poll the dbt RPC server until the dbt process is completed.
dagster_dbt.
dbt_rpc_test
= <dagster.core.definitions.solid_definition.SolidDefinition object>[source]¶The dbt models to test.
Default Value: None
The dbt models to exclude.
Default Value: None
Whether or not to run custom data tests.
Default Value: True
Whether or not to run schema tests.
Default Value: True
This solid sends the dbt test
command to a dbt RPC server and returns the request token.
This dbt RPC solid is asynchronous. The request token can be used in subsequent RPC requests to poll the progress of the running dbt process.
dagster_dbt.
dbt_rpc_test_and_wait
= <dagster.core.definitions.solid_definition.SolidDefinition object>[source]¶The dbt models to test.
Default Value: None
The dbt models to exclude.
Default Value: None
Whether or not to run custom data tests.
Default Value: True
Whether or not to run schema tests.
Default Value: True
The interval (in seconds) at which to poll the dbt rpc process.
Default Value: 10
Whether or not to return logs from the process.
Default Value: True
If True, materializations corresponding to the results of the dbt operation will be yielded when the solid executes. Default: True
Default Value: True
This solid sends the dbt test
command to a dbt RPC server and returns the result of the
executed dbt process.
This dbt RPC solid is synchronous, and will periodically poll the dbt RPC server until the dbt process is completed.