From 91367766d26bbcd6ca73e90bd48ebf01f5bb51d8 Mon Sep 17 00:00:00 2001 From: colton Date: Mon, 4 Mar 2024 12:15:11 -0500 Subject: [PATCH] [dagster-dbt] log_columns_in_relation macro to to only log table_schema 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 --- .../dbt_packages/test_columns_metadata.py | 28 +++++++++++++++++++ .../macros/log_columns_in_relation.sql | 4 ++- 2 files changed, 31 insertions(+), 1 deletion(-) diff --git a/python_modules/libraries/dagster-dbt/dagster_dbt_tests/core/dbt_packages/test_columns_metadata.py b/python_modules/libraries/dagster-dbt/dagster_dbt_tests/core/dbt_packages/test_columns_metadata.py index ba0441b89da07..abd1e09db6809 100644 --- a/python_modules/libraries/dagster-dbt/dagster_dbt_tests/core/dbt_packages/test_columns_metadata.py +++ b/python_modules/libraries/dagster-dbt/dagster_dbt_tests/core/dbt_packages/test_columns_metadata.py @@ -1,4 +1,6 @@ +import json import os +import subprocess from typing import Any, Dict, cast from dagster import ( @@ -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()] + ) diff --git a/python_modules/libraries/dagster-dbt/dbt_packages/dagster/macros/log_columns_in_relation.sql b/python_modules/libraries/dagster-dbt/dbt_packages/dagster/macros/log_columns_in_relation.sql index 4153d5a38c684..f1be3e315fc42 100644 --- a/python_modules/libraries/dagster-dbt/dbt_packages/dagster/macros/log_columns_in_relation.sql +++ b/python_modules/libraries/dagster-dbt/dbt_packages/dagster/macros/log_columns_in_relation.sql @@ -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 %}