mirror of
https://github.com/dbt-labs/dbt-core
synced 2025-12-17 19:31:34 +00:00
Compare commits
12 Commits
update-ind
...
test-adapt
| Author | SHA1 | Date | |
|---|---|---|---|
|
|
52e2e8531e | ||
|
|
ef67cffc0a | ||
|
|
a34267f54b | ||
|
|
155482851a | ||
|
|
81386a7a43 | ||
|
|
d8e38c1a1d | ||
|
|
3e37d77780 | ||
|
|
e0783c2922 | ||
|
|
c2d4643f9d | ||
|
|
84456f50f6 | ||
|
|
fb10bb4aea | ||
|
|
366d4ad04a |
6
.changes/unreleased/Docs-20240311-140344.yaml
Normal file
6
.changes/unreleased/Docs-20240311-140344.yaml
Normal 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"
|
||||
6
.changes/unreleased/Docs-20240501-021050.yaml
Normal file
6
.changes/unreleased/Docs-20240501-021050.yaml
Normal 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"
|
||||
6
.changes/unreleased/Docs-20240516-223036.yaml
Normal file
6
.changes/unreleased/Docs-20240516-223036.yaml
Normal 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"
|
||||
7
.changes/unreleased/Features-20240522-000309.yaml
Normal file
7
.changes/unreleased/Features-20240522-000309.yaml
Normal 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"
|
||||
6
.changes/unreleased/Features-20240527-124405.yaml
Normal file
6
.changes/unreleased/Features-20240527-124405.yaml
Normal 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"
|
||||
7
.changes/unreleased/Fixes-20240522-182855.yaml
Normal file
7
.changes/unreleased/Fixes-20240522-182855.yaml
Normal 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"
|
||||
6
.changes/unreleased/Fixes-20240523-204251.yaml
Normal file
6
.changes/unreleased/Fixes-20240523-204251.yaml
Normal 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"
|
||||
6
.changes/unreleased/Fixes-20240524-131135.yaml
Normal file
6
.changes/unreleased/Fixes-20240524-131135.yaml
Normal 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"
|
||||
6
.changes/unreleased/Under the Hood-20240529-102814.yaml
Normal file
6
.changes/unreleased/Under the Hood-20240529-102814.yaml
Normal 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"
|
||||
3
.github/ISSUE_TEMPLATE/config.yml
vendored
3
.github/ISSUE_TEMPLATE/config.yml
vendored
@@ -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
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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",
|
||||
|
||||
@@ -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
|
||||
|
||||
|
||||
@@ -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
|
||||
|
||||
|
||||
@@ -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.
|
||||
|
||||
@@ -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):
|
||||
|
||||
@@ -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]:
|
||||
|
||||
@@ -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
|
||||
|
||||
|
||||
@@ -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
@@ -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")
|
||||
|
||||
@@ -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()
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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.
|
||||
|
||||
@@ -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)
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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()
|
||||
|
||||
@@ -760,6 +760,14 @@ select 1 as notfun
|
||||
|
||||
"""
|
||||
|
||||
model_two_disabled_sql = """
|
||||
{{ config(
|
||||
enabled=false
|
||||
) }}
|
||||
|
||||
select 1 as notfun
|
||||
"""
|
||||
|
||||
generic_test_schema_yml = """
|
||||
|
||||
models:
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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:
|
||||
|
||||
49
tests/functional/unit_testing/test_ut_adapter_hooks.py
Normal file
49
tests/functional/unit_testing/test_ut_adapter_hooks.py
Normal 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()
|
||||
113
tests/functional/unit_testing/test_ut_diffing.py
Normal file
113
tests/functional/unit_testing/test_ut_diffing.py
Normal 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
|
||||
@@ -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)
|
||||
|
||||
@@ -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):
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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 = [
|
||||
|
||||
@@ -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]:
|
||||
|
||||
47
tests/unit/graph/test_queue.py
Normal file
47
tests/unit/graph/test_queue.py
Normal 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",
|
||||
}
|
||||
@@ -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)
|
||||
|
||||
@@ -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"),
|
||||
]
|
||||
|
||||
71
tests/unit/task/test_test.py
Normal file
71
tests/unit/task/test_test.py
Normal 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
|
||||
@@ -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={},
|
||||
)
|
||||
|
||||
50
tests/unit/utils/config.py
Normal file
50
tests/unit/utils/config.py
Normal 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={},
|
||||
)
|
||||
@@ -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=[],
|
||||
|
||||
Reference in New Issue
Block a user