Compare commits

...

20 Commits

Author SHA1 Message Date
Gerda Shank
88eb4d6b69 add @defer_state to unit-test in core/dbt/cli/main.py 2023-07-20 13:25:49 -04:00
Gerda Shank
1336104236 kludge to make mypy happy about parse_from_dict 2023-07-20 11:24:32 -04:00
Gerda Shank
2b264b248e Merge branch 'main' into arky/poc-unit-testing 2023-07-20 11:04:00 -04:00
Chenyu Li
4dc6dd395e copy nodes being referenced over 2023-05-09 10:23:11 -04:00
Michelle Ark
e203efa95b postgres fix 2023-05-09 10:11:22 -04:00
Michelle Ark
05bd06f401 null input values 2023-05-08 02:38:26 -04:00
Michelle Ark
e56538d9bb override jinja context properties 2023-05-08 02:18:17 -04:00
Michelle Ark
43ba24fbc3 hacky lineage between unit tests 2023-05-08 02:07:35 -04:00
Michelle Ark
8d586d0aab fix tests 2023-05-08 01:12:17 -04:00
Michelle Ark
19793eb9fb fix --output json 2023-05-08 00:20:17 -04:00
Michelle Ark
4cf47109f5 remove manifest.add_unit_test 2023-05-08 00:10:52 -04:00
Michelle Ark
58aa0bdb3b undo test_type:unit selector 2023-05-08 00:06:39 -04:00
Michelle Ark
f8ce09ca6c jinja refactoring 2023-05-08 00:02:51 -04:00
Michelle Ark
2c301161cb more decoupling of unit test from other parsing/execution + overrides support 2023-05-07 23:35:07 -04:00
Michelle Ark
ccc3ad3886 fix manifest artifact 2023-05-05 23:14:10 -04:00
Michelle Ark
2c953b227c move parsing to UnitTestManifestLoader + requires 2023-05-05 23:12:11 -04:00
Michelle Ark
2184a4da05 better type handling 2023-05-03 10:33:30 -04:00
Michelle Ark
f8bdd8b19b dbt.string_literal 2023-04-30 22:01:02 -04:00
Michelle Ark
8043106b9f polish up spec, get column schema from relation 2023-04-30 21:50:57 -04:00
Michelle Ark
0ab9222eda first pass 2023-04-30 14:19:50 -04:00
19 changed files with 1085 additions and 34 deletions

View File

@@ -39,6 +39,7 @@ from dbt.task.serve import ServeTask
from dbt.task.show import ShowTask
from dbt.task.snapshot import SnapshotTask
from dbt.task.test import TestTask
from dbt.task.unit_test import UnitTestTask
@dataclass
@@ -845,6 +846,52 @@ def test(ctx, **kwargs):
return results, success
# dbt test
@cli.command("unit-test")
@click.pass_context
@p.defer
@p.deprecated_defer
@p.exclude
@p.fail_fast
@p.favor_state
@p.deprecated_favor_state
@p.indirect_selection
@p.show_output_format
@p.profile
@p.profiles_dir
@p.project_dir
@p.select
@p.selector
@p.state
@p.defer_state
@p.deprecated_state
@p.store_failures
@p.target
@p.target_path
@p.threads
@p.vars
@p.version_check
@requires.postflight
@requires.preflight
@requires.profile
@requires.project
@requires.runtime_config
@requires.manifest
@requires.unit_test_collection
def unit_test(ctx, **kwargs):
"""Runs tests on data in deployed models. Run this after `dbt run`"""
task = UnitTestTask(
ctx.obj["flags"],
ctx.obj["runtime_config"],
ctx.obj["manifest"],
ctx.obj["unit_test_collection"],
)
results = task.run()
success = task.interpret_results(results)
return results, success
# Support running as a module
if __name__ == "__main__":
cli()

View File

@@ -23,6 +23,7 @@ from dbt.parser.manifest import ManifestLoader, write_manifest
from dbt.profiler import profiler
from dbt.tracking import active_user, initialize_from_flags, track_run
from dbt.utils import cast_dict_to_dict_of_strings
from dbt.parser.unit_tests import UnitTestManifestLoader
from dbt.plugins import set_up_plugin_manager, get_plugin_manager
from click import Context
@@ -265,3 +266,25 @@ def manifest(*args0, write=True, write_perf_info=False):
if len(args0) == 0:
return outer_wrapper
return outer_wrapper(args0[0])
def unit_test_collection(func):
"""A decorator used by click command functions for generating a unit test collection provided a manifest"""
def wrapper(*args, **kwargs):
ctx = args[0]
assert isinstance(ctx, Context)
req_strs = ["manifest", "runtime_config"]
reqs = [ctx.obj.get(req_str) for req_str in req_strs]
if None in reqs:
raise DbtProjectError("manifest and runtime_config required for unit_test_collection")
collection = UnitTestManifestLoader.load(ctx.obj["manifest"], ctx.obj["runtime_config"])
ctx.obj["unit_test_collection"] = collection
return func(*args, **kwargs)
return update_wrapper(wrapper, func)

View File

@@ -330,6 +330,26 @@ class MacroGenerator(BaseMacroGenerator):
return self.call_macro(*args, **kwargs)
class UnitTestMacroGenerator(MacroGenerator):
# this makes UnitTestMacroGenerator objects callable like functions
def __init__(
self,
macro_generator: MacroGenerator,
call_return_value: Any,
) -> None:
super().__init__(
macro_generator.macro,
macro_generator.context,
macro_generator.node,
macro_generator.stack,
)
self.call_return_value = call_return_value
def __call__(self, *args, **kwargs):
with self.track_call():
return self.call_return_value
class QueryStringGenerator(BaseMacroGenerator):
def __init__(self, template_str: str, context: Dict[str, Any]) -> None:
super().__init__(context)

View File

