-
Notifications
You must be signed in to change notification settings - Fork 1.3k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
feat(dbt-cloud): cache compile run id as env var for job (#11691)
### Summary & Motivation When generating software-defined assets from a dbt Cloud project, most of the latency comes from compiling the project. Rather than compiling the dbt project in an ad-hoc manner, only compile it when either: - the project changes (via git commit), or when - the dbt Cloud job commands change If one of those conditions are met, then compile the project. And save the reference/overwrite the reference to the compilation. ### How I Tested These Changes pytest, local
- Loading branch information
1 parent
48bb79a
commit 6d1c659
Showing
6 changed files
with
251 additions
and
77 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
85 changes: 85 additions & 0 deletions
85
python_modules/libraries/dagster-dbt/dagster_dbt/cloud/cli.py
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,85 @@ | ||
from typing import List | ||
|
||
import typer | ||
|
||
from dagster_dbt.cloud.asset_defs import DbtCloudCacheableAssetsDefinition | ||
from dagster_dbt.cloud.resources import DbtCloudResource | ||
|
||
app = typer.Typer() | ||
|
||
DAGSTER_DBT_COMPILE_RUN_ID_ENV_VAR = "DBT_DAGSTER_COMPILE_RUN_ID" | ||
|
||
|
||
@app.command() | ||
def cache_compile_references( | ||
auth_token: str = typer.Argument(..., envvar="DBT_CLOUD_API_KEY"), | ||
account_id: int = typer.Argument(..., envvar="DBT_CLOUD_ACCOUNT_ID"), | ||
project_id: int = typer.Argument(..., envvar="DBT_CLOUD_PROJECT_ID"), | ||
) -> None: | ||
""" | ||
Cache the latest dbt cloud compile run id for a given project. | ||
""" | ||
dbt_cloud_resource = DbtCloudResource( | ||
auth_token=auth_token, account_id=account_id, disable_schedule_on_trigger=False | ||
) | ||
|
||
# List the jobs from the project | ||
dbt_cloud_jobs = dbt_cloud_resource.list_jobs(project_id=project_id) | ||
|
||
# Compile each job with an override | ||
for dbt_cloud_job in dbt_cloud_jobs: | ||
job_id: int = dbt_cloud_job["id"] | ||
job_commands: List[str] = dbt_cloud_job["execute_steps"] | ||
|
||
# Retrieve the filters for the compile override step | ||
job_materialization_command_step = ( | ||
DbtCloudCacheableAssetsDefinition.get_job_materialization_command_step( | ||
execute_steps=job_commands | ||
) | ||
) | ||
dbt_materialization_command = job_commands[job_materialization_command_step] | ||
parsed_args = DbtCloudCacheableAssetsDefinition.parse_dbt_command( | ||
dbt_materialization_command | ||
) | ||
dbt_compile_options: List[str] = DbtCloudCacheableAssetsDefinition.get_compile_filters( | ||
parsed_args=parsed_args | ||
) | ||
dbt_compile_command = f"dbt compile {' '.join(dbt_compile_options)}" | ||
|
||
# Run the compile command | ||
dbt_cloud_compile_run = dbt_cloud_resource.run_job( | ||
job_id=job_id, | ||
cause="Generating software-defined assets for Dagster.", | ||
steps_override=[dbt_compile_command], | ||
) | ||
|
||
# Cache the compile run as a reference in the dbt Cloud job's env var | ||
dbt_cloud_compile_run_id = str(dbt_cloud_compile_run["id"]) | ||
dbt_cloud_job_env_vars = dbt_cloud_resource.get_job_environment_variables( | ||
project_id=project_id, job_id=job_id | ||
) | ||
compile_run_environment_variable_id = dbt_cloud_job_env_vars[ | ||
DAGSTER_DBT_COMPILE_RUN_ID_ENV_VAR | ||
]["job"]["id"] | ||
|
||
typer.echo( | ||
f"Updating the value of environment variable `{DAGSTER_DBT_COMPILE_RUN_ID_ENV_VAR}`" | ||
f" with id `{compile_run_environment_variable_id}` for job id `{job_id}`. Setting new" | ||
f" value to `{dbt_cloud_compile_run_id}`." | ||
) | ||
|
||
dbt_cloud_resource.set_job_environment_variable( | ||
project_id=project_id, | ||
job_id=job_id, | ||
environment_variable_id=compile_run_environment_variable_id, | ||
name=DAGSTER_DBT_COMPILE_RUN_ID_ENV_VAR, | ||
value=dbt_cloud_compile_run_id, | ||
) | ||
|
||
typer.echo("Update complete.") | ||
|
||
|
||
# https://typer.tiangolo.com/tutorial/commands/one-or-multiple/#one-command-and-one-callback | ||
@app.callback() | ||
def callback() -> None: | ||
pass |
47 changes: 47 additions & 0 deletions
47
python_modules/libraries/dagster-dbt/dagster_dbt_tests/cloud/test_cli.py
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,47 @@ | ||
import responses | ||
from dagster_dbt.cloud.cli import DAGSTER_DBT_COMPILE_RUN_ID_ENV_VAR, app | ||
from typer.testing import CliRunner | ||
|
||
from .utils import ( | ||
SAMPLE_ACCOUNT_ID, | ||
SAMPLE_API_PREFIX, | ||
SAMPLE_API_V3_PREFIX, | ||
SAMPLE_JOB_ID, | ||
SAMPLE_PROJECT_ID, | ||
sample_get_environment_variables, | ||
sample_list_job_details, | ||
sample_run_details, | ||
sample_set_environment_variable, | ||
) | ||
|
||
runner = CliRunner() | ||
|
||
|
||
@responses.activate | ||
def test_cache_compile_references(monkeypatch): | ||
monkeypatch.setenv("DBT_CLOUD_API_KEY", "test") | ||
monkeypatch.setenv("DBT_CLOUD_ACCOUNT_ID", SAMPLE_ACCOUNT_ID) | ||
monkeypatch.setenv("DBT_CLOUD_PROJECT_ID", SAMPLE_PROJECT_ID) | ||
compile_run_environment_variable_id = 3 | ||
|
||
responses.get(f"{SAMPLE_API_PREFIX}/jobs", json=sample_list_job_details()) | ||
responses.post(f"{SAMPLE_API_PREFIX}/jobs/{SAMPLE_JOB_ID}/run/", json=sample_run_details()) | ||
responses.get( | ||
f"{SAMPLE_API_V3_PREFIX}/projects/{SAMPLE_PROJECT_ID}/environment-variables/job", | ||
json=sample_get_environment_variables( | ||
environment_variable_id=compile_run_environment_variable_id, | ||
name=DAGSTER_DBT_COMPILE_RUN_ID_ENV_VAR, | ||
), | ||
) | ||
responses.post( | ||
f"{SAMPLE_API_V3_PREFIX}/projects/{SAMPLE_PROJECT_ID}/environment-variables/{compile_run_environment_variable_id}", | ||
json=sample_set_environment_variable( | ||
environment_variable_id=compile_run_environment_variable_id, | ||
name=DAGSTER_DBT_COMPILE_RUN_ID_ENV_VAR, | ||
value="500000", | ||
), | ||
) | ||
|
||
result = runner.invoke(app, ["cache-compile-references"]) | ||
|
||
assert result.exit_code == 0 |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.