Compare commits

..

15 Commits

Author SHA1 Message Date
Jeremy Cohen
93572b9291 Support 'scripts' for python models 2022-09-28 20:25:53 +02:00
Gerda Shank
3562637984 Remove parsers/source.py type ignores (#5953) 2022-09-28 11:24:59 -04:00
Callum McCann
17aca39e1c Adding metric expression validation (#5873)
* adding validation

* changie

* code formatting

* updating for review

* updating tests
2022-09-27 12:38:03 -04:00
Yoshiaki Ishihara
59744f18bb Fix typos of comments in core/dbt/adapters/ (#5693) 2022-09-27 09:10:24 -07:00
Rachel
f1326f526c Runtime: Prevent introspective queries at compile (SL only) (#5926)
* Preliminary changes to keep compile from connecting to the warehouse for runtime calls

* Adds option to lib to skip connecting to warehouse for compile; adds prelim tests

* Removes unused imports

* Simplifies test and renames to SqlCompileRunnerNoIntrospection

* Updates name in tests

* Spacing

* Updates test to check for adapter connection call instead of compile and execute

* Removes commented line

* Fixes test names

* Updates plugin to postgres type as snowflake isn't available

* Fixes docstring

* Fixes formatting

* Moves conditional logic out of class

* Fixes formatting

* Removes commented line

* Moves import

* Unmoves import

* Updates changelog

* Adds further info to method docstring
2022-09-27 09:49:55 -04:00
Jeremy Cohen
834ac716fd Prefer internal macros when called explicitly (#5907)
* Add functional test

* Prefer internal macros when called explicitly

* Add changelog entry

* update tests format

Co-authored-by: Emily Rockman <emily.rockman@dbtlabs.com>
2022-09-27 08:50:55 -04:00
Doug Beatty
0487b96098 Array macros (#5823)
* Helper macro to cast from array to string

* Default implementations and tests for array macros

* Trim Trailing Whitespace

* Changelog entry

* Remove dependence upon `cast_array_to_string` macro

* pre-commit fixes

* Remove `cast_array_to_string` macro

* pre-commit fix

* Trivial direct test; array_concat/append test non-triviallly indirectly

* Remove vestigial `lstrip`
2022-09-26 13:40:15 -06:00
FishtownBuildBot
dbd36f06e4 Add new index.html and changelog yaml files from dbt-docs (#5925) 2022-09-26 14:00:04 -05:00
dave-connors-3
38ada8a68e merge exclude columns for incremental models (#5457)
* exlcude cols like in dbt_utils.star

* dispatch macro

* changelog entry

Co-authored-by: Gerda Shank <gerda@dbtlabs.com>
2022-09-26 14:49:12 -04:00
Doug Beatty
e58edaab2d Test for Koalas DataFrames (#5928) 2022-09-26 12:41:56 -06:00
Doug Beatty
c202e005cd Tests for current_timestamp (#5935)
* Tests for `current_timestamp`

* Black formatting
2022-09-26 12:31:36 -06:00
Peter Webb
8129862b3c CT-1221 add handle to changie (#5923)
* Add 'peterallenwebb' to changie's core_team list.

* fix accidental line break
2022-09-26 11:44:20 -04:00
Drew Banin
4e8aa007cf Fix adapter reset race condition in lib.py (#5921)
* (#5919) Fix adapter reset race condition in lib.py

* run black

* changie
2022-09-26 10:26:20 -04:00
Doug Beatty
fe88bfabbf Click CLI profiles directory (#5896)
* Default directories for projects and profiles

* Re-write of get_nearest_project_dir()

* Trim Trailing Whitespace

* Functionally equivalent resolvers
2022-09-24 10:47:47 -06:00
Gerda Shank
5328a64df2 CT 815 partial parsing handling of deleted metrics (#5920)
* Update delete_schema_metric to schedule referencing nodes for reparsing

* Changie
2022-09-23 18:48:00 -04:00
45 changed files with 1049 additions and 427 deletions

View File

@@ -0,0 +1,7 @@
kind: Docs
body: Refer to exposures by their label by default.
time: 2022-09-20T15:20:40.652948-05:00
custom:
Author: emmyoop
Issue: "306"
PR: "307"

View File

@@ -0,0 +1,7 @@
kind: Features
body: merge_exclude_columns for incremental materialization
time: 2022-07-16T14:21:16.592519-05:00
custom:
Author: dave-connors-3
Issue: "5260"
PR: "5457"

View File

@@ -0,0 +1,7 @@
kind: Features
body: Array macros
time: 2022-09-12T22:22:27.475515-06:00
custom:
Author: graciegoheen dbeatty10
Issue: "5520"
PR: "5823"

View File

@@ -0,0 +1,9 @@
kind: Features
body: This conditionally no-ops warehouse connection at compile depending on an env
var, disabling introspection/queries during compilation only. This is a temporary
solution to more complex permissions requirements for the semantic layer.
time: 2022-09-26T13:06:27.591061-05:00
custom:
Author: racheldaniel
Issue: "5936"
PR: "5926"

View File

@@ -0,0 +1,7 @@
kind: Fixes
body: Fix typos of comments in core/dbt/adapters/
time: 2022-08-22T19:42:38.593923+09:00
custom:
Author: yoiki
Issue: "5690"
PR: "5693"

View File

@@ -0,0 +1,7 @@
kind: Fixes
body: Fix macro resolution order during static analysis for custom generic tests
time: 2022-09-23T14:32:26.857376+02:00
custom:
Author: jtcohen6
Issue: "5720"
PR: "5907"

View File

@@ -0,0 +1,7 @@
kind: Fixes
body: Fix race condition when invoking dbt via lib.py concurrently
time: 2022-09-23T17:45:04.405026-04:00
custom:
Author: drewbanin
Issue: "5919"
PR: "5921"

View File

@@ -0,0 +1,7 @@
kind: Under the Hood
body: Adding validation for metric expression attribute
time: 2022-09-16T15:47:12.799002-05:00
custom:
Author: callum-mcdata
Issue: "5871"
PR: "5873"

View File

@@ -0,0 +1,7 @@
kind: Under the Hood
body: Reparse references to deleted metric
time: 2022-09-23T13:35:25.681656-04:00
custom:
Author: gshank
Issue: "5444"
PR: "5920"

View File

@@ -44,7 +44,7 @@ custom:
footerFormat: |
{{- $contributorDict := dict }}
{{- /* any names added to this list should be all lowercase for later matching purposes */}}
{{- $core_team := list "emmyoop" "nathaniel-may" "gshank" "leahwicz" "chenyulinx" "stu-k" "iknox-fa" "versusfacit" "mcknight-42" "jtcohen6" "dependabot[bot]" "snyk-bot" }}
{{- $core_team := list "peterallenwebb" "emmyoop" "nathaniel-may" "gshank" "leahwicz" "chenyulinx" "stu-k" "iknox-fa" "versusfacit" "mcknight-42" "jtcohen6" "dependabot[bot]" "snyk-bot" }}
{{- range $change := .Changes }}
{{- $authorList := splitList " " $change.Custom.Author }}
{{- /* loop through all authors for a PR */}}

View File

@@ -384,7 +384,7 @@ class RelationsCache:
relation = self.relations.pop(old_key)
new_key = new_relation.key()
# relaton has to rename its innards, so it needs the _CachedRelation.
# relation has to rename its innards, so it needs the _CachedRelation.
relation.rename(new_relation)
# update all the relations that refer to it
for cached in self.relations.values():

View File

@@ -88,7 +88,7 @@ class AdapterProtocol( # type: ignore[misc]
],
):
# N.B. Technically these are ClassVars, but mypy doesn't support putting type vars in a
# ClassVar due to the restirctiveness of PEP-526
# ClassVar due to the restrictiveness of PEP-526
# See: https://github.com/python/mypy/issues/5144
AdapterSpecificConfigs: Type[AdapterConfig_T]
Column: Type[Column_T]

View File

@@ -2,6 +2,8 @@ from pathlib import Path, PurePath
import click
from dbt.cli.option_types import YAML
from dbt.cli.resolvers import default_project_dir, default_profiles_dir
# TODO: The name (reflected in flags) is a correction!
# The original name was `SEND_ANONYMOUS_USAGE_STATS` and used an env var called "DBT_SEND_ANONYMOUS_USAGE_STATS"
@@ -219,17 +221,15 @@ profiles_dir = click.option(
"--profiles-dir",
envvar="DBT_PROFILES_DIR",
help="Which directory to look in for the profiles.yml file. If not set, dbt will look in the current working directory first, then HOME/.dbt/",
default=PurePath.joinpath(Path.home(), ".dbt"),
type=click.Path(
exists=True,
),
default=default_profiles_dir(),
type=click.Path(exists=True),
)
project_dir = click.option(
"--project-dir",
envvar=None,
help="Which directory to look in for the dbt_project.yml file. Default is the current working directory and its parents.",
default=Path.cwd(),
default=default_project_dir(),
type=click.Path(exists=True),
)

11
core/dbt/cli/resolvers.py Normal file
View File

@@ -0,0 +1,11 @@
from pathlib import Path
def default_project_dir():
paths = list(Path.cwd().parents)
paths.insert(0, Path.cwd())
return next((x for x in paths if (x / "dbt_project.yml").exists()), Path.cwd())
def default_profiles_dir():
return Path.cwd() if (Path.cwd() / "profiles.yml").exists() else Path.home() / ".dbt"

View File

@@ -370,11 +370,6 @@ class Compiler:
compiled_node = _compiled_type_for(node).from_dict(data)
if compiled_node.language == ModelLanguage.python:
# TODO could we also 'minify' this code at all? just aesthetic, not functional
# quoating seems like something very specific to sql so far
# for all python implementations we are seeing there's no quating.
# TODO try to find better way to do this, given that
original_quoting = self.config.quoting
self.config.quoting = {key: False for key in original_quoting.keys()}
context = self._create_node_context(compiled_node, manifest, extra_context)
@@ -385,7 +380,19 @@ class Compiler:
node,
)
# we should NOT jinja render the python model's 'raw code'
compiled_node.compiled_code = f"{node.raw_code}\n\n{postfix}"
# if the user didn't specify an explicit `model(dbt, session)` function,
# we're going to treat the user code as a "script" and wrap it in that function now.
# TODO: this is the jankiest way of doing it, with zero AST magic
if node.meta.get("missing_model_function") is True:
raw_code_lines = node.raw_code.strip().split("\n")
raw_code_lines[-1] = f"return {raw_code_lines[-1]}"
raw_code_indented = "\n ".join(raw_code_lines)
model_code = f"def model(dbt, session):\n {raw_code_indented}"
else:
model_code = node.raw_code
compiled_node.compiled_code = f"{model_code}\n\n{postfix}"
# restore quoting settings in the end since context is lazy evaluated
self.config.quoting = original_quoting

View File

@@ -109,9 +109,15 @@ class MacroResolver:
def get_macro(self, local_package, macro_name):
local_package_macros = {}
# If the macro is explicitly prefixed with an internal namespace
# (e.g. 'dbt.some_macro'), look there first
if local_package in self.internal_package_names:
local_package_macros = self.internal_packages[local_package]
# If the macro is explicitly prefixed with a different package name
# (e.g. 'dbt_utils.some_macro'), look there first
if local_package not in self.internal_package_names and local_package in self.packages:
local_package_macros = self.packages[local_package]
# First: search the local packages for this macro
# First: search the specified package for this macro
if macro_name in local_package_macros:
return local_package_macros[macro_name]
# Now look up in the standard search order

View File

@@ -824,8 +824,8 @@ class ParsedMetric(UnparsedBaseNode, HasUniqueID, HasFqn):
description: str
label: str
calculation_method: str
expression: str
timestamp: str
expression: str
filters: List[MetricFilter]
time_grains: List[str]
dimensions: List[str]

View File

@@ -487,8 +487,8 @@ class UnparsedMetric(dbtClassMixin, Replaceable):
label: str
calculation_method: str
timestamp: str
expression: str
description: str = ""
expression: Union[str, int] = ""
time_grains: List[str] = field(default_factory=list)
dimensions: List[str] = field(default_factory=list)
window: Optional[MetricTime] = None

View File

@@ -21,28 +21,6 @@ def validator_error_message(exc):
return "at path {}: {}".format(path, exc.message)
def generic_format_node(node, include_path: bool, node_attr: str):
source_path = ''
if include_path:
source_path = " ({})".format(node.original_file_path)
node_identifier = getattr(node, node_attr)
return f"{node.resource_type.title()} '{node_identifier}'{source_path}"
def format_node(node, include_path: bool, node_attr: str = "name"):
# Sql Operations are one-off queries. If we used generic formatting,
# the node string would include a nonsense unique id & path like:
#
# sql operation.demo_data.name' (from remote system.sql)
#
# which is terribly hard to understand!
if node.resource_type == NodeType.SqlOperation:
return 'query'
else:
return generic_format_node(node, include_path, node_attr)
class Exception(builtins.Exception):
CODE = -32000
MESSAGE = "Server Error"
@@ -99,8 +77,7 @@ class RuntimeException(RuntimeError, Exception):
# out the path we know at least. This indicates an error during
# block parsing.
return "{}".format(node.path.original_file_path)
return format_node(node, include_path=True)
return "{} {} ({})".format(node.resource_type, node.name, node.original_file_path)
def process_stack(self):
lines = []
@@ -605,9 +582,14 @@ def _get_target_failure_msg(
if target_model_package is not None:
target_package_string = "in package '{}' ".format(target_model_package)
model_string = format_node(model, include_path, node_attr='unique_id')
return "{} depends on a {} named '{}' {}which {}".format(
model_string,
source_path_string = ""
if include_path:
source_path_string = " ({})".format(model.original_file_path)
return "{} '{}'{} depends on a {} named '{}' {}which {}".format(
model.resource_type.title(),
model.unique_id,
source_path_string,
target_kind,
target_name,
target_package_string,

View File

@@ -50,3 +50,31 @@
{{ return(result) }}
{% endmacro %}
{% macro get_merge_update_columns(merge_update_columns, merge_exclude_columns, dest_columns) %}
{{ return(adapter.dispatch('get_merge_update_columns', 'dbt')(merge_update_columns, merge_exclude_columns, dest_columns)) }}
{% endmacro %}
{% macro default__get_merge_update_columns(merge_update_columns, merge_exclude_columns, dest_columns) %}
{%- set default_cols = dest_columns | map(attribute="quoted") | list -%}
{%- if merge_update_columns and merge_exclude_columns -%}
{{ exceptions.raise_compiler_error(
'Model cannot specify merge_update_columns and merge_exclude_columns. Please update model to use only one config'
)}}
{%- elif merge_update_columns -%}
{%- set update_columns = merge_update_columns -%}
{%- elif merge_exclude_columns -%}
{%- set update_columns = [] -%}
{%- for column in dest_columns -%}
{% if column.column | lower not in merge_exclude_columns | map("lower") | list %}
{%- do update_columns.append(column.quoted) -%}
{% endif %}
{%- endfor -%}
{%- else -%}
{%- set update_columns = default_cols -%}
{%- endif -%}
{{ return(update_columns) }}
{% endmacro %}

View File

@@ -5,7 +5,9 @@
{% macro default__get_merge_sql(target, source, unique_key, dest_columns, predicates) -%}
{%- set predicates = [] if predicates is none else [] + predicates -%}
{%- set dest_cols_csv = get_quoted_csv(dest_columns | map(attribute="name")) -%}
{%- set update_columns = config.get('merge_update_columns', default = dest_columns | map(attribute="quoted") | list) -%}
{%- set merge_update_columns = config.get('merge_update_columns') -%}
{%- set merge_exclude_columns = config.get('merge_exclude_columns') -%}
{%- set update_columns = get_merge_update_columns(merge_update_columns, merge_exclude_columns, dest_columns) -%}
{%- set sql_header = config.get('sql_header', none) -%}
{% if unique_key %}

View File

@@ -0,0 +1,8 @@
{% macro array_append(array, new_element) -%}
{{ return(adapter.dispatch('array_append', 'dbt')(array, new_element)) }}
{%- endmacro %}
{# new_element must be the same data type as elements in array to match postgres functionality #}
{% macro default__array_append(array, new_element) -%}
array_append({{ array }}, {{ new_element }})
{%- endmacro %}

View File

@@ -0,0 +1,7 @@
{% macro array_concat(array_1, array_2) -%}
{{ return(adapter.dispatch('array_concat', 'dbt')(array_1, array_2)) }}
{%- endmacro %}
{% macro default__array_concat(array_1, array_2) -%}
array_cat({{ array_1 }}, {{ array_2 }})
{%- endmacro %}

View File

@@ -0,0 +1,12 @@
{% macro array_construct(inputs=[], data_type=api.Column.translate_type('integer')) -%}
{{ return(adapter.dispatch('array_construct', 'dbt')(inputs, data_type)) }}
{%- endmacro %}
{# all inputs must be the same data type to match postgres functionality #}
{% macro default__array_construct(inputs, data_type) -%}
{% if inputs|length > 0 %}
array[ {{ inputs|join(' , ') }} ]
{% else %}
array[]::{{data_type}}[]
{% endif %}
{%- endmacro %}

File diff suppressed because one or more lines are too long

View File

@@ -1,10 +1,64 @@
# TODO: this file is one big TODO
import os
from dbt.contracts.results import RunningStatus, collect_timing_info
from dbt.events.functions import fire_event
from dbt.events.types import NodeCompiling, NodeExecuting
from dbt.exceptions import RuntimeException
from dbt import flags
from collections import namedtuple
from dbt.task.sql import SqlCompileRunner
from dataclasses import dataclass
RuntimeArgs = namedtuple("RuntimeArgs", "project_dir profiles_dir single_threaded profile target")
@dataclass
class RuntimeArgs:
project_dir: str
profiles_dir: str
single_threaded: bool
profile: str
target: str
class SqlCompileRunnerNoIntrospection(SqlCompileRunner):
def compile_and_execute(self, manifest, ctx):
"""
This version of this method does not connect to the data warehouse.
As a result, introspective queries at compilation will not be supported
and will throw an error.
TODO: This is a temporary solution to more complex permissions requirements
for the semantic layer, and thus largely duplicates the code in the parent class
method. Once conditional credential usage is enabled, this should be removed.
"""
result = None
ctx.node._event_status["node_status"] = RunningStatus.Compiling
fire_event(
NodeCompiling(
node_info=ctx.node.node_info,
unique_id=ctx.node.unique_id,
)
)
with collect_timing_info("compile") as timing_info:
# if we fail here, we still have a compiled node to return
# this has the benefit of showing a build path for the errant
# model
ctx.node = self.compile(manifest)
ctx.timing.append(timing_info)
# for ephemeral nodes, we only want to compile, not run
if not ctx.node.is_ephemeral_model:
ctx.node._event_status["node_status"] = RunningStatus.Executing
fire_event(
NodeExecuting(
node_info=ctx.node.node_info,
unique_id=ctx.node.unique_id,
)
)
with collect_timing_info("execute") as timing_info:
result = self.run(ctx.node, manifest)
ctx.node = result.node
ctx.timing.append(timing_info)
return result
def get_dbt_config(project_dir, args=None, single_threaded=False):
@@ -17,27 +71,30 @@ def get_dbt_config(project_dir, args=None, single_threaded=False):
else:
profiles_dir = flags.DEFAULT_PROFILES_DIR
profile = args.profile if hasattr(args, "profile") else None
target = args.target if hasattr(args, "target") else None
# Construct a phony config
config = RuntimeConfig.from_args(
RuntimeArgs(project_dir, profiles_dir, single_threaded, profile, target)
runtime_args = RuntimeArgs(
project_dir=project_dir,
profiles_dir=profiles_dir,
single_threaded=single_threaded,
profile=getattr(args, "profile", None),
target=getattr(args, "target", None),
)
# Clear previously registered adapters--
# this fixes cacheing behavior on the dbt-server
# Construct a RuntimeConfig from phony args
config = RuntimeConfig.from_args(runtime_args)
# Set global flags from arguments
flags.set_from_args(args, config)
dbt.adapters.factory.reset_adapters()
# Load the relevant adapter
# This is idempotent, so we can call it repeatedly
dbt.adapters.factory.register_adapter(config)
# Set invocation id
# Make sure we have a valid invocation_id
dbt.events.functions.set_invocation_id()
return config
def get_task_by_type(type):
# TODO: we need to tell dbt-server what tasks are available
from dbt.task.run import RunTask
from dbt.task.list import ListTask
from dbt.task.seed import SeedTask
@@ -70,16 +127,13 @@ def create_task(type, args, manifest, config):
def no_op(*args, **kwargs):
pass
# TODO: yuck, let's rethink tasks a little
task = task(args, config)
# Wow! We can monkeypatch taskCls.load_manifest to return _our_ manifest
task.load_manifest = no_op
task.manifest = manifest
return task
def _get_operation_node(manifest, project_path, sql):
def _get_operation_node(manifest, project_path, sql, node_name):
from dbt.parser.manifest import process_node
from dbt.parser.sql import SqlBlockParser
import dbt.adapters.factory
@@ -92,26 +146,33 @@ def _get_operation_node(manifest, project_path, sql):
)
adapter = dbt.adapters.factory.get_adapter(config)
# TODO : This needs a real name?
sql_node = block_parser.parse_remote(sql, "name")
sql_node = block_parser.parse_remote(sql, node_name)
process_node(config, manifest, sql_node)
return config, sql_node, adapter
def compile_sql(manifest, project_path, sql):
from dbt.task.sql import SqlCompileRunner
def compile_sql(manifest, project_path, sql, node_name="query"):
config, node, adapter = _get_operation_node(manifest, project_path, sql, node_name)
allow_introspection = str(os.environ.get("__DBT_ALLOW_INTROSPECTION", "1")).lower() in (
"true",
"1",
"on",
)
config, node, adapter = _get_operation_node(manifest, project_path, sql)
runner = SqlCompileRunner(config, adapter, node, 1, 1)
if allow_introspection:
runner = SqlCompileRunner(config, adapter, node, 1, 1)
else:
runner = SqlCompileRunnerNoIntrospection(config, adapter, node, 1, 1)
return runner.safe_run(manifest)
def execute_sql(manifest, project_path, sql):
def execute_sql(manifest, project_path, sql, node_name="query"):
from dbt.task.sql import SqlExecuteRunner
config, node, adapter = _get_operation_node(manifest, project_path, sql)
config, node, adapter = _get_operation_node(manifest, project_path, sql, node_name)
runner = SqlExecuteRunner(config, adapter, node, 1, 1)
# TODO: use same interface for runner
return runner.safe_run(manifest)
@@ -128,5 +189,4 @@ def deserialize_manifest(manifest_msgpack):
def serialize_manifest(manifest):
# TODO: what should this take as an arg?
return manifest.to_msgpack()

View File

@@ -60,8 +60,8 @@ class PythonValidationVisitor(ast.NodeVisitor):
)
def check_error(self, node):
if self.num_model_def != 1:
raise ParsingException("dbt only allow one model defined per python file", node=node)
if self.num_model_def > 1:
raise ParsingException("dbt only allows one model defined per python file", node=node)
if len(self.dbt_errors) != 0:
raise ParsingException("\n".join(self.dbt_errors), node=node)
@@ -113,7 +113,7 @@ class PythonParseVisitor(ast.NodeVisitor):
return arg_literals, kwarg_literals
def visit_Call(self, node: ast.Call) -> None:
# check weather the current call could be a dbt function call
# check whether the current call could be a dbt function call
if isinstance(node.func, ast.Attribute) and node.func.attr in dbt_function_key_words:
func_name = self._flatten_attr(node.func)
# check weather the current call really is a dbt function call
@@ -204,6 +204,13 @@ class ModelParser(SimpleSQLParser[ParsedModelNode]):
dbtValidator.visit(tree)
dbtValidator.check_error(node)
# if the user didn't specify an explicit `model(dbt, session)` function,
# we're going to treat the user code as a "script" to be wrapped in that function at compile time.
# for now, we just need to recognize that fact, and save it to the node.
if dbtValidator.num_model_def == 0:
# TODO: this is silly, put this somewhere better (outside of user space)
node.meta["missing_model_function"] = True
dbtParser = PythonParseVisitor(node)
dbtParser.visit(tree)
config_keys_used = []

View File

@@ -909,8 +909,7 @@ class PartialParsing:
elif unique_id in self.saved_manifest.disabled:
self.delete_disabled(unique_id, schema_file.file_id)
# metric are created only from schema files, so just delete
# the metric or the disabled metric.
# metrics are created only from schema files, but also can be referred to by other nodes
def delete_schema_metric(self, schema_file, metric_dict):
metric_name = metric_dict["name"]
metrics = schema_file.metrics.copy()
@@ -918,6 +917,9 @@ class PartialParsing:
if unique_id in self.saved_manifest.metrics:
metric = self.saved_manifest.metrics[unique_id]
if metric.name == metric_name:
# Need to find everything that referenced this metric and schedule for parsing
if unique_id in self.saved_manifest.child_map:
self.schedule_nodes_for_parsing(self.saved_manifest.child_map[unique_id])
self.deleted_manifest.metrics[unique_id] = self.saved_manifest.metrics.pop(
unique_id
)

View File

@@ -63,7 +63,7 @@ class SourcePatcher:
self.sources[unpatched.unique_id] = unpatched
continue
# returns None if there is no patch
patch = self.get_patch_for(unpatched) # type: ignore[unreachable] # CT-564 / GH 5169
patch = self.get_patch_for(unpatched)
# returns unpatched if there is no patch
patched = self.patch_source(unpatched, patch)
@@ -213,8 +213,8 @@ class SourcePatcher:
self,
unpatched: UnpatchedSourceDefinition,
) -> Optional[SourcePatch]:
if isinstance(unpatched, ParsedSourceDefinition): # type: ignore[unreachable] # CT-564 / GH 5169
return None # type: ignore[unreachable] # CT-564 / GH 5169
if isinstance(unpatched, ParsedSourceDefinition):
return None
key = (unpatched.package_name, unpatched.source.name)
patch: Optional[SourcePatch] = self.manifest.source_patches.get(key)
if patch is None:

View File

@@ -0,0 +1,21 @@
{%
set metric_list = [
metric('number_of_people'),
metric('collective_tenure')
]
%}
{% if not execute %}
{% set metric_names = [] %}
{% for m in metric_list %}
{% do metric_names.append(m.metric_name) %}
{% endfor %}
-- this config does nothing, but it lets us check these values
{{ config(metric_names = metric_names) }}
{% endif %}
select 1 as fun

View File

@@ -0,0 +1,17 @@
version: 2
metrics:
- model: "ref('people')"
name: number_of_people
description: Total count of people
label: "Number of people"
calculation_method: count
expression: "*"
timestamp: created_at
time_grains: [day, week, month]
dimensions:
- favorite_color
- loves_dbt
meta:
my_meta: 'replaced'

View File

@@ -52,7 +52,7 @@ class BasePPTest(DBTIntegrationTest):
class MetricsTest(BasePPTest):
@use_profile('postgres')
def test_postgres_env_vars_models(self):
def test_postgres_metrics(self):
self.setup_directories()
# initial run
self.copy_file('test-files/people.sql', 'models/people.sql')
@@ -89,3 +89,18 @@ class MetricsTest(BasePPTest):
expected_depends_on_nodes = ['model.test.people']
self.assertEqual(metric_people.depends_on.nodes, expected_depends_on_nodes)
# Add model referring to metric
self.copy_file('test-files/metric_model_a.sql', 'models/metric_model_a.sql')
results = self.run_dbt(["run"])
manifest = get_manifest()
model_a = manifest.nodes['model.test.metric_model_a']
expected_depends_on_nodes = ['metric.test.number_of_people', 'metric.test.collective_tenure']
self.assertEqual(model_a.depends_on.nodes, expected_depends_on_nodes)
# Then delete a metric
self.copy_file('test-files/people_metrics3.yml', 'models/people_metrics.yml')
with self.assertRaises(CompilationException):
# We use "parse" here and not "run" because we're checking that the CompilationException
# occurs at parse time, not compilation
results = self.run_dbt(["parse"])

63
test/unit/test_lib.py Normal file
View File

@@ -0,0 +1,63 @@
import os
import unittest
from unittest import mock
from dbt.contracts.results import RunningStatus
from dbt.lib import compile_sql
from dbt.adapters.postgres import Plugin
from test.unit.utils import clear_plugin, inject_adapter
class MockContext:
def __init__(self, node):
self.timing = []
self.node = mock.MagicMock()
self.node._event_status = {
"node_status": RunningStatus.Started
}
self.node.is_ephemeral_model = True
def noop_ephemeral_result(*args):
return None
class TestSqlCompileRunnerNoIntrospection(unittest.TestCase):
def setUp(self):
self.manifest = {'mock':'manifest'}
self.adapter = Plugin.adapter({})
self.adapter.connection_for = mock.MagicMock()
self.ephemeral_result = lambda: None
inject_adapter(self.adapter, Plugin)
def tearDown(self):
clear_plugin(Plugin)
@mock.patch('dbt.lib._get_operation_node')
@mock.patch('dbt.task.sql.GenericSqlRunner.compile')
@mock.patch('dbt.task.sql.GenericSqlRunner.ephemeral_result', noop_ephemeral_result)
@mock.patch('dbt.task.base.ExecutionContext', MockContext)
def test__compile_and_execute__with_connection(self, mock_compile, mock_get_node):
"""
By default, env var for allowing introspection is true, and calling this
method should defer to the parent method.
"""
mock_get_node.return_value = ({}, None, self.adapter)
compile_sql(self.manifest, 'some/path', None)
mock_compile.assert_called_once_with(self.manifest)
self.adapter.connection_for.assert_called_once()
@mock.patch('dbt.lib._get_operation_node')
@mock.patch('dbt.task.sql.GenericSqlRunner.compile')
@mock.patch('dbt.task.sql.GenericSqlRunner.ephemeral_result', noop_ephemeral_result)
@mock.patch('dbt.task.base.ExecutionContext', MockContext)
def test__compile_and_execute__without_connection(self, mock_compile, mock_get_node):
"""
Ensure that compile is called but does not attempt warehouse connection
"""
with mock.patch.dict(os.environ, {"__DBT_ALLOW_INTROSPECTION": "0"}):
mock_get_node.return_value = ({}, None, self.adapter)
compile_sql(self.manifest, 'some/path', None)
mock_compile.assert_called_once_with(self.manifest)
self.adapter.connection_for.assert_not_called()

View File

@@ -57,6 +57,25 @@ def model(dbt, session):
return df
"""
KOALAS_MODEL = """
import databricks.koalas as ks
def model(dbt, session):
dbt.config(
materialized="table",
)
df = ks.DataFrame(
{'City': ['Buenos Aires', 'Brasilia', 'Santiago', 'Bogota', 'Caracas'],
'Country': ['Argentina', 'Brazil', 'Chile', 'Colombia', 'Venezuela'],
'Latitude': [-34.58, -15.78, -33.45, 4.60, 10.48],
'Longitude': [-58.66, -47.91, -70.66, -74.08, -66.86]}
)
return df
"""
class BasePySparkTests:
@pytest.fixture(scope="class")
@@ -65,9 +84,10 @@ class BasePySparkTests:
"pandas_df.py": PANDAS_MODEL,
"pyspark_df.py": PYSPARK_MODEL,
"pandas_on_spark_df.py": PANDAS_ON_SPARK_MODEL,
"koalas_df.py": KOALAS_MODEL,
}
def test_different_dataframes(self, project):
# test
results = run_dbt(["run"])
assert len(results) == 3
assert len(results) == 4

View File

@@ -0,0 +1,22 @@
from dbt.tests.adapter.utils.base_utils import BaseUtils
from dbt.tests.util import run_dbt, check_relations_equal, get_relation_columns
class BaseArrayUtils(BaseUtils):
def assert_columns_equal(self, project, expected_cols, actual_cols):
assert (
expected_cols == actual_cols
), f"Type difference detected: {expected_cols} vs. {actual_cols}"
def test_expected_actual(self, project):
run_dbt(["build"])
# check contents equal
check_relations_equal(project.adapter, ["expected", "actual"])
# check types equal
expected_cols = get_relation_columns(project.adapter, "expected")
actual_cols = get_relation_columns(project.adapter, "actual")
print(f"Expected: {expected_cols}")
print(f"Actual: {actual_cols}")
self.assert_columns_equal(project, expected_cols, actual_cols)

View File

@@ -0,0 +1,12 @@
# array_append
models__array_append_expected_sql = """
select 1 as id, {{ array_construct([1,2,3,4]) }} as array_col union all
select 2 as id, {{ array_construct([4]) }} as array_col
"""
models__array_append_actual_sql = """
select 1 as id, {{ array_append(array_construct([1,2,3]), 4) }} as array_col union all
select 2 as id, {{ array_append(array_construct([]), 4) }} as array_col
"""

View File

@@ -0,0 +1,14 @@
# array_concat
models__array_concat_expected_sql = """
select 1 as id, {{ array_construct([1,2,3,4,5,6]) }} as array_col union all
select 2 as id, {{ array_construct([2]) }} as array_col union all
select 3 as id, {{ array_construct([3]) }} as array_col
"""
models__array_concat_actual_sql = """
select 1 as id, {{ array_concat(array_construct([1,2,3]), array_construct([4,5,6])) }} as array_col union all
select 2 as id, {{ array_concat(array_construct([]), array_construct([2])) }} as array_col union all
select 3 as id, {{ array_concat(array_construct([3]), array_construct([])) }} as array_col
"""

View File

@@ -0,0 +1,12 @@
# array_construct
models__array_construct_expected_sql = """
select 1 as id, {{ array_construct([1,2,3]) }} as array_col union all
select 2 as id, {{ array_construct([]) }} as array_col
"""
models__array_construct_actual_sql = """
select 1 as id, {{ array_construct([1,2,3]) }} as array_col union all
select 2 as id, {{ array_construct([]) }} as array_col
"""

View File

@@ -0,0 +1,19 @@
import pytest
from dbt.tests.adapter.utils.base_array_utils import BaseArrayUtils
from dbt.tests.adapter.utils.fixture_array_append import (
models__array_append_actual_sql,
models__array_append_expected_sql,
)
class BaseArrayAppend(BaseArrayUtils):
@pytest.fixture(scope="class")
def models(self):
return {
"actual.sql": models__array_append_actual_sql,
"expected.sql": models__array_append_expected_sql,
}
class TestArrayAppend(BaseArrayAppend):
pass

View File

@@ -0,0 +1,19 @@
import pytest
from dbt.tests.adapter.utils.base_array_utils import BaseArrayUtils
from dbt.tests.adapter.utils.fixture_array_concat import (
models__array_concat_actual_sql,
models__array_concat_expected_sql,
)
class BaseArrayConcat(BaseArrayUtils):
@pytest.fixture(scope="class")
def models(self):
return {
"actual.sql": models__array_concat_actual_sql,
"expected.sql": models__array_concat_expected_sql,
}
class TestArrayConcat(BaseArrayConcat):
pass

View File

@@ -0,0 +1,19 @@
import pytest
from dbt.tests.adapter.utils.base_array_utils import BaseArrayUtils
from dbt.tests.adapter.utils.fixture_array_construct import (
models__array_construct_actual_sql,
models__array_construct_expected_sql,
)
class BaseArrayConstruct(BaseArrayUtils):
@pytest.fixture(scope="class")
def models(self):
return {
"actual.sql": models__array_construct_actual_sql,
"expected.sql": models__array_construct_expected_sql,
}
class TestArrayConstruct(BaseArrayConstruct):
pass

View File

@@ -0,0 +1,67 @@
import pytest
from datetime import datetime
from datetime import timezone
from datetime import timedelta
from dbt.tests.util import run_dbt
from dbt.tests.util import relation_from_name
models__current_ts_sql = """
select {{ dbt.current_timestamp() }} as current_ts_column
"""
def is_aware(dt: datetime) -> bool:
return dt.tzinfo is not None and dt.tzinfo.utcoffset(dt) is not None
def is_naive(dt: datetime) -> bool:
return not is_aware(dt)
class BaseCurrentTimestamp:
@pytest.fixture(scope="class")
def models(self):
return {
"current_ts.sql": models__current_ts_sql,
}
@pytest.fixture(scope="class")
def current_timestamp(self, project):
run_dbt(["build"])
relation = relation_from_name(project.adapter, "current_ts")
result = project.run_sql(f"select current_ts_column from {relation}", fetch="one")
sql_timestamp = result[0] if result is not None else None
return sql_timestamp
def test_current_timestamp_matches_utc(self, current_timestamp):
sql_timestamp = current_timestamp
now_utc = self.utcnow_matching_type(sql_timestamp)
# Plenty of wiggle room if clocks aren't perfectly sync'd, etc
tolerance = timedelta(minutes=1)
assert (sql_timestamp > (now_utc - tolerance)) and (
sql_timestamp < (now_utc + tolerance)
), f"SQL timestamp {sql_timestamp.isoformat()} is not close enough to Python UTC {now_utc.isoformat()}"
def utcnow_matching_type(self, dt: datetime) -> datetime:
"""
Current UTC datetime with the same timezone-awareness (or naiveness) as the input.
"""
return datetime.now(timezone.utc) if is_aware(dt) else datetime.utcnow()
class BaseCurrentTimestampAware(BaseCurrentTimestamp):
def test_current_timestamp_type(self, current_timestamp):
assert is_aware(current_timestamp)
class BaseCurrentTimestampNaive(BaseCurrentTimestamp):
def test_current_timestamp_type(self, current_timestamp):
assert is_naive(current_timestamp)
# Use either BaseCurrentTimestampAware or BaseCurrentTimestampNaive but not both
class TestCurrentTimestamp(BaseCurrentTimestampAware):
pass

View File

@@ -188,6 +188,42 @@ class TestInvalidMetricMissingModel:
run_dbt(["run"])
invalid_metrics__missing_expression_yml = """
version: 2
metrics:
- name: number_of_people
label: "Number of people"
model: "ref(people)"
description: Total count of people
calculation_method: count
timestamp: created_at
time_grains: [day, week, month]
dimensions:
- favorite_color
- loves_dbt
meta:
my_meta: 'testing'
"""
class TestInvalidMetricMissingExpression:
@pytest.fixture(scope="class")
def models(self):
return {
"people_metrics.yml": invalid_metrics__missing_expression_yml,
"people.sql": models__people_sql,
}
# tests that we get a ParsingException with a missing expression
def test_simple_metric(
self,
project,
):
# initial run
with pytest.raises(ParsingException):
run_dbt(["run"])
names_with_spaces_metrics_yml = """
version: 2

View File

@@ -1,7 +1,3 @@
import pytest
from dbt.tests.fixtures.project import write_project_files
wrong_specification_block__schema_yml = """
version: 2
models:
@@ -1245,264 +1241,31 @@ select 1 as "Id"
"""
alt_local_utils__macros__type_timestamp_sql = """
{%- macro type_timestamp() -%}
{{ return(adapter.dispatch('type_timestamp', 'local_utils')()) }}
{%- endmacro -%}
@pytest.fixture(scope="class")
def wrong_specification_block():
return {"schema.yml": wrong_specification_block__schema_yml}
{% macro default__type_timestamp() %}
{{ return(adapter.dispatch('type_timestamp', 'dbt')()) }}
{% endmacro %}
"""
macro_resolution_order_macros__my_custom_test_sql = """
{% test my_custom_test(model) %}
select cast(current_timestamp as {{ dbt.type_timestamp() }})
limit 0
{% endtest %}
"""
@pytest.fixture(scope="class")
def test_context_where_subq_models():
return {
"schema.yml": test_context_where_subq_models__schema_yml,
"model_a.sql": test_context_where_subq_models__model_a_sql,
}
macro_resolution_order_models__my_model_sql = """
select 1 as id
"""
@pytest.fixture(scope="class")
def test_utils():
return {
"dbt_project.yml": test_utils__dbt_project_yml,
"macros": {
"current_timestamp.sql": test_utils__macros__current_timestamp_sql,
"custom_test.sql": test_utils__macros__custom_test_sql,
},
}
@pytest.fixture(scope="class")
def local_dependency():
return {
"dbt_project.yml": local_dependency__dbt_project_yml,
"macros": {"equality.sql": local_dependency__macros__equality_sql},
}
@pytest.fixture(scope="class")
def case_sensitive_models():
return {
"schema.yml": case_sensitive_models__schema_yml,
"lowercase.sql": case_sensitive_models__lowercase_sql,
}
@pytest.fixture(scope="class")
def test_context_macros():
return {
"my_test.sql": test_context_macros__my_test_sql,
"test_my_datediff.sql": test_context_macros__test_my_datediff_sql,
"custom_schema_tests.sql": test_context_macros__custom_schema_tests_sql,
}
@pytest.fixture(scope="class")
def test_context_models_namespaced():
return {
"schema.yml": test_context_models_namespaced__schema_yml,
"model_c.sql": test_context_models_namespaced__model_c_sql,
"model_b.sql": test_context_models_namespaced__model_b_sql,
"model_a.sql": test_context_models_namespaced__model_a_sql,
}
@pytest.fixture(scope="class")
def macros_v2():
return {
"override_get_test_macros_fail": {
"get_test_sql.sql": macros_v2__override_get_test_macros_fail__get_test_sql_sql
},
"macros": {"tests.sql": macros_v2__macros__tests_sql},
"override_get_test_macros": {
"get_test_sql.sql": macros_v2__override_get_test_macros__get_test_sql_sql
},
"custom-configs": {"test.sql": macros_v2__custom_configs__test_sql},
}
@pytest.fixture(scope="class")
def test_context_macros_namespaced():
return {
"my_test.sql": test_context_macros_namespaced__my_test_sql,
"custom_schema_tests.sql": test_context_macros_namespaced__custom_schema_tests_sql,
}
@pytest.fixture(scope="class")
def seeds():
return {"some_seed.csv": seeds__some_seed_csv}
@pytest.fixture(scope="class")
def test_context_models():
return {
"schema.yml": test_context_models__schema_yml,
"model_c.sql": test_context_models__model_c_sql,
"model_b.sql": test_context_models__model_b_sql,
"model_a.sql": test_context_models__model_a_sql,
}
@pytest.fixture(scope="class")
def name_collision():
return {
"schema.yml": name_collision__schema_yml,
"base.sql": name_collision__base_sql,
"base_extension.sql": name_collision__base_extension_sql,
}
@pytest.fixture(scope="class")
def dupe_tests_collide():
return {
"schema.yml": dupe_generic_tests_collide__schema_yml,
"model_a.sql": dupe_generic_tests_collide__model_a,
}
@pytest.fixture(scope="class")
def custom_generic_test_config_custom_macros():
return {
"schema.yml": custom_generic_test_config_custom_macro__schema_yml,
"model_a.sql": custom_generic_test_config_custom_macro__model_a,
}
@pytest.fixture(scope="class")
def custom_generic_test_names():
return {
"schema.yml": custom_generic_test_names__schema_yml,
"model_a.sql": custom_generic_test_names__model_a,
}
@pytest.fixture(scope="class")
def custom_generic_test_names_alt_format():
return {
"schema.yml": custom_generic_test_names_alt_format__schema_yml,
"model_a.sql": custom_generic_test_names_alt_format__model_a,
}
@pytest.fixture(scope="class")
def test_context_where_subq_macros():
return {"custom_generic_test.sql": test_context_where_subq_macros__custom_generic_test_sql}
@pytest.fixture(scope="class")
def invalid_schema_models():
return {
"schema.yml": invalid_schema_models__schema_yml,
"model.sql": invalid_schema_models__model_sql,
}
@pytest.fixture(scope="class")
def all_models():
return {
"render_test_cli_arg_models": {
"schema.yml": models_v2__render_test_cli_arg_models__schema_yml,
"model.sql": models_v2__render_test_cli_arg_models__model_sql,
},
"override_get_test_models": {
"schema.yml": models_v2__override_get_test_models__schema_yml,
"my_model_warning.sql": models_v2__override_get_test_models__my_model_warning_sql,
"my_model_pass.sql": models_v2__override_get_test_models__my_model_pass_sql,
"my_model_failure.sql": models_v2__override_get_test_models__my_model_failure_sql,
},
"models": {
"schema.yml": models_v2__models__schema_yml,
"table_summary.sql": models_v2__models__table_summary_sql,
"table_failure_summary.sql": models_v2__models__table_failure_summary_sql,
"table_disabled.sql": models_v2__models__table_disabled_sql,
"table_failure_null_relation.sql": models_v2__models__table_failure_null_relation_sql,
"table_failure_copy.sql": models_v2__models__table_failure_copy_sql,
"table_copy.sql": models_v2__models__table_copy_sql,
},
"malformed": {
"schema.yml": models_v2__malformed__schema_yml,
"table_summary.sql": models_v2__malformed__table_summary_sql,
"table_copy.sql": models_v2__malformed__table_copy_sql,
},
"override_get_test_models_fail": {
"schema.yml": models_v2__override_get_test_models_fail__schema_yml,
"my_model.sql": models_v2__override_get_test_models_fail__my_model_sql,
},
"custom-configs": {
"schema.yml": models_v2__custom_configs__schema_yml,
"table_copy_another_one.sql": models_v2__custom_configs__table_copy_another_one_sql,
"table_copy.sql": models_v2__custom_configs__table_copy_sql,
"table.copy.with.dots.sql": models_v2__custom_configs__table_copy_with_dots_sql,
},
"render_test_configured_arg_models": {
"schema.yml": models_v2__render_test_configured_arg_models__schema_yml,
"model.sql": models_v2__render_test_configured_arg_models__model_sql,
},
"custom": {
"schema.yml": models_v2__custom__schema_yml,
"table_copy.sql": models_v2__custom__table_copy_sql,
},
"limit_null": {
"schema.yml": models_v2__limit_null__schema_yml,
"table_warning_limit_null.sql": models_v2__limit_null__table_warning_limit_null_sql,
"table_limit_null.sql": models_v2__limit_null__table_limit_null_sql,
"table_failure_limit_null.sql": models_v2__limit_null__table_failure_limit_null_sql,
},
}
@pytest.fixture(scope="class")
def local_utils():
return {
"dbt_project.yml": local_utils__dbt_project_yml,
"macros": {
"datediff.sql": local_utils__macros__datediff_sql,
"current_timestamp.sql": local_utils__macros__current_timestamp_sql,
"custom_test.sql": local_utils__macros__custom_test_sql,
},
}
@pytest.fixture(scope="class")
def ephemeral():
return {
"schema.yml": ephemeral__schema_yml,
"ephemeral.sql": ephemeral__ephemeral_sql,
}
@pytest.fixture(scope="class")
def quote_required_models():
return {
"schema.yml": quote_required_models__schema_yml,
"model_again.sql": quote_required_models__model_again_sql,
"model_noquote.sql": quote_required_models__model_noquote_sql,
"model.sql": quote_required_models__model_sql,
}
@pytest.fixture(scope="class")
def project_files(
project_root,
test_utils,
local_dependency,
test_context_macros,
macros_v2,
test_context_macros_namespaced,
seeds,
test_context_where_subq_macros,
models,
local_utils,
):
write_project_files(project_root, "test_utils", test_utils)
write_project_files(project_root, "local_dependency", local_dependency)
write_project_files(project_root, "test-context-macros", test_context_macros)
write_project_files(project_root, "macros-v2", macros_v2)
write_project_files(
project_root, "test-context-macros-namespaced", test_context_macros_namespaced
)
write_project_files(project_root, "seeds", seeds)
write_project_files(
project_root, "test-context-where-subq-macros", test_context_where_subq_macros
)
write_project_files(project_root, "models", models)
write_project_files(project_root, "local_utils", local_utils)
macro_resolution_order_models__config_yml = """
version: 2
models:
- name: my_model
tests:
- my_custom_test
"""

View File

@@ -5,31 +5,94 @@ import re
from dbt.tests.util import run_dbt, write_file
from dbt.tests.fixtures.project import write_project_files
from tests.fixtures.dbt_integration_project import dbt_integration_project # noqa: F401
from tests.functional.schema_tests.fixtures import ( # noqa: F401
wrong_specification_block,
test_context_where_subq_models,
test_utils,
local_dependency,
case_sensitive_models,
test_context_macros,
test_context_models_namespaced,
macros_v2,
test_context_macros_namespaced,
seeds,
test_context_models,
name_collision,
dupe_tests_collide,
custom_generic_test_config_custom_macros,
custom_generic_test_names,
custom_generic_test_names_alt_format,
test_context_where_subq_macros,
invalid_schema_models,
all_models,
local_utils,
ephemeral,
quote_required_models,
project_files,
from tests.functional.schema_tests.fixtures import (
wrong_specification_block__schema_yml,
test_context_where_subq_models__schema_yml,
test_context_where_subq_models__model_a_sql,
test_utils__dbt_project_yml,
test_utils__macros__current_timestamp_sql,
test_utils__macros__custom_test_sql,
local_dependency__dbt_project_yml,
local_dependency__macros__equality_sql,
case_sensitive_models__schema_yml,
case_sensitive_models__lowercase_sql,
test_context_macros__my_test_sql,
test_context_macros__test_my_datediff_sql,
test_context_macros__custom_schema_tests_sql,
test_context_models_namespaced__schema_yml,
test_context_models_namespaced__model_c_sql,
test_context_models_namespaced__model_b_sql,
test_context_models_namespaced__model_a_sql,
macros_v2__override_get_test_macros_fail__get_test_sql_sql,
macros_v2__macros__tests_sql,
macros_v2__custom_configs__test_sql,
macros_v2__override_get_test_macros__get_test_sql_sql,
test_context_macros_namespaced__my_test_sql,
test_context_macros_namespaced__custom_schema_tests_sql,
seeds__some_seed_csv,
test_context_models__schema_yml,
test_context_models__model_c_sql,
test_context_models__model_b_sql,
test_context_models__model_a_sql,
name_collision__schema_yml,
name_collision__base_sql,
name_collision__base_extension_sql,
dupe_generic_tests_collide__schema_yml,
dupe_generic_tests_collide__model_a,
custom_generic_test_config_custom_macro__schema_yml,
custom_generic_test_config_custom_macro__model_a,
custom_generic_test_names__schema_yml,
custom_generic_test_names__model_a,
custom_generic_test_names_alt_format__schema_yml,
custom_generic_test_names_alt_format__model_a,
test_context_where_subq_macros__custom_generic_test_sql,
invalid_schema_models__schema_yml,
invalid_schema_models__model_sql,
models_v2__models__schema_yml,
models_v2__models__table_summary_sql,
models_v2__models__table_failure_summary_sql,
models_v2__models__table_disabled_sql,
models_v2__models__table_failure_null_relation_sql,
models_v2__models__table_failure_copy_sql,
models_v2__models__table_copy_sql,
models_v2__limit_null__schema_yml,
models_v2__limit_null__table_warning_limit_null_sql,
models_v2__limit_null__table_limit_null_sql,
models_v2__limit_null__table_failure_limit_null_sql,
models_v2__override_get_test_models__schema_yml,
models_v2__override_get_test_models__my_model_warning_sql,
models_v2__override_get_test_models__my_model_pass_sql,
models_v2__override_get_test_models__my_model_failure_sql,
models_v2__override_get_test_models_fail__schema_yml,
models_v2__override_get_test_models_fail__my_model_sql,
models_v2__malformed__schema_yml,
models_v2__malformed__table_summary_sql,
models_v2__malformed__table_copy_sql,
models_v2__custom_configs__schema_yml,
models_v2__custom_configs__table_copy_another_one_sql,
models_v2__custom_configs__table_copy_sql,
models_v2__custom_configs__table_copy_with_dots_sql,
models_v2__custom__schema_yml,
models_v2__custom__table_copy_sql,
models_v2__render_test_cli_arg_models__schema_yml,
models_v2__render_test_cli_arg_models__model_sql,
models_v2__render_test_configured_arg_models__schema_yml,
models_v2__render_test_configured_arg_models__model_sql,
local_utils__dbt_project_yml,
local_utils__macros__datediff_sql,
local_utils__macros__current_timestamp_sql,
local_utils__macros__custom_test_sql,
ephemeral__schema_yml,
ephemeral__ephemeral_sql,
quote_required_models__schema_yml,
quote_required_models__model_again_sql,
quote_required_models__model_noquote_sql,
quote_required_models__model_sql,
case_sensitive_models__uppercase_SQL,
macro_resolution_order_macros__my_custom_test_sql,
macro_resolution_order_models__config_yml,
macro_resolution_order_models__my_model_sql,
alt_local_utils__macros__type_timestamp_sql,
)
from dbt.exceptions import ParsingException, CompilationException
from dbt.contracts.results import TestStatus
@@ -42,8 +105,16 @@ class TestSchemaTests:
project.run_sql_file(os.path.join(project.test_data_dir, "seed_failure.sql"))
@pytest.fixture(scope="class")
def models(self, all_models): # noqa: F811
return all_models["models"]
def models(self):
return {
"schema.yml": models_v2__models__schema_yml,
"table_summary.sql": models_v2__models__table_summary_sql,
"table_failure_summary.sql": models_v2__models__table_failure_summary_sql,
"table_disabled.sql": models_v2__models__table_disabled_sql,
"table_failure_null_relation.sql": models_v2__models__table_failure_null_relation_sql,
"table_failure_copy.sql": models_v2__models__table_failure_copy_sql,
"table_copy.sql": models_v2__models__table_copy_sql,
}
def assertTestFailed(self, result):
assert result.status == "fail"
@@ -117,8 +188,13 @@ class TestLimitedSchemaTests:
project.run_sql_file(os.path.join(project.test_data_dir, "seed.sql"))
@pytest.fixture(scope="class")
def models(self, all_models): # noqa: F811
return all_models["limit_null"]
def models(self):
return {
"schema.yml": models_v2__limit_null__schema_yml,
"table_warning_limit_null.sql": models_v2__limit_null__table_warning_limit_null_sql,
"table_limit_null.sql": models_v2__limit_null__table_limit_null_sql,
"table_failure_limit_null.sql": models_v2__limit_null__table_failure_limit_null_sql,
}
def assertTestFailed(self, result):
assert result.status == "fail"
@@ -163,8 +239,13 @@ class TestLimitedSchemaTests:
class TestDefaultBoolType:
# test with default True/False in get_test_sql macro
@pytest.fixture(scope="class")
def models(self, all_models): # noqa: F811
return all_models["override_get_test_models"]
def models(self):
return {
"schema.yml": models_v2__override_get_test_models__schema_yml,
"my_model_warning.sql": models_v2__override_get_test_models__my_model_warning_sql,
"my_model_pass.sql": models_v2__override_get_test_models__my_model_pass_sql,
"my_model_failure.sql": models_v2__override_get_test_models__my_model_failure_sql,
}
def assertTestFailed(self, result):
assert result.status == "fail"
@@ -207,10 +288,24 @@ class TestDefaultBoolType:
class TestOtherBoolType:
@pytest.fixture(scope="class", autouse=True)
def setUp(self, project_root):
macros_v2_file = {
"override_get_test_macros": {
"get_test_sql.sql": macros_v2__override_get_test_macros__get_test_sql_sql
},
}
write_project_files(project_root, "macros-v2", macros_v2_file)
# test with expected 0/1 in custom get_test_sql macro
@pytest.fixture(scope="class")
def models(self, all_models): # noqa: F811
return all_models["override_get_test_models"]
def models(self):
return {
"schema.yml": models_v2__override_get_test_models__schema_yml,
"my_model_warning.sql": models_v2__override_get_test_models__my_model_warning_sql,
"my_model_pass.sql": models_v2__override_get_test_models__my_model_pass_sql,
"my_model_failure.sql": models_v2__override_get_test_models__my_model_failure_sql,
}
@pytest.fixture(scope="class")
def project_config_update(self):
@@ -260,10 +355,22 @@ class TestOtherBoolType:
class TestNonBoolType:
@pytest.fixture(scope="class", autouse=True)
def setUp(self, project_root):
macros_v2_file = {
"override_get_test_macros_fail": {
"get_test_sql.sql": macros_v2__override_get_test_macros_fail__get_test_sql_sql
},
}
write_project_files(project_root, "macros-v2", macros_v2_file)
# test with invalid 'x'/'y' in custom get_test_sql macro
@pytest.fixture(scope="class")
def models(self, all_models): # noqa: F811
return all_models["override_get_test_models_fail"]
def models(self):
return {
"schema.yml": models_v2__override_get_test_models_fail__schema_yml,
"my_model.sql": models_v2__override_get_test_models_fail__my_model_sql,
}
@pytest.fixture(scope="class")
def project_config_update(self):
@@ -291,8 +398,12 @@ class TestMalformedSchemaTests:
project.run_sql_file(os.path.join(project.test_data_dir, "seed.sql"))
@pytest.fixture(scope="class")
def models(self, all_models): # noqa: F811
return all_models["malformed"]
def models(self):
return {
"schema.yml": models_v2__malformed__schema_yml,
"table_summary.sql": models_v2__malformed__table_summary_sql,
"table_copy.sql": models_v2__malformed__table_copy_sql,
}
def test_malformed_schema_will_break_run(
self,
@@ -303,14 +414,22 @@ class TestMalformedSchemaTests:
class TestCustomConfigSchemaTests:
@pytest.fixture(scope="class")
def models(self, all_models): # noqa: F811
return all_models["custom-configs"]
@pytest.fixture(scope="class", autouse=True)
def setUp(self, project):
def setUp(self, project, project_root):
project.run_sql_file(os.path.join(project.test_data_dir, "seed.sql"))
macros_v2_file = {"custom-configs": {"test.sql": macros_v2__custom_configs__test_sql}}
write_project_files(project_root, "macros-v2", macros_v2_file)
@pytest.fixture(scope="class")
def models(self):
return {
"schema.yml": models_v2__custom_configs__schema_yml,
"table_copy_another_one.sql": models_v2__custom_configs__table_copy_another_one_sql,
"table_copy.sql": models_v2__custom_configs__table_copy_sql,
"table.copy.with.dots.sql": models_v2__custom_configs__table_copy_with_dots_sql,
}
@pytest.fixture(scope="class")
def project_config_update(self):
return {
@@ -333,8 +452,11 @@ class TestCustomConfigSchemaTests:
class TestHooksInTests:
@pytest.fixture(scope="class")
def models(self, ephemeral): # noqa: F811
return ephemeral
def models(self):
return {
"schema.yml": ephemeral__schema_yml,
"ephemeral.sql": ephemeral__ephemeral_sql,
}
@pytest.fixture(scope="class")
def project_config_update(self):
@@ -359,8 +481,11 @@ class TestHooksInTests:
class TestHooksForWhich:
@pytest.fixture(scope="class")
def models(self, ephemeral): # noqa: F811
return ephemeral
def models(self):
return {
"schema.yml": ephemeral__schema_yml,
"ephemeral.sql": ephemeral__ephemeral_sql,
}
@pytest.fixture(scope="class")
def project_config_update(self):
@@ -393,6 +518,17 @@ class TestCustomSchemaTests:
write_project_files(project_root, "dbt_integration_project", dbt_integration_project)
project.run_sql_file(os.path.join(project.test_data_dir, "seed.sql"))
local_dependency_files = {
"dbt_project.yml": local_dependency__dbt_project_yml,
"macros": {"equality.sql": local_dependency__macros__equality_sql},
}
write_project_files(project_root, "local_dependency", local_dependency_files)
macros_v2_file = {
"macros": {"tests.sql": macros_v2__macros__tests_sql},
}
write_project_files(project_root, "macros-v2", macros_v2_file)
@pytest.fixture(scope="class")
def packages(self):
return {
@@ -417,8 +553,11 @@ class TestCustomSchemaTests:
}
@pytest.fixture(scope="class")
def models(self, all_models): # noqa: F811
return all_models["custom"]
def models(self):
return {
"schema.yml": models_v2__custom__schema_yml,
"table_copy.sql": models_v2__custom__table_copy_sql,
}
def test_schema_tests(
self,
@@ -443,8 +582,13 @@ class TestCustomSchemaTests:
class TestQuotedSchemaTestColumns:
@pytest.fixture(scope="class")
def models(self, quote_required_models): # noqa: F811
return quote_required_models
def models(self):
return {
"schema.yml": quote_required_models__schema_yml,
"model_again.sql": quote_required_models__model_again_sql,
"model_noquote.sql": quote_required_models__model_noquote_sql,
"model.sql": quote_required_models__model_sql,
}
def test_quote_required_column(
self,
@@ -465,9 +609,19 @@ class TestQuotedSchemaTestColumns:
class TestCliVarsSchemaTests:
@pytest.fixture(scope="class", autouse=True)
def setUp(self, project_root):
macros_v2_file = {
"macros": {"tests.sql": macros_v2__macros__tests_sql},
}
write_project_files(project_root, "macros-v2", macros_v2_file)
@pytest.fixture(scope="class")
def models(self, all_models): # noqa: F811
return all_models["render_test_cli_arg_models"]
def models(self):
return {
"schema.yml": models_v2__render_test_cli_arg_models__schema_yml,
"model.sql": models_v2__render_test_cli_arg_models__model_sql,
}
@pytest.fixture(scope="class")
def project_config_update(self):
@@ -488,9 +642,19 @@ class TestCliVarsSchemaTests:
class TestConfiguredVarsSchemaTests:
@pytest.fixture(scope="class", autouse=True)
def setUp(self, project_root):
macros_v2_file = {
"macros": {"tests.sql": macros_v2__macros__tests_sql},
}
write_project_files(project_root, "macros-v2", macros_v2_file)
@pytest.fixture(scope="class")
def models(self, all_models): # noqa: F811
return all_models["render_test_configured_arg_models"]
def models(self):
return {
"schema.yml": models_v2__render_test_configured_arg_models__schema_yml,
"model.sql": models_v2__render_test_configured_arg_models__model_sql,
}
@pytest.fixture(scope="class")
def project_config_update(self):
@@ -512,8 +676,11 @@ class TestConfiguredVarsSchemaTests:
class TestSchemaCaseInsensitive:
@pytest.fixture(scope="class")
def models(self, case_sensitive_models): # noqa: F811
return case_sensitive_models
def models(self):
return {
"schema.yml": case_sensitive_models__schema_yml,
"lowercase.sql": case_sensitive_models__lowercase_sql,
}
@pytest.fixture(scope="class", autouse=True)
def setUP(self, project):
@@ -541,9 +708,33 @@ class TestSchemaCaseInsensitive:
class TestSchemaTestContext:
@pytest.fixture(scope="class", autouse=True)
def setUp(self, project_root):
local_utils_files = {
"dbt_project.yml": local_utils__dbt_project_yml,
"macros": {
"datediff.sql": local_utils__macros__datediff_sql,
"current_timestamp.sql": local_utils__macros__current_timestamp_sql,
"custom_test.sql": local_utils__macros__custom_test_sql,
},
}
write_project_files(project_root, "local_utils", local_utils_files)
test_context_macros_files = {
"my_test.sql": test_context_macros__my_test_sql,
"test_my_datediff.sql": test_context_macros__test_my_datediff_sql,
"custom_schema_tests.sql": test_context_macros__custom_schema_tests_sql,
}
write_project_files(project_root, "test-context-macros", test_context_macros_files)
@pytest.fixture(scope="class")
def models(self, test_context_models): # noqa: F811
return test_context_models
def models(self):
return {
"schema.yml": test_context_models__schema_yml,
"model_c.sql": test_context_models__model_c_sql,
"model_b.sql": test_context_models__model_b_sql,
"model_a.sql": test_context_models__model_a_sql,
}
@pytest.fixture(scope="class")
def project_config_update(self):
@@ -557,10 +748,7 @@ class TestSchemaTestContext:
def packages(self):
return {"packages": [{"local": "local_utils"}]}
def test_test_context_tests(
self,
project,
):
def test_test_context_tests(self, project):
# This test tests the the TestContext and TestMacroNamespace
# are working correctly
run_dbt(["deps"])
@@ -586,9 +774,43 @@ class TestSchemaTestContext:
class TestSchemaTestContextWithMacroNamespace:
@pytest.fixture(scope="class", autouse=True)
def setUp(self, project_root):
test_utils_files = {
"dbt_project.yml": test_utils__dbt_project_yml,
"macros": {
"current_timestamp.sql": test_utils__macros__current_timestamp_sql,
"custom_test.sql": test_utils__macros__custom_test_sql,
},
}
write_project_files(project_root, "test_utils", test_utils_files)
local_utils_files = {
"dbt_project.yml": local_utils__dbt_project_yml,
"macros": {
"datediff.sql": local_utils__macros__datediff_sql,
"current_timestamp.sql": local_utils__macros__current_timestamp_sql,
"custom_test.sql": local_utils__macros__custom_test_sql,
},
}
write_project_files(project_root, "local_utils", local_utils_files)
test_context_macros_namespaced_file = {
"my_test.sql": test_context_macros_namespaced__my_test_sql,
"custom_schema_tests.sql": test_context_macros_namespaced__custom_schema_tests_sql,
}
write_project_files(
project_root, "test-context-macros-namespaced", test_context_macros_namespaced_file
)
@pytest.fixture(scope="class")
def models(self, test_context_models_namespaced): # noqa: F811
return test_context_models_namespaced
def models(self):
return {
"schema.yml": test_context_models_namespaced__schema_yml,
"model_c.sql": test_context_models_namespaced__model_c_sql,
"model_b.sql": test_context_models_namespaced__model_b_sql,
"model_a.sql": test_context_models_namespaced__model_a_sql,
}
@pytest.fixture(scope="class")
def project_config_update(self):
@@ -640,8 +862,12 @@ class TestSchemaTestContextWithMacroNamespace:
class TestSchemaTestNameCollision:
@pytest.fixture(scope="class")
def models(self, name_collision): # noqa: F811
return name_collision
def models(self):
return {
"schema.yml": name_collision__schema_yml,
"base.sql": name_collision__base_sql,
"base_extension.sql": name_collision__base_extension_sql,
}
def test_collision_test_names_get_hash(
self,
@@ -666,8 +892,11 @@ class TestSchemaTestNameCollision:
class TestGenericTestsCollide:
@pytest.fixture(scope="class")
def models(self, dupe_tests_collide): # noqa: F811
return dupe_tests_collide
def models(self):
return {
"schema.yml": dupe_generic_tests_collide__schema_yml,
"model_a.sql": dupe_generic_tests_collide__model_a,
}
def test_generic_test_collision(
self,
@@ -681,8 +910,11 @@ class TestGenericTestsCollide:
class TestGenericTestsConfigCustomMacros:
@pytest.fixture(scope="class")
def models(self, custom_generic_test_config_custom_macros): # noqa: F811
return custom_generic_test_config_custom_macros
def models(self):
return {
"schema.yml": custom_generic_test_config_custom_macro__schema_yml,
"model_a.sql": custom_generic_test_config_custom_macro__model_a,
}
def test_generic_test_config_custom_macros(
self,
@@ -696,8 +928,11 @@ class TestGenericTestsConfigCustomMacros:
class TestGenericTestsCustomNames:
@pytest.fixture(scope="class")
def models(self, custom_generic_test_names): # noqa: F811
return custom_generic_test_names
def models(self):
return {
"schema.yml": custom_generic_test_names__schema_yml,
"model_a.sql": custom_generic_test_names__model_a,
}
# users can define custom names for specific instances of generic tests
def test_generic_tests_with_custom_names(
@@ -723,8 +958,11 @@ class TestGenericTestsCustomNames:
class TestGenericTestsCustomNamesAltFormat(TestGenericTestsCustomNames):
@pytest.fixture(scope="class")
def models(self, custom_generic_test_names_alt_format): # noqa: F811
return custom_generic_test_names_alt_format
def models(self):
return {
"schema.yml": custom_generic_test_names_alt_format__schema_yml,
"model_a.sql": custom_generic_test_names_alt_format__model_a,
}
# exactly as above, just alternative format for yaml definition
def test_collision_test_names_get_hash(
@@ -738,8 +976,11 @@ class TestGenericTestsCustomNamesAltFormat(TestGenericTestsCustomNames):
class TestInvalidSchema:
@pytest.fixture(scope="class")
def models(self, invalid_schema_models): # noqa: F811
return invalid_schema_models
def models(self):
return {
"schema.yml": invalid_schema_models__schema_yml,
"model.sql": invalid_schema_models__model_sql,
}
def test_invalid_schema_file(
self,
@@ -752,8 +993,12 @@ class TestInvalidSchema:
class TestWrongSpecificationBlock:
@pytest.fixture(scope="class")
def models(self, wrong_specification_block): # noqa: F811
return wrong_specification_block
def models(self):
return {"schema.yml": wrong_specification_block__schema_yml}
@pytest.fixture(scope="class")
def seeds(self):
return {"some_seed.csv": seeds__some_seed_csv}
def test_wrong_specification_block(
self,
@@ -777,9 +1022,21 @@ class TestWrongSpecificationBlock:
class TestSchemaTestContextWhereSubq:
@pytest.fixture(scope="class", autouse=True)
def setUp(self, project_root):
test_context_where_subq_macros_file = {
"custom_generic_test.sql": test_context_where_subq_macros__custom_generic_test_sql
}
write_project_files(
project_root, "test-context-where-subq-macros", test_context_where_subq_macros_file
)
@pytest.fixture(scope="class")
def models(self, test_context_where_subq_models): # noqa: F811
return test_context_where_subq_models
def models(self):
return {
"schema.yml": test_context_where_subq_models__schema_yml,
"model_a.sql": test_context_where_subq_models__model_a_sql,
}
@pytest.fixture(scope="class")
def project_config_update(self):
@@ -799,3 +1056,52 @@ class TestSchemaTestContextWhereSubq:
results = run_dbt(["test"])
assert len(results) == 1
class TestCustomSchemaTestMacroResolutionOrder:
@pytest.fixture(scope="class", autouse=True)
def setUp(self, project_root):
alt_local_utils_file = {
"dbt_project.yml": local_utils__dbt_project_yml,
"macros": {
"datediff.sql": alt_local_utils__macros__type_timestamp_sql,
},
}
write_project_files(project_root, "alt_local_utils", alt_local_utils_file)
macros_resolution_order_file = {
"my_custom_test.sql": macro_resolution_order_macros__my_custom_test_sql,
}
write_project_files(
project_root, "macro_resolution_order_macros", macros_resolution_order_file
)
@pytest.fixture(scope="class")
def models(self):
return {
"schema.yml": macro_resolution_order_models__config_yml,
"my_model.sql": macro_resolution_order_models__my_model_sql,
}
@pytest.fixture(scope="class")
def project_config_update(self):
return {
"config-version": 2,
"macro-paths": ["macro_resolution_order_macros"],
}
@pytest.fixture(scope="class")
def packages(self):
return {"packages": [{"local": "alt_local_utils"}]}
def test_macro_resolution_test_namespace(
self,
project,
):
# https://github.com/dbt-labs/dbt-core/issues/5720
# Previously, macros called as 'dbt.some_macro' would not correctly
# resolve to 'some_macro' from the 'dbt' namespace during static analysis,
# if 'some_macro' also existed in an installed package,
# leading to the macro being missing in the TestNamespace
run_dbt(["deps"])
run_dbt(["parse"])