Compare commits

...

12 Commits

Author SHA1 Message Date
Michelle Ark
52e2e8531e test adapter pre hooks for unit testing 2024-06-07 12:14:34 -04:00
McKnight-42
ef67cffc0a init push for issue 10198 2024-06-03 15:02:40 -05:00
Quigley Malcolm
a34267f54b Create RuntimeConfig fixture (#10242)
* Create `runtime_config` fixture and necessary upstream fixtures

* Check for better scoped `ProjectContractError` in test_runtime tests

Previously in `test_unsupported_version_extra_config` and
`test_archive_not_allowed` we were checking for `DbtProjectError`. This
worked because `ProjectContractError` is a subclass of `DbtProjectError`.
However, if we check for `DbtProjectError` in these tests than, some tangential
failure which raises a `DbtProejctError` type error would go undetected. As
we plan on modifying these tests to be pytest in the coming commits, we want to
ensure that the tests are succeeding for the right reason.

* Convert `test_str` of `TestRuntimeConfig` to a pytest test using fixtures

* Convert `test_from_parts` of `TestRuntimeConfig` to a pytest test using fixtures

While converting `test_from_parts` I noticed the comment
>  TODO(jeb): Adapters must assert that quoting is populated?

This led me to beleive that `quoting` shouldn't be "fully" realized
in our project fixture unless we're saying that it's gone through
adapter instantiation. Thus I update the `quoting` on our project
fixture to be an empty dict. This change affected `test__str__` in
`test_project.py` which we thus needed to update accordingly.

* Convert runtime version specifier tests to pytest tests and move to test_project

We've done two things in this commit, which arguably _should_ have been done in
two commits. First we moved the version specifier tests from `test_runtime.py::TestRuntimeConfig`
to `test_project.py::TestGetRequiredVersion` this is because what is really being
tested is the `_get_required_version` method. Doing it via `RuntimeConfig.from_parts` method
made actually testing it a lot harder as it requires setting up more of the world and
running with a _full_ project config dict.

The second thing we did was convert it from the old unittest implementation to a pytest
implementation. This saves us from having to create most of the world as we were doing
previously in these tests.

Of note, I did not move the test `test_unsupported_version_range_bad_config`. This test
is a bit different from the rest of the version specifier tests. It was introduced in
[1eb5857811](1eb5857811)
of [#2726](https://github.com/dbt-labs/dbt-core/pull/2726) to resolve [#2638](https://github.com/dbt-labs/dbt-core/issues/2638).
The focus of #2726 was to ensure the version specifier checks were run _before_ the validation
of the `dbt_project.yml`. Thus what this test is actually testing for is order of
operations at parse time. As such, this is really more a _functional_ test than a
unit test. In the next commit we'll get this test moved (and renamed)

* Create a better test for checking that version checks come before project schema validation

* Convert `test_get_metadata` to pytest test

* Refactor `test_archive_not_allowed` to functional test

We do already have tests that ensure "extra" keys aren't allowed in
the dbt_project.yaml. This test is different because it's checking that
a specific key, `archive`, isn't allowed. We do this because at one point
in time `archive` _was_ an allowed key. Specifically, we stopped allowing
`archive` in dbt-core 0.15.0 via commit [f26948dd](f26948dde2).
Given that it's been 5 years and a major version, we could probably remove
this test, but let's keep it around unless we start supporting `archive` again.

* Convert `warn_for_unused_resource_config_paths` tests to use pytest
2024-05-31 13:47:28 -07:00
Doug Beatty
155482851a Button to open an issue for the docs.getdbt.com documentation repo (#10239) 2024-05-30 06:14:34 -06:00
Kshitij Aranke
81386a7a43 Move StatsItem, StatsDict, TableMetadata to dbt-common (#10238) 2024-05-30 02:22:41 +01:00
Emily Rockman
d8e38c1a1d add recording types (#10175)
* add new mode

* reflect dbt-common changes

* changelog

* rename var
2024-05-29 14:53:06 -05:00
Michelle Ark
3e37d77780 [Fix] Renaming or removing a contracted model raises a BreakingChange warning/error (#10221) 2024-05-28 15:48:21 -04:00
Michelle Ark
e0783c2922 Fix: prefer disabled project nodes to external node (#10223) 2024-05-28 11:37:33 -04:00
Michelle Ark
c2d4643f9d add --host flag to docs serve (#10228) 2024-05-28 11:11:56 -04:00
Michelle Ark
84456f50f6 Fix: Order-insensitive unit test equality assertion for expected/actual with multiple nulls (#10202) 2024-05-24 10:12:26 -04:00
Michelle Ark
fb10bb4aea Maximally parallelize dbt clone (#10129) 2024-05-23 11:06:04 -04:00
FishtownBuildBot
366d4ad04a Add most recent dbt-docs changes (#10217) 2024-05-22 15:41:57 -07:00
46 changed files with 1170 additions and 309 deletions

View File

@@ -0,0 +1,6 @@
kind: Docs
body: Enable display of unit tests
time: 2024-03-11T14:03:44.490834-04:00
custom:
Author: gshank
Issue: "501"

View File

@@ -0,0 +1,6 @@
kind: Docs
body: Unit tests not rendering
time: 2024-05-01T02:10:50.987412+02:00
custom:
Author: aranke
Issue: "506"

View File

@@ -0,0 +1,6 @@
kind: Docs
body: Add support for Saved Query node
time: 2024-05-16T22:30:36.206492-07:00
custom:
Author: ChenyuLInx
Issue: "486"

View File

@@ -0,0 +1,7 @@
kind: Features
body: Maximally parallelize dbt clone
in clone command"
time: 2024-05-22T00:03:09.765977-04:00
custom:
Author: michelleark
Issue: "7914"

View File

@@ -0,0 +1,6 @@
kind: Features
body: Add --host flag to dbt docs serve, defaulting to '127.0.0.1'
time: 2024-05-27T12:44:05.040843-04:00
custom:
Author: michelleark
Issue: "10229"

View File

@@ -0,0 +1,7 @@
kind: Fixes
body: 'Fix: Order-insensitive unit test equality assertion for expected/actual with
multiple nulls'
time: 2024-05-22T18:28:55.91733-04:00
custom:
Author: michelleark
Issue: "10167"

View File

@@ -0,0 +1,6 @@
kind: Fixes
body: Renaming or removing a contracted model should raise a BreakingChange warning/error
time: 2024-05-23T20:42:51.033946-04:00
custom:
Author: michelleark
Issue: "10116"

View File

@@ -0,0 +1,6 @@
kind: Fixes
body: prefer disabled project nodes to external node
time: 2024-05-24T13:11:35.440443-04:00
custom:
Author: michelleark
Issue: "10224"

View File

@@ -0,0 +1,6 @@
kind: Under the Hood
body: Enable record filtering by type.
time: 2024-05-29T10:28:14.547624-05:00
custom:
Author: emmyoop
Issue: "10240"

View File

@@ -1,5 +1,8 @@
blank_issues_enabled: false
contact_links:
- name: Documentation
url: https://github.com/dbt-labs/docs.getdbt.com/issues/new/choose
about: Problems and issues with dbt documentation
- name: Ask the community for help
url: https://github.com/dbt-labs/docs.getdbt.com/discussions
about: Need help troubleshooting? Check out our guide on how to ask

View File

@@ -1,2 +1,3 @@
# alias to latest
from dbt.artifacts.schemas.catalog.v1.catalog import * # noqa
from dbt_common.contracts.metadata import StatsDict, StatsItem, TableMetadata

View File

@@ -7,6 +7,7 @@ from dbt.artifacts.schemas.base import (
BaseArtifactMetadata,
schema_version,
)
from dbt_common.contracts.metadata import StatsDict, TableMetadata
from dbt_common.dataclass_schema import dbtClassMixin
from dbt_common.utils.formatting import lowercase
@@ -18,18 +19,6 @@ CatalogKey = NamedTuple(
)
@dataclass
class StatsItem(dbtClassMixin):
id: str
label: str
value: Primitive
include: bool
description: Optional[str] = None
StatsDict = Dict[str, StatsItem]
@dataclass
class ColumnMetadata(dbtClassMixin):
type: str
@@ -41,16 +30,6 @@ class ColumnMetadata(dbtClassMixin):
ColumnMap = Dict[str, ColumnMetadata]
@dataclass
class TableMetadata(dbtClassMixin):
type: str
schema: str
name: str
database: Optional[str] = None
comment: Optional[str] = None
owner: Optional[str] = None
@dataclass
class CatalogTable(dbtClassMixin):
metadata: TableMetadata

View File

@@ -274,6 +274,7 @@ def docs_generate(ctx, **kwargs):
@click.pass_context
@global_flags
@p.browser
@p.host
@p.port
@p.profiles_dir
@p.project_dir

View File

@@ -135,6 +135,14 @@ full_refresh = click.option(
is_flag=True,
)
host = click.option(
"--host",
envvar="DBT_HOST",
help="host to serve dbt docs on",
type=click.STRING,
default="127.0.0.1",
)
indirect_selection = click.option(
"--indirect-selection",
envvar="DBT_INDIRECT_SELECTION",

View File

@@ -41,7 +41,12 @@ from dbt_common.events.functions import LOG_VERSION, fire_event
from dbt_common.events.helpers import get_json_string_utcnow
from dbt_common.exceptions import DbtBaseException as DbtException
from dbt_common.invocation import reset_invocation_id
from dbt_common.record import Recorder, RecorderMode, get_record_mode_from_env
from dbt_common.record import (
Recorder,
RecorderMode,
get_record_mode_from_env,
get_record_types_from_env,
)
from dbt_common.utils import cast_dict_to_dict_of_strings
@@ -101,13 +106,14 @@ def preflight(func):
def setup_record_replay():
rec_mode = get_record_mode_from_env()
rec_types = get_record_types_from_env()
recorder: Optional[Recorder] = None
if rec_mode == RecorderMode.REPLAY:
recording_path = os.environ["DBT_REPLAY"]
recorder = Recorder(RecorderMode.REPLAY, recording_path)
recording_path = os.environ.get("DBT_RECORDER_FILE_PATH")
recorder = Recorder(RecorderMode.REPLAY, types=rec_types, recording_path=recording_path)
elif rec_mode == RecorderMode.RECORD:
recorder = Recorder(RecorderMode.RECORD)
recorder = Recorder(RecorderMode.RECORD, types=rec_types)
get_invocation_context().recorder = recorder

View File

@@ -19,6 +19,8 @@ from typing import (
from mashumaro.types import SerializableType
from dbt import deprecations
from dbt.adapters.base import ConstraintSupport
from dbt.adapters.factory import get_adapter_constraint_support
from dbt.artifacts.resources import Analysis as AnalysisResource
from dbt.artifacts.resources import (
BaseResource,
@@ -469,6 +471,13 @@ class ModelNode(ModelResource, CompiledNode):
def is_latest_version(self) -> bool:
return self.version is not None and self.version == self.latest_version
@property
def is_past_deprecation_date(self) -> bool:
return (
self.deprecation_date is not None
and self.deprecation_date < datetime.now().astimezone()
)
@property
def search_name(self):
if self.version is None:
@@ -570,6 +579,42 @@ class ModelNode(ModelResource, CompiledNode):
data = contract_state.encode("utf-8")
self.contract.checksum = hashlib.new("sha256", data).hexdigest()
def same_contract_removed(self) -> bool:
"""
self: the removed (deleted, renamed, or disabled) model node
"""
# If the contract wasn't previously enforced, no contract change has occurred
if self.contract.enforced is False:
return True
# Removed node is past its deprecation_date, so deletion does not constitute a contract change
if self.is_past_deprecation_date:
return True
# Disabled, deleted, or renamed node with previously enforced contract.
if not self.config.enabled:
breaking_change = f"Contracted model '{self.unique_id}' was disabled."
else:
breaking_change = f"Contracted model '{self.unique_id}' was deleted or renamed."
if self.version is None:
warn_or_error(
UnversionedBreakingChange(
breaking_changes=[breaking_change],
model_name=self.name,
model_file_path=self.original_file_path,
),
node=self,
)
return False
else:
raise (
ContractBreakingChangeError(
breaking_changes=[breaking_change],
node=self,
)
)
def same_contract(self, old, adapter_type=None) -> bool:
# If the contract wasn't previously enforced:
if old.contract.enforced is False and self.contract.enforced is False:
@@ -601,10 +646,6 @@ class ModelNode(ModelResource, CompiledNode):
# Breaking change: the contract was previously enforced, and it no longer is
contract_enforced_disabled = True
# TODO: this avoid the circular imports but isn't ideal
from dbt.adapters.base import ConstraintSupport
from dbt.adapters.factory import get_adapter_constraint_support
constraint_support = get_adapter_constraint_support(adapter_type)
column_constraints_exist = False

View File

@@ -25,8 +25,15 @@ class GraphQueue:
the same time, as there is an unlocked race!
"""
def __init__(self, graph: nx.DiGraph, manifest: Manifest, selected: Set[UniqueId]) -> None:
self.graph = graph
def __init__(
self,
graph: nx.DiGraph,
manifest: Manifest,
selected: Set[UniqueId],
preserve_edges: bool = True,
) -> None:
# 'create_empty_copy' returns a copy of the graph G with all of the edges removed, and leaves nodes intact.
self.graph = graph if preserve_edges else nx.classes.function.create_empty_copy(graph)
self.manifest = manifest
self._selected = selected
# store the queue as a priority queue.

View File

@@ -319,7 +319,7 @@ class NodeSelector(MethodManager):
return filtered_nodes
def get_graph_queue(self, spec: SelectionSpec) -> GraphQueue:
def get_graph_queue(self, spec: SelectionSpec, preserve_edges: bool = True) -> GraphQueue:
"""Returns a queue over nodes in the graph that tracks progress of
dependecies.
"""
@@ -330,7 +330,7 @@ class NodeSelector(MethodManager):
# Construct a new graph using the selected_nodes
new_graph = self.full_graph.get_subset_graph(selected_nodes)
# should we give a way here for consumers to mutate the graph?
return GraphQueue(new_graph.graph, self.manifest, selected_nodes)
return GraphQueue(new_graph.graph, self.manifest, selected_nodes, preserve_edges)
class ResourceTypeSelector(NodeSelector):

View File

@@ -719,7 +719,9 @@ class StateSelectorMethod(SelectorMethod):
) -> Callable[[Optional[SelectorTarget], SelectorTarget], bool]:
# get a function that compares two selector target based on compare method provided
def check_modified_contract(old: Optional[SelectorTarget], new: SelectorTarget) -> bool:
if hasattr(new, compare_method):
if new is None and hasattr(old, compare_method + "_removed"):
return getattr(old, compare_method + "_removed")()
elif hasattr(new, compare_method):
# when old body does not exist or old and new are not the same
return not old or not getattr(new, compare_method)(old, adapter_type) # type: ignore
else:
@@ -785,6 +787,22 @@ class StateSelectorMethod(SelectorMethod):
if checker(previous_node, node, **keyword_args): # type: ignore
yield unique_id
# checkers that can handle removed nodes
if checker.__name__ in ["check_modified_contract"]:
# ignore included_nodes, since those cannot contain removed nodes
for previous_unique_id, previous_node in manifest.nodes.items():
# detect removed (deleted, renamed, or disabled) nodes
removed_node = None
if previous_unique_id in self.manifest.disabled.keys():
removed_node = self.manifest.disabled[previous_unique_id][0]
elif previous_unique_id not in self.manifest.nodes.keys():
removed_node = previous_node
if removed_node:
# do not yield -- removed nodes should never be selected for downstream execution
# as they are not part of the current project's manifest.nodes
checker(removed_node, None, **keyword_args) # type: ignore
class ResultSelectorMethod(SelectorMethod):
def search(self, included_nodes: Set[UniqueId], selector: str) -> Iterator[UniqueId]:

View File

@@ -570,25 +570,21 @@ class ManifestLoader:
def check_for_model_deprecations(self):
for node in self.manifest.nodes.values():
if isinstance(node, ModelNode):
if (
node.deprecation_date
and node.deprecation_date < datetime.datetime.now().astimezone()
):
warn_or_error(
DeprecatedModel(
model_name=node.name,
model_version=version_to_str(node.version),
deprecation_date=node.deprecation_date.isoformat(),
)
if isinstance(node, ModelNode) and node.is_past_deprecation_date:
warn_or_error(
DeprecatedModel(
model_name=node.name,
model_version=version_to_str(node.version),
deprecation_date=node.deprecation_date.isoformat(),
)
)
resolved_refs = self.manifest.resolve_refs(node, self.root_project.project_name)
resolved_model_refs = [r for r in resolved_refs if isinstance(r, ModelNode)]
node.depends_on
for resolved_ref in resolved_model_refs:
if resolved_ref.deprecation_date:
if resolved_ref.deprecation_date < datetime.datetime.now().astimezone():
if resolved_ref.is_past_deprecation_date:
event_cls = DeprecatedReference
else:
event_cls = UpcomingReferenceDeprecation
@@ -808,8 +804,12 @@ class ManifestLoader:
plugin_model_nodes = pm.get_nodes().models
for node_arg in plugin_model_nodes.values():
node = ModelNode.from_args(node_arg)
# node may already exist from package or running project - in which case we should avoid clobbering it with an external node
if node.unique_id not in self.manifest.nodes:
# node may already exist from package or running project (even if it is disabled),
# in which case we should avoid clobbering it with an external node
if (
node.unique_id not in self.manifest.nodes
and node.unique_id not in self.manifest.disabled
):
self.manifest.add_node_nofile(node)
manifest_nodes_modified = True

View File

@@ -10,7 +10,7 @@ from dbt.graph import ResourceTypeSelector
from dbt.node_types import REFABLE_NODE_TYPES
from dbt.task.base import BaseRunner, resource_types_from_args
from dbt.task.run import _validate_materialization_relations_dict
from dbt.task.runnable import GraphRunnableTask
from dbt.task.runnable import GraphRunnableMode, GraphRunnableTask
from dbt_common.dataclass_schema import dbtClassMixin
from dbt_common.exceptions import CompilationError, DbtInternalError
@@ -94,6 +94,9 @@ class CloneTask(GraphRunnableTask):
def raise_on_first_error(self):
return False
def get_run_mode(self) -> GraphRunnableMode:
return GraphRunnableMode.Independent
def _get_deferred_manifest(self) -> Optional[Manifest]:
# Unlike other commands, 'clone' always requires a state manifest
# Load previous state, regardless of whether --defer flag has been set

File diff suppressed because one or more lines are too long

View File

@@ -16,11 +16,12 @@ class ServeTask(ConfiguredTask):
shutil.copyfile(DOCS_INDEX_FILE_PATH, "index.html")
port = self.args.port
host = self.args.host
if self.args.browser:
webbrowser.open_new_tab(f"http://localhost:{port}")
with socketserver.TCPServer(("127.0.0.1", port), SimpleHTTPRequestHandler) as httpd:
with socketserver.TCPServer((host, port), SimpleHTTPRequestHandler) as httpd:
click.echo(f"Serving docs at {port}")
click.echo(f"To access from your browser, navigate to: http://localhost:{port}")
click.echo("\n\n")

View File

@@ -48,6 +48,7 @@ from dbt.graph import (
from dbt.parser.manifest import write_manifest
from dbt.task.base import BaseRunner, ConfiguredTask
from dbt_common.context import _INVOCATION_CONTEXT_VAR, get_invocation_context
from dbt_common.dataclass_schema import StrEnum
from dbt_common.events.contextvars import log_contextvars, task_contextvars
from dbt_common.events.functions import fire_event, warn_or_error
from dbt_common.events.types import Formatting
@@ -58,6 +59,11 @@ from .printer import print_run_end_messages, print_run_result_error
RESULT_FILE_NAME = "run_results.json"
class GraphRunnableMode(StrEnum):
Topological = "topological"
Independent = "independent"
class GraphRunnableTask(ConfiguredTask):
MARK_DEPENDENT_ERRORS_STATUSES = [NodeStatus.Error]
@@ -135,7 +141,15 @@ class GraphRunnableTask(ConfiguredTask):
selector = self.get_node_selector()
# Following uses self.selection_arg and self.exclusion_arg
spec = self.get_selection_spec()
return selector.get_graph_queue(spec)
preserve_edges = True
if self.get_run_mode() == GraphRunnableMode.Independent:
preserve_edges = False
return selector.get_graph_queue(spec, preserve_edges)
def get_run_mode(self) -> GraphRunnableMode:
return GraphRunnableMode.Topological
def _runtime_initialize(self):
self.compile_manifest()

View File

@@ -126,6 +126,8 @@ class TestRunner(CompileRunner):
def execute_data_test(self, data_test: TestNode, manifest: Manifest) -> TestResultData:
context = generate_runtime_model_context(data_test, self.config, manifest)
hook_ctx = self.adapter.pre_model_hook(context)
materialization_macro = manifest.find_materialization_macro_by_name(
self.config.project_name, data_test.get_materialization(), self.adapter.type()
)
@@ -140,10 +142,13 @@ class TestRunner(CompileRunner):
"Invalid materialization context generated, missing config: {}".format(context)
)
# generate materialization macro
macro_func = MacroGenerator(materialization_macro, context)
# execute materialization macro
macro_func()
try:
# generate materialization macro
macro_func = MacroGenerator(materialization_macro, context)
# execute materialization macro
macro_func()
finally:
self.adapter.post_model_hook(context, hook_ctx)
# load results from context
# could eventually be returned directly by materialization
result = context["load_result"]("main")
@@ -198,6 +203,8 @@ class TestRunner(CompileRunner):
# materialization, not compile the node.compiled_code
context = generate_runtime_model_context(unit_test_node, self.config, unit_test_manifest)
hook_ctx = self.adapter.pre_model_hook(context)
materialization_macro = unit_test_manifest.find_materialization_macro_by_name(
self.config.project_name, unit_test_node.get_materialization(), self.adapter.type()
)
@@ -213,16 +220,18 @@ class TestRunner(CompileRunner):
"Invalid materialization context generated, missing config: {}".format(context)
)
# generate materialization macro
macro_func = MacroGenerator(materialization_macro, context)
# execute materialization macro
try:
# generate materialization macro
macro_func = MacroGenerator(materialization_macro, context)
# execute materialization macro
macro_func()
except DbtBaseException as e:
raise DbtRuntimeError(
f"An error occurred during execution of unit test '{unit_test_def.name}'. "
f"There may be an error in the unit test definition: check the data types.\n {e}"
)
finally:
self.adapter.post_model_hook(context, hook_ctx)
# load results from context
# could eventually be returned directly by materialization
@@ -338,16 +347,17 @@ class TestRunner(CompileRunner):
def _get_daff_diff(
self, expected: "agate.Table", actual: "agate.Table", ordered: bool = False
) -> daff.TableDiff:
expected_daff_table = daff.PythonTableView(list_rows_from_table(expected))
actual_daff_table = daff.PythonTableView(list_rows_from_table(actual))
alignment = daff.Coopy.compareTables(expected_daff_table, actual_daff_table).align()
result = daff.PythonTableView([])
# Sort expected and actual inputs prior to creating daff diff to ensure order insensitivity
# https://github.com/paulfitz/daff/issues/200
expected_daff_table = daff.PythonTableView(list_rows_from_table(expected, sort=True))
actual_daff_table = daff.PythonTableView(list_rows_from_table(actual, sort=True))
flags = daff.CompareFlags()
flags.ordered = ordered
alignment = daff.Coopy.compareTables(expected_daff_table, actual_daff_table, flags).align()
result = daff.PythonTableView([])
diff = daff.TableDiff(alignment, flags)
diff.hilite(result)
return diff
@@ -408,10 +418,25 @@ def json_rows_from_table(table: "agate.Table") -> List[Dict[str, Any]]:
# This was originally in agate_helper, but that was moved out into dbt_common
def list_rows_from_table(table: "agate.Table") -> List[Any]:
"Convert a table to a list of lists, where the first element represents the header"
rows = [[col.name for col in table.columns]]
def list_rows_from_table(table: "agate.Table", sort: bool = False) -> List[Any]:
"""
Convert given table to a list of lists, where the first element represents the header
By default, sort is False and no sort order is applied to the non-header rows of the given table.
If sort is True, sort the non-header rows hierarchically, treating None values as lower in order.
Examples:
* [['a','b','c'],[4,5,6],[1,2,3]] -> [['a','b','c'],[1,2,3],[4,5,6]]
* [['a','b','c'],[4,5,6],[1,null,3]] -> [['a','b','c'],[1,null,3],[4,5,6]]
* [['a','b','c'],[4,5,6],[null,2,3]] -> [['a','b','c'],[4,5,6],[null,2,3]]
"""
header = [col.name for col in table.columns]
rows = []
for row in table.rows:
rows.append(list(row.values()))
return rows
if sort:
rows = sorted(rows, key=lambda x: [(elem is None, elem) for elem in x])
return [header] + rows

View File

@@ -71,7 +71,7 @@ setup(
"minimal-snowplow-tracker>=0.0.2,<0.1",
"dbt-semantic-interfaces>=0.5.1,<0.6",
# Minor versions for these are expected to be backwards-compatible
"dbt-common>=1.1.0,<2.0",
"dbt-common>=1.2.0,<2.0",
"dbt-adapters>=1.1.1,<2.0",
# ----
# Expect compatibility with all new versions of these packages, so lower bounds only.

View File

@@ -4,7 +4,8 @@ from pathlib import Path
import pytest
import yaml
from dbt.exceptions import ProjectContractError
from dbt.cli.main import dbtRunner
from dbt.exceptions import DbtProjectError, ProjectContractError
from dbt.tests.util import run_dbt, update_config_file, write_config_file
simple_model_sql = """
@@ -118,3 +119,51 @@ class TestProjectDbtCloudConfigString:
with pytest.raises(ProjectContractError) as excinfo:
run_dbt()
assert expected_err in str(excinfo.value)
class TestVersionSpecifierChecksComeBeforeYamlValidation:
def test_version_specifier_checks_before_yaml_validation(self, project) -> None:
runner = dbtRunner()
# if no version specifier error, we should get a yaml validation error
config_update = {"this-is-not-a-valid-key": "my-value-for-invalid-key"}
update_config_file(config_update, "dbt_project.yml")
result = runner.invoke(["parse"])
assert result.exception is not None
assert isinstance(result.exception, ProjectContractError)
assert "Additional properties are not allowed" in str(result.exception)
# add bad version specifier, and assert we get the error for that
update_config_file({"require-dbt-version": [">0.0.0", "<=0.0.1"]}, "dbt_project.yml")
result = runner.invoke(["parse"])
assert result.exception is not None
assert isinstance(result.exception, DbtProjectError)
assert "This version of dbt is not supported"
class TestArchiveNotAllowed:
"""At one point in time we supported an 'archive' key in projects, but no longer"""
def test_archive_not_allowed(self, project):
runner = dbtRunner()
config_update = {
"archive": {
"source_schema": "a",
"target_schema": "b",
"tables": [
{
"source_table": "seed",
"target_table": "archive_actual",
"updated_at": "updated_at",
"unique_key": """id || '-' || first_name""",
},
],
}
}
update_config_file(config_update, "dbt_project.yml")
result = runner.invoke(["parse"])
assert result.exception is not None
assert isinstance(result.exception, ProjectContractError)
assert "Additional properties are not allowed" in str(result.exception)

View File

@@ -108,6 +108,25 @@ models:
data_type: text
"""
disabled_contract_schema_yml = """
version: 2
models:
- name: table_model
config:
contract:
enforced: True
enabled: False
columns:
- name: id
data_type: integer
data_tests:
- unique:
severity: error
- not_null
- name: name
data_type: text
"""
modified_contract_schema_yml = """
version: 2
models:
@@ -126,7 +145,7 @@ models:
data_type: text
"""
disabled_contract_schema_yml = """
unenforced_contract_schema_yml = """
version: 2
models:
- name: table_model
@@ -144,6 +163,25 @@ models:
data_type: text
"""
disabled_unenforced_contract_schema_yml = """
version: 2
models:
- name: table_model
config:
contract:
enforced: False
enabled: False
columns:
- name: id
data_type: integer
data_tests:
- unique:
severity: error
- not_null
- name: name
data_type: text
"""
versioned_no_contract_schema_yml = """
version: 2
models:
@@ -182,6 +220,27 @@ models:
data_type: text
"""
disabled_versioned_contract_schema_yml = """
version: 2
models:
- name: table_model
config:
contract:
enforced: True
enabled: False
versions:
- v: 1
columns:
- name: id
data_type: integer
data_tests:
- unique:
severity: error
- not_null
- name: name
data_type: text
"""
versioned_modified_contract_schema_yml = """
version: 2
models:
@@ -202,7 +261,28 @@ models:
data_type: text
"""
versioned_disabled_contract_schema_yml = """
disabled_versioned_unenforced_contract_schema_yml = """
version: 2
models:
- name: table_model
config:
contract:
enforced: False
enabled: False
versions:
- v: 1
columns:
- name: id
data_type: integer
data_tests:
- unique:
severity: error
- not_null
- name: name
data_type: text
"""
versioned_unenforced_contract_schema_yml = """
version: 2
models:
- name: table_model

View File

@@ -8,6 +8,7 @@ import pytest
from dbt.exceptions import CompilationError, ContractBreakingChangeError
from dbt.tests.util import (
get_manifest,
rm_file,
run_dbt,
run_dbt_and_capture,
update_config_file,
@@ -17,6 +18,9 @@ from tests.functional.defer_state.fixtures import (
constraint_schema_yml,
contract_schema_yml,
disabled_contract_schema_yml,
disabled_unenforced_contract_schema_yml,
disabled_versioned_contract_schema_yml,
disabled_versioned_unenforced_contract_schema_yml,
ephemeral_model_sql,
exposures_yml,
infinite_macros_sql,
@@ -33,10 +37,11 @@ from tests.functional.defer_state.fixtures import (
table_model_now_incremental_sql,
table_model_now_view_sql,
table_model_sql,
unenforced_contract_schema_yml,
versioned_contract_schema_yml,
versioned_disabled_contract_schema_yml,
versioned_modified_contract_schema_yml,
versioned_no_contract_schema_yml,
versioned_unenforced_contract_schema_yml,
view_model_now_table_sql,
view_model_sql,
)
@@ -507,7 +512,7 @@ class TestChangedContractUnversioned(BaseModifiedState):
MODEL_UNIQUE_ID = "model.test.table_model"
CONTRACT_SCHEMA_YML = contract_schema_yml
MODIFIED_SCHEMA_YML = modified_contract_schema_yml
DISABLED_SCHEMA_YML = disabled_contract_schema_yml
UNENFORCED_SCHEMA_YML = unenforced_contract_schema_yml
NO_CONTRACT_SCHEMA_YML = no_contract_schema_yml
def test_changed_contract(self, project):
@@ -570,8 +575,8 @@ class TestChangedContractUnversioned(BaseModifiedState):
expected_warning = "While comparing to previous project state, dbt detected a breaking change to an unversioned model"
expected_change = "Contract enforcement was removed"
# Now disable the contract. Should throw a warning - force warning into an error.
write_file(self.DISABLED_SCHEMA_YML, "models", "schema.yml")
# Now unenforce the contract. Should throw a warning - force warning into an error.
write_file(self.UNENFORCED_SCHEMA_YML, "models", "schema.yml")
with pytest.raises(CompilationError):
_, logs = run_dbt_and_capture(
[
@@ -591,7 +596,7 @@ class TestChangedContractVersioned(BaseModifiedState):
MODEL_UNIQUE_ID = "model.test.table_model.v1"
CONTRACT_SCHEMA_YML = versioned_contract_schema_yml
MODIFIED_SCHEMA_YML = versioned_modified_contract_schema_yml
DISABLED_SCHEMA_YML = versioned_disabled_contract_schema_yml
UNENFORCED_SCHEMA_YML = versioned_unenforced_contract_schema_yml
NO_CONTRACT_SCHEMA_YML = versioned_no_contract_schema_yml
def test_changed_contract_versioned(self, project):
@@ -643,12 +648,138 @@ class TestChangedContractVersioned(BaseModifiedState):
with pytest.raises(ContractBreakingChangeError):
results = run_dbt(["run", "--models", "state:modified.contract", "--state", "./state"])
# Now disable the contract. Should raise an error.
write_file(self.DISABLED_SCHEMA_YML, "models", "schema.yml")
# Now unenforce the contract. Should raise an error.
write_file(self.UNENFORCED_SCHEMA_YML, "models", "schema.yml")
with pytest.raises(ContractBreakingChangeError):
results = run_dbt(["run", "--models", "state:modified.contract", "--state", "./state"])
class TestDeleteUnversionedContractedModel(BaseModifiedState):
MODEL_UNIQUE_ID = "model.test.table_model"
CONTRACT_SCHEMA_YML = contract_schema_yml
def test_delete_unversioned_contracted_model(self, project):
# ensure table_model is contracted
write_file(self.CONTRACT_SCHEMA_YML, "models", "schema.yml")
self.run_and_save_state()
# delete versioned contracted model
rm_file(project.project_root, "models", "table_model.sql")
# since the models are unversioned, they raise a warning but not an error
_, logs = run_dbt_and_capture(
["run", "--models", "state:modified.contract", "--state", "./state"]
)
expected_warning = "While comparing to previous project state, dbt detected a breaking change to an unversioned model"
expected_change = "Contracted model 'model.test.table_model' was deleted or renamed"
assert expected_warning in logs
assert expected_change in logs
class TestDeleteVersionedContractedModel(BaseModifiedState):
MODEL_UNIQUE_ID = "model.test.table_model.v1"
CONTRACT_SCHEMA_YML = versioned_contract_schema_yml
def test_delete_versioned_contracted_model(self, project):
# ensure table_model is versioned + contracted
write_file(self.CONTRACT_SCHEMA_YML, "models", "schema.yml")
self.run_and_save_state()
# delete versioned contracted model
rm_file(project.project_root, "models", "table_model.sql")
# since the models are versioned, they raise an error
with pytest.raises(ContractBreakingChangeError) as e:
run_dbt(["run", "--models", "state:modified.contract", "--state", "./state"])
assert "Contracted model 'model.test.table_model.v1' was deleted or renamed." in str(
e.value
)
class TestDisableUnversionedContractedModel(BaseModifiedState):
MODEL_UNIQUE_ID = "model.test.table_model"
CONTRACT_SCHEMA_YML = contract_schema_yml
DISABLED_CONTRACT_SCHEMA_YML = disabled_contract_schema_yml
def test_disable_unversioned_contracted_model(self, project):
# ensure table_model is contracted and enabled
write_file(self.CONTRACT_SCHEMA_YML, "models", "schema.yml")
self.run_and_save_state()
# disable unversioned + contracted model
write_file(self.DISABLED_CONTRACT_SCHEMA_YML, "models", "schema.yml")
# since the models are unversioned, they raise a warning but not an error
_, logs = run_dbt_and_capture(
["run", "--models", "state:modified.contract", "--state", "./state"]
)
expected_warning = "While comparing to previous project state, dbt detected a breaking change to an unversioned model"
expected_change = "Contracted model 'model.test.table_model' was disabled"
assert expected_warning in logs
assert expected_change in logs
class TestDisableVersionedContractedModel(BaseModifiedState):
MODEL_UNIQUE_ID = "model.test.table_model.v1"
CONTRACT_SCHEMA_YML = versioned_contract_schema_yml
DISABLED_CONTRACT_SCHEMA_YML = disabled_versioned_contract_schema_yml
def test_disable_versioned_contracted_model(self, project):
# ensure table_model is versioned + contracted
write_file(self.CONTRACT_SCHEMA_YML, "models", "schema.yml")
self.run_and_save_state()
# disable versioned + contracted model
write_file(self.DISABLED_CONTRACT_SCHEMA_YML, "models", "schema.yml")
# since the models are versioned, they raise an error
with pytest.raises(ContractBreakingChangeError) as e:
run_dbt(["run", "--models", "state:modified.contract", "--state", "./state"])
assert "Contracted model 'model.test.table_model.v1' was disabled." in str(e.value)
class TestDisableUnversionedUncontractedModel(BaseModifiedState):
MODEL_UNIQUE_ID = "model.test.table_model"
NO_CONTRACT_SCHEMA_YML = unenforced_contract_schema_yml
DISABLED_NO_CONTRACT_SCHEMA_YML = disabled_unenforced_contract_schema_yml
def test_delete_versioned_contracted_model(self, project):
# ensure table_model is not contracted
write_file(self.NO_CONTRACT_SCHEMA_YML, "models", "schema.yml")
self.run_and_save_state()
# disable uncontracted model
write_file(self.DISABLED_NO_CONTRACT_SCHEMA_YML, "models", "schema.yml")
# since the models are unversioned, no warning or error is raised
_, logs = run_dbt_and_capture(
["run", "--models", "state:modified.contract", "--state", "./state"]
)
assert "breaking change" not in logs.lower()
class TestDisableVersionedUncontractedModel(BaseModifiedState):
MODEL_UNIQUE_ID = "model.test.table_model.v1"
NO_CONTRACT_SCHEMA_YML = versioned_unenforced_contract_schema_yml
DISABLED_NO_CONTRACT_SCHEMA_YML = disabled_versioned_unenforced_contract_schema_yml
def test_delete_versioned_contracted_model(self, project):
# ensure table_model is not contracted
write_file(self.NO_CONTRACT_SCHEMA_YML, "models", "schema.yml")
self.run_and_save_state()
# disable uncontracted model
write_file(self.DISABLED_NO_CONTRACT_SCHEMA_YML, "models", "schema.yml")
# since the models are unversioned, no warning or error is raised
run_dbt_and_capture(["run", "--models", "state:modified.contract", "--state", "./state"])
class TestChangedConstraintUnversioned(BaseModifiedState):
def test_changed_constraint(self, project):
self.run_and_save_state()

View File

@@ -760,6 +760,14 @@ select 1 as notfun
"""
model_two_disabled_sql = """
{{ config(
enabled=false
) }}
select 1 as notfun
"""
generic_test_schema_yml = """
models:

View File

@@ -50,6 +50,7 @@ from tests.functional.partial_parsing.fixtures import (
model_three_disabled_sql,
model_three_modified_sql,
model_three_sql,
model_two_disabled_sql,
model_two_sql,
models_schema1_yml,
models_schema2_yml,
@@ -695,6 +696,15 @@ class TestExternalModels:
schema="test_schema",
)
@pytest.fixture(scope="class")
def external_model_node_merge(self):
return ModelNodeArgs(
name="model_two",
package_name="test",
identifier="test_identifier",
schema="test_schema",
)
@pytest.fixture(scope="class")
def models(self):
return {"model_one.sql": model_one_sql}
@@ -708,6 +718,7 @@ class TestExternalModels:
external_model_node_versioned,
external_model_node_depends_on,
external_model_node_depends_on_parent,
external_model_node_merge,
):
# initial plugin - one external model
external_nodes = PluginNodes()
@@ -724,12 +735,30 @@ class TestExternalModels:
assert len(manifest.external_node_unique_ids) == 1
assert manifest.external_node_unique_ids == ["model.external.external_model"]
# add a model file
# add a model file - test.model_two
write_file(model_two_sql, project.project_root, "models", "model_two.sql")
manifest = run_dbt(["--partial-parse", "parse"])
assert len(manifest.nodes) == 3
# add an external model
# add an external model that is already in project - test.model_two
# project model should be preferred to external model
external_nodes.add_model(external_model_node_merge)
manifest = run_dbt(["--partial-parse", "parse"])
assert len(manifest.nodes) == 3
assert len(manifest.external_node_unique_ids) == 1
# disable test.model_two in project
# project models should still be preferred to external model
write_file(model_two_disabled_sql, project.project_root, "models", "model_two.sql")
manifest = run_dbt(["--partial-parse", "parse"])
assert len(manifest.nodes) == 2
assert len(manifest.disabled) == 1
assert len(manifest.external_node_unique_ids) == 1
# re-enable model_2.sql
write_file(model_two_sql, project.project_root, "models", "model_two.sql")
# add a new external model
external_nodes.add_model(external_model_node_versioned)
manifest = run_dbt(["--partial-parse", "parse"])
assert len(manifest.nodes) == 4

View File

@@ -116,6 +116,23 @@ unit_tests:
tags: test_this
"""
test_my_model_pass_yml = """
unit_tests:
- name: test_my_model
model: my_model
given:
- input: ref('my_model_a')
rows:
- {id: 1, a: 1}
- input: ref('my_model_b')
rows:
- {id: 1, b: 2}
- {id: 2, b: 2}
expect:
rows:
- {c: 3}
"""
test_my_model_simple_fixture_yml = """
unit_tests:

View File

@@ -0,0 +1,49 @@
from unittest import mock
import pytest
from dbt.tests.util import run_dbt
from dbt_common.exceptions import CompilationError
from tests.functional.unit_testing.fixtures import (
my_model_a_sql,
my_model_b_sql,
my_model_sql,
test_my_model_pass_yml,
)
class BaseUnitTestAdapterHook:
@pytest.fixture(scope="class")
def models(self):
return {
"my_model.sql": my_model_sql,
"my_model_a.sql": my_model_a_sql,
"my_model_b.sql": my_model_b_sql,
"test_my_model.yml": test_my_model_pass_yml,
}
class TestUnitTestAdapterHookPasses(BaseUnitTestAdapterHook):
def test_unit_test_runs_adapter_pre_hook(self, project):
results = run_dbt(["run"])
assert len(results) == 3
mock_pre_model_hook = mock.Mock()
with mock.patch.object(type(project.adapter), "pre_model_hook", mock_pre_model_hook):
results = run_dbt(["test", "--select", "test_name:test_my_model"], expect_pass=True)
assert len(results) == 1
mock_pre_model_hook.assert_called_once()
class TestUnitTestAdapterHookFails(BaseUnitTestAdapterHook):
def test_unit_test_runs_adapter_pre_hook_fails(self, project):
results = run_dbt(["run"])
assert len(results) == 3
mock_pre_model_hook = mock.Mock()
mock_pre_model_hook.side_effect = CompilationError("exception from adapter.pre_model_hook")
with mock.patch.object(type(project.adapter), "pre_model_hook", mock_pre_model_hook):
run_dbt(["test", "--select", "test_name:test_my_model"], expect_pass=False)
mock_pre_model_hook.assert_called_once()

View File

@@ -0,0 +1,113 @@
import pytest
from dbt.tests.util import run_dbt
my_input_model = """
SELECT 1 as id, 'some string' as status
"""
my_model = """
SELECT * FROM {{ ref("my_input_model") }}
"""
test_my_model_order_insensitive = """
unit_tests:
- name: unordered_no_nulls
model: my_model
given:
- input: ref("my_input_model")
rows:
- {"id": 1, "status": 'B'}
- {"id": 2, "status": 'B'}
- {"id": 3, "status": 'A'}
expect:
rows:
- {"id": 3, "status": 'A'}
- {"id": 2, "status": 'B'}
- {"id": 1, "status": 'B'}
- name: unordered_with_nulls
model: my_model
given:
- input: ref("my_input_model")
rows:
- {"id": , "status": 'B'}
- {"id": , "status": 'B'}
- {"id": 3, "status": 'A'}
expect:
rows:
- {"id": 3, "status": 'A'}
- {"id": , "status": 'B'}
- {"id": , "status": 'B'}
- name: unordered_with_nulls_2
model: my_model
given:
- input: ref("my_input_model")
rows:
- {"id": 3, "status": 'A'}
- {"id": , "status": 'B'}
- {"id": , "status": 'B'}
expect:
rows:
- {"id": , "status": 'B'}
- {"id": , "status": 'B'}
- {"id": 3, "status": 'A'}
- name: unordered_with_nulls_mixed_columns
model: my_model
given:
- input: ref("my_input_model")
rows:
- {"id": 3, "status": 'A'}
- {"id": , "status": 'B'}
- {"id": 1, "status": }
expect:
rows:
- {"id": 1, "status": }
- {"id": , "status": 'B'}
- {"id": 3, "status": 'A'}
- name: unordered_with_null
model: my_model
given:
- input: ref("my_input_model")
rows:
- {"id": 3, "status": 'A'}
- {"id": , "status": 'B'}
expect:
rows:
- {"id": , "status": 'B'}
- {"id": 3, "status": 'A'}
- name: ordered_with_nulls
model: my_model
given:
- input: ref("my_input_model")
rows:
- {"id": 3, "status": 'A'}
- {"id": , "status": 'B'}
- {"id": , "status": 'B'}
expect:
rows:
- {"id": 3, "status": 'A'}
- {"id": , "status": 'B'}
- {"id": , "status": 'B'}
"""
class TestUnitTestingDiffIsOrderAgnostic:
@pytest.fixture(scope="class")
def models(self):
return {
"my_input_model.sql": my_input_model,
"my_model.sql": my_model,
"test_my_model.yml": test_my_model_order_insensitive,
}
def test_unit_testing_diff_is_order_insensitive(self, project):
run_dbt(["run"])
# Select by model name
results = run_dbt(["test", "--select", "my_model"], expect_pass=True)
assert len(results) == 6

View File

@@ -2,6 +2,7 @@ import json
import os
import unittest
from copy import deepcopy
from typing import Any, Dict
from unittest import mock
import pytest
@@ -10,7 +11,7 @@ import dbt.config
import dbt.exceptions
from dbt.adapters.contracts.connection import DEFAULT_QUERY_COMMENT, QueryComment
from dbt.adapters.factory import load_plugin
from dbt.config.project import Project
from dbt.config.project import Project, _get_required_version
from dbt.constants import DEPENDENCIES_FILE_NAME
from dbt.contracts.project import GitPackage, LocalPackage, PackageConfig
from dbt.flags import set_from_args
@@ -42,7 +43,7 @@ class TestProjectMethods:
def test__str__(self, project: Project):
assert (
str(project)
== "{'name': 'test_project', 'version': 1.0, 'project-root': 'doesnt/actually/exist', 'profile': 'test_profile', 'model-paths': ['models'], 'macro-paths': ['macros'], 'seed-paths': ['seeds'], 'test-paths': ['tests'], 'analysis-paths': ['analyses'], 'docs-paths': ['docs'], 'asset-paths': ['assets'], 'target-path': 'target', 'snapshot-paths': ['snapshots'], 'clean-targets': ['target'], 'log-path': 'path/to/project/logs', 'quoting': {'database': True, 'schema': True, 'identifier': True}, 'models': {}, 'on-run-start': [], 'on-run-end': [], 'dispatch': [{'macro_namespace': 'dbt_utils', 'search_order': ['test_project', 'dbt_utils']}], 'seeds': {}, 'snapshots': {}, 'sources': {}, 'data_tests': {}, 'unit_tests': {}, 'metrics': {}, 'semantic-models': {}, 'saved-queries': {}, 'exposures': {}, 'vars': {}, 'require-dbt-version': ['=0.0.0'], 'restrict-access': False, 'dbt-cloud': {}, 'query-comment': {'comment': \"\\n{%- set comment_dict = {} -%}\\n{%- do comment_dict.update(\\n app='dbt',\\n dbt_version=dbt_version,\\n profile_name=target.get('profile_name'),\\n target_name=target.get('target_name'),\\n) -%}\\n{%- if node is not none -%}\\n {%- do comment_dict.update(\\n node_id=node.unique_id,\\n ) -%}\\n{% else %}\\n {# in the node context, the connection name is the node_id #}\\n {%- do comment_dict.update(connection_name=connection_name) -%}\\n{%- endif -%}\\n{{ return(tojson(comment_dict)) }}\\n\", 'append': False, 'job-label': False}, 'packages': []}"
== "{'name': 'test_project', 'version': 1.0, 'project-root': 'doesnt/actually/exist', 'profile': 'test_profile', 'model-paths': ['models'], 'macro-paths': ['macros'], 'seed-paths': ['seeds'], 'test-paths': ['tests'], 'analysis-paths': ['analyses'], 'docs-paths': ['docs'], 'asset-paths': ['assets'], 'target-path': 'target', 'snapshot-paths': ['snapshots'], 'clean-targets': ['target'], 'log-path': 'path/to/project/logs', 'quoting': {}, 'models': {}, 'on-run-start': [], 'on-run-end': [], 'dispatch': [{'macro_namespace': 'dbt_utils', 'search_order': ['test_project', 'dbt_utils']}], 'seeds': {}, 'snapshots': {}, 'sources': {}, 'data_tests': {}, 'unit_tests': {}, 'metrics': {}, 'semantic-models': {}, 'saved-queries': {}, 'exposures': {}, 'vars': {}, 'require-dbt-version': ['=0.0.0'], 'restrict-access': False, 'dbt-cloud': {}, 'query-comment': {'comment': \"\\n{%- set comment_dict = {} -%}\\n{%- do comment_dict.update(\\n app='dbt',\\n dbt_version=dbt_version,\\n profile_name=target.get('profile_name'),\\n target_name=target.get('target_name'),\\n) -%}\\n{%- if node is not none -%}\\n {%- do comment_dict.update(\\n node_id=node.unique_id,\\n ) -%}\\n{% else %}\\n {# in the node context, the connection name is the node_id #}\\n {%- do comment_dict.update(connection_name=connection_name) -%}\\n{%- endif -%}\\n{{ return(tojson(comment_dict)) }}\\n\", 'append': False, 'job-label': False}, 'packages': []}"
)
def test_get_selector(self, project: Project):
@@ -534,3 +535,53 @@ class TestMultipleProjectFlags(BaseConfigTest):
def test_setting_multiple_flags(self):
with pytest.raises(dbt.exceptions.DbtProjectError):
set_from_args(self.args, None)
class TestGetRequiredVersion:
@pytest.fixture
def project_dict(self) -> Dict[str, Any]:
return {
"name": "test_project",
"require-dbt-version": ">0.0.0",
}
def test_supported_version(self, project_dict: Dict[str, Any]) -> None:
specifiers = _get_required_version(project_dict=project_dict, verify_version=True)
assert set(x.to_version_string() for x in specifiers) == {">0.0.0"}
def test_unsupported_version(self, project_dict: Dict[str, Any]) -> None:
project_dict["require-dbt-version"] = ">99999.0.0"
with pytest.raises(
dbt.exceptions.DbtProjectError, match="This version of dbt is not supported"
):
_get_required_version(project_dict=project_dict, verify_version=True)
def test_unsupported_version_no_check(self, project_dict: Dict[str, Any]) -> None:
project_dict["require-dbt-version"] = ">99999.0.0"
specifiers = _get_required_version(project_dict=project_dict, verify_version=False)
assert set(x.to_version_string() for x in specifiers) == {">99999.0.0"}
def test_supported_version_range(self, project_dict: Dict[str, Any]) -> None:
project_dict["require-dbt-version"] = [">0.0.0", "<=99999.0.0"]
specifiers = _get_required_version(project_dict=project_dict, verify_version=True)
assert set(x.to_version_string() for x in specifiers) == {">0.0.0", "<=99999.0.0"}
def test_unsupported_version_range(self, project_dict: Dict[str, Any]) -> None:
project_dict["require-dbt-version"] = [">0.0.0", "<=0.0.1"]
with pytest.raises(
dbt.exceptions.DbtProjectError, match="This version of dbt is not supported"
):
_get_required_version(project_dict=project_dict, verify_version=True)
def test_unsupported_version_range_no_check(self, project_dict: Dict[str, Any]) -> None:
project_dict["require-dbt-version"] = [">0.0.0", "<=0.0.1"]
specifiers = _get_required_version(project_dict=project_dict, verify_version=False)
assert set(x.to_version_string() for x in specifiers) == {">0.0.0", "<=0.0.1"}
def test_impossible_version_range(self, project_dict: Dict[str, Any]) -> None:
project_dict["require-dbt-version"] = [">99999.0.0", "<=0.0.1"]
with pytest.raises(
dbt.exceptions.DbtProjectError,
match="The package version requirement can never be satisfied",
):
_get_required_version(project_dict=project_dict, verify_version=True)

View File

@@ -1,244 +1,118 @@
import os
import tempfile
from argparse import Namespace
from typing import Any, Dict
from unittest import mock
import pytest
from pytest_mock import MockerFixture
import dbt.config
import dbt.exceptions
from dbt import tracking
from dbt.config.profile import Profile
from dbt.config.project import Project
from dbt.config.runtime import RuntimeConfig
from dbt.contracts.project import PackageConfig
from dbt.events.types import UnusedResourceConfigPath
from dbt.flags import set_from_args
from dbt.tests.util import safe_set_invocation_context
from tests.unit.config import (
BaseConfigTest,
empty_profile_renderer,
project_from_config_norender,
temp_cd,
)
from dbt_common.events.event_manager_client import add_callback_to_manager
from tests.unit.config import BaseConfigTest, temp_cd
from tests.utils import EventCatcher
class TestRuntimeConfig(BaseConfigTest):
def get_project(self):
return project_from_config_norender(
self.default_project_data,
project_root=self.project_dir,
verify_version=self.args.version_check,
class TestRuntimeConfig:
@pytest.fixture
def args(self) -> Namespace:
return Namespace(
profiles_dir=tempfile.mkdtemp(),
cli_vars={},
version_check=True,
project_dir=tempfile.mkdtemp(),
target=None,
threads=None,
profile=None,
)
def get_profile(self):
renderer = empty_profile_renderer()
return dbt.config.Profile.from_raw_profiles(
self.default_profile_data, self.default_project_data["profile"], renderer
)
def test_str(self, profile: Profile, project: Project) -> None:
config = dbt.config.RuntimeConfig.from_parts(project, profile, {})
def from_parts(self, exc=None):
with self.assertRaisesOrReturns(exc) as err:
project = self.get_project()
profile = self.get_profile()
# to make sure nothing terrible happens
str(config)
result = dbt.config.RuntimeConfig.from_parts(project, profile, self.args)
def test_from_parts(self, args: Namespace, profile: Profile, project: Project):
config = dbt.config.RuntimeConfig.from_parts(project, profile, args)
if exc is None:
return result
else:
return err
def test_from_parts(self):
project = self.get_project()
profile = self.get_profile()
config = dbt.config.RuntimeConfig.from_parts(project, profile, self.args)
self.assertEqual(config.cli_vars, {})
self.assertEqual(config.to_profile_info(), profile.to_profile_info())
assert config.cli_vars == {}
assert config.to_profile_info() == profile.to_profile_info()
# we should have the default quoting set in the full config, but not in
# the project
# TODO(jeb): Adapters must assert that quoting is populated?
expected_project = project.to_project_config()
self.assertEqual(expected_project["quoting"], {})
assert expected_project["quoting"] == {}
expected_project["quoting"] = {
"database": True,
"identifier": True,
"schema": True,
}
self.assertEqual(config.to_project_config(), expected_project)
def test_str(self):
project = self.get_project()
profile = self.get_profile()
config = dbt.config.RuntimeConfig.from_parts(project, profile, {})
# to make sure nothing terrible happens
str(config)
def test_supported_version(self):
self.default_project_data["require-dbt-version"] = ">0.0.0"
conf = self.from_parts()
self.assertEqual(set(x.to_version_string() for x in conf.dbt_version), {">0.0.0"})
def test_unsupported_version(self):
self.default_project_data["require-dbt-version"] = ">99999.0.0"
raised = self.from_parts(dbt.exceptions.DbtProjectError)
self.assertIn("This version of dbt is not supported", str(raised.exception))
def test_unsupported_version_no_check(self):
self.default_project_data["require-dbt-version"] = ">99999.0.0"
self.args.version_check = False
set_from_args(self.args, None)
conf = self.from_parts()
self.assertEqual(set(x.to_version_string() for x in conf.dbt_version), {">99999.0.0"})
def test_supported_version_range(self):
self.default_project_data["require-dbt-version"] = [">0.0.0", "<=99999.0.0"]
conf = self.from_parts()
self.assertEqual(
set(x.to_version_string() for x in conf.dbt_version), {">0.0.0", "<=99999.0.0"}
)
def test_unsupported_version_range(self):
self.default_project_data["require-dbt-version"] = [">0.0.0", "<=0.0.1"]
raised = self.from_parts(dbt.exceptions.DbtProjectError)
self.assertIn("This version of dbt is not supported", str(raised.exception))
def test_unsupported_version_range_bad_config(self):
self.default_project_data["require-dbt-version"] = [">0.0.0", "<=0.0.1"]
self.default_project_data["some-extra-field-not-allowed"] = True
raised = self.from_parts(dbt.exceptions.DbtProjectError)
self.assertIn("This version of dbt is not supported", str(raised.exception))
def test_unsupported_version_range_no_check(self):
self.default_project_data["require-dbt-version"] = [">0.0.0", "<=0.0.1"]
self.args.version_check = False
set_from_args(self.args, None)
conf = self.from_parts()
self.assertEqual(
set(x.to_version_string() for x in conf.dbt_version), {">0.0.0", "<=0.0.1"}
)
def test_impossible_version_range(self):
self.default_project_data["require-dbt-version"] = [">99999.0.0", "<=0.0.1"]
raised = self.from_parts(dbt.exceptions.DbtProjectError)
self.assertIn(
"The package version requirement can never be satisfied", str(raised.exception)
)
def test_unsupported_version_extra_config(self):
self.default_project_data["some-extra-field-not-allowed"] = True
raised = self.from_parts(dbt.exceptions.DbtProjectError)
self.assertIn("Additional properties are not allowed", str(raised.exception))
def test_archive_not_allowed(self):
self.default_project_data["archive"] = [
{
"source_schema": "a",
"target_schema": "b",
"tables": [
{
"source_table": "seed",
"target_table": "archive_actual",
"updated_at": "updated_at",
"unique_key": """id || '-' || first_name""",
},
],
}
]
with self.assertRaises(dbt.exceptions.DbtProjectError):
self.get_project()
def test__warn_for_unused_resource_config_paths_empty(self):
project = self.from_parts()
dbt.flags.WARN_ERROR = True
try:
project.warn_for_unused_resource_config_paths(
{
"models": frozenset(
(
("my_test_project", "foo", "bar"),
("my_test_project", "foo", "baz"),
)
)
},
[],
)
finally:
dbt.flags.WARN_ERROR = False
@mock.patch.object(tracking, "active_user")
def test_get_metadata(self, mock_user):
project = self.get_project()
profile = self.get_profile()
config = dbt.config.RuntimeConfig.from_parts(project, profile, self.args)
assert config.to_project_config() == expected_project
def test_get_metadata(self, mocker: MockerFixture, runtime_config: RuntimeConfig) -> None:
mock_user = mocker.patch.object(tracking, "active_user")
mock_user.id = "cfc9500f-dc7f-4c83-9ea7-2c581c1b38cf"
set_from_args(Namespace(SEND_ANONYMOUS_USAGE_STATS=False), None)
metadata = config.get_metadata()
metadata = runtime_config.get_metadata()
# ensure user_id and send_anonymous_usage_stats are set correctly
self.assertEqual(metadata.user_id, mock_user.id)
self.assertFalse(metadata.send_anonymous_usage_stats)
assert metadata.user_id == mock_user.id
assert not metadata.send_anonymous_usage_stats
@pytest.fixture
def used_fqns(self) -> Dict[str, Any]:
return {"models": frozenset((("my_test_project", "foo", "bar"),))}
class TestRuntimeConfigWithConfigs(BaseConfigTest):
def setUp(self):
self.profiles_dir = "/invalid-profiles-path"
self.project_dir = "/invalid-root-path"
super().setUp()
self.default_project_data["project-root"] = self.project_dir
self.default_project_data["models"] = {
"enabled": True,
def test_warn_for_unused_resource_config_paths(
self,
runtime_config: RuntimeConfig,
used_fqns: Dict[str, Any],
):
catcher = EventCatcher(event_to_catch=UnusedResourceConfigPath)
add_callback_to_manager(catcher.catch)
runtime_config.models = {
"my_test_project": {
"foo": {
"materialized": "view",
"bar": {
"materialized": "table",
},
},
"baz": {
"materialized": "table",
},
},
}
self.used = {
"models": frozenset(
(
("my_test_project", "foo", "bar"),
("my_test_project", "foo", "baz"),
)
)
"baz": {
"materialized": "table",
},
}
}
}
def get_project(self):
return project_from_config_norender(
self.default_project_data, project_root=self.project_dir, verify_version=True
)
runtime_config.warn_for_unused_resource_config_paths(used_fqns, [])
len(catcher.caught_events) == 1
expected_msg = "models.my_test_project.foo.baz"
assert expected_msg in str(catcher.caught_events[0].data)
def get_profile(self):
renderer = empty_profile_renderer()
return dbt.config.Profile.from_raw_profiles(
self.default_profile_data, self.default_project_data["profile"], renderer
)
def test_warn_for_unused_resource_config_paths_empty_models(
self,
runtime_config: RuntimeConfig,
used_fqns: Dict[str, Any],
) -> None:
catcher = EventCatcher(event_to_catch=UnusedResourceConfigPath)
add_callback_to_manager(catcher.catch)
def from_parts(self, exc=None):
with self.assertRaisesOrReturns(exc) as err:
project = self.get_project()
profile = self.get_profile()
# models should already be empty, but lets ensure it
runtime_config.models = {}
result = dbt.config.RuntimeConfig.from_parts(project, profile, self.args)
if exc is None:
return result
else:
return err
def test__warn_for_unused_resource_config_paths(self):
project = self.from_parts()
with mock.patch("dbt.config.runtime.warn_or_error") as warn_or_error_patch:
project.warn_for_unused_resource_config_paths(self.used, [])
warn_or_error_patch.assert_called_once()
event = warn_or_error_patch.call_args[0][0]
assert type(event).__name__ == "UnusedResourceConfigPath"
msg = event.message()
expected_msg = "- models.my_test_project.baz"
assert expected_msg in msg
runtime_config.warn_for_unused_resource_config_paths(used_fqns, ())
assert len(catcher.caught_events) == 0
class TestRuntimeConfigFiles(BaseConfigTest):

View File

@@ -6,6 +6,7 @@ from dbt.contracts.graph.nodes import SourceDefinition
# All manifest related fixtures.
from tests.unit.utils.adapter import * # noqa
from tests.unit.utils.config import * # noqa
from tests.unit.utils.event_manager import * # noqa
from tests.unit.utils.flags import * # noqa
from tests.unit.utils.manifest import * # noqa

View File

@@ -50,6 +50,7 @@ from tests.unit.utils import (
MockNode,
MockSource,
inject_plugin,
make_manifest,
)
REQUIRED_PARSED_NODE_KEYS = frozenset(
@@ -1091,20 +1092,6 @@ class TestManifestSearch(unittest.TestCase):
)
def make_manifest(nodes=[], sources=[], macros=[], docs=[]):
return Manifest(
nodes={n.unique_id: n for n in nodes},
macros={m.unique_id: m for m in macros},
sources={s.unique_id: s for s in sources},
docs={d.unique_id: d for d in docs},
disabled={},
files={},
exposures={},
metrics={},
selectors={},
)
FindMacroSpec = namedtuple("FindMacroSpec", "macros,expected")
macro_parameter_sets = [

View File

@@ -1,4 +1,5 @@
from copy import deepcopy
from datetime import datetime
from typing import List
import pytest
@@ -8,11 +9,19 @@ from dbt_semantic_interfaces.type_enums import (
DimensionType,
EntityType,
)
from freezegun import freeze_time
from dbt.artifacts.resources import Defaults, Dimension, Entity, Measure, TestMetadata
from dbt.artifacts.resources import (
Defaults,
Dimension,
Entity,
FileHash,
Measure,
TestMetadata,
)
from dbt.artifacts.resources.v1.semantic_model import NodeRelation
from dbt.contracts.graph.model_config import TestConfig
from dbt.contracts.graph.nodes import ColumnInfo, SemanticModel
from dbt.contracts.graph.nodes import ColumnInfo, ModelNode, SemanticModel
from dbt.node_types import NodeType
from dbt_common.contracts.constraints import (
ColumnLevelConstraint,
@@ -22,6 +31,44 @@ from dbt_common.contracts.constraints import (
from tests.unit.fixtures import generic_test_node, model_node
class TestModelNode:
@pytest.fixture(scope="class")
def default_model_node(self):
return ModelNode(
resource_type=NodeType.Model,
unique_id="model.test_package.test_name",
name="test_name",
package_name="test_package",
schema="test_schema",
alias="test_alias",
fqn=["models", "test_name"],
original_file_path="test_original_file_path",
checksum=FileHash.from_contents("checksum"),
path="test_path",
database=None,
)
@pytest.mark.parametrize(
"deprecation_date,current_date,expected_is_past_deprecation_date",
[
(None, "2024-05-02", False),
("2024-05-01", "2024-05-02", True),
("2024-05-01", "2024-05-01", False),
("2024-05-01", "2024-04-30", False),
],
)
def test_is_past_deprecation_date(
self, default_model_node, deprecation_date, current_date, expected_is_past_deprecation_date
):
with freeze_time(current_date):
if deprecation_date is not None:
default_model_node.deprecation_date = datetime.strptime(
deprecation_date, "%Y-%m-%d"
).astimezone()
assert default_model_node.is_past_deprecation_date is expected_is_past_deprecation_date
class TestSemanticModel:
@pytest.fixture(scope="function")
def dimensions(self) -> List[Dimension]:

View File

@@ -0,0 +1,47 @@
import networkx as nx
import pytest
from dbt.contracts.graph.manifest import Manifest
from dbt.graph.queue import GraphQueue
from tests.unit.utils import MockNode, make_manifest
class TestGraphQueue:
@pytest.fixture(scope="class")
def manifest(self) -> Manifest:
return make_manifest(
nodes=[
MockNode(package="test_package", name="upstream_model"),
MockNode(package="test_package", name="downstream_model"),
]
)
@pytest.fixture(scope="class")
def graph(self) -> nx.DiGraph:
graph = nx.DiGraph()
graph.add_edge("model.test_package.upstream_model", "model.test_package.downstream_model")
return graph
def test_init_graph_queue(self, manifest, graph):
graph_queue = GraphQueue(graph=graph, manifest=manifest, selected={})
assert graph_queue.manifest == manifest
assert graph_queue.graph == graph
assert graph_queue.inner.queue == [(0, "model.test_package.upstream_model")]
assert graph_queue.in_progress == set()
assert graph_queue.queued == {"model.test_package.upstream_model"}
assert graph_queue.lock
def test_init_graph_queue_preserve_edges_false(self, manifest, graph):
graph_queue = GraphQueue(graph=graph, manifest=manifest, selected={}, preserve_edges=False)
# when preserve_edges is set to false, dependencies between nodes are no longer tracked in the priority queue
assert list(graph_queue.graph.edges) == []
assert graph_queue.inner.queue == [
(0, "model.test_package.downstream_model"),
(0, "model.test_package.upstream_model"),
]
assert graph_queue.queued == {
"model.test_package.upstream_model",
"model.test_package.downstream_model",
}

View File

@@ -12,6 +12,7 @@ def serve_task():
task = ServeTask(config=MagicMock(), args=MagicMock())
task.config.project_target_path = "."
task.args.port = 8000
task.args.host = "127.0.0.1"
return task
@@ -21,3 +22,13 @@ def test_serve_bind_to_127(serve_task):
patched_TCPServer.return_value = MagicMock()
serve_task.run()
patched_TCPServer.assert_called_once_with(("127.0.0.1", 8000), SimpleHTTPRequestHandler)
def test_serve_bind_to_all(serve_task):
serve_task.args.browser = False
serve_task.args.host = ""
with patch("dbt.task.docs.serve.socketserver.TCPServer") as patched_TCPServer:
patched_TCPServer.return_value = MagicMock()
serve_task.run()
patched_TCPServer.assert_called_once_with(("", 8000), SimpleHTTPRequestHandler)

View File

@@ -1,10 +1,14 @@
from dataclasses import dataclass
from typing import AbstractSet, Any, Dict, Optional
from typing import AbstractSet, Any, Dict, List, Optional, Tuple
import networkx as nx
import pytest
from dbt.task.runnable import GraphRunnableTask
from dbt.artifacts.resources.types import NodeType
from dbt.graph import Graph, ResourceTypeSelector
from dbt.task.runnable import GraphRunnableMode, GraphRunnableTask
from dbt.tests.util import safe_set_invocation_context
from tests.unit.utils import MockNode, make_manifest
@dataclass
@@ -14,6 +18,9 @@ class MockArgs:
state: Optional[Dict[str, Any]] = None
defer_state: Optional[Dict[str, Any]] = None
write_json: bool = False
selector: Optional[str] = None
select: Tuple[str] = ()
exclude: Tuple[str] = ()
@dataclass
@@ -23,12 +30,28 @@ class MockConfig:
threads: int = 1
target_name: str = "mock_config_target_name"
def get_default_selector_name(self):
return None
class MockRunnableTask(GraphRunnableTask):
def __init__(self, exception_class: Exception = Exception):
def __init__(
self,
exception_class: Exception = Exception,
nodes: Optional[List[MockNode]] = None,
edges: Optional[List[Tuple[str, str]]] = None,
):
nodes = nodes or []
edges = edges or []
self.forced_exception_class = exception_class
self.did_cancel: bool = False
super().__init__(args=MockArgs(), config=MockConfig(), manifest=None)
self.manifest = make_manifest(nodes=nodes)
digraph = nx.DiGraph()
for edge in edges:
digraph.add_edge(edge[0], edge[1])
self.graph = Graph(digraph)
def run_queue(self, pool):
"""Override `run_queue` to raise a system exit"""
@@ -40,13 +63,25 @@ class MockRunnableTask(GraphRunnableTask):
def get_node_selector(self):
"""This is an `abstract_method` on `GraphRunnableTask`, thus we must implement it"""
return None
selector = ResourceTypeSelector(
graph=self.graph,
manifest=self.manifest,
previous_state=self.previous_state,
resource_types=[NodeType.Model],
include_empty_nodes=True,
)
return selector
def defer_to_manifest(self, adapter, selected_uids: AbstractSet[str]):
"""This is an `abstract_method` on `GraphRunnableTask`, thus we must implement it"""
return None
class MockRunnableTaskIndependent(MockRunnableTask):
def get_run_mode(self) -> GraphRunnableMode:
return GraphRunnableMode.Independent
def test_graph_runnable_task_cancels_connection_on_system_exit():
safe_set_invocation_context()
@@ -81,3 +116,36 @@ def test_graph_runnable_task_doesnt_cancel_connection_on_generic_exception():
# If `did_cancel` is True, that means `_cancel_connections` was called
assert task.did_cancel is False
def test_graph_runnable_preserves_edges_by_default():
task = MockRunnableTask(
nodes=[
MockNode("test", "upstream_node", fqn="model.test.upstream_node"),
MockNode("test", "downstream_node", fqn="model.test.downstream_node"),
],
edges=[("model.test.upstream_node", "model.test.downstream_node")],
)
assert task.get_run_mode() == GraphRunnableMode.Topological
graph_queue = task.get_graph_queue()
assert graph_queue.queued == {"model.test.upstream_node"}
assert graph_queue.inner.queue == [(0, "model.test.upstream_node")]
def test_graph_runnable_preserves_edges_false():
task = MockRunnableTaskIndependent(
nodes=[
MockNode("test", "upstream_node", fqn="model.test.upstream_node"),
MockNode("test", "downstream_node", fqn="model.test.downstream_node"),
],
edges=[("model.test.upstream_node", "model.test.downstream_node")],
)
assert task.get_run_mode() == GraphRunnableMode.Independent
graph_queue = task.get_graph_queue()
assert graph_queue.queued == {"model.test.downstream_node", "model.test.upstream_node"}
assert graph_queue.inner.queue == [
(0, "model.test.downstream_node"),
(0, "model.test.upstream_node"),
]

View File

@@ -0,0 +1,71 @@
import agate
import pytest
from dbt.task.test import list_rows_from_table
class TestListRowsFromTable:
@pytest.mark.parametrize(
"agate_table_cols,agate_table_rows,expected_list_rows",
[
(["a", "b", "c"], [], [["a", "b", "c"]]), # no rows
(["a", "b", "c"], [[1, 2, 3]], [["a", "b", "c"], [1, 2, 3]]), # single row, no nulls
(
["a", "b", "c"],
[[1, 2, 3], [2, 3, 4]],
[["a", "b", "c"], [1, 2, 3], [2, 3, 4]],
), # multiple rows
(
["a", "b", "c"],
[[None, 2, 3], [2, None, 4]],
[["a", "b", "c"], [None, 2, 3], [2, None, 4]],
), # multiple rows, with nulls
],
)
def test_list_rows_from_table_no_sort(
self, agate_table_cols, agate_table_rows, expected_list_rows
):
table = agate.Table(rows=agate_table_rows, column_names=agate_table_cols)
list_rows = list_rows_from_table(table)
assert list_rows == expected_list_rows
@pytest.mark.parametrize(
"agate_table_cols,agate_table_rows,expected_list_rows",
[
(["a", "b", "c"], [], [["a", "b", "c"]]), # no rows
(["a", "b", "c"], [[1, 2, 3]], [["a", "b", "c"], [1, 2, 3]]), # single row, no nulls
(
["a", "b", "c"],
[[1, 2, 3], [2, 3, 4]],
[["a", "b", "c"], [1, 2, 3], [2, 3, 4]],
), # multiple rows, in order
(
["a", "b", "c"],
[[2, 3, 4], [1, 2, 3]],
[["a", "b", "c"], [1, 2, 3], [2, 3, 4]],
), # multiple rows, out of order
(
["a", "b", "c"],
[[None, 2, 3], [2, 3, 4]],
[["a", "b", "c"], [2, 3, 4], [None, 2, 3]],
), # multiple rows, out of order with nulls in first position
(
["a", "b", "c"],
[[4, 5, 6], [1, None, 3]],
[["a", "b", "c"], [1, None, 3], [4, 5, 6]],
), # multiple rows, out of order with null in non-first position
(
["a", "b", "c"],
[[None, 5, 6], [1, None, 3]],
[["a", "b", "c"], [1, None, 3], [None, 5, 6]],
), # multiple rows, out of order with nulls in many positions
],
)
def test_list_rows_from_table_with_sort(
self, agate_table_cols, agate_table_rows, expected_list_rows
):
table = agate.Table(rows=agate_table_rows, column_names=agate_table_cols)
list_rows = list_rows_from_table(table, sort=True)
assert list_rows == expected_list_rows

View File

@@ -11,6 +11,7 @@ import agate
import pytest
from dbt.config.project import PartialProject
from dbt.contracts.graph.manifest import Manifest
from dbt_common.dataclass_schema import ValidationError
@@ -387,3 +388,17 @@ def replace_config(n, **kwargs):
config=n.config.replace(**kwargs),
unrendered_config=dict_replace(n.unrendered_config, **kwargs),
)
def make_manifest(nodes=[], sources=[], macros=[], docs=[]) -> Manifest:
return Manifest(
nodes={n.unique_id: n for n in nodes},
macros={m.unique_id: m for m in macros},
sources={s.unique_id: s for s in sources},
docs={d.unique_id: d for d in docs},
disabled={},
files={},
exposures={},
metrics={},
selectors={},
)

View File

@@ -0,0 +1,50 @@
import pytest
from dbt.adapters.postgres.connections import PostgresCredentials
from dbt.config.profile import Profile
from dbt.config.project import Project
from dbt.config.renderer import ProfileRenderer
from dbt.config.runtime import RuntimeConfig
@pytest.fixture
def credentials() -> PostgresCredentials:
return PostgresCredentials(
database="test_database",
schema="test_schema",
host="test_host",
user="test_user",
port=1337,
password="test_password",
)
@pytest.fixture
def profile() -> Profile:
profile_yaml = {
"target": "postgres",
"outputs": {
"postgres": {
"type": "postgres",
"host": "postgres-db-hostname",
"port": 5555,
"user": "db_user",
"pass": "db_pass",
"dbname": "postgres-db-name",
"schema": "postgres-schema",
"threads": 7,
},
},
}
return Profile.from_raw_profile_info(
raw_profile=profile_yaml, profile_name="test_profile", renderer=ProfileRenderer({})
)
@pytest.fixture
def runtime_config(project: Project, profile: Profile) -> RuntimeConfig:
return RuntimeConfig.from_parts(
project=project,
profile=profile,
args={},
)

View File

@@ -45,7 +45,7 @@ def project(selector_config: SelectorConfig) -> Project:
log_path="path/to/project/logs",
packages_install_path="dbt_packages",
packages_specified_path="packages.yml",
quoting={"database": True, "schema": True, "identifier": True},
quoting={},
models={},
on_run_start=[],
on_run_end=[],