Compare commits

...

5 Commits

Author SHA1 Message Date
Emily Rockman
ac780b2538 better handling + documentation of edge cases 2025-11-27 10:48:13 -05:00
Emily Rockman
4946afc11e add comparison operators, test 2025-11-27 10:48:13 -05:00
Emily Rockman
4f68bef57a Use Jinja AST instead of a regex 2025-11-27 10:48:12 -05:00
Emily Rockman
463a2bdf3e first pass solution. uses regex :( 2025-11-26 14:45:29 -05:00
Emily Rockman
7016142e74 add tests 2025-11-26 14:41:08 -05:00
7 changed files with 979 additions and 1 deletions

View File

@@ -243,3 +243,345 @@ def construct_static_kwarg_value(kwarg) -> str:
# This is still useful to be able to detect changes in unrendered configs, even if it is
# not an exact representation of the user input.
return str(kwarg)
def statically_extract_sql_header(source: str) -> Optional[str]:
"""
Extract the unrendered template from a {% call set_sql_header(config) %} block.
Returns the template string that should be re-rendered at runtime, or None if no
set_sql_header block is found OR if the template contains unsupported Jinja constructs.
This is needed to fix issue #2793 where ref(), source(), etc. in sql_header
resolve incorrectly at parse time. By extracting and storing the unrendered template,
we can re-render it at runtime with the correct context.
Similar to statically_parse_unrendered_config(), but for CallBlock nodes instead
of Call nodes.
For example, given:
"{% call set_sql_header(config) %}
select * from {{ ref('my_model') }};
{% endcall %}
select 1 as id"
returns: "select * from {{ ref('my_model') }};"
No set_sql_header block:
"select 1 as id"
returns: None
Unsupported Jinja construct (e.g., {% for %} loop):
"{% call set_sql_header(config) %}
{% for item in items %}
select * from {{ ref(item) }};
{% endfor %}
{% endcall %}"
returns: None (triggers fallback to parse-time rendering)
Note: If None is returned due to unsupported constructs, the sql_header will be
rendered at parse time, which may cause ref(), source(), and this to resolve
incorrectly. Users should simplify their sql_header or report the issue.
"""
# Return early to avoid creating jinja environment if no set_sql_header in source
if "set_sql_header" not in source:
return None
# Parse the source using Jinja2 AST
env = get_environment(None, capture_macros=True)
try:
parsed = env.parse(source)
except Exception:
# If parsing fails, return None rather than raising
return None
# Find all CallBlock nodes ({% call ... %}...{% endcall %})
call_blocks = list(parsed.find_all(jinja2.nodes.CallBlock))
for call_block in call_blocks:
# Check if this is a call to set_sql_header
if (
hasattr(call_block.call, "node")
and hasattr(call_block.call.node, "name")
and call_block.call.node.name == "set_sql_header"
):
# Extract the body content by reconstructing from AST nodes
# The body is the template between {% call ... %} and {% endcall %}
template_parts = []
unsupported_node_found = False
def extract_template_from_nodes(nodes):
"""Recursively extract template string from AST nodes.
Currently only supports to most common node types.
Returns False if an unsupported node type is encountered.
"""
nonlocal unsupported_node_found
for node in nodes:
# Early exit if we've hit an unsupported node
if unsupported_node_found:
return
if isinstance(node, jinja2.nodes.Output):
# Output nodes contain the actual template content
if hasattr(node, "nodes"):
extract_template_from_nodes(node.nodes)
elif isinstance(node, jinja2.nodes.TemplateData):
# Raw text/whitespace in the template
template_parts.append(node.data)
elif isinstance(node, jinja2.nodes.Call):
# Function call like {{ ref('model') }}
template_parts.append("{{ ")
template_parts.append(_reconstruct_jinja_call(node))
template_parts.append(" }}")
elif isinstance(node, jinja2.nodes.Name):
# Variable reference like {{ my_var }}
template_parts.append("{{ ")
template_parts.append(node.name)
template_parts.append(" }}")
elif isinstance(node, jinja2.nodes.Getattr):
# Attribute access like {{ obj.attr }}
template_parts.append("{{ ")
template_parts.append(_reconstruct_getattr(node))
template_parts.append(" }}")
elif isinstance(node, jinja2.nodes.If):
# {% if ... %} blocks
template_parts.append("{% if ")
template_parts.append(_reconstruct_test(node.test))
template_parts.append(" %}")
extract_template_from_nodes(node.body)
if node.else_:
template_parts.append("{% else %}")
extract_template_from_nodes(node.else_)
template_parts.append("{% endif %}")
elif isinstance(node, jinja2.nodes.Compare):
# Comparison like {% if a > b %}
template_parts.append(_reconstruct_comparison(node))
elif isinstance(node, (jinja2.nodes.And, jinja2.nodes.Or)):
# Boolean operators
template_parts.append(_reconstruct_boolean_op(node))
elif isinstance(node, jinja2.nodes.Not):
# Negation
template_parts.append("not ")
template_parts.append(_reconstruct_test(node.node))
else:
# Unsupported node type - we can't reliably reconstruct this template.
# This triggers fallback to parse-time rendering (existing behavior).
#
# Known unsupported constructs that trigger this:
# - {% for %} loops
# - {{ value | filter }} filters
# - {% set var = value %} assignments
# - Complex expressions
#
# If ref(), source(), or this are used within sql_header and we hit this,
# they will resolve incorrectly at parse time, potentially causing
# "relation does not exist" errors at runtime.
#
# Users experiencing this should:
# 1. Simplify their sql_header to use only supported constructs
# 2. Or report the issue so we can add support for the construct
#
# Supported: Output, TemplateData, Call, Name, Getattr, If, Compare, And, Or, Not
# Node type encountered: {type(node).__name__}
unsupported_node_found = True
return
def _reconstruct_jinja_call(call_node):
"""Reconstruct a Jinja function call from AST"""
nonlocal unsupported_node_found
if not hasattr(call_node, "node"):
unsupported_node_found = True
return ""
# Get function name
func_parts = []
if isinstance(call_node.node, jinja2.nodes.Name):
func_parts.append(call_node.node.name)
elif isinstance(call_node.node, jinja2.nodes.Getattr):
func_parts.append(_reconstruct_getattr(call_node.node))
else:
# Unknown function node type - trigger fallback
unsupported_node_found = True
return ""
# Reconstruct arguments
args = []
for arg in call_node.args:
if isinstance(arg, jinja2.nodes.Const):
# String/number literal
args.append(repr(arg.value))
elif isinstance(arg, jinja2.nodes.Name):
# Variable reference
args.append(arg.name)
elif isinstance(arg, jinja2.nodes.Call):
# Nested function call
args.append(_reconstruct_jinja_call(arg))
else:
# Unknown argument type - trigger fallback
unsupported_node_found = True
return ""
# Reconstruct keyword arguments
for kwarg in call_node.kwargs:
key = kwarg.key
if isinstance(kwarg.value, jinja2.nodes.Const):
args.append(f"{key}={repr(kwarg.value.value)}")
elif isinstance(kwarg.value, jinja2.nodes.Name):
args.append(f"{key}={kwarg.value.name}")
else:
# Unknown kwarg value type - trigger fallback
unsupported_node_found = True
return ""
func_parts.append(f"({', '.join(args)})")
return "".join(func_parts)
def _reconstruct_getattr(node):
"""Reconstruct attribute access like obj.attr"""
nonlocal unsupported_node_found
if isinstance(node.node, jinja2.nodes.Name):
return f"{node.node.name}.{node.attr}"
elif isinstance(node.node, jinja2.nodes.Getattr):
return f"{_reconstruct_getattr(node.node)}.{node.attr}"
else:
# Unknown node type - trigger fallback
unsupported_node_found = True
return ""
def _reconstruct_comparison(comp_node):
"""Reconstruct comparison expressions like {{ a > b }}"""
nonlocal unsupported_node_found
# Comparisons have: expr (left side), ops (list of Operand objects)
# Each Operand has: op (operator type), expr (right side expression)
parts = []
# Start with the left expression
if isinstance(comp_node.expr, jinja2.nodes.Name):
parts.append(comp_node.expr.name)
elif isinstance(comp_node.expr, jinja2.nodes.Call):
parts.append(_reconstruct_jinja_call(comp_node.expr))
elif isinstance(comp_node.expr, jinja2.nodes.Const):
parts.append(repr(comp_node.expr.value))
else:
# Unknown left expression type - trigger fallback
unsupported_node_found = True
return ""
# Add operators and operands
for operand in comp_node.ops:
# operand has .op and .expr
op_map = {
"eq": "==",
"ne": "!=",
"lt": "<",
"lteq": "<=",
"gt": ">",
"gteq": ">=",
"in": "in",
"notin": "not in",
}
op_str = op_map.get(operand.op, operand.op)
parts.append(f" {op_str} ")
# Add the right side expression
if isinstance(operand.expr, jinja2.nodes.Name):
parts.append(operand.expr.name)
elif isinstance(operand.expr, jinja2.nodes.Call):
parts.append(_reconstruct_jinja_call(operand.expr))
elif isinstance(operand.expr, jinja2.nodes.Const):
parts.append(repr(operand.expr.value))
else:
# Unknown right expression type - trigger fallback
unsupported_node_found = True
return ""
return "".join(parts)
def _reconstruct_boolean_op(bool_node):
"""Reconstruct boolean operators like {{ a and b }}"""
nonlocal unsupported_node_found
op_name = "and" if isinstance(bool_node, jinja2.nodes.And) else "or"
parts = []
# And/Or nodes have 'left' and 'right' attributes
def add_operand(operand):
nonlocal unsupported_node_found
if isinstance(operand, jinja2.nodes.Name):
parts.append(operand.name)
elif isinstance(operand, jinja2.nodes.Call):
parts.append(_reconstruct_jinja_call(operand))
elif isinstance(operand, jinja2.nodes.Compare):
parts.append(_reconstruct_comparison(operand))
elif isinstance(operand, jinja2.nodes.Not):
parts.append("not ")
if isinstance(operand.node, jinja2.nodes.Name):
parts.append(operand.node.name)
elif isinstance(operand.node, jinja2.nodes.Call):
parts.append(_reconstruct_jinja_call(operand.node))
else:
# Unknown Not operand type - trigger fallback
unsupported_node_found = True
elif isinstance(operand, (jinja2.nodes.And, jinja2.nodes.Or)):
# Nested boolean operators
parts.append("(")
parts.append(_reconstruct_boolean_op(operand))
parts.append(")")
else:
# Unknown operand type - trigger fallback
unsupported_node_found = True
add_operand(bool_node.left)
parts.append(f" {op_name} ")
add_operand(bool_node.right)
return "".join(parts)
def _reconstruct_test(test_node):
"""Reconstruct test expressions for {% if %} blocks"""
nonlocal unsupported_node_found
if isinstance(test_node, jinja2.nodes.Call):
return _reconstruct_jinja_call(test_node)
elif isinstance(test_node, jinja2.nodes.Name):
return test_node.name
elif isinstance(test_node, jinja2.nodes.Compare):
return _reconstruct_comparison(test_node)
elif isinstance(test_node, (jinja2.nodes.And, jinja2.nodes.Or)):
return _reconstruct_boolean_op(test_node)
elif isinstance(test_node, jinja2.nodes.Not):
result = "not "
if isinstance(test_node.node, jinja2.nodes.Name):
result += test_node.node.name
elif isinstance(test_node.node, jinja2.nodes.Call):
result += _reconstruct_jinja_call(test_node.node)
else:
# Unknown Not operand type - trigger fallback
unsupported_node_found = True
return ""
return result
else:
# Unknown test type - trigger fallback
unsupported_node_found = True
return ""
# Extract template from the CallBlock body
extract_template_from_nodes(call_block.body)
# If we encountered an unsupported node type, return None
# This causes fallback to parse-time rendering (existing behavior)
if unsupported_node_found:
return None
# Join and strip the result
template = "".join(template_parts).strip()
return template if template else None
return None

View File

@@ -558,6 +558,13 @@ class RuntimeConfigObject(Config):
def __init__(self, model, context_config: Optional[ContextConfig] = None):
self.model = model
# we never use or get a config, only the parser cares
self._runtime_context: Optional[Dict[str, Any]] = (
None # Set by ModelRunner for sql_header re-rendering
)
def set_runtime_context(self, context: Dict[str, Any]):
"""Called by ModelRunner to inject the full runtime context for sql_header re-rendering."""
self._runtime_context = context
def __call__(self, *args, **kwargs):
return ""
@@ -569,6 +576,25 @@ class RuntimeConfigObject(Config):
validator(value)
def _lookup(self, name, default=_MISSING):
# Special handling for sql_header: re-render at runtime if unrendered template exists
# This fixes issue #2793 where ref(), source(), this, etc. resolve incorrectly at parse time
if name == "sql_header" and self._runtime_context is not None:
unrendered_sql_header = getattr(self.model, "unrendered_config", {}).get("sql_header")
if unrendered_sql_header:
try:
from dbt.clients import jinja
rendered_sql_header = jinja.get_rendered(
unrendered_sql_header,
self._runtime_context,
self.model,
)
return rendered_sql_header
except Exception:
# Fall back to parse-time version if re-rendering fails
pass
# Default behavior for all other config values
# if this is a macro, there might be no `model.config`.
if not hasattr(self.model, "config"):
result = default

View File

@@ -7,6 +7,7 @@ from dbt import deprecations, hooks, utils
from dbt.adapters.factory import get_adapter # noqa: F401
from dbt.artifacts.resources import Contract
from dbt.clients.jinja import MacroGenerator, get_rendered
from dbt.clients.jinja_static import statically_extract_sql_header
from dbt.config import RuntimeConfig
from dbt.context.context_config import ContextConfig
from dbt.context.providers import (
@@ -442,6 +443,19 @@ class ConfiguredParser(
# This updates the node database/schema/alias/relation_name
self.update_parsed_node_relation_names(parsed_node, config_dict)
# Extract and store the unrendered sql_header template for runtime re-rendering
# This fixes issue #2793 where ref(), source(), etc. in sql_header resolve incorrectly at parse time
if hasattr(parsed_node, "raw_code") and parsed_node.raw_code:
sql_header_template = statically_extract_sql_header(parsed_node.raw_code)
if sql_header_template:
parsed_node.unrendered_config["sql_header"] = sql_header_template
# Re-render sql_header at parse time with parse context to capture dependencies
# Similar to how hooks are handled (see below for hook rendering)
if not context:
context = self._context_for(parsed_node, config)
get_rendered(sql_header_template, context, parsed_node, capture_macros=True)
# tests don't have hooks
if parsed_node.resource_type == NodeType.Test:
return

View File

@@ -322,6 +322,11 @@ class ModelRunner(CompileRunner):
)
context_config = context["config"]
# Inject runtime context for sql_header re-rendering (fixes issue #2793)
# This allows ref(), source(), this, is_incremental(), etc. to resolve correctly in sql_header
if hasattr(context_config, "set_runtime_context"):
context_config.set_runtime_context(context)
mat_has_supported_langs = hasattr(materialization_macro, "supported_languages")
model_lang_supported = model.language in materialization_macro.supported_languages
if mat_has_supported_langs and not model_lang_supported:

View File

@@ -33,7 +33,13 @@ dbt wants to understand and define each SQL model as an object in an internal da
- Simple Jinja-SQL models (using just `ref()`, `source()`, &/or `config()` with literal inputs) are also [statically analyzed](https://docs.getdbt.com/reference/parsing#static-parser), using [a thing we built](https://github.com/dbt-labs/dbt-extractor). This is **very** fast (~0.3 ms).
- More complex Jinja-SQL models are parsed by actually rendering the Jinja, and "capturing" any instances of `ref()`, `source()`, &/or `config()`. This is kinda slow, but its more capable than our static parser. Those macros can receive `set` variables, or call other macros in turn, and we can still capture the right results because **were actually using real Jinja to render it.**
- We capture any other macros called in `depends_on.macros`. This enables us to do clever things later on, such as select models downstream of changed macros (`state:modified.macros`).
- **However:** If `ref()` is nested inside a conditional block that is false at parse time (e.g. `{% if execute %}`), we will miss capturing that macro call then. If the same conditional block resolves to true at runtime, were screwed! So [we have a runtime check](https://github.com/dbt-labs/dbt-core/blob/16f529e1d4e067bdbb6a659a622bead442f24b4e/core/dbt/context/providers.py#L495-L500) to validate that any `ref()` we see again at compile/runtime, is one we also previously captured at parse time. If we find a new `ref()` we werent expecting, theres a risk that were running the DAG out of order!
- **However:** If `ref()` is nested inside a conditional block that is false at parse time (e.g. `{% if execute %}`), we will miss capturing that macro call then. If the same conditional block resolves to true at runtime, we're screwed! So [we have a runtime check](https://github.com/dbt-labs/dbt-core/blob/16f529e1d4e067bdbb6a659a622bead442f24b4e/core/dbt/context/providers.py#L495-L500) to validate that any `ref()` we see again at compile/runtime, is one we also previously captured at parse time. If we find a new `ref()` we weren't expecting, there's a risk that we're running the DAG out of order!
- **`set_sql_header` blocks** use two-phase rendering (similar to hooks) to correctly resolve `ref()`, `source()`, and `this`:
- At parse time, the unrendered template is extracted from `raw_code` and stored in `unrendered_config`. It's then rendered with parse context to capture dependencies (`ref()`, `source()`, macros) for DAG construction.
- At runtime, the template is re-rendered with runtime context, so `ref()` and `source()` resolve to their actual target relations (not placeholder values), and `is_incremental()` evaluates correctly.
- This two-phase approach ensures both correct dependency tracking and correct SQL generation.
- Without this fix, `ref('model_a')` in a `set_sql_header` block would incorrectly resolve to the current model at both parse time and runtime, causing "relation does not exist" errors.
- This behavior was fixed as part of [dbt-core#2793](https://github.com/dbt-labs/dbt-core/issues/2793).
</details>

View File

@@ -198,3 +198,255 @@ class BaseConfigProject:
"failing.sql": tests__failing_sql,
"sleeper_agent.sql": tests__sleeper_agent_sql,
}
# ========================================
# Fixtures for set_sql_header tests
# ========================================
# Fixtures for Issue #2793: Basic ref() in set_sql_header
models__test_tmp_1 = """
{{
config(materialized="table")
}}
select 1 as key, 100 as value
"""
models__test_tmp_2 = """
{{
config(materialized="table")
}}
{% call set_sql_header(config) %}
select * from {{ ref('test_tmp_1') }};
{% endcall %}
select * from {{ ref('test_tmp_1') }}
"""
# Fixtures for Issue #3264: is_incremental() in set_sql_header
models__incremental_header = """
{{
config(
materialized='incremental',
unique_key='id'
)
}}
{% call set_sql_header(config) %}
{% if is_incremental() %}
-- This should only appear on incremental runs
select 1 as incremental_marker;
{% else %}
-- This should only appear on full refresh
select 0 as full_refresh_marker;
{% endif %}
{% endcall %}
select 1 as id, 'initial' as status
{% if is_incremental() %}
union all
select 2 as id, 'incremental' as status
{% endif %}
"""
# Fixtures for Issue #4692: Nested macro calls
macros__custom_ref_macro = """
{% macro get_ref_in_macro(model_name) %}
{{ return(ref(model_name)) }}
{% endmacro %}
"""
models__base_model = """
select 1 as id, 'base' as name
"""
models__nested_macro_header = """
{% call set_sql_header(config) %}
select * from {{ get_ref_in_macro('base_model') }};
{% endcall %}
select * from {{ ref('base_model') }}
"""
# Fixtures for Issue #6058: source() in set_sql_header
seeds__source_seed = """id,name
1,alice
2,bob
"""
sources__schema_yml = """
version: 2
sources:
- name: test_source
schema: "{{ target.schema }}"
tables:
- name: source_seed
"""
models__source_in_header = """
{% call set_sql_header(config) %}
select count(*) from {{ source('test_source', 'source_seed') }};
{% endcall %}
select * from {{ source('test_source', 'source_seed') }}
"""
# NOTE: source_in_set_block is NOT using set_sql_header, so it's out of scope for issue #2793
# It uses {% set %} blocks which is a different pattern with different resolution behavior
# Left here for reference but not tested
models__source_in_set_block = """
{% set my_source_query %}
select count(*) from {{ source('test_source', 'source_seed') }}
{% endset %}
{{
config(
pre_hook=my_source_query
)
}}
select * from {{ source('test_source', 'source_seed') }}
"""
# Fixtures for Issue #7151: this with custom generate_alias_name
macros__custom_alias = """
{% macro generate_alias_name(custom_alias_name=none, node=none) -%}
{%- if custom_alias_name is not none -%}
{{ return(custom_alias_name | trim) }}
{%- else -%}
{{ return('custom_' ~ node.name) }}
{%- endif -%}
{%- endmacro %}
"""
models__this_with_alias = """
{% call set_sql_header(config) %}
-- Reference to current model using this
select 'header: ' || '{{ this }}' as header_this;
{% endcall %}
select '{{ this }}' as body_this, 1 as id
"""
# Fixtures for multiple refs in set_sql_header
models__ref_a = """
select 1 as id, 'a' as source
"""
models__ref_b = """
select 2 as id, 'b' as source
"""
models__multiple_refs_header = """
{% call set_sql_header(config) %}
select * from {{ ref('ref_a') }};
select * from {{ ref('ref_b') }};
{% endcall %}
select * from {{ ref('ref_a') }}
union all
select * from {{ ref('ref_b') }}
"""
# Fixtures for combination of ref, source, and this
models__combination_header = """
{% call set_sql_header(config) %}
-- Using ref
select count(*) from {{ ref('base_model') }};
-- Using source
select count(*) from {{ source('test_source', 'source_seed') }};
-- Using this (should be current model)
select '{{ this }}' as current_model;
{% endcall %}
select * from {{ ref('base_model') }}
"""
# Fixtures for different materializations
models__view_with_header = """
{{
config(materialized='view')
}}
{% call set_sql_header(config) %}
select 1 as view_header;
{% endcall %}
select * from {{ ref('base_model') }}
"""
models__ephemeral_with_header = """
{{
config(materialized='ephemeral')
}}
{% call set_sql_header(config) %}
select 1 as ephemeral_header;
{% endcall %}
select * from {{ ref('base_model') }}
"""
# Fixtures for Issue #2921: ref() with custom database/schema
models__custom_schema_model = """
{{
config(
schema='custom_schema'
)
}}
select 1 as id, 'custom_schema' as source
"""
models__ref_custom_schema = """
{% call set_sql_header(config) %}
-- Reference model with custom schema
select count(*) from {{ ref('custom_schema_model') }};
{% endcall %}
select * from {{ ref('custom_schema_model') }}
"""
# Fixtures for comparison and boolean operators in set_sql_header
models__conditional_header = """
{{
config(
materialized='incremental',
unique_key='id'
)
}}
{% call set_sql_header(config) %}
{% if is_incremental() and var('enable_optimization', false) %}
-- Boolean AND: Only runs on incremental with optimization enabled
select 'incremental_and_optimized' as header_status, '{{ this }}' as target_table;
{% endif %}
{% if var('threshold', 0) > 50 %}
-- Comparison operator: Only runs when threshold > 50
select 'threshold_exceeded' as header_status, '{{ this }}' as target_table;
{% endif %}
{% if not is_incremental() or var('force_refresh', false) %}
-- Boolean NOT and OR: Runs on full refresh or when force_refresh is true
select 'full_refresh_or_forced' as header_status, '{{ this }}' as target_table;
{% endif %}
{% endcall %}
select 1 as id, 'data' as value
{% if is_incremental() %}
union all
select 2 as id, 'incremental' as value
{% endif %}
"""

View File

@@ -0,0 +1,333 @@
"""
Tests for ref(), source(), and this resolution in set_sql_header blocks.
This addresses the core issue (#2793) where ref(), source(), this, and is_incremental()
resolve incorrectly at parse time within set_sql_header blocks, causing runtime errors.
The fix implements two-phase rendering:
1. Parse time: Extract and render the sql_header template to capture dependencies for the DAG
2. Runtime: Re-render the template with runtime context for correct SQL generation
Note: These tests only cover set_sql_header blocks. Other Jinja patterns like
{% set %} blocks with pre_hooks have different resolution behavior and are out of scope.
"""
import pytest
from dbt.tests.util import get_manifest, run_dbt
from tests.functional.configs.fixtures import (
macros__custom_alias,
macros__custom_ref_macro,
models__base_model,
models__combination_header,
models__conditional_header,
models__custom_schema_model,
models__ephemeral_with_header,
models__incremental_header,
models__multiple_refs_header,
models__nested_macro_header,
models__ref_a,
models__ref_b,
models__ref_custom_schema,
models__source_in_header,
models__test_tmp_1,
models__test_tmp_2,
models__this_with_alias,
models__view_with_header,
seeds__source_seed,
sources__schema_yml,
)
class TestBasicRefInSqlHeader:
"""Test for issue #2793: ref() in set_sql_header resolves to wrong model"""
@pytest.fixture(scope="class")
def models(self):
return {
"test_tmp_1.sql": models__test_tmp_1,
"test_tmp_2.sql": models__test_tmp_2,
}
def test_ref_resolves_correctly(self, project):
"""Verify ref('test_tmp_1') in set_sql_header resolves to test_tmp_1, not test_tmp_2"""
# Before the fix, this would fail with "relation does not exist" error
# because ref('test_tmp_1') would incorrectly resolve to test_tmp_2
results = run_dbt(["run"])
assert len(results) == 2
assert all(r.status == "success" for r in results)
# Verify dependencies are tracked correctly
manifest = get_manifest(project.project_root)
model_id = "model.test.test_tmp_2"
assert model_id in manifest.nodes
# test_tmp_2 should depend on test_tmp_1
deps = manifest.nodes[model_id].depends_on.nodes
assert "model.test.test_tmp_1" in deps
class TestIsIncrementalInSqlHeader:
"""Test for issue #3264: is_incremental() in set_sql_header"""
@pytest.fixture(scope="class")
def models(self):
return {
"incremental_header.sql": models__incremental_header,
}
def test_is_incremental_evaluates_correctly(self, project):
"""Verify is_incremental() evaluates correctly in set_sql_header"""
# First run - should not be incremental
results = run_dbt(["run"])
assert len(results) == 1
assert results[0].status == "success"
# Second run - should be incremental
results = run_dbt(["run"])
assert len(results) == 1
assert results[0].status == "success"
# The fact that both runs succeeded means is_incremental() is working
# If it wasn't, the SQL in sql_header would be malformed
class TestNestedMacroInSqlHeader:
"""Test for issue #4692: Nested macro calls in set_sql_header"""
@pytest.fixture(scope="class")
def models(self):
return {
"base_model.sql": models__base_model,
"nested_macro_header.sql": models__nested_macro_header,
}
@pytest.fixture(scope="class")
def macros(self):
return {
"custom_ref_macro.sql": macros__custom_ref_macro,
}
def test_nested_macro_ref_resolves(self, project):
"""Verify macro that calls ref() works in set_sql_header"""
results = run_dbt(["run"])
assert len(results) == 2
assert all(r.status == "success" for r in results)
# Verify macro dependency is tracked
manifest = get_manifest(project.project_root)
model_id = "model.test.nested_macro_header"
assert "macro.test.get_ref_in_macro" in manifest.nodes[model_id].depends_on.macros
# Verify ref dependency is tracked
assert "model.test.base_model" in manifest.nodes[model_id].depends_on.nodes
class TestSourceInSqlHeader:
"""Test for issue #6058: source() in set_sql_header"""
@pytest.fixture(scope="class")
def seeds(self):
return {
"source_seed.csv": seeds__source_seed,
}
@pytest.fixture(scope="class")
def models(self):
return {
"schema.yml": sources__schema_yml,
"source_in_header.sql": models__source_in_header,
# Note: source_in_set_block is NOT using set_sql_header, so it's out of scope for this fix
# "source_in_set_block.sql": models__source_in_set_block,
}
def test_source_in_header_resolves(self, project):
"""Verify source() in set_sql_header resolves correctly"""
run_dbt(["seed"])
results = run_dbt(["run"])
assert len(results) == 1 # Changed from 2
assert all(r.status == "success" for r in results)
# Verify source dependency is tracked
manifest = get_manifest(project.project_root)
model_id = "model.test.source_in_header"
assert "source.test.test_source.source_seed" in manifest.nodes[model_id].depends_on.nodes
# Removed test_source_in_set_block_resolves - it doesn't use set_sql_header so it's out of scope
class TestThisWithCustomAlias:
"""Test for issue #7151: this with custom generate_alias_name"""
@pytest.fixture(scope="class")
def macros(self):
return {
"custom_alias.sql": macros__custom_alias,
}
@pytest.fixture(scope="class")
def models(self):
return {
"this_with_alias.sql": models__this_with_alias,
}
def test_this_uses_custom_alias(self, project):
"""Verify {{ this }} in set_sql_header uses custom alias"""
results = run_dbt(["run"])
assert len(results) == 1
assert results[0].status == "success"
manifest = get_manifest(project.project_root)
model_id = "model.test.this_with_alias"
node = manifest.nodes[model_id]
# Verify custom alias is used
assert node.alias == "custom_this_with_alias"
class TestMultipleRefsInSqlHeader:
"""Test multiple ref() calls in set_sql_header"""
@pytest.fixture(scope="class")
def models(self):
return {
"ref_a.sql": models__ref_a,
"ref_b.sql": models__ref_b,
"multiple_refs_header.sql": models__multiple_refs_header,
}
def test_multiple_refs_resolve(self, project):
"""Verify multiple ref() calls in set_sql_header all resolve correctly"""
results = run_dbt(["run"])
assert len(results) == 3
assert all(r.status == "success" for r in results)
# Verify dependencies are tracked
manifest = get_manifest(project.project_root)
model_id = "model.test.multiple_refs_header"
deps = manifest.nodes[model_id].depends_on.nodes
assert "model.test.ref_a" in deps
assert "model.test.ref_b" in deps
class TestCombinationRefSourceThis:
"""Test combination of ref, source, and this in set_sql_header"""
@pytest.fixture(scope="class")
def seeds(self):
return {
"source_seed.csv": seeds__source_seed,
}
@pytest.fixture(scope="class")
def models(self):
return {
"schema.yml": sources__schema_yml,
"base_model.sql": models__base_model,
"combination_header.sql": models__combination_header,
}
def test_combination_resolves_correctly(self, project):
"""Verify ref, source, and this all work together in set_sql_header"""
run_dbt(["seed"])
results = run_dbt(["run"])
assert len(results) == 2
assert all(r.status == "success" for r in results)
# Verify dependencies are tracked
manifest = get_manifest(project.project_root)
model_id = "model.test.combination_header"
deps = manifest.nodes[model_id].depends_on.nodes
assert "model.test.base_model" in deps
assert "source.test.test_source.source_seed" in deps
class TestDifferentMaterializations:
"""Test set_sql_header with different materializations"""
@pytest.fixture(scope="class")
def models(self):
return {
"base_model.sql": models__base_model,
"view_with_header.sql": models__view_with_header,
"ephemeral_with_header.sql": models__ephemeral_with_header,
}
def test_materializations_with_header(self, project):
"""Verify set_sql_header works with view and ephemeral materializations"""
results = run_dbt(["run"])
# base_model and view_with_header should create relations
# ephemeral models don't show up in run results (they're only compiled)
assert len(results) == 2
assert all(r.status == "success" for r in results)
manifest = get_manifest(project.project_root)
# Check view
view_id = "model.test.view_with_header"
assert manifest.nodes[view_id].config.materialized == "view"
assert "model.test.base_model" in manifest.nodes[view_id].depends_on.nodes
# Check ephemeral
ephemeral_id = "model.test.ephemeral_with_header"
assert manifest.nodes[ephemeral_id].config.materialized == "ephemeral"
assert "model.test.base_model" in manifest.nodes[ephemeral_id].depends_on.nodes
class TestRefWithCustomDatabaseSchema:
"""Test for issue #2921: ref() with custom database/schema in set_sql_header"""
@pytest.fixture(scope="class")
def models(self):
return {
"custom_schema_model.sql": models__custom_schema_model,
"ref_custom_schema.sql": models__ref_custom_schema,
}
def test_ref_with_custom_schema_resolves(self, project):
"""Verify ref() resolves correctly when referencing model with custom schema"""
results = run_dbt(["run"])
assert len(results) == 2
assert all(r.status == "success" for r in results)
manifest = get_manifest(project.project_root)
# Check that custom_schema_model has custom schema
custom_model_id = "model.test.custom_schema_model"
assert manifest.nodes[custom_model_id].schema != project.test_schema
# Check that ref_custom_schema depends on custom_schema_model
ref_model_id = "model.test.ref_custom_schema"
assert "model.test.custom_schema_model" in manifest.nodes[ref_model_id].depends_on.nodes
class TestComparisonAndBooleanOperators:
"""Test comparison (>, <, ==) and boolean (and, or, not) operators in set_sql_header"""
@pytest.fixture(scope="class")
def models(self):
return {
"conditional_header.sql": models__conditional_header,
}
def test_comparison_and_boolean_operators(self, project):
"""Verify comparison and boolean operators work in set_sql_header blocks"""
# First run (full refresh) - should succeed
results = run_dbt(["run"])
assert len(results) == 1
assert results[0].status == "success"
# Second run (incremental) - should also succeed
results = run_dbt(["run"])
assert len(results) == 1
assert results[0].status == "success"
# Run with var overrides to test different branches
results = run_dbt(["run", "--vars", '{"enable_optimization": true, "threshold": 100}'])
assert len(results) == 1
assert results[0].status == "success"
manifest = get_manifest(project.project_root)
model_id = "model.test.conditional_header"
assert model_id in manifest.nodes
assert manifest.nodes[model_id].config.materialized == "incremental"