@@ -13,7 +13,10 @@ from dbt.flags import get_flags
from dbt.adapters.factory import get_adapter
from dbt.clients import jinja
from dbt.clients.system import make_directory
from dbt.context.providers import generate_runtime_model_context
from dbt.context.providers import (
generate_runtime_model_context,
generate_runtime_unit_test_context,
)
from dbt.contracts.graph.manifest import Manifest, UniqueID
from dbt.contracts.graph.nodes import (
ManifestNode,
@@ -22,6 +25,7 @@ from dbt.contracts.graph.nodes import (
GraphMemberNode,
InjectedCTE,
SeedNode,
UnitTestNode,
)
from dbt.exceptions import (
GraphDependencyNotFoundError,
@@ -44,6 +48,7 @@ def print_compile_stats(stats):
names = {
NodeType.Model: "model",
NodeType.Test: "test",
NodeType.Unit: "unit test",
NodeType.Snapshot: "snapshot",
NodeType.Analysis: "analysis",
NodeType.Macro: "macro",
@@ -289,8 +294,10 @@ class Compiler:
manifest: Manifest,
extra_context: Dict[str, Any],
) -> Dict[str, Any]:
context = generate_runtime_model_context(node, self.config, manifest)
if isinstance(node, UnitTestNode):
context = generate_runtime_unit_test_context(node, self.config, manifest)
else:
context = generate_runtime_model_context(node, self.config, manifest)
context.update(extra_context)
if isinstance(node, GenericTestNode):

View File

@@ -1,4 +1,5 @@
import abc
from copy import deepcopy
import os
from typing import (
Callable,
@@ -17,7 +18,7 @@ from typing_extensions import Protocol
from dbt.adapters.base.column import Column
from dbt.adapters.factory import get_adapter, get_adapter_package_names, get_adapter_type_names
from dbt.clients import agate_helper
from dbt.clients.jinja import get_rendered, MacroGenerator, MacroStack
from dbt.clients.jinja import get_rendered, MacroGenerator, MacroStack, UnitTestMacroGenerator
from dbt.config import RuntimeConfig, Project
from dbt.constants import SECRET_ENV_PREFIX, DEFAULT_ENV_PLACEHOLDER
from dbt.context.base import contextmember, contextproperty, Var
@@ -39,6 +40,7 @@ from dbt.contracts.graph.nodes import (
RefArgs,
AccessType,
SemanticModel,
UnitTestNode,
)
from dbt.contracts.graph.metrics import MetricReference, ResolvedMetricReference
from dbt.contracts.graph.unparsed import NodeVersion
@@ -566,6 +568,17 @@ class OperationRefResolver(RuntimeRefResolver):
return super().create_relation(target_model)
class RuntimeUnitTestRefResolver(RuntimeRefResolver):
def resolve(
self,
target_name: str,
target_package: Optional[str] = None,
target_version: Optional[NodeVersion] = None,
) -> RelationProxy:
target_name = f"{self.model.name}__{target_name}"
return super().resolve(target_name, target_package, target_version)
# `source` implementations
class ParseSourceResolver(BaseSourceResolver):
def resolve(self, source_name: str, table_name: str):
@@ -670,6 +683,22 @@ class RuntimeVar(ModelConfiguredVar):
pass
class UnitTestVar(RuntimeVar):
def __init__(
self,
context: Dict[str, Any],
config: RuntimeConfig,
node: Resource,
) -> None:
config_copy = None
assert isinstance(node, UnitTestNode)
if node.overrides and node.overrides.vars:
config_copy = deepcopy(config)
config_copy.cli_vars.update(node.overrides.vars)
super().__init__(context, config_copy or config, node=node)
# Providers
class Provider(Protocol):
execute: bool
@@ -711,6 +740,16 @@ class RuntimeProvider(Provider):
metric = RuntimeMetricResolver
class RuntimeUnitTestProvider(Provider):
execute = True
Config = RuntimeConfigObject
DatabaseWrapper = RuntimeDatabaseWrapper
Var = UnitTestVar
ref = RuntimeUnitTestRefResolver
source = RuntimeSourceResolver # TODO: RuntimeUnitTestSourceResolver
metric = RuntimeMetricResolver
class OperationProvider(RuntimeProvider):
ref = OperationRefResolver
@@ -1359,7 +1398,7 @@ class ModelContext(ProviderContext):
@contextproperty
def pre_hooks(self) -> List[Dict[str, Any]]:
if self.model.resource_type in [NodeType.Source, NodeType.Test]:
if self.model.resource_type in [NodeType.Source, NodeType.Test, NodeType.Unit]:
return []
# TODO CT-211
return [
@@ -1368,7 +1407,7 @@ class ModelContext(ProviderContext):
@contextproperty
def post_hooks(self) -> List[Dict[str, Any]]:
if self.model.resource_type in [NodeType.Source, NodeType.Test]:
if self.model.resource_type in [NodeType.Source, NodeType.Test, NodeType.Unit]:
return []
# TODO CT-211
return [
@@ -1461,6 +1500,25 @@ class ModelContext(ProviderContext):
return None
class UnitTestContext(ModelContext):
model: UnitTestNode
@contextmember
def env_var(self, var: str, default: Optional[str] = None) -> str:
"""The env_var() function. Return the overriden unit test environment variable named 'var'.
If there is no unit test override, return the environment variable named 'var'.
If there is no such environment variable set, return the default.
If the default is None, raise an exception for an undefined variable.
"""
if self.model.overrides and var in self.model.overrides.env_vars:
return self.model.overrides.env_vars[var]
else:
return super().env_var(var, default)
# This is called by '_context_for', used in 'render_with_context'
def generate_parser_model_context(
model: ManifestNode,
@@ -1505,6 +1563,24 @@ def generate_runtime_macro_context(
return ctx.to_dict()
def generate_runtime_unit_test_context(
unit_test: UnitTestNode,
config: RuntimeConfig,
manifest: Manifest,
) -> Dict[str, Any]:
ctx = UnitTestContext(unit_test, config, manifest, RuntimeUnitTestProvider(), None)
ctx_dict = ctx.to_dict()
if unit_test.overrides and unit_test.overrides.macros:
for macro_name, macro_value in unit_test.overrides.macros.items():
context_value = ctx_dict.get(macro_name)
if isinstance(context_value, MacroGenerator):
ctx_dict[macro_name] = UnitTestMacroGenerator(context_value, macro_value)
else:
ctx_dict[macro_name] = macro_value
return ctx_dict
class ExposureRefResolver(BaseResolver):
def __call__(self, *args, **kwargs) -> str:
package = None

View File

@@ -661,6 +661,7 @@ RESOURCE_TYPES: Dict[NodeType, Type[BaseConfig]] = {
NodeType.Source: SourceConfig,
NodeType.Seed: SeedConfig,
NodeType.Test: TestConfig,
NodeType.Unit: TestConfig,
NodeType.Model: NodeConfig,
NodeType.Snapshot: SnapshotConfig,
}

View File

@@ -34,6 +34,7 @@ from dbt.contracts.graph.unparsed import (
UnparsedSourceDefinition,
UnparsedSourceTableDefinition,
UnparsedColumn,
UnparsedUnitTestOverrides,
)
from dbt.contracts.graph.node_args import ModelNodeArgs
from dbt.contracts.util import Replaceable, AdditionalPropertiesMixin
@@ -942,6 +943,15 @@ class TestMetadata(dbtClassMixin, Replaceable):
namespace: Optional[str] = None
@dataclass
class UnitTestMetadata(dbtClassMixin, Replaceable):
# kwargs are the args that are left in the test builder after
# removing configs. They are set from the test builder when
# the test node is created.
kwargs: Dict[str, Any] = field(default_factory=dict)
namespace: Optional[str] = None
# This has to be separated out because it has no default and so
# has to be included as a superclass, not an attribute
@dataclass
@@ -949,6 +959,11 @@ class HasTestMetadata(dbtClassMixin):
test_metadata: TestMetadata
@dataclass
class HasUnitTestMetadata(dbtClassMixin):
unit_test_metadata: UnitTestMetadata
@dataclass
class GenericTestNode(TestShouldStoreFailures, CompiledNode, HasTestMetadata):
resource_type: NodeType = field(metadata={"restrict": [NodeType.Test]})
@@ -970,6 +985,17 @@ class GenericTestNode(TestShouldStoreFailures, CompiledNode, HasTestMetadata):
return "generic"
@dataclass
class UnitTestNode(CompiledNode):
resource_type: NodeType = field(metadata={"restrict": [NodeType.Unit]})
attached_node: Optional[str] = None
overrides: Optional[UnparsedUnitTestOverrides] = None
@property
def test_node_type(self):
return "unit"
# ====================================
# Snapshot node
# ====================================
@@ -1628,6 +1654,7 @@ ManifestSQLNode = Union[
SqlNode,
GenericTestNode,
SnapshotNode,
UnitTestNode,
]
# All SQL nodes plus SeedNode (csv files)
@@ -1657,7 +1684,4 @@ Resource = Union[
Group,
]
TestNode = Union[
SingularTestNode,
GenericTestNode,
]
TestNode = Union[SingularTestNode, GenericTestNode]

View File

@@ -671,6 +671,34 @@ class UnparsedGroup(dbtClassMixin, Replaceable):
raise ValidationError("Group owner must have at least one of 'name' or 'email'.")
@dataclass
class UnparsedInputFixture(dbtClassMixin):
input: str
rows: List[Dict[str, Any]] = field(default_factory=list)
@dataclass
class UnparsedUnitTestOverrides(dbtClassMixin):
macros: Dict[str, Any] = field(default_factory=dict)
vars: Dict[str, Any] = field(default_factory=dict)
env_vars: Dict[str, Any] = field(default_factory=dict)
@dataclass
class UnparsedUnitTestCase(dbtClassMixin):
name: str
given: Sequence[UnparsedInputFixture]
expect: List[Dict[str, Any]]
description: str = ""
overrides: Optional[UnparsedUnitTestOverrides] = None
@dataclass
class UnparsedUnitTestSuite(dbtClassMixin):
model: str # name of the model being unit tested
tests: Sequence[UnparsedUnitTestCase]
#
# semantic interfaces unparsed objects
#

View File

@@ -2158,7 +2158,7 @@ class SQLCompiledPath(InfoLevel):
return "Z026"
def message(self) -> str:
return f" compiled Code at {self.path}"
return f" compiled code at {self.path}"
class CheckNodeTestFailure(InfoLevel):

View File

@@ -178,7 +178,7 @@ class DbtDatabaseError(DbtRuntimeError):
lines = []
if hasattr(self.node, "build_path") and self.node.build_path:
lines.append(f"compiled Code at {self.node.build_path}")
lines.append(f"compiled code at {self.node.build_path}")
return lines + DbtRuntimeError.process_stack(self)

View File

@@ -12,3 +12,31 @@
{{ "limit " ~ limit if limit != none }}
) dbt_internal_test
{%- endmacro %}
{% macro get_unit_test_sql(main_sql, expected_fixture_sql, expected_column_names) -%}
{{ adapter.dispatch('get_unit_test_sql', 'dbt')(main_sql, expected_fixture_sql, expected_column_names) }}
{%- endmacro %}
{% macro default__get_unit_test_sql(main_sql, expected_fixture_sql, expected_column_names) -%}
-- Build actual result given inputs
with dbt_internal_unit_test_actual AS (
select
{% for expected_column_name in expected_column_names %}{{expected_column_name}}{% if not loop.last -%},{% endif %}{%- endfor -%}, {{ dbt.string_literal("actual") }} as actual_or_expected
from (
{{ main_sql }}
) _dbt_internal_unit_test_actual
),
-- Build expected result
dbt_internal_unit_test_expected AS (
select
{% for expected_column_name in expected_column_names %}{{expected_column_name}}{% if not loop.last -%}, {% endif %}{%- endfor -%}, {{ dbt.string_literal("expected") }} as actual_or_expected
from (
{{ expected_fixture_sql }}
) _dbt_internal_unit_test_expected
)
-- Union actual and expected results
select * from dbt_internal_unit_test_actual
union all
select * from dbt_internal_unit_test_expected
{%- endmacro %}

View File

@@ -0,0 +1,29 @@
{%- materialization unit, default -%}
{% set relations = [] %}
{% set expected_rows = config.get('expected_rows') %}
{% set tested_expected_column_names = expected_rows[0].keys() if (expected_rows | length ) > 0 else get_columns_in_query(sql) %} %}
{%- set target_relation = this.incorporate(type='table') -%}
{%- set temp_relation = make_temp_relation(target_relation)-%}
{% do run_query(get_create_table_as_sql(True, temp_relation, get_empty_subquery_sql(sql))) %}
{%- set columns_in_relation = adapter.get_columns_in_relation(temp_relation) -%}
{%- set column_name_to_data_types = {} -%}
{%- for column in columns_in_relation -%}
{%- do column_name_to_data_types.update({column.name: column.dtype}) -%}
{%- endfor -%}
{% set unit_test_sql = get_unit_test_sql(sql, get_expected_sql(expected_rows, column_name_to_data_types), tested_expected_column_names) %}
{% call statement('main', fetch_result=True) -%}
{{ unit_test_sql }}
{%- endcall %}
{% do adapter.drop_relation(temp_relation) %}
{{ return({'relations': relations}) }}
{%- endmaterialization -%}

View File

@@ -0,0 +1,77 @@
{% macro get_fixture_sql(rows, column_name_to_data_types) %}
-- Fixture for {{ model.name }}
{% set default_row = {} %}
{%- if not column_name_to_data_types -%}
{%- set columns_in_relation = adapter.get_columns_in_relation(this) -%}
{%- set column_name_to_data_types = {} -%}
{%- for column in columns_in_relation -%}
{%- do column_name_to_data_types.update({column.name: column.dtype}) -%}
{%- endfor -%}
{%- endif -%}
{%- if not column_name_to_data_types -%}
{{ exceptions.raise_compiler_error("columns not available for" ~ model.name) }}
{%- endif -%}
{%- for column_name, column_type in column_name_to_data_types.items() -%}
{%- do default_row.update({column_name: (safe_cast("null", column_type) | trim )}) -%}
{%- endfor -%}
{%- for row in rows -%}
{%- do format_row(row, column_name_to_data_types) -%}
{%- set default_row_copy = default_row.copy() -%}
{%- do default_row_copy.update(row) -%}
select
{%- for column_name, column_value in default_row_copy.items() %} {{ column_value }} AS {{ column_name }}{% if not loop.last -%}, {%- endif %}
{%- endfor %}
{%- if not loop.last %}
union all
{% endif %}
{%- endfor -%}
{%- if (rows | length) == 0 -%}
select
{%- for column_name, column_value in default_row.items() %} {{ column_value }} AS {{ column_name }}{% if not loop.last -%},{%- endif %}
{%- endfor %}
limit 0
{%- endif -%}
{% endmacro %}
{% macro get_expected_sql(rows, column_name_to_data_types) %}
{%- if (rows | length) == 0 -%}
select * FROM dbt_internal_unit_test_actual
limit 0
{%- else -%}
{%- for row in rows -%}
{%- do format_row(row, column_name_to_data_types) -%}
select
{%- for column_name, column_value in row.items() %} {{ column_value }} AS {{ column_name }}{% if not loop.last -%}, {%- endif %}
{%- endfor %}
{%- if not loop.last %}
union all
{% endif %}
{%- endfor -%}
{%- endif -%}
{% endmacro %}
{%- macro format_row(row, column_name_to_data_types) -%}
{#-- wrap yaml strings in quotes, apply cast --#}
{%- for column_name, column_value in row.items() -%}
{% set row_update = {column_name: column_value} %}
{%- if column_value is string -%}
{%- set row_update = {column_name: safe_cast(dbt.string_literal(column_value), column_name_to_data_types[column_name]) } -%}
{%- elif column_value is none -%}
{%- set row_update = {column_name: safe_cast('null', column_name_to_data_types[column_name]) } -%}
{%- else -%}
{%- set row_update = {column_name: safe_cast(column_value, column_name_to_data_types[column_name]) } -%}
{%- endif -%}
{%- do row.update(row_update) -%}
{%- endfor -%}
{%- endmacro -%}

View File

@@ -34,6 +34,7 @@ class NodeType(StrEnum):
Metric = "metric"
Group = "group"
SemanticModel = "semantic_model"
Unit = "unit test"
@classmethod
def executable(cls) -> List["NodeType"]:
@@ -47,6 +48,7 @@ class NodeType(StrEnum):
cls.Documentation,
cls.RPCCall,
cls.SqlOperation,
cls.Unit,
]
@classmethod

View File

@@ -138,6 +138,11 @@ class SchemaParser(SimpleParser[YamlBlock, ModelNode]):
self.root_project, self.project.project_name, self.schema_yaml_vars
)
# This is unnecessary, but mypy was requiring it. Clean up parser code so
# we don't have to do this.
def parse_from_dict(self, dct):
pass
@classmethod
def get_compiled_path(cls, block: FileBlock) -> str:
# should this raise an error?
@@ -297,7 +302,7 @@ class YamlReader(metaclass=ABCMeta):
if coerce_dict_str(entry) is None:
raise YamlParseListError(path, self.key, data, "expected a dict with string keys")
if "name" not in entry:
if "name" not in entry and "model" not in entry:
raise ParsingError("Entry did not contain a name")
# Render the data (except for tests and descriptions).

View File

@@ -0,0 +1,211 @@
from dbt.contracts.graph.unparsed import UnparsedUnitTestSuite
from dbt.contracts.graph.nodes import NodeConfig
from dbt_extractor import py_extract_from_source # type: ignore
from dbt.contracts.graph.nodes import (
ModelNode,
UnitTestNode,
)
from dbt.contracts.graph.manifest import Manifest
from dbt.parser.schemas import (
SchemaParser,
YamlBlock,
ValidationError,
JSONValidationError,
YamlParseDictError,
YamlReader,
)
from dbt.exceptions import (
ParsingError,
)
from dbt.parser.search import FileBlock
from dbt.contracts.files import FileHash, SchemaSourceFile
from dbt.node_types import NodeType
from dbt.context.providers import generate_parse_exposure, get_rendered
def _is_model_node(node_id, manifest):
return manifest.nodes[node_id].resource_type == NodeType.Model
class UnitTestManifestLoader:
@classmethod
def load(cls, manifest, root_project) -> Manifest:
unit_test_manifest = Manifest(macros=manifest.macros)
for file in manifest.files.values():
block = FileBlock(file)
if isinstance(file, SchemaSourceFile):
dct = file.dict_from_yaml
if "unit" in dct:
yaml_block = YamlBlock.from_file_block(block, dct)
# TODO: first root_project should be project, or we should only parse unit tests from root_project
schema_parser = SchemaParser(root_project, manifest, root_project)
parser = UnitTestParser(schema_parser, yaml_block, unit_test_manifest)
parser.parse()
model_to_unit_tests = {}
for node in unit_test_manifest.nodes.values():
if isinstance(node, UnitTestNode):
model_name = node.name.split("__")[0]
if model_name not in model_to_unit_tests:
model_to_unit_tests[model_name] = [node.unique_id]
else:
model_to_unit_tests[model_name].append(node.unique_id)
for node in unit_test_manifest.nodes.values():
if isinstance(node, UnitTestNode):
# a unit test should depend on its fixture nodes, and any unit tests on its ref'd nodes
for ref in node.refs:
for unique_id in model_to_unit_tests.get(ref.name, []):
node.depends_on.nodes.append(unique_id)
return unit_test_manifest
class UnitTestParser(YamlReader):
def __init__(self, schema_parser: SchemaParser, yaml: YamlBlock, unit_test_manifest: Manifest):
super().__init__(schema_parser, yaml, "unit")
self.yaml = yaml
self.unit_test_manifest = unit_test_manifest
def parse_unit_test(self, unparsed: UnparsedUnitTestSuite):
package_name = self.project.project_name
path = self.yaml.path.relative_path
# TODO: fix
checksum = "f8f57c9e32eafaacfb002a4d03a47ffb412178f58f49ba58fd6f436f09f8a1d6"
unit_test_node_ids = []
for unit_test in unparsed.tests:
input_nodes = []
original_input_nodes = []
for given in unit_test.given:
original_input_node = self._get_original_input_node(given.input)
original_input_nodes.append(original_input_node)
original_input_node_columns = None
if (
original_input_node.resource_type == NodeType.Model
and original_input_node.config.contract.enforced
):
original_input_node_columns = {
column.name: column.data_type for column in original_input_node.columns
}
# TODO: package_name?
input_name = f"{unparsed.model}__{unit_test.name}__{original_input_node.name}"
input_unique_id = f"model.{package_name}.{input_name}"
input_node = ModelNode(
raw_code=self._build_raw_code(given.rows, original_input_node_columns),
resource_type=NodeType.Model,
package_name=package_name,
path=path,
# original_file_path=self.yaml.path.original_file_path,
original_file_path=f"models_unit_test/{input_name}.sql",
unique_id=input_unique_id,
name=input_name,
config=NodeConfig(materialized="ephemeral"),
database=original_input_node.database,
schema=original_input_node.schema,
alias=original_input_node.alias,
fqn=input_unique_id.split("."),
checksum=FileHash(name="sha256", checksum=checksum),
)
input_nodes.append(input_node)
actual_node = self.manifest.ref_lookup.perform_lookup(
f"model.{package_name}.{unparsed.model}", self.manifest
)
unit_test_unique_id = f"unit.{package_name}.{unit_test.name}.{unparsed.model}"
unit_test_node = UnitTestNode(
resource_type=NodeType.Unit,
package_name=package_name,
path=f"{unparsed.model}.sql",
# original_file_path=self.yaml.path.original_file_path,
original_file_path=f"models_unit_test/{unparsed.model}.sql",
unique_id=unit_test_unique_id,
name=f"{unparsed.model}__{unit_test.name}",
# TODO: merge with node config
config=NodeConfig(materialized="unit", _extra={"expected_rows": unit_test.expect}),
raw_code=actual_node.raw_code,
database=actual_node.database,
schema=actual_node.schema,
alias=f"{unparsed.model}__{unit_test.name}",
fqn=unit_test_unique_id.split("."),
checksum=FileHash(name="sha256", checksum=checksum),
attached_node=actual_node.unique_id,
overrides=unit_test.overrides,
)
# TODO: generalize this method
ctx = generate_parse_exposure(
unit_test_node, # type: ignore
self.root_project,
self.schema_parser.manifest,
package_name,
)
get_rendered(unit_test_node.raw_code, ctx, unit_test_node, capture_macros=True)
# unit_test_node now has a populated refs/sources
# during compilation, refs will resolve to fixtures,
# so add original input node ids to depends on explicitly to preserve lineage
for original_input_node in original_input_nodes:
# TODO: consider nulling out the original_input_node.raw_code
self.unit_test_manifest.nodes[original_input_node.unique_id] = original_input_node
unit_test_node.depends_on.nodes.append(original_input_node.unique_id)
self.unit_test_manifest.nodes[unit_test_node.unique_id] = unit_test_node
# self.unit_test_manifest.nodes[actual_node.unique_id] = actual_node
for input_node in input_nodes:
self.unit_test_manifest.nodes[input_node.unique_id] = input_node
# should be a process_refs / process_sources call isntead?
unit_test_node.depends_on.nodes.append(input_node.unique_id)
unit_test_node_ids.append(unit_test_node.unique_id)
# find out all nodes that are referenced but not in unittest manifest
all_depends_on = set()
for node_id in self.unit_test_manifest.nodes:
if _is_model_node(node_id, self.unit_test_manifest):
all_depends_on.update(self.unit_test_manifest.nodes[node_id].depends_on.nodes) # type: ignore
not_in_manifest = all_depends_on - set(self.unit_test_manifest.nodes.keys())
# copy those node also over into unit_test_manifest
for node_id in not_in_manifest:
self.unit_test_manifest.nodes[node_id] = self.manifest.nodes[node_id]
def parse(self):
for data in self.get_key_dicts():
try:
UnparsedUnitTestSuite.validate(data)
unparsed = UnparsedUnitTestSuite.from_dict(data)
except (ValidationError, JSONValidationError) as exc:
raise YamlParseDictError(self.yaml.path, self.key, data, exc)
self.parse_unit_test(unparsed)
def _build_raw_code(self, rows, column_name_to_data_types) -> str:
return ("{{{{ get_fixture_sql({rows}, {column_name_to_data_types}) }}}}").format(
rows=rows, column_name_to_data_types=column_name_to_data_types
)
def _get_original_input_node(self, input: str):
statically_parsed = py_extract_from_source(f"{{{{ {input} }}}}")
if statically_parsed["refs"]:
ref = statically_parsed["refs"][0]
if len(ref) == 2:
input_package_name, input_model_name = ref
else:
input_package_name, input_model_name = None, ref[0]
# TODO: disabled lookup, versioned lookup, public models
original_input_node = self.manifest.ref_lookup.find(
input_model_name, input_package_name, None, self.manifest
)
elif statically_parsed["sources"]:
input_package_name, input_source_name = statically_parsed["sources"][0]
original_input_node = self.manifest.source_lookup.find(
input_source_name, input_package_name, self.manifest
)
else:
raise ParsingError("given input must be ref or source")
return original_input_node

191
core/dbt/task/unit_test.py Normal file
View File

@@ -0,0 +1,191 @@
from dataclasses import dataclass
from dbt.dataclass_schema import dbtClassMixin
import threading
from typing import Dict, Any, Optional
import io
from .compile import CompileRunner
from .run import RunTask
from dbt.contracts.graph.nodes import UnitTestNode
from dbt.contracts.graph.manifest import Manifest
from dbt.contracts.results import TestStatus, RunResult
from dbt.context.providers import generate_runtime_model_context
from dbt.clients.jinja import MacroGenerator
from dbt.events.functions import fire_event
from dbt.events.types import (
LogTestResult,
LogStartLine,
)
from dbt.graph import ResourceTypeSelector
from dbt.exceptions import (
DbtInternalError,
MissingMaterializationError,
)
from dbt.node_types import NodeType
@dataclass
class UnitTestResultData(dbtClassMixin):
should_error: bool
adapter_response: Dict[str, Any]
diff: Optional[str] = None
class UnitTestRunner(CompileRunner):
def describe_node(self):
return f"{self.node.resource_type} {self.node.name}"
def print_result_line(self, result):
model = result.node
fire_event(
LogTestResult(
name=model.name,
status=str(result.status),
index=self.node_index,
num_models=self.num_nodes,
execution_time=result.execution_time,
node_info=model.node_info,
num_failures=result.failures,
),
level=LogTestResult.status_to_level(str(result.status)),
)
def print_start_line(self):
fire_event(
LogStartLine(
description=self.describe_node(),
index=self.node_index,
total=self.num_nodes,
node_info=self.node.node_info,
)
)
def before_execute(self):
self.print_start_line()
def execute_unit_test(self, node: UnitTestNode, manifest: Manifest) -> UnitTestResultData:
# generate_runtime_unit_test_context not strictly needed - this is to run the 'unit' materialization, not compile the node.compield_code
context = generate_runtime_model_context(node, self.config, manifest)
materialization_macro = manifest.find_materialization_macro_by_name(
self.config.project_name, node.get_materialization(), self.adapter.type()
)
if materialization_macro is None:
raise MissingMaterializationError(
materialization=node.get_materialization(), adapter_type=self.adapter.type()
)
if "config" not in context:
raise DbtInternalError(
"Invalid materialization context generated, missing config: {}".format(context)
)
# generate materialization macro
macro_func = MacroGenerator(materialization_macro, context)
# execute materialization macro
macro_func()
# load results from context
# could eventually be returned directly by materialization
result = context["load_result"]("main")
adapter_response = result["response"].to_dict(omit_none=True)
table = result["table"]
actual = self._get_unit_test_table(table, "actual")
expected = self._get_unit_test_table(table, "expected")
should_error = actual.rows != expected.rows
diff = None
if should_error:
actual_output = self._agate_table_to_str(actual)
expected_output = self._agate_table_to_str(expected)
diff = f"\n\nActual:\n{actual_output}\n\nExpected:\n{expected_output}\n"
return UnitTestResultData(
diff=diff,
should_error=should_error,
adapter_response=adapter_response,
)
def execute(self, node: UnitTestNode, manifest: Manifest):
result = self.execute_unit_test(node, manifest)
thread_id = threading.current_thread().name
status = TestStatus.Pass
message = None
failures = 0
if result.should_error:
status = TestStatus.Fail
message = result.diff
failures = 1
return RunResult(
node=node,
status=status,
timing=[],
thread_id=thread_id,
execution_time=0,
message=message,
adapter_response=result.adapter_response,
failures=failures,
)
def after_execute(self, result):
self.print_result_line(result)
def _get_unit_test_table(self, result_table, actual_or_expected: str):
unit_test_table = result_table.where(
lambda row: row["actual_or_expected"] == actual_or_expected
)
columns = list(unit_test_table.columns.keys())
columns.remove("actual_or_expected")
return unit_test_table.select(columns)
def _agate_table_to_str(self, table) -> str:
# Hack to get Agate table output as string
output = io.StringIO()
if self.config.args.output == "json":
table.to_json(path=output)
else:
table.print_table(output=output, max_rows=None)
return output.getvalue().strip()
class UnitTestSelector(ResourceTypeSelector):
def __init__(self, graph, manifest, previous_state):
super().__init__(
graph=graph,
manifest=manifest,
previous_state=previous_state,
resource_types=[NodeType.Unit],
)
class UnitTestTask(RunTask):
"""
Unit testing:
Read schema files + custom data tests and validate that
constraints are satisfied.
"""
def __init__(self, args, config, manifest, collection):
super().__init__(args, config, collection)
self.collection = collection
self.original_manifest = manifest
__test__ = False
def raise_on_first_error(self):
return False
def get_node_selector(self) -> UnitTestSelector:
if self.manifest is None or self.graph is None:
raise DbtInternalError("manifest and graph must be set to get perform node selection")
return UnitTestSelector(
graph=self.graph,
manifest=self.manifest,
previous_state=self.previous_state,
)
def get_runner_type(self, _):
return UnitTestRunner

View File

@@ -44,6 +44,9 @@
{
"$ref": "#/definitions/SnapshotNode"
},
{
"$ref": "#/definitions/UnitTestNode"
},
{
"$ref": "#/definitions/SeedNode"
}
@@ -129,6 +132,9 @@
{
"$ref": "#/definitions/SnapshotNode"
},
{
"$ref": "#/definitions/UnitTestNode"
},
{
"$ref": "#/definitions/SeedNode"
},
@@ -204,7 +210,7 @@
}
},
"additionalProperties": false,
"description": "WritableManifest(metadata: dbt.contracts.graph.manifest.ManifestMetadata, nodes: Mapping[str, Union[dbt.contracts.graph.nodes.AnalysisNode, dbt.contracts.graph.nodes.SingularTestNode, dbt.contracts.graph.nodes.HookNode, dbt.contracts.graph.nodes.ModelNode, dbt.contracts.graph.nodes.RPCNode, dbt.contracts.graph.nodes.SqlNode, dbt.contracts.graph.nodes.GenericTestNode, dbt.contracts.graph.nodes.SnapshotNode, dbt.contracts.graph.nodes.SeedNode]], sources: Mapping[str, dbt.contracts.graph.nodes.SourceDefinition], macros: Mapping[str, dbt.contracts.graph.nodes.Macro], docs: Mapping[str, dbt.contracts.graph.nodes.Documentation], exposures: Mapping[str, dbt.contracts.graph.nodes.Exposure], metrics: Mapping[str, dbt.contracts.graph.nodes.Metric], groups: Mapping[str, dbt.contracts.graph.nodes.Group], selectors: Mapping[str, Any], disabled: Optional[Mapping[str, List[Union[dbt.contracts.graph.nodes.AnalysisNode, dbt.contracts.graph.nodes.SingularTestNode, dbt.contracts.graph.nodes.HookNode, dbt.contracts.graph.nodes.ModelNode, dbt.contracts.graph.nodes.RPCNode, dbt.contracts.graph.nodes.SqlNode, dbt.contracts.graph.nodes.GenericTestNode, dbt.contracts.graph.nodes.SnapshotNode, dbt.contracts.graph.nodes.SeedNode, dbt.contracts.graph.nodes.SourceDefinition, dbt.contracts.graph.nodes.Exposure, dbt.contracts.graph.nodes.Metric]]]], parent_map: Optional[Dict[str, List[str]]], child_map: Optional[Dict[str, List[str]]], group_map: Optional[Dict[str, List[str]]])",
"description": "WritableManifest(metadata: dbt.contracts.graph.manifest.ManifestMetadata, nodes: Mapping[str, Union[dbt.contracts.graph.nodes.AnalysisNode, dbt.contracts.graph.nodes.SingularTestNode, dbt.contracts.graph.nodes.HookNode, dbt.contracts.graph.nodes.ModelNode, dbt.contracts.graph.nodes.RPCNode, dbt.contracts.graph.nodes.SqlNode, dbt.contracts.graph.nodes.GenericTestNode, dbt.contracts.graph.nodes.SnapshotNode, dbt.contracts.graph.nodes.UnitTestNode, dbt.contracts.graph.nodes.SeedNode]], sources: Mapping[str, dbt.contracts.graph.nodes.SourceDefinition], macros: Mapping[str, dbt.contracts.graph.nodes.Macro], docs: Mapping[str, dbt.contracts.graph.nodes.Documentation], exposures: Mapping[str, dbt.contracts.graph.nodes.Exposure], metrics: Mapping[str, dbt.contracts.graph.nodes.Metric], groups: Mapping[str, dbt.contracts.graph.nodes.Group], selectors: Mapping[str, Any], disabled: Optional[Mapping[str, List[Union[dbt.contracts.graph.nodes.AnalysisNode, dbt.contracts.graph.nodes.SingularTestNode, dbt.contracts.graph.nodes.HookNode, dbt.contracts.graph.nodes.ModelNode, dbt.contracts.graph.nodes.RPCNode, dbt.contracts.graph.nodes.SqlNode, dbt.contracts.graph.nodes.GenericTestNode, dbt.contracts.graph.nodes.SnapshotNode, dbt.contracts.graph.nodes.UnitTestNode, dbt.contracts.graph.nodes.SeedNode, dbt.contracts.graph.nodes.SourceDefinition, dbt.contracts.graph.nodes.Exposure, dbt.contracts.graph.nodes.Metric]]]], parent_map: Optional[Dict[str, List[str]]], child_map: Optional[Dict[str, List[str]]], group_map: Optional[Dict[str, List[str]]])",
"definitions": {
"ManifestMetadata": {
"type": "object",
@@ -216,12 +222,12 @@
},
"dbt_version": {
"type": "string",
"default": "1.5.0b5"
"default": "1.5.0rc1"
},
"generated_at": {
"type": "string",
"format": "date-time",
"default": "2023-04-12T03:35:01.188035Z"
"default": "2023-05-06T03:13:38.958167Z"
},
"invocation_id": {
"oneOf": [
@@ -232,7 +238,7 @@
"type": "null"
}
],
"default": "8aa1596d-f52f-40bc-ad4b-f5e48fc7e6c2"
"default": "f56f309a-b2ce-41ce-8086-15126638957d"
},
"env": {
"type": "object",
@@ -451,7 +457,7 @@
},
"created_at": {
"type": "number",
"default": 1681270501.189703
"default": 1683342818.9598248
},
"config_call_dict": {
"type": "object",
@@ -1155,7 +1161,7 @@
},
"created_at": {
"type": "number",
"default": 1681270501.19095
"default": 1683342818.96106
},
"config_call_dict": {
"type": "object",
@@ -1542,7 +1548,7 @@
},
"created_at": {
"type": "number",
"default": 1681270501.191555
"default": 1683342818.96234
},
"config_call_dict": {
"type": "object",
@@ -1817,7 +1823,7 @@
},
"created_at": {
"type": "number",
"default": 1681270501.192162
"default": 1683342818.962948
},
"config_call_dict": {
"type": "object",
@@ -2180,7 +2186,7 @@
},
"created_at": {
"type": "number",
"default": 1681270501.192949
"default": 1683342818.963712
},
"config_call_dict": {
"type": "object",
@@ -2445,7 +2451,7 @@
},
"created_at": {
"type": "number",
"default": 1681270501.1935291
"default": 1683342818.9643068
},
"config_call_dict": {
"type": "object",
@@ -2703,7 +2709,7 @@
},
"created_at": {
"type": "number",
"default": 1681270501.19419
"default": 1683342818.964963
},
"config_call_dict": {
"type": "object",
@@ -2998,7 +3004,7 @@
},
"created_at": {
"type": "number",
"default": 1681270501.1952698
"default": 1683342818.966057
},
"config_call_dict": {
"type": "object",
@@ -3323,6 +3329,281 @@
"additionalProperties": true,
"description": "SnapshotConfig(_extra: Dict[str, Any] = <factory>, enabled: bool = True, alias: Optional[str] = None, schema: Optional[str] = None, database: Optional[str] = None, tags: Union[List[str], str] = <factory>, meta: Dict[str, Any] = <factory>, group: Optional[str] = None, materialized: str = 'snapshot', incremental_strategy: Optional[str] = None, persist_docs: Dict[str, Any] = <factory>, post_hook: List[dbt.contracts.graph.model_config.Hook] = <factory>, pre_hook: List[dbt.contracts.graph.model_config.Hook] = <factory>, quoting: Dict[str, Any] = <factory>, column_types: Dict[str, Any] = <factory>, full_refresh: Optional[bool] = None, unique_key: Optional[str] = None, on_schema_change: Optional[str] = 'ignore', grants: Dict[str, Any] = <factory>, packages: List[str] = <factory>, docs: dbt.contracts.graph.unparsed.Docs = <factory>, contract: dbt.contracts.graph.model_config.ContractConfig = <factory>, strategy: Optional[str] = None, target_schema: Optional[str] = None, target_database: Optional[str] = None, updated_at: Optional[str] = None, check_cols: Union[str, List[str], NoneType] = None)"
},
"UnitTestNode": {
"type": "object",
"required": [
"schema",
"name",
"resource_type",
"package_name",
"path",
"original_file_path",
"unique_id",
"fqn",
"alias",
"checksum"
],
"properties": {
"database": {
"oneOf": [
{
"type": "string"
},
{
"type": "null"
}
]
},
"schema": {
"type": "string"
},
"name": {
"type": "string"
},
"resource_type": {
"type": "string",
"enum": [
"unit test"
]
},
"package_name": {
"type": "string"
},
"path": {
"type": "string"
},
"original_file_path": {
"type": "string"
},
"unique_id": {
"type": "string"
},
"fqn": {
"type": "array",
"items": {
"type": "string"
}
},
"alias": {
"type": "string"
},
"checksum": {
"$ref": "#/definitions/FileHash"
},
"config": {
"$ref": "#/definitions/NodeConfig",
"default": {
"enabled": true,
"alias": null,
"schema": null,
"database": null,
"tags": [],
"meta": {},
"group": null,
"materialized": "view",
"incremental_strategy": null,
"persist_docs": {},
"quoting": {},
"column_types": {},
"full_refresh": null,
"unique_key": null,
"on_schema_change": "ignore",
"grants": {},
"packages": [],
"docs": {
"show": true,
"node_color": null
},
"contract": {
"enforced": false
},
"post-hook": [],
"pre-hook": []
}
},
"tags": {
"type": "array",
"items": {
"type": "string"
},
"default": []
},
"description": {
"type": "string",
"default": ""
},
"columns": {
"type": "object",
"additionalProperties": {
"$ref": "#/definitions/ColumnInfo"
},
"default": {}
},
"meta": {
"type": "object",
"default": {}
},
"group": {
"oneOf": [
{
"type": "string"
},
{
"type": "null"
}
]
},
"docs": {
"$ref": "#/definitions/Docs",
"default": {
"show": true,
"node_color": null
}
},
"patch_path": {
"oneOf": [
{
"type": "string"
},
{
"type": "null"
}
]
},
"build_path": {
"oneOf": [
{
"type": "string"
},
{
"type": "null"
}
]
},
"deferred": {
"type": "boolean",
"default": false
},
"unrendered_config": {
"type": "object",
"default": {}
},
"created_at": {
"type": "number",
"default": 1683342818.9666371
},
"config_call_dict": {
"type": "object",
"default": {}
},
"relation_name": {
"oneOf": [
{
"type": "string"
},
{
"type": "null"
}
]
},
"raw_code": {
"type": "string",
"default": ""
},
"language": {
"type": "string",
"default": "sql"
},
"refs": {
"type": "array",
"items": {
"$ref": "#/definitions/RefArgs"
},
"default": []
},
"sources": {
"type": "array",
"items": {
"type": "array",
"items": {
"type": "string"
}
},
"default": []
},
"metrics": {
"type": "array",
"items": {
"type": "array",
"items": {
"type": "string"
}
},
"default": []
},
"depends_on": {
"$ref": "#/definitions/DependsOn",
"default": {
"macros": [],
"nodes": []
}
},
"compiled_path": {
"oneOf": [
{
"type": "string"
},
{
"type": "null"
}
]
},
"compiled": {
"type": "boolean",
"default": false
},
"compiled_code": {
"oneOf": [
{
"type": "string"
},
{
"type": "null"
}
]
},
"extra_ctes_injected": {
"type": "boolean",
"default": false
},
"extra_ctes": {
"type": "array",
"items": {
"$ref": "#/definitions/InjectedCTE"
},
"default": []
},
"contract": {
"$ref": "#/definitions/Contract",
"default": {
"enforced": false,
"checksum": null
}
},
"attached_node": {
"oneOf": [
{
"type": "string"
},
{
"type": "null"
}
]
}
},
"additionalProperties": false,
"description": "UnitTestNode(database: Optional[str], schema: str, name: str, resource_type: dbt.node_types.NodeType, package_name: str, path: str, original_file_path: str, unique_id: str, fqn: List[str], alias: str, checksum: dbt.contracts.files.FileHash, config: dbt.contracts.graph.model_config.NodeConfig = <factory>, _event_status: Dict[str, Any] = <factory>, tags: List[str] = <factory>, description: str = '', columns: Dict[str, dbt.contracts.graph.nodes.ColumnInfo] = <factory>, meta: Dict[str, Any] = <factory>, group: Optional[str] = None, docs: dbt.contracts.graph.unparsed.Docs = <factory>, patch_path: Optional[str] = None, build_path: Optional[str] = None, deferred: bool = False, unrendered_config: Dict[str, Any] = <factory>, created_at: float = <factory>, config_call_dict: Dict[str, Any] = <factory>, relation_name: Optional[str] = None, raw_code: str = '', language: str = 'sql', refs: List[dbt.contracts.graph.nodes.RefArgs] = <factory>, sources: List[List[str]] = <factory>, metrics: List[List[str]] = <factory>, depends_on: dbt.contracts.graph.nodes.DependsOn = <factory>, compiled_path: Optional[str] = None, compiled: bool = False, compiled_code: Optional[str] = None, extra_ctes_injected: bool = False, extra_ctes: List[dbt.contracts.graph.nodes.InjectedCTE] = <factory>, _pre_injected_sql: Optional[str] = None, contract: dbt.contracts.graph.nodes.Contract = <factory>, attached_node: Optional[str] = None)"
},
"SeedNode": {
"type": "object",
"required": [
@@ -3485,7 +3766,7 @@
},
"created_at": {
"type": "number",
"default": 1681270501.1968079
"default": 1683342818.9676318
},
"config_call_dict": {
"type": "object",
@@ -3887,7 +4168,7 @@
},
"created_at": {
"type": "number",
"default": 1681270501.197819
"default": 1683342818.968619
}
},
"additionalProperties": false,
@@ -3997,12 +4278,12 @@
},
"dbt_version": {
"type": "string",
"default": "1.5.0b5"
"default": "1.5.0rc1"
},
"generated_at": {
"type": "string",
"format": "date-time",
"default": "2023-04-12T03:35:01.185979Z"
"default": "2023-05-06T03:13:38.956088Z"
},
"invocation_id": {
"oneOf": [
@@ -4013,7 +4294,7 @@
"type": "null"
}
],
"default": "8aa1596d-f52f-40bc-ad4b-f5e48fc7e6c2"
"default": "f56f309a-b2ce-41ce-8086-15126638957d"
},
"env": {
"type": "object",
@@ -4024,7 +4305,7 @@
}
},
"additionalProperties": false,
"description": "FreshnessMetadata(dbt_schema_version: str = <factory>, dbt_version: str = '1.5.0b5', generated_at: datetime.datetime = <factory>, invocation_id: Optional[str] = <factory>, env: Dict[str, str] = <factory>)"
"description": "FreshnessMetadata(dbt_schema_version: str = <factory>, dbt_version: str = '1.5.0rc1', generated_at: datetime.datetime = <factory>, invocation_id: Optional[str] = <factory>, env: Dict[str, str] = <factory>)"
},
"SourceFreshnessRuntimeError": {
"type": "object",
@@ -4366,7 +4647,7 @@
},
"created_at": {
"type": "number",
"default": 1681270501.198105
"default": 1683342818.968908
},
"supported_languages": {
"oneOf": [
@@ -4606,7 +4887,7 @@
},
"created_at": {
"type": "number",
"default": 1681270501.198782
"default": 1683342818.969573
}
},
"additionalProperties": false,
@@ -4826,7 +5107,7 @@
},
"created_at": {
"type": "number",
"default": 1681270501.199492
"default": 1683342818.970261
},
"group": {
"oneOf": [

View File

@@ -17,6 +17,7 @@ node_type_pluralizations = {
NodeType.Metric: "metrics",
NodeType.Group: "groups",
NodeType.SemanticModel: "semantic_models",
NodeType.Unit: "unit tests",
}