The redshift_maintenance macro throws a "Compilation Error" if a project has overridden the built in ref() macro (example of how/why one might do that).
Steps to reproduce
Any Redshift based dbt project.
Add this package to it.
Add a ref() macro (e.g. macros/not_really_custom_ref.sql) override that doesn't really do anything:
{% macro ref(model_name) %}
{% set rel = builtins.ref(model_name) %}
{% set newrel = rel %}
{% do return(newrel) %}
{% endmacro %}
4. Try vacuuming via `dbt run-operation redshift_maintenance`.
### Expected results
Expected tables to be vacuumed and stdout to having something like:
22:21:04 Running with dbt=1.0.1
22:21:08 11:21:08 + 1 of 1 Vacuuming "analytics"."dbt_jyeo"."my_table"
22:21:08 11:21:08 + 1 of 1 Analyzing "analytics"."dbt_jyeo"."my_table"
22:21:09 11:21:09 + 1 of 1 Finished "analytics"."dbt_jyeo"."my_table" in 0.59s
_Note: above is the result of NOT having a custom `ref()` override._
### Actual results
A `dbt.exceptions.CompilationException` is raised instead:
22:36:12 Running with dbt=1.0.1
22:36:12 Change detected to override macro used during parsing. Starting full parse.
22:36:16 Encountered an error while running operation: Compilation Error in macro redshift_maintenance (macros/redshift_maintenance_operation.sql)
'dbt.clients.jinja.MacroGenerator object' has no attribute 'config'
### Screenshots and log output
Stdout with `--debug`:
<details>
<summary>Click to expand!</summary>
```shell
============================== 2022-02-13 22:36:54.114097 | 191d1b44-d1b6-4bc0-ad7c-59207c87bd30 ==============================
22:36:54.114097 [info ] [MainThread]: Running with dbt=1.0.1
22:36:54.115352 [debug] [MainThread]: running dbt with arguments Namespace(record_timing_info=None, debug=True, log_format=None, write_json=None, use_colors=None, printer_width=None, warn_error=None, version_check=None, partial_parse=None, single_threaded=False, use_experimental_parser=None, static_parser=None, profiles_dir='/Users/jeremy/.dbt', send_anonymous_usage_stats=None, fail_fast=None, event_buffer_size=None, project_dir=None, profile=None, target=None, vars='{}', log_cache_events=False, macro='redshift_maintenance', args='{}', defer=None, state=None, cls=<class 'dbt.task.run_operation.RunOperationTask'>, which='run-operation', rpc_method='run-operation')
22:36:54.115846 [debug] [MainThread]: Tracking: tracking
22:36:54.139491 [debug] [MainThread]: Sending event: {'category': 'dbt', 'action': 'invocation', 'label': 'start', 'context': [<snowplow_tracker.self_describing_json.SelfDescribingJson object at 0x106a90d60>, <snowplow_tracker.self_describing_json.SelfDescribingJson object at 0x106a90880>, <snowplow_tracker.self_describing_json.SelfDescribingJson object at 0x106a85e80>]}
22:36:54.291982 [debug] [MainThread]: Partial parsing enabled: 0 files deleted, 0 files added, 0 files changed.
22:36:54.292516 [debug] [MainThread]: Partial parsing enabled, no changes found, skipping parsing
22:36:54.303169 [debug] [MainThread]: Sending event: {'category': 'dbt', 'action': 'load_project', 'label': '191d1b44-d1b6-4bc0-ad7c-59207c87bd30', 'context': [<snowplow_tracker.self_describing_json.SelfDescribingJson object at 0x1071450d0>]}
22:36:54.314694 [debug] [MainThread]: Acquiring new redshift connection "macro_redshift_maintenance"
22:36:54.315138 [debug] [MainThread]: Using redshift connection "macro_redshift_maintenance"
22:36:54.315434 [debug] [MainThread]: On macro_redshift_maintenance: BEGIN
22:36:54.315789 [debug] [MainThread]: Opening a new connection, currently in state init
22:36:54.316666 [debug] [MainThread]: Redshift adapter: Connecting to Redshift using 'database' credentials
22:36:56.453905 [debug] [MainThread]: SQL status: BEGIN in 2.14 seconds
22:36:56.454777 [debug] [MainThread]: On macro_redshift_maintenance: COMMIT
22:36:56.455268 [debug] [MainThread]: Using redshift connection "macro_redshift_maintenance"
22:36:56.455952 [debug] [MainThread]: On macro_redshift_maintenance: COMMIT
22:36:56.668484 [debug] [MainThread]: SQL status: COMMIT in 0.21 seconds
22:36:56.693942 [debug] [MainThread]: Postgres adapter: Error running SQL: macro redshift_maintenance
22:36:56.694449 [debug] [MainThread]: Postgres adapter: Rolling back transaction.
22:36:56.694901 [debug] [MainThread]: On macro_redshift_maintenance: Close
22:36:56.696044 [error] [MainThread]: Encountered an error while running operation: Compilation Error in macro redshift_maintenance (macros/redshift_maintenance_operation.sql)
'dbt.clients.jinja.MacroGenerator object' has no attribute 'config'
22:36:56.696788 [debug] [MainThread]:
Traceback (most recent call last):
File "/Users/jeremy/src/dbt-sandcastles/venv_dbt_1.0.1/lib/python3.9/site-packages/dbt/clients/jinja.py", line 304, in exception_handler
yield
File "/Users/jeremy/src/dbt-sandcastles/venv_dbt_1.0.1/lib/python3.9/site-packages/dbt/clients/jinja.py", line 259, in call_macro
return macro(*args, **kwargs)
File "/Users/jeremy/src/dbt-sandcastles/venv_dbt_1.0.1/lib/python3.9/site-packages/jinja2/runtime.py", line 675, in __call__
return self._invoke(arguments, autoescape)
File "/Users/jeremy/src/dbt-sandcastles/venv_dbt_1.0.1/lib/python3.9/site-packages/jinja2/runtime.py", line 679, in _invoke
rv = self._func(*arguments)
File "<template>", line 23, in macro
File "/Users/jeremy/src/dbt-sandcastles/venv_dbt_1.0.1/lib/python3.9/site-packages/jinja2/sandbox.py", line 407, in getattr
value = getattr(obj, attribute)
File "/Users/jeremy/src/dbt-sandcastles/venv_dbt_1.0.1/lib/python3.9/site-packages/jinja2/runtime.py", line 753, in __getattr__
return self._fail_with_undefined_error()
File "/Users/jeremy/src/dbt-sandcastles/venv_dbt_1.0.1/lib/python3.9/site-packages/jinja2/runtime.py", line 747, in _fail_with_undefined_error
raise self._undefined_exception(self._undefined_message)
jinja2.exceptions.UndefinedError: 'dbt.clients.jinja.MacroGenerator object' has no attribute 'config'
During handling of the above exception, another exception occurred:
Traceback (most recent call last):
File "/Users/jeremy/src/dbt-sandcastles/venv_dbt_1.0.1/lib/python3.9/site-packages/dbt/task/run_operation.py", line 57, in run
self._run_unsafe()
File "/Users/jeremy/src/dbt-sandcastles/venv_dbt_1.0.1/lib/python3.9/site-packages/dbt/task/run_operation.py", line 44, in _run_unsafe
res = adapter.execute_macro(
File "/Users/jeremy/src/dbt-sandcastles/venv_dbt_1.0.1/lib/python3.9/site-packages/dbt/adapters/base/impl.py", line 992, in execute_macro
result = macro_function(**kwargs)
File "/Users/jeremy/src/dbt-sandcastles/venv_dbt_1.0.1/lib/python3.9/site-packages/dbt/clients/jinja.py", line 332, in __call__
return self.call_macro(*args, **kwargs)
File "/Users/jeremy/src/dbt-sandcastles/venv_dbt_1.0.1/lib/python3.9/site-packages/dbt/clients/jinja.py", line 261, in call_macro
return e.value
File "/Users/jeremy/.pyenv/versions/3.9.7/lib/python3.9/contextlib.py", line 137, in __exit__
self.gen.throw(typ, value, traceback)
File "/Users/jeremy/src/dbt-sandcastles/venv_dbt_1.0.1/lib/python3.9/site-packages/dbt/clients/jinja.py", line 306, in exception_handler
raise_compiler_error(str(e), self.macro)
File "/Users/jeremy/src/dbt-sandcastles/venv_dbt_1.0.1/lib/python3.9/site-packages/dbt/exceptions.py", line 454, in raise_compiler_error
raise CompilationException(msg, node)
dbt.exceptions.CompilationException: Compilation Error in macro redshift_maintenance (macros/redshift_maintenance_operation.sql)
'dbt.clients.jinja.MacroGenerator object' has no attribute 'config'
22:36:56.709326 [debug] [MainThread]: Sending event: {'category': 'dbt', 'action': 'invocation', 'label': 'end', 'context': [<snowplow_tracker.self_describing_json.SelfDescribingJson object at 0x10722b8b0>, <snowplow_tracker.self_describing_json.SelfDescribingJson object at 0x10722b7f0>, <snowplow_tracker.self_describing_json.SelfDescribingJson object at 0x10722b730>]}
22:36:56.711625 [debug] [MainThread]: Flushing usage events
22:36:57.600768 [debug] [MainThread]: Connection 'macro_redshift_maintenance' was properly closed.
This looks to be caused by the following 2 lines that try to get the root project name from the ref.config attribute - which is unavailable if a user manually overrides ref with their own.
As of dbt 0.16, the project_name looks to be available without taking it from ref.config so we can change the above 2 lines to be something like:
{% set root_project = context.project_name %}
{% if context.get(root_project, {}).get('vacuumable_tables_sql') %}
{% set vacuumable_tables_sql=context[root_project].vacuumable_tables_sql(**kwargs) %}
P.s. I did test the above code change and found that it worked - submitting PR after this issue :)
Describe the bug
The
redshift_maintenance
macro throws a "Compilation Error" if a project has overridden the built inref()
macro (example of how/why one might do that).Steps to reproduce
Add a
ref()
macro (e.g.macros/not_really_custom_ref.sql
) override that doesn't really do anything:{% endmacro %}
22:21:04 Running with dbt=1.0.1 22:21:08 11:21:08 + 1 of 1 Vacuuming "analytics"."dbt_jyeo"."my_table" 22:21:08 11:21:08 + 1 of 1 Analyzing "analytics"."dbt_jyeo"."my_table" 22:21:09 11:21:09 + 1 of 1 Finished "analytics"."dbt_jyeo"."my_table" in 0.59s
22:36:12 Running with dbt=1.0.1 22:36:12 Change detected to override macro used during parsing. Starting full parse. 22:36:16 Encountered an error while running operation: Compilation Error in macro redshift_maintenance (macros/redshift_maintenance_operation.sql) 'dbt.clients.jinja.MacroGenerator object' has no attribute 'config'
System information
The contents of your
packages.yml
file:Which database are you using dbt with?
The output of
dbt --version
:The operating system you're using:
The output of
python --version
:Additional context
This looks to be caused by the following 2 lines that try to get the root project name from the
ref.config
attribute - which is unavailable if a user manually overridesref
with their own.https://github.com/dbt-labs/redshift/blob/48d6be2af1140c73bbba1ba5a6949225403cee43/macros/redshift_maintenance_operation.sql#L32-L33
As of dbt 0.16, the
project_name
looks to be available without taking it fromref.config
so we can change the above 2 lines to be something like:P.s. I did test the above code change and found that it worked - submitting PR after this issue :)