diff --git a/tests/unit/graph/test_selector.py b/tests/unit/graph/test_selector.py index 677fb1c46bd..acbb69dfbba 100644 --- a/tests/unit/graph/test_selector.py +++ b/tests/unit/graph/test_selector.py @@ -1,9 +1,7 @@ -import os import string -import unittest from argparse import Namespace from queue import Empty -from unittest.mock import MagicMock, patch +from unittest.mock import MagicMock import networkx as nx import pytest @@ -17,23 +15,11 @@ import dbt.parser.manifest import dbt.utils import dbt_common.exceptions -from dbt import tracking -from dbt.adapters.factory import register_adapter, reset_adapters -from dbt.adapters.postgres import Plugin as PostgresPlugin -from dbt.cli.flags import convert_config -from dbt.contracts.files import FileHash, FilePath, SourceFile -from dbt.contracts.graph.manifest import MacroManifest, ManifestStateCheck -from dbt.contracts.project import ProjectFlags -from dbt.events.logging import setup_event_logger -from dbt.flags import get_flags, set_from_args +from dbt.config.runtime import RuntimeConfig +from dbt.flags import set_from_args from dbt.graph import NodeSelector, parse_difference -from dbt.mp_context import get_mp_context from dbt.node_types import NodeType -from tests.unit.utils import ( - config_from_parts_or_dicts, - generate_name_macros, - inject_plugin, -) +from tests.unit.utils.manifest import make_manifest, make_model set_from_args(Namespace(WARN_ERROR=False), None) @@ -82,7 +68,7 @@ def graph(): @pytest.fixture -def manifest(graph): +def mock_manifest(graph): return _get_manifest(graph) @@ -142,8 +128,8 @@ def id_macro(arg): @pytest.mark.parametrize("include,exclude,expected", run_specs, ids=id_macro) -def test_run_specs(include, exclude, expected, graph, manifest): - selector = graph_selector.NodeSelector(graph, manifest) +def test_run_specs(include, exclude, expected, graph, mock_manifest): + selector = graph_selector.NodeSelector(graph, mock_manifest) spec = graph_cli.parse_difference(include, exclude) selected, _ = selector.select_nodes(spec) @@ -231,320 +217,70 @@ def test_invalid_specs(invalid): graph_selector.SelectionCriteria.from_single_spec(invalid) -class GraphTest(unittest.TestCase): - def tearDown(self): - self.mock_filesystem_search.stop() - self.load_state_check.stop() - self.load_source_file_patcher.stop() - reset_adapters() - - def setUp(self): - # create various attributes - self.graph_result = None - tracking.do_not_track() - self.profile = { - "outputs": { - "test": { - "type": "postgres", - "threads": 4, - "host": "thishostshouldnotexist", - "port": 5432, - "user": "root", - "pass": "password", - "dbname": "dbt", - "schema": "dbt_test", - } - }, - "target": "test", - } - self.macro_manifest = MacroManifest( - {n.unique_id: n for n in generate_name_macros("test_models_compile")} - ) - self.mock_models = [] # used by filesystem_searcher - - # Create file filesystem searcher - self.filesystem_search = patch("dbt.parser.read_files.filesystem_search") - - def mock_filesystem_search(project, relative_dirs, extension, ignore_spec): - if "sql" not in extension: - return [] - if "models" not in relative_dirs: - return [] - return [model.path for model in self.mock_models] - - self.mock_filesystem_search = self.filesystem_search.start() - self.mock_filesystem_search.side_effect = mock_filesystem_search - - # Create the Manifest.state_check patcher - @patch("dbt.parser.manifest.ManifestLoader.build_manifest_state_check") - def _mock_state_check(self): - all_projects = self.all_projects - return ManifestStateCheck( - project_env_vars_hash=FileHash.from_contents(""), - profile_env_vars_hash=FileHash.from_contents(""), - vars_hash=FileHash.from_contents("vars"), - project_hashes={name: FileHash.from_contents(name) for name in all_projects}, - profile_hash=FileHash.from_contents("profile"), - ) - - self.load_state_check = patch( - "dbt.parser.manifest.ManifestLoader.build_manifest_state_check" - ) - self.mock_state_check = self.load_state_check.start() - self.mock_state_check.side_effect = _mock_state_check - - # Create the source file patcher - self.load_source_file_patcher = patch("dbt.parser.read_files.load_source_file") - self.mock_source_file = self.load_source_file_patcher.start() - - def mock_load_source_file(path, parse_file_type, project_name, saved_files): - for sf in self.mock_models: - if sf.path == path: - source_file = sf - source_file.project_name = project_name - source_file.parse_file_type = parse_file_type - return source_file - - self.mock_source_file.side_effect = mock_load_source_file - - # Create hookparser source file patcher - self.load_source_file_manifest_patcher = patch("dbt.parser.manifest.load_source_file") - self.mock_source_file_manifest = self.load_source_file_manifest_patcher.start() - - def mock_load_source_file_manifest(path, parse_file_type, project_name, saved_files): - return [] - - self.mock_source_file_manifest.side_effect = mock_load_source_file_manifest - - def get_config(self, extra_cfg=None): - if extra_cfg is None: - extra_cfg = {} - - cfg = { - "name": "test_models_compile", - "version": "0.1", - "profile": "test", - "project-root": os.path.abspath("."), - "config-version": 2, - } - cfg.update(extra_cfg) - - config = config_from_parts_or_dicts(project=cfg, profile=self.profile) - set_from_args(Namespace(), ProjectFlags()) - flags = get_flags() - setup_event_logger(flags) - object.__setattr__(flags, "PARTIAL_PARSE", False) - for arg_name, args_param_value in vars(flags).items(): - args_param_value = convert_config(arg_name, args_param_value) - object.__setattr__(config.args, arg_name.upper(), args_param_value) - object.__setattr__(config.args, arg_name.lower(), args_param_value) - - return config - - def get_compiler(self, project): - return dbt.compilation.Compiler(project) - - def use_models(self, models): - for k, v in models.items(): - path = FilePath( - searched_path="models", - project_root=os.path.normcase(os.getcwd()), - relative_path="{}.sql".format(k), - modification_time=0.0, - ) - # FileHash can't be empty or 'search_key' will be None - source_file = SourceFile(path=path, checksum=FileHash.from_contents("abc")) - source_file.contents = v - self.mock_models.append(source_file) - - def load_manifest(self, config): - inject_plugin(PostgresPlugin) - register_adapter(config, get_mp_context()) - loader = dbt.parser.manifest.ManifestLoader(config, {config.project_name: config}) - loader.manifest.macros = self.macro_manifest.macros - loader.load() - return loader.manifest - - def test__single_model(self): - self.use_models( - { - "model_one": "select * from events", - } - ) - - config = self.get_config() - manifest = self.load_manifest(config) +class TestCompiler: + def test_single_model(self, runtime_config: RuntimeConfig): + model = make_model(pkg="pkg", name="model_one", code="SELECT * FROM events") + manifest = make_manifest(nodes=[model]) - compiler = self.get_compiler(config) + compiler = dbt.compilation.Compiler(config=runtime_config) linker = compiler.compile(manifest) - self.assertEqual(list(linker.nodes()), ["model.test_models_compile.model_one"]) + assert linker.nodes() == {model.unique_id} + assert linker.edges() == set() - self.assertEqual(list(linker.edges()), []) - - def test__two_models_simple_ref(self): - self.use_models( - { - "model_one": "select * from events", - "model_two": "select * from {{ref('model_one')}}", - } + def test_two_models_simple_ref(self, runtime_config: RuntimeConfig): + model_one = make_model(pkg="pkg", name="model_one", code="SELECT * FROM events") + model_two = make_model( + pkg="pkg", + name="model_two", + code="SELECT * FROM {{ref('model_one')}}", + refs=[model_one], ) + models = [model_one, model_two] + manifest = make_manifest(nodes=models) - config = self.get_config() - manifest = self.load_manifest(config) - compiler = self.get_compiler(config) + compiler = dbt.compilation.Compiler(config=runtime_config) linker = compiler.compile(manifest) - self.assertCountEqual( - linker.nodes(), - [ - "model.test_models_compile.model_one", - "model.test_models_compile.model_two", - ], - ) - - self.assertCountEqual( - linker.edges(), - [ - ( - "model.test_models_compile.model_one", - "model.test_models_compile.model_two", - ) - ], - ) - - def test__two_models_package_ref(self): - self.use_models( - { - "model_one": "select * from events", - "model_two": "select * from {{ref('test_models_compile', 'model_one')}}", - } - ) + expected_nodes = [model.unique_id for model in models] + assert linker.nodes() == set(expected_nodes) + assert list(linker.edges()) == [tuple(expected_nodes)] - config = self.get_config() - manifest = self.load_manifest(config) - compiler = self.get_compiler(config) - linker = compiler.compile(manifest) - self.assertCountEqual( - linker.nodes(), - [ - "model.test_models_compile.model_one", - "model.test_models_compile.model_two", - ], +class TestNodeSelector: + def test_dependency_list(self, runtime_config: RuntimeConfig): + model_one = make_model(pkg="pkg", name="model_one", code="SELECT * FROM events") + model_two = make_model( + pkg="pkg", + name="model_two", + code="SELECT * FROM {{ref('model_one')}}", + refs=[model_one], ) - - self.assertCountEqual( - linker.edges(), - [ - ( - "model.test_models_compile.model_one", - "model.test_models_compile.model_two", - ) - ], - ) - - def test__model_materializations(self): - self.use_models( - { - "model_one": "select * from events", - "model_two": "select * from {{ref('model_one')}}", - "model_three": "select * from events", - "model_four": "select * from events", - } - ) - - cfg = { - "models": { - "materialized": "table", - "test_models_compile": { - "model_one": {"materialized": "table"}, - "model_two": {"materialized": "view"}, - "model_three": {"materialized": "ephemeral"}, - }, - } - } - - config = self.get_config(cfg) - manifest = self.load_manifest(config) - - expected_materialization = { - "model_one": "table", - "model_two": "view", - "model_three": "ephemeral", - "model_four": "table", - } - - for model, expected in expected_materialization.items(): - key = "model.test_models_compile.{}".format(model) - actual = manifest.nodes[key].config.materialized - self.assertEqual(actual, expected) - - def test__model_incremental(self): - self.use_models({"model_one": "select * from events"}) - - cfg = { - "models": { - "test_models_compile": { - "model_one": {"materialized": "incremental", "unique_key": "id"}, - } - } - } - - config = self.get_config(cfg) - manifest = self.load_manifest(config) - compiler = self.get_compiler(config) - linker = compiler.compile(manifest) - - node = "model.test_models_compile.model_one" - - self.assertEqual(list(linker.nodes()), [node]) - self.assertEqual(list(linker.edges()), []) - - self.assertEqual(manifest.nodes[node].config.materialized, "incremental") - - def test__dependency_list(self): - self.use_models( - { - "model_1": "select * from events", - "model_2": 'select * from {{ ref("model_1") }}', - "model_3": """ - select * from {{ ref("model_1") }} + model_three = make_model( + pkg="pkg", + name="model_three", + code=""" + SELECT * FROM {{ ref("model_1") }} union all - select * from {{ ref("model_2") }} + SELECT * FROM {{ ref("model_2") }} """, - "model_4": 'select * from {{ ref("model_3") }}', - } + refs=[model_one, model_two], + ) + model_four = make_model( + pkg="pkg", + name="model_four", + code="SELECT * FROM {{ref('model_three')}}", + refs=[model_three], ) + models = [model_one, model_two, model_three, model_four] + manifest = make_manifest(nodes=models) - config = self.get_config() - manifest = self.load_manifest(config) - compiler = self.get_compiler(config) + # Get the graph + compiler = dbt.compilation.Compiler(runtime_config) graph = compiler.compile(manifest) - models = ("model_1", "model_2", "model_3", "model_4") - model_ids = ["model.test_models_compile.{}".format(m) for m in models] - - manifest = MagicMock( - nodes={ - n: MagicMock( - unique_id=n, - name=n.split(".")[-1], - package_name="test_models_compile", - fqn=["test_models_compile", n], - empty=False, - config=MagicMock(enabled=True), - ) - for n in model_ids - } - ) - manifest.expect.side_effect = lambda n: MagicMock(unique_id=n) + # Create the selector and get the queue selector = NodeSelector(graph, manifest) - # TODO: The "eager" string below needs to be replaced with programatic access - # to the default value for the indirect selection parameter in - # dbt.cli.params.indirect_selection - # - # Doing that is actually a little tricky, so I'm punting it to a new ticket GH #6397 queue = selector.get_graph_queue( parse_difference( None, @@ -552,29 +288,11 @@ def test__dependency_list(self): ) ) - for model_id in model_ids: - self.assertFalse(queue.empty()) + for model in models: + assert not queue.empty() got = queue.get(block=False) - self.assertEqual(got.unique_id, model_id) - with self.assertRaises(Empty): + assert got.unique_id == model.unique_id + with pytest.raises(Empty): queue.get(block=False) queue.mark_done(got.unique_id) - self.assertTrue(queue.empty()) - - def test__partial_parse(self): - config = self.get_config() - - manifest = self.load_manifest(config) - - # we need a loader to compare the two manifests - loader = dbt.parser.manifest.ManifestLoader(config, {config.project_name: config}) - loader.manifest = manifest.deepcopy() - - is_partial_parsable, _ = loader.is_partial_parsable(manifest) - self.assertTrue(is_partial_parsable) - manifest.metadata.dbt_version = "0.0.1a1" - is_partial_parsable, _ = loader.is_partial_parsable(manifest) - self.assertFalse(is_partial_parsable) - manifest.metadata.dbt_version = "99999.99.99" - is_partial_parsable, _ = loader.is_partial_parsable(manifest) - self.assertFalse(is_partial_parsable) + assert queue.empty() diff --git a/tests/unit/parser/test_manifest.py b/tests/unit/parser/test_manifest.py index 1f10ee04f25..4c887bce5c0 100644 --- a/tests/unit/parser/test_manifest.py +++ b/tests/unit/parser/test_manifest.py @@ -4,8 +4,9 @@ import pytest from pytest_mock import MockerFixture +from dbt.artifacts.resources.base import FileHash from dbt.config import RuntimeConfig -from dbt.contracts.graph.manifest import Manifest +from dbt.contracts.graph.manifest import Manifest, ManifestStateCheck from dbt.flags import set_from_args from dbt.parser.manifest import ManifestLoader from dbt.parser.read_files import FileDiff @@ -38,6 +39,39 @@ def test_profile_hash_change(self, mock_project): manifest = ManifestLoader(mock_project, {}) assert manifest.manifest.state_check.profile_hash.checksum != profile_hash + @patch("dbt.parser.manifest.ManifestLoader.build_manifest_state_check") + @patch("dbt.parser.manifest.os.path.exists") + @patch("dbt.parser.manifest.open") + def test_partial_parse_by_version( + self, + patched_open, + patched_os_exist, + patched_state_check, + runtime_config: RuntimeConfig, + manifest: Manifest, + ): + file_hash = FileHash.from_contents("test contests") + manifest.state_check = ManifestStateCheck( + vars_hash=file_hash, + profile_hash=file_hash, + profile_env_vars_hash=file_hash, + project_env_vars_hash=file_hash, + ) + # we need a loader to compare the two manifests + loader = ManifestLoader(runtime_config, {runtime_config.project_name: runtime_config}) + loader.manifest = manifest.deepcopy() + + is_partial_parsable, _ = loader.is_partial_parsable(manifest) + assert is_partial_parsable + + manifest.metadata.dbt_version = "0.0.1a1" + is_partial_parsable, _ = loader.is_partial_parsable(manifest) + assert not is_partial_parsable + + manifest.metadata.dbt_version = "99999.99.99" + is_partial_parsable, _ = loader.is_partial_parsable(manifest) + assert not is_partial_parsable + class TestFailedPartialParse: @patch("dbt.tracking.track_partial_parser") diff --git a/tests/unit/utils/config.py b/tests/unit/utils/config.py index 72cb4fa024c..3041bbada7c 100644 --- a/tests/unit/utils/config.py +++ b/tests/unit/utils/config.py @@ -5,6 +5,7 @@ from dbt.config.project import Project from dbt.config.renderer import ProfileRenderer from dbt.config.runtime import RuntimeConfig +from dbt.flags import get_flags @pytest.fixture @@ -42,9 +43,10 @@ def profile() -> Profile: @pytest.fixture -def runtime_config(project: Project, profile: Profile) -> RuntimeConfig: +def runtime_config(project: Project, profile: Profile, set_test_flags) -> RuntimeConfig: + args = get_flags() return RuntimeConfig.from_parts( project=project, profile=profile, - args={}, + args=args, ) diff --git a/tests/unit/utils/manifest.py b/tests/unit/utils/manifest.py index a7c269cdab2..98c00007cd5 100644 --- a/tests/unit/utils/manifest.py +++ b/tests/unit/utils/manifest.py @@ -1,3 +1,5 @@ +from typing import Any, Dict, List + import pytest from dbt_semantic_interfaces.type_enums import MetricType @@ -14,15 +16,18 @@ TestMetadata, ) from dbt.artifacts.resources.v1.model import ModelConfig -from dbt.contracts.files import FileHash +from dbt.contracts.files import AnySourceFile, FileHash from dbt.contracts.graph.manifest import Manifest, ManifestMetadata from dbt.contracts.graph.nodes import ( AccessType, DependsOn, + Documentation, Exposure, GenericTestNode, + GraphMemberNode, Group, Macro, + ManifestNode, Metric, ModelNode, NodeConfig, @@ -501,41 +506,41 @@ def make_saved_query(pkg: str, name: str, metric: str, path=None): @pytest.fixture -def macro_test_unique(): +def macro_test_unique() -> Macro: return make_macro( "dbt", "test_unique", "blablabla", depends_on_macros=["macro.dbt.default__test_unique"] ) @pytest.fixture -def macro_default_test_unique(): +def macro_default_test_unique() -> Macro: return make_macro("dbt", "default__test_unique", "blablabla") @pytest.fixture -def macro_test_not_null(): +def macro_test_not_null() -> Macro: return make_macro( "dbt", "test_not_null", "blablabla", depends_on_macros=["macro.dbt.default__test_not_null"] ) @pytest.fixture -def macro_default_test_not_null(): +def macro_default_test_not_null() -> Macro: return make_macro("dbt", "default__test_not_null", "blabla") @pytest.fixture -def seed(): +def seed() -> SeedNode: return make_seed("pkg", "seed") @pytest.fixture -def source(): +def source() -> SourceDefinition: return make_source("pkg", "raw", "seed", identifier="seed") @pytest.fixture -def ephemeral_model(source): +def ephemeral_model(source) -> ModelNode: return make_model( "pkg", "ephemeral_model", @@ -546,7 +551,7 @@ def ephemeral_model(source): @pytest.fixture -def view_model(ephemeral_model): +def view_model(ephemeral_model) -> ModelNode: return make_model( "pkg", "view_model", @@ -558,7 +563,7 @@ def view_model(ephemeral_model): @pytest.fixture -def table_model(ephemeral_model): +def table_model(ephemeral_model) -> ModelNode: return make_model( "pkg", "table_model", @@ -580,7 +585,7 @@ def table_model(ephemeral_model): @pytest.fixture -def table_model_py(seed): +def table_model_py(seed) -> ModelNode: return make_model( "pkg", "table_model_py", @@ -593,7 +598,7 @@ def table_model_py(seed): @pytest.fixture -def table_model_csv(seed): +def table_model_csv(seed) -> ModelNode: return make_model( "pkg", "table_model_csv", @@ -606,7 +611,7 @@ def table_model_csv(seed): @pytest.fixture -def ext_source(): +def ext_source() -> SourceDefinition: return make_source( "ext", "ext_raw", @@ -615,7 +620,7 @@ def ext_source(): @pytest.fixture -def ext_source_2(): +def ext_source_2() -> SourceDefinition: return make_source( "ext", "ext_raw", @@ -624,7 +629,7 @@ def ext_source_2(): @pytest.fixture -def ext_source_other(): +def ext_source_other() -> SourceDefinition: return make_source( "ext", "raw", @@ -633,7 +638,7 @@ def ext_source_other(): @pytest.fixture -def ext_source_other_2(): +def ext_source_other_2() -> SourceDefinition: return make_source( "ext", "raw", @@ -642,7 +647,7 @@ def ext_source_other_2(): @pytest.fixture -def ext_model(ext_source): +def ext_model(ext_source) -> ModelNode: return make_model( "ext", "ext_model", @@ -652,7 +657,7 @@ def ext_model(ext_source): @pytest.fixture -def union_model(seed, ext_source): +def union_model(seed, ext_source) -> ModelNode: return make_model( "pkg", "union_model", @@ -667,7 +672,7 @@ def union_model(seed, ext_source): @pytest.fixture -def versioned_model_v1(seed): +def versioned_model_v1(seed) -> ModelNode: return make_model( "pkg", "versioned_model", @@ -682,7 +687,7 @@ def versioned_model_v1(seed): @pytest.fixture -def versioned_model_v2(seed): +def versioned_model_v2(seed) -> ModelNode: return make_model( "pkg", "versioned_model", @@ -697,7 +702,7 @@ def versioned_model_v2(seed): @pytest.fixture -def versioned_model_v3(seed): +def versioned_model_v3(seed) -> ModelNode: return make_model( "pkg", "versioned_model", @@ -712,7 +717,7 @@ def versioned_model_v3(seed): @pytest.fixture -def versioned_model_v12_string(seed): +def versioned_model_v12_string(seed) -> ModelNode: return make_model( "pkg", "versioned_model", @@ -727,7 +732,7 @@ def versioned_model_v12_string(seed): @pytest.fixture -def versioned_model_v4_nested_dir(seed): +def versioned_model_v4_nested_dir(seed) -> ModelNode: return make_model( "pkg", "versioned_model", @@ -743,27 +748,27 @@ def versioned_model_v4_nested_dir(seed): @pytest.fixture -def table_id_unique(table_model): +def table_id_unique(table_model) -> GenericTestNode: return make_unique_test("pkg", table_model, "id") @pytest.fixture -def table_id_not_null(table_model): +def table_id_not_null(table_model) -> GenericTestNode: return make_not_null_test("pkg", table_model, "id") @pytest.fixture -def view_id_unique(view_model): +def view_id_unique(view_model) -> GenericTestNode: return make_unique_test("pkg", view_model, "id") @pytest.fixture -def ext_source_id_unique(ext_source): +def ext_source_id_unique(ext_source) -> GenericTestNode: return make_unique_test("ext", ext_source, "id") @pytest.fixture -def view_test_nothing(view_model): +def view_test_nothing(view_model) -> SingularTestNode: return make_singular_test( "pkg", "view_test_nothing", @@ -773,7 +778,7 @@ def view_test_nothing(view_model): @pytest.fixture -def unit_test_table_model(table_model): +def unit_test_table_model(table_model) -> UnitTestDefinition: return make_unit_test( "pkg", "unit_test_table_model", @@ -783,12 +788,12 @@ def unit_test_table_model(table_model): # Support dots as namespace separators @pytest.fixture -def namespaced_seed(): +def namespaced_seed() -> SeedNode: return make_seed("pkg", "mynamespace.seed") @pytest.fixture -def namespace_model(source): +def namespace_model(source) -> ModelNode: return make_model( "pkg", "mynamespace.ephemeral_model", @@ -799,7 +804,7 @@ def namespace_model(source): @pytest.fixture -def namespaced_union_model(seed, ext_source): +def namespaced_union_model(seed, ext_source) -> ModelNode: return make_model( "pkg", "mynamespace.union_model", @@ -910,7 +915,7 @@ def nodes( namespaced_seed, namespace_model, namespaced_union_model, -) -> list: +) -> List[ManifestNode]: return [ seed, ephemeral_model, @@ -953,7 +958,7 @@ def macros( macro_default_test_unique, macro_test_not_null, macro_default_test_not_null, -) -> list: +) -> List[Macro]: return [ macro_test_unique, macro_default_test_unique, @@ -963,25 +968,58 @@ def macros( @pytest.fixture -def unit_tests(unit_test_table_model) -> list: +def unit_tests(unit_test_table_model) -> List[UnitTestDefinition]: return [unit_test_table_model] @pytest.fixture -def metrics() -> list: +def metrics() -> List[Metric]: return [] @pytest.fixture -def semantic_models() -> list: +def semantic_models() -> List[SemanticModel]: return [] @pytest.fixture -def files() -> dict: +def files() -> Dict[str, AnySourceFile]: return {} +def make_manifest( + disabled: Dict[str, List[GraphMemberNode]] = {}, + docs: List[Documentation] = [], + exposures: List[Exposure] = [], + files: Dict[str, AnySourceFile] = {}, + groups: List[Group] = [], + macros: List[Macro] = [], + metrics: List[Metric] = [], + nodes: List[ModelNode] = [], + selectors: Dict[str, Any] = {}, + semantic_models: List[SemanticModel] = [], + sources: List[SourceDefinition] = [], + unit_tests: List[UnitTestDefinition] = [], +) -> Manifest: + manifest = Manifest( + nodes={n.unique_id: n for n in nodes}, + sources={s.unique_id: s for s in sources}, + macros={m.unique_id: m for m in macros}, + unit_tests={t.unique_id: t for t in unit_tests}, + semantic_models={s.unique_id: s for s in semantic_models}, + docs={d.unique_id: d for d in docs}, + files=files, + exposures={e.unique_id: e for e in exposures}, + metrics={m.unique_id: m for m in metrics}, + disabled=disabled, + selectors=selectors, + groups={g.unique_id: g for g in groups}, + metadata=ManifestMetadata(adapter_type="postgres", project_name="pkg"), + ) + manifest.build_parent_and_child_maps() + return manifest + + @pytest.fixture def manifest( metric, @@ -994,20 +1032,12 @@ def manifest( semantic_models, files, ) -> Manifest: - manifest = Manifest( - nodes={n.unique_id: n for n in nodes}, - sources={s.unique_id: s for s in sources}, - macros={m.unique_id: m for m in macros}, - unit_tests={t.unique_id: t for t in unit_tests}, - semantic_models={s.unique_id: s for s in semantic_models}, - docs={}, + return make_manifest( + nodes=nodes, + sources=sources, + macros=macros, + unit_tests=unit_tests, + semantic_models=semantic_models, files=files, - exposures={}, - metrics={m.unique_id: m for m in metrics}, - disabled={}, - selectors={}, - groups={}, - metadata=ManifestMetadata(adapter_type="postgres", project_name="pkg"), + metrics=metrics, ) - manifest.build_parent_and_child_maps() - return manifest