Skip to content

Commit

Permalink
[dagster-dbt] log_columns_in_relation macro to to only log table_sche…
Browse files Browse the repository at this point in the history
…ma if present (#20088)

## Summary & Motivation

When running the `dbt parse` command on a project like
`dagster-open-platform`, the post hook `log_columns_in_relation` macro
produces many empty log statements. This updates the macro to only log
`table_schema` when it is non-null.

```
make manifest
cd dbt && dbt parse && cd ..
15:24:05  Running with dbt=1.7.6
15:24:09  Registered adapter: snowflake=1.7.1
15:24:09  Unable to do partial parsing because profile has changed
15:24:09  Unable to do partial parsing because a project dependency has been added
15:24:09  Unable to do partial parsing because a project config has changed
15:24:09  {}
15:24:09  {}
15:24:09  {}
15:24:09  {}
15:24:09  {}
15:24:09  {}
15:24:09  {}
15:24:09  {}
```

## How I Tested These Changes

Only tested happy-path by updating the macro in the `dbt_modules`
directory and running the project.

I confirmed that the logs were _not_ produced in my project, but I was
unable to test when they _are_ produced.

---------

Co-authored-by: Rex Ledesma <[email protected]>
  • Loading branch information
cmpadden and rexledesma authored Mar 4, 2024
1 parent af0eee9 commit 9136776
Show file tree
Hide file tree
Showing 2 changed files with 31 additions and 1 deletion.
Original file line number Diff line number Diff line change
@@ -1,4 +1,6 @@
import json
import os
import subprocess
from typing import Any, Dict, cast

from dagster import (
Expand Down Expand Up @@ -74,3 +76,29 @@ def assert_columns_metadata(context: AssetExecutionContext, dbt: DbtCliResource)
)

assert result.success


def test_dbt_cli_no_jinja_log_info() -> None:
dbt = DbtCliResource(project_dir=os.fspath(test_metadata_path))
dbt_cli_parse_invocation = dbt.cli(["parse"])

assert dbt_cli_parse_invocation.is_successful()
assert not any(
event.raw_event["info"]["name"] == "JinjaLogInfo"
for event in dbt_cli_parse_invocation.stream_raw_events()
)


def test_dbt_raw_cli_no_empty_jinja_log_info() -> None:
# Ensure `log_columns_in_relation.sql` does not produce empty `{}` statements when run outside of the context of `dagster-dbt`
result = subprocess.run(
["dbt", "--log-format", "json", "--no-partial-parse", "parse"],
capture_output=True,
text=True,
cwd=test_metadata_path,
check=False,
)

assert not any(
[json.loads(line)["info"]["name"] == "JinjaLogInfo" for line in result.stdout.splitlines()]
)
Original file line number Diff line number Diff line change
Expand Up @@ -7,5 +7,7 @@
{%- set _ = table_schema.update(serializable_column) -%}
{% endfor %}

{% do log(tojson(table_schema), info=true) %}
{% if table_schema %}
{% do log(tojson(table_schema), info=true) %}
{% endif %}
{% endmacro %}

0 comments on commit 9136776

Please sign in to comment.