diff --git a/core/dbt/contracts/files.py b/core/dbt/contracts/files.py index 4ca2444274b..570f84a41ae 100644 --- a/core/dbt/contracts/files.py +++ b/core/dbt/contracts/files.py @@ -1,13 +1,12 @@ import hashlib import os from dataclasses import dataclass, field +from mashumaro.types import SerializableType from typing import List, Optional, Union, Dict, Any from dbt.dataclass_schema import dbtClassMixin, StrEnum -from dbt.exceptions import InternalException - -from .util import MacroKey, SourceKey +from .util import SourceKey MAXIMUM_SEED_SIZE = 1 * 1024 * 1024 @@ -23,7 +22,20 @@ class ParseFileType(StrEnum): Seed = 'seed' Documentation = 'docs' Schema = 'schema' - Hook = 'hook' + Hook = 'hook' # not a real filetype, from dbt_project.yml + + +parse_file_type_to_parser = { + ParseFileType.Macro: 'MacroParser', + ParseFileType.Model: 'ModelParser', + ParseFileType.Snapshot: 'SnapshotParser', + ParseFileType.Analysis: 'AnalysisParser', + ParseFileType.Test: 'DataTestParser', + ParseFileType.Seed: 'SeedParser', + ParseFileType.Documentation: 'DocumentationParser', + ParseFileType.Schema: 'SchemaParser', + ParseFileType.Hook: 'HookParser', +} @dataclass @@ -122,7 +134,7 @@ def original_file_path(self): @dataclass -class SourceFile(dbtClassMixin): +class BaseSourceFile(dbtClassMixin, SerializableType): """Define a source file in dbt""" path: Union[FilePath, RemoteFile] # the path information checksum: FileHash @@ -131,45 +143,57 @@ class SourceFile(dbtClassMixin): # Parse file type: i.e. which parser will process this file parse_file_type: Optional[ParseFileType] = None # we don't want to serialize this - _contents: Optional[str] = None + contents: Optional[str] = None # the unique IDs contained in this file - nodes: List[str] = field(default_factory=list) - docs: List[str] = field(default_factory=list) - macros: List[str] = field(default_factory=list) - sources: List[str] = field(default_factory=list) - exposures: List[str] = field(default_factory=list) - # any node patches in this file. The entries are names, not unique ids! - patches: List[str] = field(default_factory=list) - # any macro patches in this file. The entries are package, name pairs. - macro_patches: List[MacroKey] = field(default_factory=list) - # any source patches in this file. The entries are package, name pairs - source_patches: List[SourceKey] = field(default_factory=list) - # temporary - for schema source files only - dict_from_yaml: Optional[Dict[str, Any]] = None @property - def search_key(self) -> Optional[str]: + def file_id(self): if isinstance(self.path, RemoteFile): return None if self.checksum.name == 'none': return None - return self.path.search_key + return f'{self.project_name}://{self.path.original_file_path}' - @property - def contents(self) -> str: - if self._contents is None: - raise InternalException('SourceFile has no contents!') - return self._contents - - @contents.setter - def contents(self, value): - self._contents = value + def _serialize(self): + dct = self.to_dict() + if 'pp_files' in dct: + del dct['pp_files'] + if 'pp_test_index' in dct: + del dct['pp_test_index'] + return dct @classmethod - def empty(cls, path: FilePath) -> 'SourceFile': - self = cls(path=path, checksum=FileHash.empty()) - self.contents = '' - return self + def _deserialize(cls, dct: Dict[str, int]): + if dct['parse_file_type'] == 'schema': + # TODO: why are these keys even here + if 'pp_files' in dct: + del dct['pp_files'] + if 'pp_test_index' in dct: + del dct['pp_test_index'] + sf = SchemaSourceFile.from_dict(dct) + else: + sf = SourceFile.from_dict(dct) + return sf + + def __post_serialize__(self, dct): + dct = super().__post_serialize__(dct) + # remove empty lists to save space + dct_keys = list(dct.keys()) + for key in dct_keys: + if isinstance(dct[key], list) and not dct[key]: + del dct[key] + # remove contents. Schema files will still have 'dict_from_yaml' + # from the contents + if 'contents' in dct: + del dct['contents'] + return dct + + +@dataclass +class SourceFile(BaseSourceFile): + nodes: List[str] = field(default_factory=list) + docs: List[str] = field(default_factory=list) + macros: List[str] = field(default_factory=list) @classmethod def big_seed(cls, path: FilePath) -> 'SourceFile': @@ -178,8 +202,63 @@ def big_seed(cls, path: FilePath) -> 'SourceFile': self.contents = '' return self + # TODO: do this a different way. This remote file kludge isn't going + # to work long term @classmethod - def remote(cls, contents: str) -> 'SourceFile': - self = cls(path=RemoteFile(), checksum=FileHash.empty()) - self.contents = contents + def remote(cls, contents: str, project_name: str) -> 'SourceFile': + self = cls( + path=RemoteFile(), + checksum=FileHash.from_contents(contents), + project_name=project_name, + contents=contents, + ) return self + + +@dataclass +class SchemaSourceFile(BaseSourceFile): + dfy: Dict[str, Any] = field(default_factory=dict) + # these are in the manifest.nodes dictionary + tests: List[str] = field(default_factory=list) + sources: List[str] = field(default_factory=list) + exposures: List[str] = field(default_factory=list) + # node patches contain models, seeds, snapshots, analyses + ndp: List[str] = field(default_factory=list) + # any macro patches in this file by macro unique_id. + mcp: List[str] = field(default_factory=list) + # any source patches in this file. The entries are package, name pairs + # Patches are only against external sources. Sources can be + # created too, but those are in 'sources' + sop: List[SourceKey] = field(default_factory=list) + pp_dict: Optional[Dict[str, Any]] = None + pp_test_index: Optional[Dict[str, Any]] = None + + @property + def dict_from_yaml(self): + return self.dfy + + @property + def node_patches(self): + return self.ndp + + @property + def macro_patches(self): + return self.mcp + + @property + def source_patches(self): + return self.sop + + def __post_serialize__(self, dct): + dct = super().__post_serialize__(dct) + if 'pp_files' in dct: + del dct['pp_files'] + if 'pp_test_index' in dct: + del dct['pp_test_index'] + return dct + + def append_patch(self, yaml_key, unique_id): + self.node_patches.append(unique_id) + + +AnySourceFile = Union[SchemaSourceFile, SourceFile] diff --git a/core/dbt/contracts/graph/manifest.py b/core/dbt/contracts/graph/manifest.py index 7c90cb10ecc..9f9786e0885 100644 --- a/core/dbt/contracts/graph/manifest.py +++ b/core/dbt/contracts/graph/manifest.py @@ -1,11 +1,11 @@ -import abc import enum from dataclasses import dataclass, field from itertools import chain, islice +from mashumaro import DataClassMessagePackMixin from multiprocessing.synchronize import Lock from typing import ( Dict, List, Optional, Union, Mapping, MutableMapping, Any, Set, Tuple, - TypeVar, Callable, Iterable, Generic, cast, AbstractSet + TypeVar, Callable, Iterable, Generic, cast, AbstractSet, ClassVar ) from typing_extensions import Protocol from uuid import UUID @@ -19,22 +19,21 @@ UnpatchedSourceDefinition, ManifestNodes ) from dbt.contracts.graph.unparsed import SourcePatch -from dbt.contracts.files import SourceFile, FileHash, RemoteFile +from dbt.contracts.files import SourceFile, SchemaSourceFile, FileHash, AnySourceFile from dbt.contracts.util import ( - BaseArtifactMetadata, MacroKey, SourceKey, ArtifactMixin, schema_version + BaseArtifactMetadata, SourceKey, ArtifactMixin, schema_version ) from dbt.dataclass_schema import dbtClassMixin from dbt.exceptions import ( - InternalException, CompilationException, + CompilationException, raise_duplicate_resource_name, raise_compiler_error, warn_or_error, - raise_invalid_patch, raise_duplicate_patch_name, + raise_duplicate_patch_name, raise_duplicate_macro_patch_name, raise_duplicate_source_patch_name, ) from dbt.helper_types import PathSet from dbt.logger import GLOBAL_LOGGER as logger from dbt.node_types import NodeType from dbt.ui import line_wrap_message -from dbt import deprecations from dbt import flags from dbt import tracking import dbt.utils @@ -46,72 +45,70 @@ UniqueID = str -K_T = TypeVar('K_T') -V_T = TypeVar('V_T') +def find_unique_id_for_package(storage, key, package: Optional[PackageName]): + if key not in storage: + return None + pkg_dct: Mapping[PackageName, UniqueID] = storage[key] -class PackageAwareCache(Generic[K_T, V_T]): - def __init__(self, manifest: 'Manifest'): - self.storage: Dict[K_T, Dict[PackageName, UniqueID]] = {} - self._manifest = manifest - self.populate() - - @abc.abstractmethod - def populate(self): - pass - - @abc.abstractmethod - def perform_lookup(self, unique_id: UniqueID) -> V_T: - pass - - def find_cached_value( - self, key: K_T, package: Optional[PackageName] - ) -> Optional[V_T]: - unique_id = self.find_unique_id_for_package(key, package) - if unique_id is not None: - return self.perform_lookup(unique_id) + if package is None: + if not pkg_dct: + return None + else: + return next(iter(pkg_dct.values())) + elif package in pkg_dct: + return pkg_dct[package] + else: return None - def find_unique_id_for_package( - self, key: K_T, package: Optional[PackageName] - ) -> Optional[UniqueID]: - if key not in self.storage: - return None - pkg_dct: Mapping[PackageName, UniqueID] = self.storage[key] +class DocLookup(dbtClassMixin): + def __init__(self, manifest: 'Manifest'): + self.storage: Dict[str, Dict[PackageName, UniqueID]] = {} + self.populate(manifest) - if package is None: - if not pkg_dct: - return None - else: - return next(iter(pkg_dct.values())) - elif package in pkg_dct: - return pkg_dct[package] - else: - return None + def get_unique_id(self, key, package: Optional[PackageName]): + return find_unique_id_for_package(self.storage, key, package) + def find(self, key, package: Optional[PackageName], manifest: 'Manifest'): + unique_id = self.get_unique_id(key, package) + if unique_id is not None: + return self.perform_lookup(unique_id, manifest) + return None -class DocCache(PackageAwareCache[DocName, ParsedDocumentation]): def add_doc(self, doc: ParsedDocumentation): if doc.name not in self.storage: self.storage[doc.name] = {} self.storage[doc.name][doc.package_name] = doc.unique_id - def populate(self): - for doc in self._manifest.docs.values(): + def populate(self, manifest): + for doc in manifest.docs.values(): self.add_doc(doc) def perform_lookup( - self, unique_id: UniqueID + self, unique_id: UniqueID, manifest ) -> ParsedDocumentation: - if unique_id not in self._manifest.docs: + if unique_id not in manifest.docs: raise dbt.exceptions.InternalException( f'Doc {unique_id} found in cache but not found in manifest' ) - return self._manifest.docs[unique_id] + return manifest.docs[unique_id] + + +class SourceLookup(dbtClassMixin): + def __init__(self, manifest: 'Manifest'): + self.storage: Dict[Tuple[str, str], Dict[PackageName, UniqueID]] = {} + self.populate(manifest) + def get_unique_id(self, key, package: Optional[PackageName]): + return find_unique_id_for_package(self.storage, key, package) + + def find(self, key, package: Optional[PackageName], manifest: 'Manifest'): + unique_id = self.get_unique_id(key, package) + if unique_id is not None: + return self.perform_lookup(unique_id, manifest) + return None -class SourceCache(PackageAwareCache[SourceKey, ParsedSourceDefinition]): def add_source(self, source: ParsedSourceDefinition): key = (source.source_name, source.name) if key not in self.storage: @@ -119,47 +116,63 @@ def add_source(self, source: ParsedSourceDefinition): self.storage[key][source.package_name] = source.unique_id - def populate(self): - for source in self._manifest.sources.values(): + def populate(self, manifest): + for source in manifest.sources.values(): if hasattr(source, 'source_name'): self.add_source(source) def perform_lookup( - self, unique_id: UniqueID + self, unique_id: UniqueID, manifest: 'Manifest' ) -> ParsedSourceDefinition: - if unique_id not in self._manifest.sources: + if unique_id not in manifest.sources: raise dbt.exceptions.InternalException( f'Source {unique_id} found in cache but not found in manifest' ) - return self._manifest.sources[unique_id] + return manifest.sources[unique_id] -class RefableCache(PackageAwareCache[RefName, ManifestNode]): +class RefableLookup(dbtClassMixin): + # model, seed, snapshot + _lookup_types: ClassVar[set] = set(NodeType.refable()) + # refables are actually unique, so the Dict[PackageName, UniqueID] will # only ever have exactly one value, but doing 3 dict lookups instead of 1 # is not a big deal at all and retains consistency def __init__(self, manifest: 'Manifest'): - self._cached_types = set(NodeType.refable()) - super().__init__(manifest) + self.storage: Dict[str, Dict[PackageName, UniqueID]] = {} + self.populate(manifest) + + def get_unique_id(self, key, package: Optional[PackageName]): + return find_unique_id_for_package(self.storage, key, package) + + def find(self, key, package: Optional[PackageName], manifest: 'Manifest'): + unique_id = self.get_unique_id(key, package) + if unique_id is not None: + return self.perform_lookup(unique_id, manifest) + return None def add_node(self, node: ManifestNode): - if node.resource_type in self._cached_types: + if node.resource_type in self._lookup_types: if node.name not in self.storage: self.storage[node.name] = {} self.storage[node.name][node.package_name] = node.unique_id - def populate(self): - for node in self._manifest.nodes.values(): + def populate(self, manifest): + for node in manifest.nodes.values(): self.add_node(node) def perform_lookup( - self, unique_id: UniqueID + self, unique_id: UniqueID, manifest ) -> ManifestNode: - if unique_id not in self._manifest.nodes: + if unique_id not in manifest.nodes: raise dbt.exceptions.InternalException( f'Node {unique_id} found in cache but not found in manifest' ) - return self._manifest.nodes[unique_id] + return manifest.nodes[unique_id] + + +class AnalysisLookup(RefableLookup): + _lookup_types: ClassVar[set] = set(NodeType.Analysis) def _search_packages( @@ -514,39 +527,55 @@ def _find_macros_by_name( @dataclass class ManifestStateCheck(dbtClassMixin): - vars_hash: FileHash - profile_hash: FileHash - project_hashes: MutableMapping[str, FileHash] + vars_hash: FileHash = field(default_factory=FileHash.empty) + profile_hash: FileHash = field(default_factory=FileHash.empty) + project_hashes: MutableMapping[str, FileHash] = field(default_factory=dict) @dataclass -class Manifest(MacroMethods): +class Manifest(MacroMethods, DataClassMessagePackMixin, dbtClassMixin): """The manifest for the full graph, after parsing and during compilation. """ # These attributes are both positional and by keyword. If an attribute # is added it must all be added in the __reduce_ex__ method in the # args tuple in the right position. - nodes: MutableMapping[str, ManifestNode] - sources: MutableMapping[str, ParsedSourceDefinition] - macros: MutableMapping[str, ParsedMacro] - docs: MutableMapping[str, ParsedDocumentation] - exposures: MutableMapping[str, ParsedExposure] - selectors: MutableMapping[str, Any] - disabled: List[CompileResultNode] - files: MutableMapping[str, SourceFile] + nodes: MutableMapping[str, ManifestNode] = field(default_factory=dict) + sources: MutableMapping[str, ParsedSourceDefinition] = field(default_factory=dict) + macros: MutableMapping[str, ParsedMacro] = field(default_factory=dict) + docs: MutableMapping[str, ParsedDocumentation] = field(default_factory=dict) + exposures: MutableMapping[str, ParsedExposure] = field(default_factory=dict) + selectors: MutableMapping[str, Any] = field(default_factory=dict) + disabled: List[CompileResultNode] = field(default_factory=list) + files: MutableMapping[str, AnySourceFile] = field(default_factory=dict) metadata: ManifestMetadata = field(default_factory=ManifestMetadata) flat_graph: Dict[str, Any] = field(default_factory=dict) - state_check: Optional[ManifestStateCheck] = None + state_check: ManifestStateCheck = field(default_factory=ManifestStateCheck) # Moved from the ParseResult object - macro_patches: MutableMapping[MacroKey, ParsedMacroPatch] = field(default_factory=dict) - patches: MutableMapping[str, ParsedNodePatch] = field(default_factory=dict) source_patches: MutableMapping[SourceKey, SourcePatch] = field(default_factory=dict) # following is from ParseResult _disabled: MutableMapping[str, List[CompileResultNode]] = field(default_factory=dict) - _docs_cache: Optional[DocCache] = None - _sources_cache: Optional[SourceCache] = None - _refs_cache: Optional[RefableCache] = None - _lock: Lock = field(default_factory=flags.MP_CONTEXT.Lock) + _doc_lookup: Optional[DocLookup] = field( + default=None, metadata={'serialize': lambda x: None, 'deserialize': lambda x: None} + ) + _source_lookup: Optional[SourceLookup] = field( + default=None, metadata={'serialize': lambda x: None, 'deserialize': lambda x: None} + ) + _ref_lookup: Optional[RefableLookup] = field( + default=None, metadata={'serialize': lambda x: None, 'deserialize': lambda x: None} + ) + _analysis_lookup: Optional[AnalysisLookup] = field( + default=None, metadata={'serialize': lambda x: None, 'deserialize': lambda x: None} + ) + _lock: Lock = field( + default_factory=flags.MP_CONTEXT.Lock, + metadata={'serialize': lambda x: None, 'deserialize': lambda x: flags.MP_CONTEXT.Lock} + ) + + def __pre_serialize__(self): + # serialization won't work with anything except an empty source_patches because + # tuple keys are not supported, so ensure it's empty + self.source_patches = {} + return self def sync_update_node( self, new_node: NonSourceCompiledNode @@ -662,102 +691,59 @@ def get_resource_fqns(self) -> Mapping[str, PathSet]: resource_fqns[resource_type_plural].add(tuple(resource.fqn)) return resource_fqns - def add_nodes(self, new_nodes: Mapping[str, ManifestNode]): - """Add the given dict of new nodes to the manifest.""" - for unique_id, node in new_nodes.items(): - if unique_id in self.nodes: - raise_duplicate_resource_name(node, self.nodes[unique_id]) - self.nodes[unique_id] = node - # fixup the cache if it exists. - if self._refs_cache is not None: - if node.resource_type in NodeType.refable(): - self._refs_cache.add_node(node) - + # This is called by 'parse_patch' in the NodePatchParser def add_patch( - self, source_file: SourceFile, patch: ParsedNodePatch, + self, source_file: SchemaSourceFile, patch: ParsedNodePatch, ) -> None: + if patch.yaml_key in ['models', 'seeds', 'snapshots']: + unique_id = self.ref_lookup.get_unique_id(patch.name, None) + elif patch.yaml_key == 'analyses': + unique_id = self.analysis_lookup.get_unique_id(patch.name, None) + else: + raise dbt.exceptions.InternalException( + f'Unexpected yaml_key {patch.yaml_key} for patch in ' + f'file {source_file.path.original_file_path}' + ) + if unique_id is None: + # This will usually happen when a node is disabled + return + # patches can't be overwritten - if patch.name in self.patches: - raise_duplicate_patch_name(patch, self.patches[patch.name]) - self.patches[patch.name] = patch - self.get_file(source_file).patches.append(patch.name) + node = self.nodes.get(unique_id) + if node: + if node.patch_path: + package_name, existing_file_path = node.patch_path.split('://') + raise_duplicate_patch_name(patch, existing_file_path) + source_file.append_patch(patch.yaml_key, unique_id) + node.patch(patch) def add_macro_patch( - self, source_file: SourceFile, patch: ParsedMacroPatch, + self, source_file: SchemaSourceFile, patch: ParsedMacroPatch, ) -> None: # macros are fully namespaced - key = (patch.package_name, patch.name) - if key in self.macro_patches: - raise_duplicate_macro_patch_name(patch, self.macro_patches[key]) - self.macro_patches[key] = patch - self.get_file(source_file).macro_patches.append(key) + unique_id = f'macro.{patch.package_name}.{patch.name}' + macro = self.macros.get(unique_id) + if not macro: + warn_or_error( + f'WARNING: Found documentation for macro "{patch.name}" ' + f'which was not found' + ) + return + if macro.patch_path: + package_name, existing_file_path = macro.patch_path.split('://') + raise_duplicate_macro_patch_name(patch, existing_file_path) + source_file.macro_patches.append(unique_id) + macro.patch(patch) def add_source_patch( - self, source_file: SourceFile, patch: SourcePatch, + self, source_file: SchemaSourceFile, patch: SourcePatch, ) -> None: # source patches must be unique key = (patch.overrides, patch.name) if key in self.source_patches: raise_duplicate_source_patch_name(patch, self.source_patches[key]) self.source_patches[key] = patch - self.get_file(source_file).source_patches.append(key) - - def patch_macros(self) -> None: - for macro in self.macros.values(): - key = (macro.package_name, macro.name) - patch = self.macro_patches.pop(key, None) - if not patch: - continue - macro.patch(patch) - - if self.macro_patches: - for patch in self.macro_patches.values(): - warn_or_error( - f'WARNING: Found documentation for macro "{patch.name}" ' - f'which was not found' - ) - - def patch_nodes(self) -> None: - """Patch nodes with the given dict of patches. Note that this consumes - the input! - This relies on the fact that all nodes have unique _name_ fields, not - just unique unique_id fields. - """ - # because we don't have any mapping from node _names_ to nodes, and we - # only have the node name in the patch, we have to iterate over all the - # nodes looking for matching names. We could use a NameSearcher if we - # were ok with doing an O(n*m) search (one nodes scan per patch) - # Q: could we save patches by node unique_ids instead, or convert - # between names and node ids? - for node in self.nodes.values(): - patch = self.patches.pop(node.name, None) - if not patch: - continue - - expected_key = node.resource_type.pluralize() - if expected_key != patch.yaml_key: - if patch.yaml_key == 'models': - deprecations.warn( - 'models-key-mismatch', - patch=patch, node=node, expected_key=expected_key - ) - else: - raise_invalid_patch( - node, patch.yaml_key, patch.original_file_path - ) - - node.patch(patch) - - # If anything is left in self.patches, it means that the node for - # that patch wasn't found. - if self.patches: - for patch in self.patches.values(): - # since patches aren't nodes, we can't use the existing - # target_not_found warning - logger.debug(( - 'WARNING: Found documentation for resource "{}" which was ' - 'not found or is disabled').format(patch.name) - ) + source_file.source_patches.append(key) def get_used_schemas(self, resource_types=None): return frozenset({ @@ -808,14 +794,11 @@ def writable_manifest(self): parent_map=backward_edges, ) - # When 'to_dict' is called on the Manifest, it substitues a - # WritableManifest - def __pre_serialize__(self): - return self.writable_manifest() - def write(self, path): self.writable_manifest().write(path) + # Called in dbt.compilation.Linker.write_graph and + # dbt.graph.queue.get and ._include_in_cost def expect(self, unique_id: str) -> GraphMemberNode: if unique_id in self.nodes: return self.nodes[unique_id] @@ -830,29 +813,31 @@ def expect(self, unique_id: str) -> GraphMemberNode: ) @property - def docs_cache(self) -> DocCache: - if self._docs_cache is not None: - return self._docs_cache - cache = DocCache(self) - self._docs_cache = cache - return cache + def doc_lookup(self) -> DocLookup: + if self._doc_lookup is None: + self._doc_lookup = DocLookup(self) + return self._doc_lookup @property - def source_cache(self) -> SourceCache: - if self._sources_cache is not None: - return self._sources_cache - cache = SourceCache(self) - self._sources_cache = cache - return cache + def source_lookup(self) -> SourceLookup: + if self._source_lookup is None: + self._source_lookup = SourceLookup(self) + return self._source_lookup @property - def refs_cache(self) -> RefableCache: - if self._refs_cache is not None: - return self._refs_cache - cache = RefableCache(self) - self._refs_cache = cache - return cache + def ref_lookup(self) -> RefableLookup: + if self._ref_lookup is None: + self._ref_lookup = RefableLookup(self) + return self._ref_lookup + @property + def analysis_lookup(self) -> AnalysisLookup: + if self._analysis_lookup is None: + self._analysis_lookup = AnalysisLookup(self) + return self._analysis_lookup + + # Called by dbt.parser.manifest._resolve_refs_for_exposure + # and dbt.parser.manifest._process_refs_for_node def resolve_ref( self, target_model_name: str, @@ -868,7 +853,7 @@ def resolve_ref( current_project, node_package, target_model_package ) for pkg in candidates: - node = self.refs_cache.find_cached_value(target_model_name, pkg) + node = self.ref_lookup.find(target_model_name, pkg, self) if node is not None and node.config.enabled: return node @@ -883,6 +868,8 @@ def resolve_ref( return Disabled(disabled) return None + # Called by dbt.parser.manifest._resolve_sources_for_exposure + # and dbt.parser.manifest._process_source_for_node def resolve_source( self, target_source_name: str, @@ -897,7 +884,7 @@ def resolve_source( disabled: Optional[ParsedSourceDefinition] = None for pkg in candidates: - source = self.source_cache.find_cached_value(key, pkg) + source = self.source_lookup.find(key, pkg, self) if source is not None and source.config.enabled: return source @@ -910,6 +897,7 @@ def resolve_source( return Disabled(disabled) return None + # Called by DocsRuntimeContext.doc def resolve_doc( self, name: str, @@ -926,11 +914,12 @@ def resolve_doc( ) for pkg in candidates: - result = self.docs_cache.find_cached_value(name, pkg) + result = self.doc_lookup.find(name, pkg, self) if result is not None: return result return None + # Called by RunTask.defer_to_manifest def merge_from_artifact( self, adapter, @@ -964,13 +953,6 @@ def merge_from_artifact( ) # Methods that were formerly in ParseResult - def get_file(self, source_file: SourceFile) -> SourceFile: - key = source_file.search_key - if key is None: - return source_file - if key not in self.files: - self.files[key] = source_file - return self.files[key] def add_macro(self, source_file: SourceFile, macro: ParsedMacro): if macro.unique_id in self.macros: @@ -997,10 +979,10 @@ def add_macro(self, source_file: SourceFile, macro: ParsedMacro): raise_compiler_error(msg) self.macros[macro.unique_id] = macro - self.get_file(source_file).macros.append(macro.unique_id) + source_file.macros.append(macro.unique_id) def has_file(self, source_file: SourceFile) -> bool: - key = source_file.search_key + key = source_file.file_id if key is None: return False if key not in self.files: @@ -1009,26 +991,29 @@ def has_file(self, source_file: SourceFile) -> bool: return my_checksum == source_file.checksum def add_source( - self, source_file: SourceFile, source: UnpatchedSourceDefinition + self, source_file: SchemaSourceFile, source: UnpatchedSourceDefinition ): # sources can't be overwritten! _check_duplicates(source, self.sources) self.sources[source.unique_id] = source # type: ignore - self.get_file(source_file).sources.append(source.unique_id) + source_file.sources.append(source.unique_id) def add_node_nofile(self, node: ManifestNodes): # nodes can't be overwritten! _check_duplicates(node, self.nodes) self.nodes[node.unique_id] = node - def add_node(self, source_file: SourceFile, node: ManifestNodes): + def add_node(self, source_file: AnySourceFile, node: ManifestNodes): self.add_node_nofile(node) - self.get_file(source_file).nodes.append(node.unique_id) + if isinstance(source_file, SchemaSourceFile): + source_file.tests.append(node.unique_id) + else: + source_file.nodes.append(node.unique_id) - def add_exposure(self, source_file: SourceFile, exposure: ParsedExposure): + def add_exposure(self, source_file: SchemaSourceFile, exposure: ParsedExposure): _check_duplicates(exposure, self.exposures) self.exposures[exposure.unique_id] = exposure - self.get_file(source_file).exposures.append(exposure.unique_id) + source_file.exposures.append(exposure.unique_id) def add_disabled_nofile(self, node: CompileResultNode): if node.unique_id in self._disabled: @@ -1036,137 +1021,18 @@ def add_disabled_nofile(self, node: CompileResultNode): else: self._disabled[node.unique_id] = [node] - def add_disabled(self, source_file: SourceFile, node: CompileResultNode): + def add_disabled(self, source_file: AnySourceFile, node: CompileResultNode): self.add_disabled_nofile(node) - self.get_file(source_file).nodes.append(node.unique_id) + if isinstance(source_file, SchemaSourceFile): + source_file.tests.append(node.unique_id) + else: + source_file.nodes.append(node.unique_id) def add_doc(self, source_file: SourceFile, doc: ParsedDocumentation): _check_duplicates(doc, self.docs) self.docs[doc.unique_id] = doc - self.get_file(source_file).docs.append(doc.unique_id) + source_file.docs.append(doc.unique_id) - def _get_disabled( - self, - unique_id: str, - match_file: SourceFile, - ) -> List[CompileResultNode]: - if unique_id not in self._disabled: - raise InternalException( - 'called _get_disabled with id={}, but it does not exist' - .format(unique_id) - ) - return [ - n for n in self._disabled[unique_id] - if n.original_file_path == match_file.path.original_file_path - ] - - # This is only used by 'sanitized_update' which processes "old_manifest" - def _process_node( - self, - node_id: str, - source_file: SourceFile, - old_file: SourceFile, - old_manifest: Any, - ) -> None: - """Nodes are a special kind of complicated - there can be multiple - with the same name, as long as all but one are disabled. - - Only handle nodes where the matching node has the same resource type - as the current parser. - """ - source_path = source_file.path.original_file_path - found: bool = False - if node_id in old_manifest.nodes: - old_node = old_manifest.nodes[node_id] - if old_node.original_file_path == source_path: - self.add_node(source_file, old_node) - found = True - - if node_id in old_manifest._disabled: - matches = old_manifest._get_disabled(node_id, source_file) - for match in matches: - self.add_disabled(source_file, match) - found = True - - if not found: - raise CompilationException( - 'Expected to find "{}" in cached "manifest.nodes" or ' - '"manifest.disabled" based on cached file information: {}!' - .format(node_id, old_file) - ) - - # This is called by ManifestLoader._get_cached/parse_with_cache, - # which handles updating the ManifestLoader results with information - # from the "old_manifest", i.e. the pickle file if the checksums are - # the same. - def sanitized_update( - self, - source_file: SourceFile, - old_manifest: Any, - resource_type: NodeType, - ) -> bool: - - if isinstance(source_file.path, RemoteFile): - return False - - old_file = old_manifest.get_file(source_file) - for doc_id in old_file.docs: - doc = _expect_value(doc_id, old_manifest.docs, old_file, "docs") - self.add_doc(source_file, doc) - - for macro_id in old_file.macros: - macro = _expect_value( - macro_id, old_manifest.macros, old_file, "macros" - ) - self.add_macro(source_file, macro) - - for source_id in old_file.sources: - source = _expect_value( - source_id, old_manifest.sources, old_file, "sources" - ) - self.add_source(source_file, source) - - # because we know this is how we _parsed_ the node, we can safely - # assume if it's disabled it was done by the project or file, and - # we can keep our old data - # the node ID could be in old_manifest.disabled AND in old_manifest.nodes. - # In that case, we have to make sure the path also matches. - for node_id in old_file.nodes: - # cheat: look at the first part of the node ID and compare it to - # the parser resource type. On a mismatch, bail out. - if resource_type != node_id.split('.')[0]: - continue - self._process_node(node_id, source_file, old_file, old_manifest) - - for exposure_id in old_file.exposures: - exposure = _expect_value( - exposure_id, old_manifest.exposures, old_file, "exposures" - ) - self.add_exposure(source_file, exposure) - - # Note: There shouldn't be any patches in here after the cleanup. - # The pickled Manifest should have had all patches applied. - patched = False - for name in old_file.patches: - patch = _expect_value( - name, old_manifest.patches, old_file, "patches" - ) - self.add_patch(source_file, patch) - patched = True - if patched: - self.get_file(source_file).patches.sort() - - macro_patched = False - for key in old_file.macro_patches: - macro_patch = _expect_value( - key, old_manifest.macro_patches, old_file, "macro_patches" - ) - self.add_macro_patch(source_file, macro_patch) - macro_patched = True - if macro_patched: - self.get_file(source_file).macro_patches.sort() - - return True # end of methods formerly in ParseResult # Provide support for copy.deepcopy() - we just need to avoid the lock! @@ -1189,13 +1055,11 @@ def __reduce_ex__(self, protocol): self.metadata, self.flat_graph, self.state_check, - self.macro_patches, - self.patches, self.source_patches, self._disabled, - self._docs_cache, - self._sources_cache, - self._refs_cache, + self._doc_lookup, + self._source_lookup, + self._ref_lookup, ) return self.__class__, args @@ -1266,6 +1130,10 @@ def _check_duplicates( raise_duplicate_resource_name(value, src[value.unique_id]) +K_T = TypeVar('K_T') +V_T = TypeVar('V_T') + + def _expect_value( key: K_T, src: Mapping[K_T, V_T], old_file: SourceFile, name: str ) -> V_T: diff --git a/core/dbt/contracts/graph/model_config.py b/core/dbt/contracts/graph/model_config.py index 580b24f8f03..d61ecfec6aa 100644 --- a/core/dbt/contracts/graph/model_config.py +++ b/core/dbt/contracts/graph/model_config.py @@ -2,14 +2,13 @@ from enum import Enum from itertools import chain from typing import ( - Any, List, Optional, Dict, MutableMapping, Union, Type, - TypeVar, Callable, + Any, List, Optional, Dict, Union, Type, TypeVar ) from dbt.dataclass_schema import ( dbtClassMixin, ValidationError, register_pattern, ) from dbt.contracts.graph.unparsed import AdditionalPropertiesAllowed -from dbt.exceptions import CompilationException, InternalException +from dbt.exceptions import InternalException from dbt.contracts.util import Replaceable, list_str from dbt import hooks from dbt.node_types import NodeType @@ -182,53 +181,29 @@ class Hook(dbtClassMixin, Replaceable): @dataclass class BaseConfig( - AdditionalPropertiesAllowed, Replaceable, MutableMapping[str, Any] + AdditionalPropertiesAllowed, Replaceable ): - # Implement MutableMapping so this config will behave as some macros expect - # during parsing (notably, syntax like `{{ node.config['schema'] }}`) + + # enable syntax like: config['key'] def __getitem__(self, key): - """Handle parse-time use of `config` as a dictionary, making the extra - values available during parsing. - """ + return self.get(key) + + # like doing 'get' on a dictionary + def get(self, key, default=None): if hasattr(self, key): return getattr(self, key) - else: + elif key in self._extra: return self._extra[key] + else: + return default + # enable syntax like: config['key'] = value def __setitem__(self, key, value): if hasattr(self, key): setattr(self, key, value) else: self._extra[key] = value - def __delitem__(self, key): - if hasattr(self, key): - msg = ( - 'Error, tried to delete config key "{}": Cannot delete ' - 'built-in keys' - ).format(key) - raise CompilationException(msg) - else: - del self._extra[key] - - def _content_iterator(self, include_condition: Callable[[Field], bool]): - seen = set() - for fld, _ in self._get_fields(): - seen.add(fld.name) - if include_condition(fld): - yield fld.name - - for key in self._extra: - if key not in seen: - seen.add(key) - yield key - - def __iter__(self): - yield from self._content_iterator(include_condition=lambda f: True) - - def __len__(self): - return len(self._get_fields()) + len(self._extra) - @staticmethod def compare_key( unrendered: Dict[str, Any], diff --git a/core/dbt/contracts/graph/parsed.py b/core/dbt/contracts/graph/parsed.py index c6d8cd9fd1a..58c6990fc43 100644 --- a/core/dbt/contracts/graph/parsed.py +++ b/core/dbt/contracts/graph/parsed.py @@ -1,5 +1,7 @@ import os +import time from dataclasses import dataclass, field +from mashumaro.types import SerializableType from pathlib import Path from typing import ( Optional, @@ -131,7 +133,7 @@ def patch(self, patch: 'ParsedNodePatch'): """Given a ParsedNodePatch, add the new information to the node.""" # explicitly pick out the parts to update so we don't inadvertently # step on the model name or anything - self.patch_path: Optional[str] = patch.original_file_path + self.patch_path: Optional[str] = patch.file_id() self.description = patch.description self.columns = patch.columns self.meta = patch.meta @@ -183,6 +185,7 @@ class ParsedNodeDefaults(ParsedNodeMandatory): build_path: Optional[str] = None deferred: bool = False unrendered_config: Dict[str, Any] = field(default_factory=dict) + created_at: int = field(default_factory=lambda: int(time.time())) def write_node(self, target_path: str, subdirectory: str, payload: str): if (os.path.basename(self.path) == @@ -204,7 +207,39 @@ def write_node(self, target_path: str, subdirectory: str, payload: str): @dataclass -class ParsedNode(ParsedNodeDefaults, ParsedNodeMixins): +class ParsedNode(ParsedNodeDefaults, ParsedNodeMixins, SerializableType): + + def _serialize(self): + return self.to_dict() + + @classmethod + def _deserialize(cls, dct: Dict[str, int]): + # The serialized ParsedNodes do not differ from each other + # in fields that would allow 'from_dict' to distinguis + # between them. + resource_type = dct['resource_type'] + if resource_type == 'model': + return ParsedModelNode.from_dict(dct) + elif resource_type == 'analysis': + return ParsedAnalysisNode.from_dict(dct) + elif resource_type == 'seed': + return ParsedSeedNode.from_dict(dct) + elif resource_type == 'rpc': + return ParsedRPCNode.from_dict(dct) + elif resource_type == 'test': + if 'test_metadata' in dct: + return ParsedSchemaTestNode.from_dict(dct) + else: + return ParsedDataTestNode.from_dict(dct) + elif resource_type == 'operation': + return ParsedHookNode.from_dict(dct) + elif resource_type == 'seed': + return ParsedSeedNode.from_dict(dct) + elif resource_type == 'snapshot': + return ParsedSnapshotNode.from_dict(dct) + else: + return cls.from_dict(dct) + def _persist_column_docs(self) -> bool: return bool(self.config.persist_docs.get('columns')) @@ -442,12 +477,13 @@ class ParsedMacro(UnparsedBaseNode, HasUniqueID): docs: Docs = field(default_factory=Docs) patch_path: Optional[str] = None arguments: List[MacroArgument] = field(default_factory=list) + created_at: int = field(default_factory=lambda: int(time.time())) def local_vars(self): return {} def patch(self, patch: ParsedMacroPatch): - self.patch_path: Optional[str] = patch.original_file_path + self.patch_path: Optional[str] = patch.file_id() self.description = patch.description self.meta = patch.meta self.docs = patch.docs @@ -568,6 +604,7 @@ class ParsedSourceDefinition( patch_path: Optional[Path] = None unrendered_config: Dict[str, Any] = field(default_factory=dict) relation_name: Optional[str] = None + created_at: int = field(default_factory=lambda: int(time.time())) def same_database_representation( self, other: 'ParsedSourceDefinition' @@ -672,6 +709,7 @@ class ParsedExposure(UnparsedBaseNode, HasUniqueID, HasFqn): depends_on: DependsOn = field(default_factory=DependsOn) refs: List[List[str]] = field(default_factory=list) sources: List[List[str]] = field(default_factory=list) + created_at: int = field(default_factory=lambda: int(time.time())) @property def depends_on_nodes(self): diff --git a/core/dbt/contracts/graph/unparsed.py b/core/dbt/contracts/graph/unparsed.py index e6656c92a40..a2439c6dc63 100644 --- a/core/dbt/contracts/graph/unparsed.py +++ b/core/dbt/contracts/graph/unparsed.py @@ -25,6 +25,9 @@ class UnparsedBaseNode(dbtClassMixin, Replaceable): path: str original_file_path: str + def file_id(self): + return f'{self.package_name}://{self.original_file_path}' + @dataclass class HasSQL: @@ -116,6 +119,9 @@ class HasYamlMetadata(dbtClassMixin): yaml_key: str package_name: str + def file_id(self): + return f'{self.package_name}://{self.original_file_path}' + @dataclass class UnparsedAnalysisUpdate(HasColumnDocs, HasDocs, HasYamlMetadata): diff --git a/core/dbt/contracts/util.py b/core/dbt/contracts/util.py index 5dd329d193e..ef4a8fca5a5 100644 --- a/core/dbt/contracts/util.py +++ b/core/dbt/contracts/util.py @@ -14,7 +14,6 @@ from dbt.tracking import get_invocation_id from dbt.dataclass_schema import dbtClassMixin -MacroKey = Tuple[str, str] SourceKey = Tuple[str, str] diff --git a/core/dbt/exceptions.py b/core/dbt/exceptions.py index 82cd9267bc4..500b6bba812 100644 --- a/core/dbt/exceptions.py +++ b/core/dbt/exceptions.py @@ -845,11 +845,11 @@ def _fix_dupe_msg(path_1: str, path_2: str, name: str, type_name: str) -> str: ) -def raise_duplicate_patch_name(patch_1, patch_2): +def raise_duplicate_patch_name(patch_1, existing_patch_path): name = patch_1.name fix = _fix_dupe_msg( patch_1.original_file_path, - patch_2.original_file_path, + existing_patch_path, name, 'resource', ) @@ -860,12 +860,12 @@ def raise_duplicate_patch_name(patch_1, patch_2): ) -def raise_duplicate_macro_patch_name(patch_1, patch_2): +def raise_duplicate_macro_patch_name(patch_1, existing_patch_path): package_name = patch_1.package_name name = patch_1.name fix = _fix_dupe_msg( patch_1.original_file_path, - patch_2.original_file_path, + existing_patch_path, name, 'macros' ) diff --git a/core/dbt/parser/macros.py b/core/dbt/parser/macros.py index 2b5b8f420ff..4bffd99be1e 100644 --- a/core/dbt/parser/macros.py +++ b/core/dbt/parser/macros.py @@ -92,12 +92,8 @@ def parse_unparsed_macros( yield node def parse_file(self, block: FileBlock): - # mark the file as seen, even if there are no macros in it - self.manifest.get_file(block.file) source_file = block.file - original_file_path = source_file.path.original_file_path - logger.debug("Parsing {}".format(original_file_path)) # this is really only used for error messages diff --git a/core/dbt/parser/manifest.py b/core/dbt/parser/manifest.py index 04eb6619b1b..ad32f316141 100644 --- a/core/dbt/parser/manifest.py +++ b/core/dbt/parser/manifest.py @@ -1,7 +1,6 @@ from dataclasses import dataclass from dataclasses import field import os -import pickle from typing import ( Dict, Optional, Mapping, Callable, Any, List, Type, Union ) @@ -27,6 +26,7 @@ from dbt.context.base import generate_base_context from dbt.contracts.files import FileHash, ParseFileType from dbt.parser.read_files import read_files, load_source_file +from dbt.parser.partial import PartialParsing from dbt.contracts.graph.compiled import ManifestNode from dbt.contracts.graph.manifest import ( Manifest, Disabled, MacroManifest, ManifestStateCheck @@ -42,7 +42,7 @@ get_source_not_found_or_disabled_msg, warn_or_error, ) -from dbt.parser.base import BaseParser, Parser +from dbt.parser.base import Parser from dbt.parser.analysis import AnalysisParser from dbt.parser.data_test import DataTestParser from dbt.parser.docs import DocumentationParser @@ -53,13 +53,13 @@ from dbt.parser.search import FileBlock from dbt.parser.seeds import SeedParser from dbt.parser.snapshots import SnapshotParser -from dbt.parser.sources import patch_sources +from dbt.parser.sources import SourcePatcher from dbt.ui import warning_tag from dbt.version import __version__ from dbt.dataclass_schema import dbtClassMixin -PARTIAL_PARSE_FILE_NAME = 'partial_parse.pickle' +PARTIAL_PARSE_FILE_NAME = 'partial_parse.msgpack' PARSING_STATE = DbtProcessState('parsing') DEFAULT_PARTIAL_PARSE = False @@ -112,8 +112,9 @@ def __init__( ) -> None: self.root_project: RuntimeConfig = root_project self.all_projects: Mapping[str, Project] = all_projects - self.manifest: Manifest = Manifest({}, {}, {}, {}, {}, {}, [], {}) + self.manifest: Manifest = Manifest() self.manifest.metadata = root_project.get_metadata() + self.started_at = int(time.time()) # This is a MacroQueryStringSetter callable, which is called # later after we set the MacroManifest in the adapter. It sets # up the query headers. @@ -123,14 +124,19 @@ def __init__( else: self.macro_hook = macro_hook - # State check determines whether the old_manifest and the current + self._perf_info = self.build_perf_info() + + # State check determines whether the saved_manifest and the current # manifest match well enough to do partial parsing self.manifest.state_check = self.build_manifest_state_check() + # We need to know if we're actually partially parsing. It could + # have been enabled, but not happening because of some issue. + self.partially_parsing = False self._perf_info = self.build_perf_info() # This is a saved manifest from a previous run that's used for partial parsing - self.old_manifest: Optional[Manifest] = self.read_saved_manifest() + self.saved_manifest: Optional[Manifest] = self.read_manifest_for_partial_parse() # This is the method that builds a complete manifest. We sometimes # use an abbreviated process in tests. @@ -156,12 +162,8 @@ def get_full_manifest( projects = config.load_dependencies() loader = ManifestLoader(config, projects, macro_hook) - loader.load() - # The goal is to move partial parse writing to after update_manifest - loader.write_manifest_for_partial_parse() - manifest = loader.update_manifest() - # Move write_manifest_for_partial_parse here + manifest = loader.load() _check_manifest(manifest, config) manifest.build_flat_graph() @@ -181,9 +183,6 @@ def get_full_manifest( # This is where the main action happens def load(self): - if self.old_manifest is not None: - logger.debug('Got an acceptable saved parse result') - # Read files creates a dictionary of projects to a dictionary # of parsers to lists of file strings. The file strings are # used to get the SourceFiles from the manifest files. @@ -197,37 +196,118 @@ def load(self): read_files(project, self.manifest.files, project_parser_files) self._perf_info.read_files_elapsed = (time.perf_counter() - start_read_files) - # We need to parse the macros first, so they're resolvable when - # the other files are loaded - start_load_macros = time.perf_counter() - for project in self.all_projects.values(): - parser = MacroParser(project, self.manifest) - parser_files = project_parser_files[project.project_name] - for search_key in parser_files['MacroParser']: - block = FileBlock(self.manifest.files[search_key]) - self.parse_with_cache(block, parser) - self.reparse_macros() - # This is where a loop over self.manifest.macros should be performed - # to set the 'depends_on' information from static rendering. - self._perf_info.load_macros_elapsed = (time.perf_counter() - start_load_macros) - - # Load the rest of the files except for schema yaml files - parser_types: List[Type[Parser]] = [ - ModelParser, SnapshotParser, AnalysisParser, DataTestParser, - SeedParser, DocumentationParser, HookParser] - for project in self.all_projects.values(): - if project.project_name not in project_parser_files: - continue - self.parse_project(project, project_parser_files[project.project_name], parser_types) + skip_parsing = False + if self.saved_manifest is not None: + partial_parsing = PartialParsing(self.saved_manifest, self.manifest.files) + skip_parsing = partial_parsing.skip_parsing() + if not skip_parsing: + # files are different, we need to create a new set of + # project_parser_files. + project_parser_files = partial_parsing.get_parsing_files() + self.manifest = self.saved_manifest + self.partially_parsing = True + + if skip_parsing: + logger.info("Partial parsing enabled, no changes found, skipping parsing") + self.manifest = self.saved_manifest - # Load yaml files - parser_types = [SchemaParser] - for project in self.all_projects.values(): - if project.project_name not in project_parser_files: - continue - self.parse_project(project, project_parser_files[project.project_name], parser_types) + else: + # Load Macros + # We need to parse the macros first, so they're resolvable when + # the other files are loaded + start_load_macros = time.perf_counter() + for project in self.all_projects.values(): + if project.project_name not in project_parser_files: + continue + parser_files = project_parser_files[project.project_name] + if 'MacroParser' not in parser_files: + continue + parser = MacroParser(project, self.manifest) + for file_id in parser_files['MacroParser']: + block = FileBlock(self.manifest.files[file_id]) + parser.parse_file(block) + # Look at changed macros and update the macro.depends_on.macros + self.macro_depends_on() + self._perf_info.load_macros_elapsed = (time.perf_counter() - start_load_macros) + + # Now that the macros are parsed, parse the rest of the files. + # This is currently done on a per project basis. + start_parse_projects = time.perf_counter() + + # Load the rest of the files except for schema yaml files + parser_types: List[Type[Parser]] = [ + ModelParser, SnapshotParser, AnalysisParser, DataTestParser, + SeedParser, DocumentationParser, HookParser] + for project in self.all_projects.values(): + if project.project_name not in project_parser_files: + continue + self.parse_project( + project, + project_parser_files[project.project_name], + parser_types + ) + + # Now that we've loaded most of the nodes (except for schema tests and sources) + # load up the Lookup objects to resolve them by name, so the SourceFiles store + # the unique_id instead of the name. Sources are loaded from yaml files, so + # aren't in place yet + self.manifest.ref_lookup + self.manifest.doc_lookup + + # Load yaml files + parser_types = [SchemaParser] + for project in self.all_projects.values(): + if project.project_name not in project_parser_files: + continue + self.parse_project( + project, + project_parser_files[project.project_name], + parser_types + ) + + self._perf_info.parse_project_elapsed = (time.perf_counter() - start_parse_projects) + + # patch_sources converts the UnparsedSourceDefinitions in the + # Manifest.sources to ParsedSourceDefinition via 'patch_source' + # in SourcePatcher + start_patch = time.perf_counter() + patcher = SourcePatcher(self.root_project, self.manifest) + patcher.construct_sources() + self.manifest.sources = patcher.sources + self._perf_info.patch_sources_elapsed = ( + time.perf_counter() - start_patch + ) - # Parse every file in this project, except macros (already done) + # ParseResults had a 'disabled' attribute which was a dictionary + # which is now named '_disabled'. This used to copy from + # ParseResults to the Manifest. + # TODO: normalize to only one disabled + disabled = [] + for value in self.manifest._disabled.values(): + disabled.extend(value) + self.manifest.disabled = disabled + + # copy the selectors from the root_project to the manifest + self.manifest.selectors = self.root_project.manifest_selectors + + # update the refs, sources, and docs + # These check the created_at time on the nodes to + # determine whether they need processinga. + start_process = time.perf_counter() + self.process_sources(self.root_project.project_name) + self.process_refs(self.root_project.project_name) + self.process_docs(self.root_project) + self._perf_info.process_manifest_elapsed = ( + time.perf_counter() - start_process + ) + + # write out the fully parsed manifest + self.write_manifest_for_partial_parse() + + return self.manifest + + # Parse the files in the 'parser_files' dictionary, for parsers listed in + # 'parser_types' def parse_project( self, project: Project, @@ -255,7 +335,10 @@ def parse_project( for file_id in parser_files[parser_name]: block = FileBlock(self.manifest.files[file_id]) if isinstance(parser, SchemaParser): - dct = block.file.dict_from_yaml + if self.partially_parsing: + dct = block.file.pp_dict + else: + dct = block.file.dict_from_yaml parser.parse_file(block, dct=dct) else: parser.parse_file(block) @@ -271,6 +354,9 @@ def parse_project( # HookParser doesn't run from loaded files, just dbt_project.yml, # so do separately + # This shouldn't need to be parsed again if we're starting from + # a saved manifest, because that won't be allowed if dbt_project.yml + # changed, but leave for now. if HookParser in parser_types: hook_parser = HookParser(project, self.manifest, self.root_project) path = hook_parser.get_path() @@ -289,7 +375,7 @@ def parse_project( # Loop through macros in the manifest and statically parse # the 'macro_sql' to find depends_on.macros - def reparse_macros(self): + def macro_depends_on(self): internal_package_names = get_adapter_package_names( self.root_project.credentials.type ) @@ -300,6 +386,8 @@ def reparse_macros(self): ) base_ctx = generate_base_context({}) for macro in self.manifest.macros.values(): + if macro.created_at < self.started_at: + continue possible_macro_calls = statically_extract_macro_calls(macro.macro_sql, base_ctx) for macro_name in possible_macro_calls: # adapter.dispatch calls can generate a call with the same name as the macro @@ -313,74 +401,34 @@ def reparse_macros(self): if dep_macro_id: macro.depends_on.add_macro(dep_macro_id) # will check for dupes - # This is where we use the partial-parse state from the - # pickle file (if it exists) - def parse_with_cache( - self, - block: FileBlock, - parser: BaseParser, - ) -> None: - # _get_cached actually copies the nodes, etc, that were - # generated from the file to the results, in 'sanitized_update' - if not self._get_cached(block, parser): - parser.parse_file(block) - - # check if we have a stored parse file, then check if - # file checksums are the same or not and either return - # the old ... stuff or return false (not cached) - def _get_cached( - self, - block: FileBlock, - parser: BaseParser, - ) -> bool: - # TODO: handle multiple parsers w/ same files, by - # tracking parser type vs node type? Or tracking actual - # parser type during parsing? - if self.old_manifest is None: - return False - # The 'has_file' method is where we check to see if - # the checksum of the old file is the same as the new - # file. If the checksum is different, 'has_file' returns - # false. If it's the same, the file and the things that - # were generated from it are used. - if self.old_manifest.has_file(block.file): - return self.manifest.sanitized_update( - block.file, self.old_manifest, parser.resource_type - ) - return False - def write_manifest_for_partial_parse(self): path = os.path.join(self.root_project.target_path, PARTIAL_PARSE_FILE_NAME) - make_directory(self.root_project.target_path) - with open(path, 'wb') as fp: - pickle.dump(self.manifest, fp) + try: + manifest_msgpack = self.manifest.to_msgpack() + make_directory(os.path.dirname(path)) + with open(path, 'wb') as fp: + fp.write(manifest_msgpack) + except Exception: + raise def matching_parse_results(self, manifest: Manifest) -> bool: """Compare the global hashes of the read-in parse results' values to the known ones, and return if it is ok to re-use the results. """ - try: - if manifest.metadata.dbt_version != __version__: - logger.debug( - 'dbt version mismatch: {} != {}, cache invalidated' - .format(manifest.metadata.dbt_version, __version__) - ) - return False - except AttributeError as exc: - logger.debug(f"malformed result file, cache invalidated: {exc}") - return False - valid = True - if not self.manifest.state_check or not manifest.state_check: - return False - + if manifest.metadata.dbt_version != __version__: + logger.info("Unable to do partial parsing because of a dbt version mismatch") + return False # If the version is wrong, the other checks might not work if self.manifest.state_check.vars_hash != manifest.state_check.vars_hash: - logger.debug('vars hash mismatch, cache invalidated') + logger.info("Unable to do partial parsing because config vars, " + "config profile, or config target have changed") valid = False if self.manifest.state_check.profile_hash != manifest.state_check.profile_hash: - logger.debug('profile hash mismatch, cache invalidated') + # Note: This should be made more granular. We shouldn't need to invalidate + # partial parsing if a non-used profile section has changed. + logger.info("Unable to do partial parsing because profile has changed") valid = False missing_keys = { @@ -388,21 +436,15 @@ def matching_parse_results(self, manifest: Manifest) -> bool: if k not in manifest.state_check.project_hashes } if missing_keys: - logger.debug( - 'project hash mismatch: values missing, cache invalidated: {}' - .format(missing_keys) - ) + logger.info("Unable to do partial parsing because a project dependency has been added") valid = False for key, new_value in self.manifest.state_check.project_hashes.items(): if key in manifest.state_check.project_hashes: old_value = manifest.state_check.project_hashes[key] if new_value != old_value: - logger.debug( - 'For key {}, hash mismatch ({} -> {}), cache ' - 'invalidated' - .format(key, old_value, new_value) - ) + logger.info("Unable to do partial parsing because " + "a project config has changed") valid = False return valid @@ -416,7 +458,7 @@ def _partial_parse_enabled(self): else: return DEFAULT_PARTIAL_PARSE - def read_saved_manifest(self) -> Optional[Manifest]: + def read_manifest_for_partial_parse(self) -> Optional[Manifest]: if not self._partial_parse_enabled(): logger.debug('Partial parsing not enabled') return None @@ -426,7 +468,8 @@ def read_saved_manifest(self) -> Optional[Manifest]: if os.path.exists(path): try: with open(path, 'rb') as fp: - manifest: Manifest = pickle.load(fp) + manifest_mp = fp.read() + manifest: Manifest = Manifest.from_msgpack(manifest_mp) # type: ignore # keep this check inside the try/except in case something about # the file has changed in weird ways, perhaps due to being a # different version of dbt @@ -440,49 +483,6 @@ def read_saved_manifest(self) -> Optional[Manifest]: ) return None - # This find the sources, refs, and docs and resolves them - # for nodes and exposures - def process_manifest(self): - project_name = self.root_project.project_name - process_sources(self.manifest, project_name) - process_refs(self.manifest, project_name) - process_docs(self.manifest, self.root_project) - - def update_manifest(self) -> Manifest: - start_patch = time.perf_counter() - # patch_sources converts the UnparsedSourceDefinitions in the - # Manifest.sources to ParsedSourceDefinition via 'patch_source' - # in SourcePatcher - sources = patch_sources(self.root_project, self.manifest) - self.manifest.sources = sources - # ParseResults had a 'disabled' attribute which was a dictionary - # which is now named '_disabled'. This used to copy from - # ParseResults to the Manifest. Can this be normalized so - # there's only one disabled? - disabled = [] - for value in self.manifest._disabled.values(): - disabled.extend(value) - self.manifest.disabled = disabled - self._perf_info.patch_sources_elapsed = ( - time.perf_counter() - start_patch - ) - - self.manifest.selectors = self.root_project.manifest_selectors - - # do the node and macro patches - self.manifest.patch_nodes() - self.manifest.patch_macros() - - # process_manifest updates the refs, sources, and docs - start_process = time.perf_counter() - self.process_manifest() - - self._perf_info.process_manifest_elapsed = ( - time.perf_counter() - start_process - ) - - return self.manifest - def build_perf_info(self): mli = ManifestLoaderInfo( is_partial_parse_enabled=self._partial_parse_enabled() @@ -553,7 +553,7 @@ def create_macro_manifest(self): block = FileBlock(source_file) # This does not add the file to the manifest.files, # but that shouldn't be necessary here. - self.parse_with_cache(block, macro_parser) + macro_parser.parse_file(block) macro_manifest = MacroManifest(self.manifest.macros) return macro_manifest @@ -600,6 +600,80 @@ def track_project_load(self): ), }) + # Takes references in 'refs' array of nodes and exposures, finds the target + # node, and updates 'depends_on.nodes' with the unique id + def process_refs(self, current_project: str): + for node in self.manifest.nodes.values(): + if node.created_at < self.started_at: + continue + _process_refs_for_node(self.manifest, current_project, node) + for exposure in self.manifest.exposures.values(): + if exposure.created_at < self.started_at: + continue + _process_refs_for_exposure(self.manifest, current_project, exposure) + + # nodes: node and column descriptions + # sources: source and table descriptions, column descriptions + # macros: macro argument descriptions + # exposures: exposure descriptions + def process_docs(self, config: RuntimeConfig): + for node in self.manifest.nodes.values(): + if node.created_at < self.started_at: + continue + ctx = generate_runtime_docs( + config, + node, + self.manifest, + config.project_name, + ) + _process_docs_for_node(ctx, node) + for source in self.manifest.sources.values(): + if source.created_at < self.started_at: + continue + ctx = generate_runtime_docs( + config, + source, + self.manifest, + config.project_name, + ) + _process_docs_for_source(ctx, source) + for macro in self.manifest.macros.values(): + if macro.created_at < self.started_at: + continue + ctx = generate_runtime_docs( + config, + macro, + self.manifest, + config.project_name, + ) + _process_docs_for_macro(ctx, macro) + for exposure in self.manifest.exposures.values(): + if exposure.created_at < self.started_at: + continue + ctx = generate_runtime_docs( + config, + exposure, + self.manifest, + config.project_name, + ) + _process_docs_for_exposure(ctx, exposure) + + # Loops through all nodes and exposures, for each element in + # 'sources' array finds the source node and updates the + # 'depends_on.nodes' array with the unique id + def process_sources(self, current_project: str): + for node in self.manifest.nodes.values(): + if node.resource_type == NodeType.Source: + continue + assert not isinstance(node, ParsedSourceDefinition) + if node.created_at < self.started_at: + continue + _process_sources_for_node(self.manifest, current_project, node) + for exposure in self.manifest.exposures.values(): + if exposure.created_at < self.started_at: + continue + _process_sources_for_exposure(self.manifest, current_project, exposure) + def invalid_ref_fail_unless_test(node, target_model_name, target_model_package, disabled): @@ -773,45 +847,6 @@ def _process_docs_for_exposure( exposure.description = get_rendered(exposure.description, context) -# nodes: node and column descriptions -# sources: source and table descriptions, column descriptions -# macros: macro argument descriptions -# exposures: exposure descriptions -def process_docs(manifest: Manifest, config: RuntimeConfig): - for node in manifest.nodes.values(): - ctx = generate_runtime_docs( - config, - node, - manifest, - config.project_name, - ) - _process_docs_for_node(ctx, node) - for source in manifest.sources.values(): - ctx = generate_runtime_docs( - config, - source, - manifest, - config.project_name, - ) - _process_docs_for_source(ctx, source) - for macro in manifest.macros.values(): - ctx = generate_runtime_docs( - config, - macro, - manifest, - config.project_name, - ) - _process_docs_for_macro(ctx, macro) - for exposure in manifest.exposures.values(): - ctx = generate_runtime_docs( - config, - exposure, - manifest, - config.project_name, - ) - _process_docs_for_exposure(ctx, exposure) - - def _process_refs_for_exposure( manifest: Manifest, current_project: str, exposure: ParsedExposure ): @@ -899,16 +934,6 @@ def _process_refs_for_node( manifest.update_node(node) -# Takes references in 'refs' array of nodes and exposures, finds the target -# node, and updates 'depends_on.nodes' with the unique id -def process_refs(manifest: Manifest, current_project: str): - for node in manifest.nodes.values(): - _process_refs_for_node(manifest, current_project, node) - for exposure in manifest.exposures.values(): - _process_refs_for_exposure(manifest, current_project, exposure) - return manifest - - def _process_sources_for_exposure( manifest: Manifest, current_project: str, exposure: ParsedExposure ): @@ -960,20 +985,6 @@ def _process_sources_for_node( manifest.update_node(node) -# Loops through all nodes and exposures, for each element in -# 'sources' array finds the source node and updates the -# 'depends_on.nodes' array with the unique id -def process_sources(manifest: Manifest, current_project: str): - for node in manifest.nodes.values(): - if node.resource_type == NodeType.Source: - continue - assert not isinstance(node, ParsedSourceDefinition) - _process_sources_for_node(manifest, current_project, node) - for exposure in manifest.exposures.values(): - _process_sources_for_exposure(manifest, current_project, exposure) - return manifest - - # This is called in task.rpc.sql_commands when a "dynamic" node is # created in the manifest, in 'add_refs' def process_macro( diff --git a/core/dbt/parser/partial.py b/core/dbt/parser/partial.py new file mode 100644 index 00000000000..cf0d28c24d4 --- /dev/null +++ b/core/dbt/parser/partial.py @@ -0,0 +1,547 @@ +from typing import MutableMapping, Dict +from dbt.contracts.graph.manifest import Manifest +from dbt.contracts.files import ( + AnySourceFile, ParseFileType, parse_file_type_to_parser, +) +from dbt.logger import GLOBAL_LOGGER as logger + + +mssat_files = ( + ParseFileType.Model, + ParseFileType.Seed, + ParseFileType.Snapshot, + ParseFileType.Analysis, + ParseFileType.Test, +) + + +key_to_prefix = { + 'models': 'model', + 'seeds': 'seed', + 'snapshots': 'snapshot', + 'analyses': 'analysis', +} + + +parse_file_type_to_key = { + ParseFileType.Model: 'models', + ParseFileType.Seed: 'seeds', + ParseFileType.Snapshot: 'snapshots', + ParseFileType.Analysis: 'analyses', +} + + +# Partial parsing. Create a diff of files from saved manifest and current +# files and produce a project_parser_file dictionary to drive parsing of +# only the necessary changes. +# Will produce a 'skip_parsing' method, and a project_parser_file dictionary +class PartialParsing: + def __init__(self, saved_manifest: Manifest, new_files: MutableMapping[str, AnySourceFile]): + self.saved_manifest = saved_manifest + self.new_files = new_files + self.project_parser_files: Dict = {} + self.saved_files = self.saved_manifest.files + self.project_parser_files = {} + self.deleted_manifest = Manifest() + self.build_file_diff() + + def skip_parsing(self): + return ( + not self.file_diff['deleted'] and + not self.file_diff['added'] and + not self.file_diff['changed'] and + not self.file_diff['changed_schema_files'] and + not self.file_diff['deleted_schema_files'] + ) + + # Compare the previously saved manifest files and the just-loaded manifest + # files to see if anything changed + def build_file_diff(self): + saved_file_ids = set(self.saved_files.keys()) + new_file_ids = set(self.new_files.keys()) + deleted_all_files = saved_file_ids.difference(new_file_ids) + added = new_file_ids.difference(saved_file_ids) + common = saved_file_ids.intersection(new_file_ids) + + # separate out deleted schema files + deleted_schema_files = [] + deleted = [] + for file_id in deleted_all_files: + if self.saved_files[file_id].parse_file_type == ParseFileType.Schema: + deleted_schema_files.append(file_id) + else: + deleted.append(file_id) + + changed = [] + changed_schema_files = [] + unchanged = [] + for file_id in common: + if self.saved_files[file_id].checksum == self.new_files[file_id].checksum: + unchanged.append(file_id) + else: + # separate out changed schema files + if self.saved_files[file_id].parse_file_type == ParseFileType.Schema: + sf = self.saved_files[file_id] + if type(sf).__name__ != 'SchemaSourceFile': + raise Exception(f"Serialization failure for {file_id}") + changed_schema_files.append(file_id) + else: + changed.append(file_id) + file_diff = { + "deleted": deleted, + "deleted_schema_files": deleted_schema_files, + "added": added, + "changed": changed, + "changed_schema_files": changed_schema_files, + "unchanged": unchanged, + } + logger.info(f"Partial parsing enabled: " + f"{len(deleted) + len(deleted_schema_files)} files deleted, " + f"{len(added)} files added, " + f"{len(changed) + len(changed_schema_files)} files changed.") + self.file_diff = file_diff + + # generate the list of files that need parsing + # uses self.manifest.files generated by 'read_files' + def get_parsing_files(self): + if self.skip_parsing(): + return {} + # Need to add new files first, because changes in schema files + # might refer to them + for file_id in self.file_diff['added']: + self.add_to_saved(file_id) + # Need to process schema files next, because the dictionaries + # need to be in place for handling SQL file changes + for file_id in self.file_diff['changed_schema_files']: + self.change_schema_file(file_id) + for file_id in self.file_diff['deleted_schema_files']: + self.delete_schema_file(file_id) + for file_id in self.file_diff['deleted']: + self.delete_from_saved(file_id) + for file_id in self.file_diff['changed']: + self.update_in_saved(file_id) + return self.project_parser_files + + # new files are easy, just add them to parse list + def add_to_saved(self, file_id): + # add file object to saved manifest.files + source_file = self.new_files[file_id] + if source_file.parse_file_type == ParseFileType.Schema: + source_file.pp_dict = source_file.dict_from_yaml.copy() + self.saved_files[file_id] = source_file + # update pp_files to parse + self.add_to_pp_files(source_file) + logger.debug(f"Partial parsing: added file: {file_id}") + + # This handles all non-schema files + def delete_from_saved(self, file_id): + # Look at all things touched by file, remove those + # nodes, and update pp_files to parse unless the + # file creating those nodes has also been deleted + saved_source_file = self.saved_files[file_id] + + # SQL file: models, seeds, snapshots, analyses, tests: SQL files, except + # macros/tests + if saved_source_file.parse_file_type in mssat_files: + self.delete_mssat_file(saved_source_file) + + # macros + if saved_source_file.parse_file_type == ParseFileType.Macro: + self.delete_macro_file(saved_source_file) + + # docs + if saved_source_file.parse_file_type == ParseFileType.Documentation: + self.delete_doc_file(saved_source_file) + + self.deleted_manifest.files[file_id] = self.saved_manifest.files.pop(file_id) + logger.debug(f"Partial parsing: deleted file: {file_id}") + + # schema files already updated + def update_in_saved(self, file_id): + new_source_file = self.new_files[file_id] + old_source_file = self.saved_files[file_id] + + if new_source_file.parse_file_type in mssat_files: + self.update_mssat_in_saved(new_source_file, old_source_file) + elif new_source_file.parse_file_type == ParseFileType.Macro: + self.update_macro_in_saved(new_source_file, old_source_file) + elif new_source_file.parse_file_type == ParseFileType.Documentation: + self.update_doc_in_saved(new_source_file, old_source_file) + else: + raise Exception(f"Invalid parse_file_type in source_file {file_id}") + logger.debug(f"Partial parsing: updated file: {file_id}") + + # This is models, seeds, snapshots, analyses, tests. + # Models, seeds, snapshots: patches and tests + # analyses: patches, no tests + # tests: not touched by schema files (no patches, no tests) + # Updated schema files should have been processed already. + def update_mssat_in_saved(self, new_source_file, old_source_file): + + # These files only have one node. + unique_id = old_source_file.nodes[0] + + # replace source_file in saved and add to parsing list + file_id = new_source_file.file_id + self.deleted_manifest.files[file_id] = old_source_file + self.saved_files[file_id] = new_source_file + self.add_to_pp_files(new_source_file) + self.delete_node_in_saved(new_source_file, unique_id) + + def delete_node_in_saved(self, source_file, unique_id): + # delete node in saved + node = self.saved_manifest.nodes.pop(unique_id) + self.deleted_manifest.nodes[unique_id] = node + + # look at patch_path in model node to see if we need + # to reapply a patch from a schema_file. + if node.patch_path: + file_id = node.patch_path + # it might be changed... then what? + if file_id not in self.file_diff['deleted']: + # schema_files should already be updated + schema_file = self.saved_files[file_id] + dict_key = parse_file_type_to_key[source_file.parse_file_type] + # look for a matching list dictionary + for elem in schema_file.dict_from_yaml[dict_key]: + if elem['name'] == node.name: + elem_patch = elem + if elem_patch: + self.delete_schema_mssa_links(schema_file, dict_key, elem_patch) + self.merge_patch(schema_file, dict_key, elem_patch) + self.add_to_pp_files(schema_file) + if unique_id in schema_file.node_patches: + schema_file.node_patches.remove(unique_id) + + def update_macro_in_saved(self, new_source_file, old_source_file): + self.handle_macro_file_links(old_source_file) + file_id = new_source_file.file_id + self.saved_files[file_id] = new_source_file + self.add_to_pp_files(new_source_file) + + def update_doc_in_saved(self, new_source_file, old_source_file): + self.saved_files[new_source_file.file_id] = new_source_file + self.add_to_pp_files(new_source_file) + logger.warning("Partial parse is enabled and a doc file was updated, " + "but references to the doc will not be rebuilt. Please rebuild " + "without partial parsing.") + + def change_schema_file(self, file_id): + saved_schema_file = self.saved_files[file_id] + new_schema_file = self.new_files[file_id] + saved_yaml_dict = saved_schema_file.dict_from_yaml + new_yaml_dict = new_schema_file.dict_from_yaml + saved_schema_file.pp_dict = {"version": saved_yaml_dict['version']} + self.handle_schema_file_changes(saved_schema_file, saved_yaml_dict, new_yaml_dict) + + # copy from new schema_file to saved_schema_file to preserve references + # that weren't removed + saved_schema_file.contents = new_schema_file.contents + saved_schema_file.checksum = new_schema_file.checksum + saved_schema_file.dfy = new_schema_file.dfy + # schedule parsing + self.add_to_pp_files(saved_schema_file) + # schema_file pp_dict should have been generated already + logger.debug(f"Partial parsing: update schema file: {file_id}") + + # This is a variation on changed schema files + def delete_schema_file(self, file_id): + saved_schema_file = self.saved_files[file_id] + saved_yaml_dict = saved_schema_file.dict_from_yaml + new_yaml_dict = {} + self.handle_schema_file_changes(saved_schema_file, saved_yaml_dict, new_yaml_dict) + self.deleted_manifest.files[file_id] = self.saved_manifest.files.pop(file_id) + + # For each key in a schema file dictionary, process the changed, deleted, and added + # elemnts for the key lists + def handle_schema_file_changes(self, schema_file, saved_yaml_dict, new_yaml_dict): + # loop through comparing previous dict_from_yaml with current dict_from_yaml + # Need to do the deleted/added/changed thing, just like the files lists + + # models, seeds, snapshots, analyses + for dict_key in ['models', 'seeds', 'snapshots', 'analyses']: + key_diff = self.get_diff_for(dict_key, saved_yaml_dict, new_yaml_dict) + if key_diff['changed']: + for elem in key_diff['changed']: + self.delete_schema_mssa_links(schema_file, dict_key, elem) + self.merge_patch(schema_file, dict_key, elem) + if key_diff['deleted']: + for elem in key_diff['deleted']: + self.delete_schema_mssa_links(schema_file, dict_key, elem) + # patches will go away when new is copied to saved + if key_diff['added']: + for elem in key_diff['added']: + self.merge_patch(schema_file, dict_key, elem) + + # sources + source_diff = self.get_diff_for('sources', saved_yaml_dict, new_yaml_dict) + if source_diff['changed']: + for source in source_diff['changed']: + if 'override' in source: + raise Exception(f"Partial parsing does not handle changed " + f"source overrides: {schema_file.file_id}") + self.delete_schema_source(schema_file, source) + self.merge_patch(schema_file, 'sources', source) + if source_diff['deleted']: + for source in source_diff['deleted']: + if 'override' in source: + raise Exception(f"Partial parsing does not handle deleted " + f"source overrides: {schema_file.file_id}") + self.delete_schema_source(schema_file, source) + if source_diff['added']: + for source in source_diff['added']: + if 'override' in source: + raise Exception(f"Partial parsing does not handle new " + f"source overrides: {schema_file.file_id}") + self.merge_patch(schema_file, 'sources', source) + + # macros + macro_diff = self.get_diff_for('macros', saved_yaml_dict, new_yaml_dict) + if macro_diff['changed']: + for macro in macro_diff['changed']: + self.delete_schema_macro_patch(schema_file, macro) + self.merge_patch(schema_file, 'macros', elem) + if macro_diff['deleted']: + for macro in macro_diff['deleted']: + self.delete_schema_macro_patch(schema_file, macro) + if macro_diff['added']: + for elem in macro_diff['added']: + self.merge_patch(schema_file, 'macros', elem) + + # exposures + exposure_diff = self.get_diff_for('exposures', saved_yaml_dict, new_yaml_dict) + if exposure_diff['changed']: + for exposure in exposure_diff['changed']: + self.delete_schema_exposure(schema_file, exposure) + self.merge_patch(schema_file, 'exposures', exposure) + if exposure_diff['deleted']: + for exposure in exposure_diff['deleted']: + self.delete_schema_exposure(schema_file, exposure) + if exposure_diff['added']: + for exposure in exposure_diff['added']: + self.merge_patch(schema_file, 'exposures', exposure) + + # Take a "section" of the schema file yaml dictionary from saved and new schema files + # and determine which parts have changed + def get_diff_for(self, key, saved_yaml_dict, new_yaml_dict): + if key in saved_yaml_dict or key in new_yaml_dict: + saved_elements = saved_yaml_dict[key] if key in saved_yaml_dict else [] + new_elements = new_yaml_dict[key] if key in new_yaml_dict else [] + else: + return {'deleted': [], 'added': [], 'changed': []} + # for each set of keys, need to create a dictionary of names pointing to entry + saved_elements_by_name = {} + new_elements_by_name = {} + # sources have two part names? + for element in saved_elements: + saved_elements_by_name[element['name']] = element + for element in new_elements: + new_elements_by_name[element['name']] = element + + # now determine which elements, by name, are added, deleted or changed + saved_element_names = set(saved_elements_by_name.keys()) + new_element_names = set(new_elements_by_name.keys()) + deleted = saved_element_names.difference(new_element_names) + added = new_element_names.difference(saved_element_names) + common = saved_element_names.intersection(new_element_names) + changed = [] + for element_name in common: + if saved_elements_by_name[element_name] != new_elements_by_name[element_name]: + changed.append(element_name) + + # make lists of yaml elements to return as diffs + deleted_elements = [saved_elements_by_name[name].copy() for name in deleted] + added_elements = [new_elements_by_name[name].copy() for name in added] + changed_elements = [new_elements_by_name[name].copy() for name in changed] + + diff = { + "deleted": deleted_elements, + "added": added_elements, + "changed": changed_elements, + } + return diff + + # Add the file to the project parser dictionaries to schedule parsing + def add_to_pp_files(self, source_file): + file_id = source_file.file_id + parser_name = parse_file_type_to_parser[source_file.parse_file_type] + project_name = source_file.project_name + if not parser_name or not project_name: + raise Exception(f"Did not find parse_file_type or project_name " + f"in SourceFile for {source_file.file_id}") + if project_name not in self.project_parser_files: + self.project_parser_files[project_name] = {} + if parser_name not in self.project_parser_files[project_name]: + self.project_parser_files[project_name][parser_name] = [] + if (file_id not in self.project_parser_files[project_name][parser_name] and + file_id not in self.file_diff['deleted']): + self.project_parser_files[project_name][parser_name].append(file_id) + + # Merge a patch file into the pp_dict in a schema file + def merge_patch(self, schema_file, key, patch): + if not schema_file.pp_dict: + schema_file.pp_dict = {"version": schema_file.dict_from_yaml['version']} + pp_dict = schema_file.pp_dict + if key not in pp_dict: + pp_dict[key] = [patch] + else: + # check that this patch hasn't already been saved + found = False + for elem in pp_dict[key]: + if elem['name'] == patch['name']: + found = True + if not found: + pp_dict[key].append(patch) + + # For model, seed, snapshot, analysis schema dictionary keys, + # delete the patches and tests from the patch + def delete_schema_mssa_links(self, schema_file, dict_key, elem): + # find elem node unique_id in node_patches + + prefix = key_to_prefix[dict_key] + elem_unique_id = '' + for unique_id in schema_file.node_patches: + if not unique_id.startswith(prefix): + continue + parts = unique_id.split('.') + elem_name = parts[-1] + if elem_name == elem['name']: + elem_unique_id = unique_id + break + + # remove elem node and remove unique_id from node_patches + if elem_unique_id: + # might have been already removed + if elem_unique_id in self.saved_manifest.nodes: + node = self.saved_manifest.nodes.pop(elem_unique_id) + self.deleted_manifest.nodes[elem_unique_id] = node + # need to add the node source_file to pp_files + file_id = node.file_id() + # need to copy new file to saved files in order to get content + if self.new_files[file_id]: + self.saved_files[file_id] = self.new_files[file_id] + if self.saved_files[file_id]: + source_file = self.saved_files[file_id] + self.add_to_pp_files(source_file) + # TODO: should removing patches be here or with the 'merge_patch' code? + # remove from patches + schema_file.node_patches.remove(elem_unique_id) + + # for models, seeds, snapshots (not analyses) + if dict_key in ['models', 'seeds', 'snapshots']: + # find related tests and remove them + tests = self.get_tests_for(schema_file, elem['name']) + for test_unique_id in tests: + node = self.saved_manifest.nodes.pop(test_unique_id) + self.deleted_manifest.nodes[test_unique_id] = node + schema_file.tests.remove(test_unique_id) + + # Create a pp_test_index in the schema file if it doesn't exist + # and look for test names related to this yaml dict element name + def get_tests_for(self, schema_file, node_name): + if not schema_file.pp_test_index: + pp_test_index = {} + for test_unique_id in schema_file.tests: + test_node = self.saved_manifest.nodes[test_unique_id] + tested_node_id = test_node.depends_on.nodes[0] + parts = tested_node_id.split('.') + elem_name = parts[-1] + if elem_name in pp_test_index: + pp_test_index[elem_name].append(test_unique_id) + else: + pp_test_index[elem_name] = [test_unique_id] + schema_file.pp_test_index = pp_test_index + if node_name in schema_file.pp_test_index: + return schema_file.pp_test_index[node_name] + return [] + + def delete_mssat_file(self, source_file): + # nodes [unique_ids] -- SQL files + # There should always be a node for a SQL file + if not source_file.nodes: + raise Exception(f"No nodes found for source file {source_file.file_id}") + # There is generally only 1 node for SQL files, except for macros + for unique_id in source_file.nodes: + self.delete_node_in_saved(source_file, unique_id) + + def delete_macro_file(self, source_file): + self.handle_macro_file_links(source_file) + file_id = source_file.file_id + self.deleted_manifest.files[file_id] = self.saved_files.pop(file_id) + + def handle_macro_file_links(self, source_file): + # remove the macros in the 'macros' dictionary + for unique_id in source_file.macros: + self.deleted_manifest.macros[unique_id] = self.saved_manifest.macros.pop(unique_id) + # loop through all macros, finding references to this macro: macro.depends_on.macros + for macro in self.saved_manifest.macros.values(): + for macro_unique_id in macro.depends_on.macros: + if (macro_unique_id == unique_id and + macro_unique_id in self.saved_manifest.macros): + # schedule file for parsing + dep_file_id = macro.file_id() + if dep_file_id in self.saved_files: + source_file = self.saved_files[dep_file_id] + self.add_to_pp_files(source_file) + # loop through all nodes, finding references to this macro: node.depends_on.macros + for node in self.saved_manifest.nodes.values(): + for macro_unique_id in node.depends_on.macros: + if (macro_unique_id == unique_id and + macro_unique_id in self.saved_manifest.macros): + # schedule file for parsing + dep_file_id = node.file_id() + if dep_file_id in self.saved_files: + source_file = self.saved_files[dep_file_id] + self.add_to_pp_files(source_file) + + def delete_doc_file(self, source_file): + file_id = source_file.file_id + # remove the nodes in the 'docs' dictionary + for unique_id in source_file.docs: + self.deleted_manifest.docs[unique_id] = self.saved_manifest.docs.pop(unique_id) + logger.warning(f"Doc file {file_id} was deleted, but partial parsing cannot update " + f"doc references. Please rebuild to regenerate docs.") + # remove the file from the saved_files + self.deleted_manifest.files[file_id] = self.saved_files.pop(file_id) + + def delete_schema_source(self, schema_file, source_dict): + # both patches, tests, and source nodes + source_name = source_dict['name'] + # There may be multiple sources for each source dict, since + # there will be a separate source node for each table. + # ParsedSourceDefinition name = table name, dict name is source_name + for unique_id in schema_file.sources: + if unique_id in self.saved_manifest.sources: + source = self.saved_manifest.sources[unique_id] + if source.source_name == source_name: + source = self.saved_manifest.exposures.pop(unique_id) + self.deleted_manifest.sources[unique_id] = source + logger.debug(f"Partial parsing: deleted source {unique_id}") + + def delete_schema_macro_patch(self, schema_file, macro): + # This is just macro patches that need to be reapplied + for unique_id in schema_file.macro_patches: + parts = unique_id.split('.') + macro_name = parts[-1] + if macro_name == macro['name']: + macro_unique_id = unique_id + break + if macro_unique_id and macro_unique_id in self.saved_manifest.macros: + macro = self.saved_manifest.macros.pop(macro_unique_id) + self.deleted_manifest.macros[macro_unique_id] = macro + macro_file_id = macro.file_id() + self.add_to_pp_files(self.saved_files[macro_file_id]) + if macro_unique_id in schema_file.macro_patches: + schema_file.macro_patches.remove(macro_unique_id) + + # exposures are created only from schema files, so just delete + # the exposure. + def delete_schema_exposure(self, schema_file, exposure_dict): + exposure_name = exposure_dict['name'] + for unique_id in schema_file.exposures: + exposure = self.saved_manifest.exposures[unique_id] + if unique_id in self.saved_manifest.exposures: + if exposure.name == exposure_name: + self.deleted_manifest.exposures[unique_id] = \ + self.saved_manifest.exposures.pop(unique_id) + logger.debug(f"Partial parsing: deleted exposure {unique_id}") diff --git a/core/dbt/parser/read_files.py b/core/dbt/parser/read_files.py index f288c16b172..7900d0da85a 100644 --- a/core/dbt/parser/read_files.py +++ b/core/dbt/parser/read_files.py @@ -1,5 +1,7 @@ from dbt.clients.system import load_file_contents -from dbt.contracts.files import FilePath, ParseFileType, SourceFile, FileHash +from dbt.contracts.files import ( + FilePath, ParseFileType, SourceFile, FileHash, AnySourceFile, SchemaSourceFile +) from dbt.parser.schemas import yaml_from_file from dbt.parser.search import FilesystemSearcher @@ -8,14 +10,15 @@ # This loads the files contents and creates the SourceFile object def load_source_file( path: FilePath, parse_file_type: ParseFileType, - project_name: str) -> SourceFile: + project_name: str) -> AnySourceFile: file_contents = load_file_contents(path.absolute_path, strip=False) checksum = FileHash.from_contents(file_contents) - source_file = SourceFile(path=path, checksum=checksum, - parse_file_type=parse_file_type, project_name=project_name) + sf_cls = SchemaSourceFile if parse_file_type == ParseFileType.Schema else SourceFile + source_file = sf_cls(path=path, checksum=checksum, + parse_file_type=parse_file_type, project_name=project_name) source_file.contents = file_contents.strip() if parse_file_type == ParseFileType.Schema: - source_file.dict_from_yaml = yaml_from_file(source_file) + source_file.dfy = yaml_from_file(source_file) return source_file @@ -58,8 +61,8 @@ def read_files_for_parser(project, files, dirs, extension, parse_ft): project, dirs, extension, parse_ft ) for sf in source_files: - files[sf.search_key] = sf - parser_files.append(sf.search_key) + files[sf.file_id] = sf + parser_files.append(sf.file_id) return parser_files diff --git a/core/dbt/parser/rpc.py b/core/dbt/parser/rpc.py index 997b82471db..144987c1b1d 100644 --- a/core/dbt/parser/rpc.py +++ b/core/dbt/parser/rpc.py @@ -42,7 +42,7 @@ def get_compiled_path(cls, block: FileBlock): return os.path.join('rpc', block.name) def parse_remote(self, sql: str, name: str) -> ParsedRPCNode: - source_file = SourceFile.remote(contents=sql) + source_file = SourceFile.remote(sql, self.project.project_name) contents = RPCBlock(rpc_name=name, file=source_file) return self.parse_node(contents) diff --git a/core/dbt/parser/schemas.py b/core/dbt/parser/schemas.py index 8a04067a344..91816b5e781 100644 --- a/core/dbt/parser/schemas.py +++ b/core/dbt/parser/schemas.py @@ -14,10 +14,7 @@ from dbt.clients.yaml_helper import load_yaml_text from dbt.config.renderer import SchemaYamlRenderer from dbt.context.context_config import ( - BaseContextConfigGenerator, ContextConfig, - ContextConfigGenerator, - UnrenderedConfigGenerator, ) from dbt.context.configured import generate_schema_yml from dbt.context.target import generate_target_context @@ -26,11 +23,9 @@ ) from dbt.context.macro_resolver import MacroResolver from dbt.contracts.files import FileHash -from dbt.contracts.graph.manifest import SourceFile -from dbt.contracts.graph.model_config import SourceConfig +from dbt.contracts.graph.manifest import SchemaSourceFile from dbt.contracts.graph.parsed import ( ParsedNodePatch, - ParsedSourceDefinition, ColumnInfo, ParsedSchemaTestNode, ParsedMacroPatch, @@ -38,7 +33,6 @@ ParsedExposure, ) from dbt.contracts.graph.unparsed import ( - FreshnessThreshold, HasColumnDocs, HasColumnTests, HasDocs, @@ -53,7 +47,7 @@ from dbt.exceptions import ( validator_error_message, JSONValidationException, raise_invalid_schema_yml_version, ValidationException, - CompilationException, InternalException + CompilationException, ) from dbt.node_types import NodeType from dbt.parser.base import SimpleParser @@ -98,7 +92,7 @@ def error_context( def yaml_from_file( - source_file: SourceFile + source_file: SchemaSourceFile ) -> Optional[Dict[str, Any]]: """If loading the yaml fails, raise an exception. """ @@ -163,17 +157,6 @@ def _trimmed(inp: str) -> str: return inp[:44] + '...' + inp[-3:] -def merge_freshness( - base: Optional[FreshnessThreshold], update: Optional[FreshnessThreshold] -) -> Optional[FreshnessThreshold]: - if base is not None and update is not None: - return base.merged(update) - elif base is None and update is not None: - return update - else: - return None - - class SchemaParser(SimpleParser[SchemaTestBlock, ParsedSchemaTestNode]): def __init__( self, project, manifest, root_project, @@ -245,101 +228,6 @@ def parse_column_tests( for test in column.tests: self.parse_test(block, test, column) - def _generate_source_config(self, fqn: List[str], rendered: bool): - generator: BaseContextConfigGenerator - if rendered: - generator = ContextConfigGenerator(self.root_project) - else: - generator = UnrenderedConfigGenerator( - self.root_project - ) - - return generator.calculate_node_config( - config_calls=[], - fqn=fqn, - resource_type=NodeType.Source, - project_name=self.project.project_name, - base=False, - ) - - def _get_relation_name(self, node: ParsedSourceDefinition): - adapter = get_adapter(self.root_project) - relation_cls = adapter.Relation - return str(relation_cls.create_from(self.root_project, node)) - - # This converts an UnpatchedSourceDefinition to a ParsedSourceDefinition - # it is used by the SourcePatcher. - def parse_source( - self, target: UnpatchedSourceDefinition - ) -> ParsedSourceDefinition: - source = target.source - table = target.table - refs = ParserRef.from_target(table) - unique_id = target.unique_id - description = table.description or '' - meta = table.meta or {} - source_description = source.description or '' - loaded_at_field = table.loaded_at_field or source.loaded_at_field - - freshness = merge_freshness(source.freshness, table.freshness) - quoting = source.quoting.merged(table.quoting) - # path = block.path.original_file_path - source_meta = source.meta or {} - - # make sure we don't do duplicate tags from source + table - tags = sorted(set(itertools.chain(source.tags, table.tags))) - - config = self._generate_source_config( - fqn=target.fqn, - rendered=True, - ) - - unrendered_config = self._generate_source_config( - fqn=target.fqn, - rendered=False, - ) - - if not isinstance(config, SourceConfig): - raise InternalException( - f'Calculated a {type(config)} for a source, but expected ' - f'a SourceConfig' - ) - - default_database = self.root_project.credentials.database - - parsed_source = ParsedSourceDefinition( - package_name=target.package_name, - database=(source.database or default_database), - schema=(source.schema or source.name), - identifier=(table.identifier or table.name), - root_path=target.root_path, - path=target.path, - original_file_path=target.original_file_path, - columns=refs.column_info, - unique_id=unique_id, - name=table.name, - description=description, - external=table.external, - source_name=source.name, - source_description=source_description, - source_meta=source_meta, - meta=meta, - loader=source.loader, - loaded_at_field=loaded_at_field, - freshness=freshness, - quoting=quoting, - resource_type=NodeType.Source, - fqn=target.fqn, - tags=tags, - config=config, - unrendered_config=unrendered_config, - ) - - # relation name is added after instantiation because the adapter does - # not provide the relation name for a UnpatchedSourceDefinition object - parsed_source.relation_name = self._get_relation_name(parsed_source) - return parsed_source - def create_test_node( self, target: Union[UnpatchedSourceDefinition, UnparsedNodeUpdate], @@ -518,42 +406,6 @@ def render_test_update(self, node, config, builder): msg = validator_error_message(exc) raise CompilationException(msg, node=node) from exc - def parse_source_test( - self, - target: UnpatchedSourceDefinition, - test: Dict[str, Any], - column: Optional[UnparsedColumn], - ) -> ParsedSchemaTestNode: - column_name: Optional[str] - if column is None: - column_name = None - else: - column_name = column.name - should_quote = ( - column.quote or - (column.quote is None and target.quote_columns) - ) - if should_quote: - column_name = get_adapter(self.root_project).quote(column_name) - - tags_sources = [target.source.tags, target.table.tags] - if column is not None: - tags_sources.append(column.tags) - tags = list(itertools.chain.from_iterable(tags_sources)) - - node = self._parse_generic_test( - target=target, - test=test, - tags=tags, - column_name=column_name - ) - # we can't go through result.add_node - no file... instead! - if node.config.enabled: - self.manifest.add_node_nofile(node) - else: - self.manifest.add_disabled_nofile(node) - return node - def parse_node(self, block: SchemaTestBlock) -> ParsedSchemaTestNode: """In schema parsing, we rewrite most of the part of parse_node that builds the initial node to be parsed, but rendering is basically the @@ -622,21 +474,13 @@ def parse_tests(self, block: TestBlock) -> None: for test in block.tests: self.parse_test(block, test, None) - def parse_exposures(self, block: YamlBlock) -> None: - parser = ExposureParser(self, block) - for node in parser.parse(): - self.manifest.add_exposure(block.file, node) - def parse_file(self, block: FileBlock, dct: Dict = None) -> None: if not dct: dct = yaml_from_file(block.file) - # mark the file as seen, in Manifest.files - self.manifest.get_file(block.file) - if dct: try: - # This does a deep_map to check for circular references + # This does a deep_map which will fail if there are circular references dct = self.raw_renderer.render_data(dct) except CompilationException as exc: raise CompilationException( @@ -681,21 +525,21 @@ def parse_file(self, block: FileBlock, dct: Dict = None) -> None: parser = SourceParser(self, yaml_block, 'sources') parser.parse() - # NonSourceParser.parse() + # NonSourceParser.parse() (but never test_blocks) if 'macros' in dct: parser = MacroPatchParser(self, yaml_block, 'macros') - for test_block in parser.parse(): - self.parse_tests(test_block) + parser.parse() - # NonSourceParser.parse() + # NonSourceParser.parse() (but never test_blocks) if 'analyses' in dct: parser = AnalysisPatchParser(self, yaml_block, 'analyses') - for test_block in parser.parse(): - self.parse_tests(test_block) + parser.parse() # parse exposures if 'exposures' in dct: - self.parse_exposures(yaml_block) + exp_parser = ExposureParser(self, yaml_block) + for node in exp_parser.parse(): + self.manifest.add_exposure(yaml_block.file, node) Parsed = TypeVar( @@ -819,7 +663,7 @@ def add_source_definitions(self, source: UnparsedSourceDefinition) -> None: fqn = self.schema_parser.get_fqn_prefix(fqn_path) fqn.extend([source.name, table.name]) - result = UnpatchedSourceDefinition( + source_def = UnpatchedSourceDefinition( source=source, table=table, path=original_file_path, @@ -830,7 +674,7 @@ def add_source_definitions(self, source: UnparsedSourceDefinition) -> None: resource_type=NodeType.Source, fqn=fqn, ) - self.manifest.add_source(self.yaml.file, result) + self.manifest.add_source(self.yaml.file, source_def) # This class has three main subclasses: TestablePatchParser (models, @@ -876,6 +720,7 @@ def parse(self) -> List[TestBlock]: # This adds the node_block to self.manifest # as a ParsedNodePatch or ParsedMacroPatch self.parse_patch(node_block, refs) + # This will always be empty if the node a macro or analysis return test_blocks def get_unparsed_target(self) -> Iterable[NonSourceTarget]: diff --git a/core/dbt/parser/search.py b/core/dbt/parser/search.py index 46b5a87b35d..a63981c0b46 100644 --- a/core/dbt/parser/search.py +++ b/core/dbt/parser/search.py @@ -1,13 +1,13 @@ import os from dataclasses import dataclass from typing import ( - List, Callable, Iterable, Set, Union, Iterator, TypeVar, Generic + List, Callable, Iterable, Set, Union, Iterator, TypeVar, Generic, Any ) from dbt.clients.jinja import extract_toplevel_blocks, BlockTag from dbt.clients.system import find_matching from dbt.config import Project -from dbt.contracts.files import SourceFile, FilePath +from dbt.contracts.files import FilePath, AnySourceFile from dbt.exceptions import CompilationException, InternalException @@ -15,7 +15,9 @@ # Could it be removed? @dataclass class FileBlock: - file: SourceFile + # This is either a SchemaSourceFile or SourceFile, but mypy + # had a hard time sorting it out. Using AnySourceFile didn't help. + file: Any @property def name(self): @@ -37,7 +39,7 @@ def path(self): # difference is what 'contents' returns? @dataclass class BlockContents(FileBlock): - file: SourceFile # if you remove this, mypy will get upset + file: AnySourceFile # if you remove this, mypy will get upset block: BlockTag @property @@ -51,7 +53,7 @@ def contents(self): @dataclass class FullBlock(FileBlock): - file: SourceFile # if you remove this, mypy will get upset + file: AnySourceFile # if you remove this, mypy will get upset block: BlockTag @property @@ -93,7 +95,7 @@ def __iter__(self) -> Iterator[FilePath]: BlockSearchResult = TypeVar('BlockSearchResult', BlockContents, FullBlock) -BlockSearchResultFactory = Callable[[SourceFile, BlockTag], BlockSearchResult] +BlockSearchResultFactory = Callable[[AnySourceFile, BlockTag], BlockSearchResult] class BlockSearcher(Generic[BlockSearchResult], Iterable[BlockSearchResult]): diff --git a/core/dbt/parser/snapshots.py b/core/dbt/parser/snapshots.py index dac871a6523..2815b9343c5 100644 --- a/core/dbt/parser/snapshots.py +++ b/core/dbt/parser/snapshots.py @@ -78,7 +78,3 @@ def parse_file(self, file_block: FileBlock) -> None: ) for block in blocks: self.parse_node(block) - # in case there are no snapshots declared, we still want to mark this - # file as seen. But after we've finished, because we don't want to add - # files with syntax errors - self.manifest.get_file(file_block.file) diff --git a/core/dbt/parser/sources.py b/core/dbt/parser/sources.py index 97d3bd1907f..d5f2d503e0a 100644 --- a/core/dbt/parser/sources.py +++ b/core/dbt/parser/sources.py @@ -1,13 +1,17 @@ +import itertools from pathlib import Path from typing import ( - Iterable, - Dict, - Optional, - Set, - Union, + Iterable, Dict, Optional, Set, List, Any ) +from dbt.adapters.factory import get_adapter from dbt.config import RuntimeConfig +from dbt.context.context_config import ( + BaseContextConfigGenerator, + ContextConfigGenerator, + UnrenderedConfigGenerator, +) from dbt.contracts.graph.manifest import Manifest, SourceKey +from dbt.contracts.graph.model_config import SourceConfig from dbt.contracts.graph.parsed import ( UnpatchedSourceDefinition, ParsedSourceDefinition, @@ -18,13 +22,24 @@ SourcePatch, SourceTablePatch, UnparsedSourceTableDefinition, + FreshnessThreshold, + UnparsedColumn, ) -from dbt.exceptions import warn_or_error +from dbt.exceptions import warn_or_error, InternalException +from dbt.node_types import NodeType from dbt.parser.schemas import SchemaParser, ParserRef from dbt import ui +# An UnparsedSourceDefinition is taken directly from the yaml +# file. It can affect multiple tables, all of which will eventually +# have their own source node. An UnparsedSourceDefinition will +# generate multiple UnpatchedSourceDefinition nodes (one per +# table) in the SourceParser.add_source_definitions. The +# SourcePatcher takes an UnparsedSourceDefinition and the +# SourcePatch and produces a ParsedSourceDefinition. Each +# SourcePatch can be applied to multiple UnpatchedSourceDefinitions. class SourcePatcher: def __init__( self, @@ -37,11 +52,44 @@ def __init__( self.patches_used: Dict[SourceKey, Set[str]] = {} self.sources: Dict[str, ParsedSourceDefinition] = {} + # This method calls the 'parse_source' method which takes + # the UnpatchedSourceDefinitions in the manifest and combines them + # with SourcePatches to produce ParsedSourceDefinitions. + def construct_sources(self) -> None: + for unique_id, unpatched in self.manifest.sources.items(): + if isinstance(unpatched, ParsedSourceDefinition): + continue + # returns None if there is no patch + patch = self.get_patch_for(unpatched) + + # returns unpatched if there is no patch + patched = self.patch_source(unpatched, patch) + + # now use the patched UnpatchedSourceDefinition to extract test + # data. 'get_source_tests' uses the schema parser + for test in self.get_source_tests(patched): + if test.config.enabled: + self.manifest.add_node_nofile(test) + else: + self.manifest.add_disabled_nofile(test) + + # Convert UnpatchedSourceDefinition to a ParsedSourceDefinition + parsed = self.parse_source(patched) + if parsed.config.enabled: + self.sources[unique_id] = parsed + else: + self.manifest.add_disabled_nofile(parsed) + + self.warn_unused() + def patch_source( self, unpatched: UnpatchedSourceDefinition, patch: Optional[SourcePatch], ) -> UnpatchedSourceDefinition: + + # This skips patching if no patch exists because of the + # performance overhead of converting to and from dicts if patch is None: return unpatched @@ -65,15 +113,83 @@ def patch_source( source=source, table=table, patch_path=patch_path ) - def parse_source_docs(self, block: UnpatchedSourceDefinition) -> ParserRef: - refs = ParserRef() - for column in block.columns: - description = column.description - data_type = column.data_type - meta = column.meta - refs.add(column, description, data_type, meta) - return refs + # This converts an UnpatchedSourceDefinition to a ParsedSourceDefinition + def parse_source( + self, target: UnpatchedSourceDefinition + ) -> ParsedSourceDefinition: + source = target.source + table = target.table + refs = ParserRef.from_target(table) + unique_id = target.unique_id + description = table.description or '' + meta = table.meta or {} + source_description = source.description or '' + loaded_at_field = table.loaded_at_field or source.loaded_at_field + + freshness = merge_freshness(source.freshness, table.freshness) + quoting = source.quoting.merged(table.quoting) + # path = block.path.original_file_path + source_meta = source.meta or {} + + # make sure we don't do duplicate tags from source + table + tags = sorted(set(itertools.chain(source.tags, table.tags))) + config = self._generate_source_config( + fqn=target.fqn, + rendered=True, + project_name=target.package_name, + ) + + unrendered_config = self._generate_source_config( + fqn=target.fqn, + rendered=False, + project_name=target.package_name, + ) + + if not isinstance(config, SourceConfig): + raise InternalException( + f'Calculated a {type(config)} for a source, but expected ' + f'a SourceConfig' + ) + + default_database = self.root_project.credentials.database + + parsed_source = ParsedSourceDefinition( + package_name=target.package_name, + database=(source.database or default_database), + schema=(source.schema or source.name), + identifier=(table.identifier or table.name), + root_path=target.root_path, + path=target.path, + original_file_path=target.original_file_path, + columns=refs.column_info, + unique_id=unique_id, + name=table.name, + description=description, + external=table.external, + source_name=source.name, + source_description=source_description, + source_meta=source_meta, + meta=meta, + loader=source.loader, + loaded_at_field=loaded_at_field, + freshness=freshness, + quoting=quoting, + resource_type=NodeType.Source, + fqn=target.fqn, + tags=tags, + config=config, + unrendered_config=unrendered_config, + ) + + # relation name is added after instantiation because the adapter does + # not provide the relation name for a UnpatchedSourceDefinition object + parsed_source.relation_name = self._get_relation_name(parsed_source) + return parsed_source + + # This code uses the SchemaParser because it shares the '_parse_generic_test' + # code. It might be nice to separate out the generic test code + # and make it common to the schema parser and source patcher. def get_schema_parser_for(self, package_name: str) -> 'SchemaParser': if package_name in self.schema_parsers: schema_parser = self.schema_parsers[package_name] @@ -89,9 +205,8 @@ def get_schema_parser_for(self, package_name: str) -> 'SchemaParser': def get_source_tests( self, target: UnpatchedSourceDefinition ) -> Iterable[ParsedSchemaTestNode]: - schema_parser = self.get_schema_parser_for(target.package_name) for test, column in target.get_tests(): - yield schema_parser.parse_source_test( + yield self.parse_source_test( target=target, test=test, column=column, @@ -99,7 +214,7 @@ def get_source_tests( def get_patch_for( self, - unpatched: Union[UnpatchedSourceDefinition, ParsedSourceDefinition], + unpatched: UnpatchedSourceDefinition, ) -> Optional[SourcePatch]: if isinstance(unpatched, ParsedSourceDefinition): return None @@ -114,31 +229,67 @@ def get_patch_for( self.patches_used[key].add(unpatched.table.name) return patch - def construct_sources(self) -> None: - # given the UnpatchedSourceDefinition and SourcePatches, combine them - # to make a beautiful baby ParsedSourceDefinition. - for unique_id, unpatched in self.manifest.sources.items(): - if isinstance(unpatched, ParsedSourceDefinition): - continue - patch = self.get_patch_for(unpatched) + # This calls _parse_generic_test in the SchemaParser + def parse_source_test( + self, + target: UnpatchedSourceDefinition, + test: Dict[str, Any], + column: Optional[UnparsedColumn], + ) -> ParsedSchemaTestNode: + column_name: Optional[str] + if column is None: + column_name = None + else: + column_name = column.name + should_quote = ( + column.quote or + (column.quote is None and target.quote_columns) + ) + if should_quote: + column_name = get_adapter(self.root_project).quote(column_name) - patched = self.patch_source(unpatched, patch) - # now use the patched UnpatchedSourceDefinition to extract test - # data. - for test in self.get_source_tests(patched): - if test.config.enabled: - self.manifest.add_node_nofile(test) - else: - self.manifest.add_disabled_nofile(test) + tags_sources = [target.source.tags, target.table.tags] + if column is not None: + tags_sources.append(column.tags) + tags = list(itertools.chain.from_iterable(tags_sources)) - schema_parser = self.get_schema_parser_for(unpatched.package_name) - parsed = schema_parser.parse_source(patched) - if parsed.config.enabled: - self.sources[unique_id] = parsed - else: - self.manifest.add_disabled_nofile(parsed) + # TODO: make the generic_test code common so we don't need to + # create schema parsers to handle the tests + schema_parser = self.get_schema_parser_for(target.package_name) + node = schema_parser._parse_generic_test( + target=target, + test=test, + tags=tags, + column_name=column_name + ) + # we can't go through result.add_node - no file... instead! + if node.config.enabled: + self.manifest.add_node_nofile(node) + else: + self.manifest.add_disabled_nofile(node) + return node - self.warn_unused() + def _generate_source_config(self, fqn: List[str], rendered: bool, project_name: str): + generator: BaseContextConfigGenerator + if rendered: + generator = ContextConfigGenerator(self.root_project) + else: + generator = UnrenderedConfigGenerator( + self.root_project + ) + + return generator.calculate_node_config( + config_calls=[], + fqn=fqn, + resource_type=NodeType.Source, + project_name=project_name, + base=False, + ) + + def _get_relation_name(self, node: ParsedSourceDefinition): + adapter = get_adapter(self.root_project) + relation_cls = adapter.Relation + return str(relation_cls.create_from(self.root_project, node)) def warn_unused(self) -> None: unused_tables: Dict[SourceKey, Optional[Set[str]]] = {} @@ -159,6 +310,8 @@ def warn_unused(self) -> None: msg = self.get_unused_msg(unused_tables) warn_or_error(msg, log_fmt=ui.warning_tag('{}')) + self.manifest.source_patches = {} + def get_unused_msg( self, unused_tables: Dict[SourceKey, Optional[Set[str]]], @@ -184,16 +337,12 @@ def get_unused_msg( return '\n'.join(msg) -def patch_sources( - root_project: RuntimeConfig, - manifest: Manifest, -) -> Dict[str, ParsedSourceDefinition]: - """Patch all the sources found in the results. Updates results.disabled and - results.nodes. - - Return a dict of ParsedSourceDefinitions, suitable for use in - manifest.sources. - """ - patcher = SourcePatcher(root_project, manifest) - patcher.construct_sources() - return patcher.sources +def merge_freshness( + base: Optional[FreshnessThreshold], update: Optional[FreshnessThreshold] +) -> Optional[FreshnessThreshold]: + if base is not None and update is not None: + return base.merged(update) + elif base is None and update is not None: + return update + else: + return None diff --git a/core/dbt/task/parse.py b/core/dbt/task/parse.py index 3563fd6b7cc..985db062093 100644 --- a/core/dbt/task/parse.py +++ b/core/dbt/task/parse.py @@ -61,12 +61,8 @@ def get_full_manifest(self): print_timestamped_line("Dependencies loaded") loader = ManifestLoader(root_config, projects, macro_hook) print_timestamped_line("ManifestLoader created") - loader.load() + manifest = loader.load() print_timestamped_line("Manifest loaded") - loader.write_manifest_for_partial_parse() - print_timestamped_line("Manifest for partial parse saved") - manifest = loader.update_manifest() - print_timestamped_line("Manifest updated") _check_manifest(manifest, root_config) print_timestamped_line("Manifest checked") manifest.build_flat_graph() @@ -93,5 +89,6 @@ def run(self): if self.args.write_manifest: print_timestamped_line('Writing manifest.') self.write_manifest() + self.write_perf_info() print_timestamped_line('Done.') diff --git a/core/dbt/utils.py b/core/dbt/utils.py index def02a55f6c..f14273d5192 100644 --- a/core/dbt/utils.py +++ b/core/dbt/utils.py @@ -404,6 +404,8 @@ def pluralize(count, string: Union[str, NodeType]): return f'{count} {pluralized}' +# Note that this only affects hologram json validation. +# It has no effect on mashumaro serialization. def restrict_to(*restrictions): """Create the metadata for a restricted dataclass field""" return {'restrict': list(restrictions)} diff --git a/core/setup.py b/core/setup.py index 72e77ede3d0..7ea2402741e 100644 --- a/core/setup.py +++ b/core/setup.py @@ -72,7 +72,7 @@ def read(fname): 'dataclasses>=0.6,<0.9;python_version<"3.7"', 'hologram==0.0.14', 'logbook>=1.5,<1.6', - 'mashumaro==2.0', + 'mashumaro==2.5', 'typing-extensions>=3.7.4,<3.8', # the following are all to match snowflake-connector-python 'requests<3.0.0', diff --git a/test/integration/012_deprecation_tests/test_deprecations.py b/test/integration/012_deprecation_tests/test_deprecations.py index a8376be1145..ae59355ae5e 100644 --- a/test/integration/012_deprecation_tests/test_deprecations.py +++ b/test/integration/012_deprecation_tests/test_deprecations.py @@ -60,27 +60,6 @@ def test_postgres_deprecations(self): self.assertEqual(expected, deprecations.active_deprecations) -class TestModelsKeyMismatchDeprecation(BaseTestDeprecations): - @property - def models(self): - return self.dir('models-key-mismatch') - - @use_profile('postgres') - def test_postgres_deprecations_fail(self): - # this should fail at compile_time - with self.assertRaises(dbt.exceptions.CompilationException) as exc: - self.run_dbt(strict=True) - exc_str = ' '.join(str(exc.exception).split()) # flatten all whitespace - self.assertIn('"seed" is a seed node, but it is specified in the models section', exc_str) - - @use_profile('postgres') - def test_postgres_deprecations(self): - self.assertEqual(deprecations.active_deprecations, set()) - self.run_dbt(strict=False) - expected = {'models-key-mismatch'} - self.assertEqual(expected, deprecations.active_deprecations) - - class TestAdapterMacroDeprecation(BaseTestDeprecations): @property def models(self): diff --git a/test/integration/029_docs_generate_tests/test_docs_generate.py b/test/integration/029_docs_generate_tests/test_docs_generate.py index 5529531be00..ff163dd619f 100644 --- a/test/integration/029_docs_generate_tests/test_docs_generate.py +++ b/test/integration/029_docs_generate_tests/test_docs_generate.py @@ -1021,7 +1021,7 @@ def _verify_generic_macro_structure(self, manifest): 'path', 'original_file_path', 'package_name', 'root_path', 'name', 'unique_id', 'tags', 'resource_type', 'depends_on', 'meta', 'description', 'patch_path', 'arguments', - 'macro_sql', 'docs' + 'macro_sql', 'docs', 'created_at', } ) # Don't compare the sql, just make sure it exists @@ -1043,6 +1043,7 @@ def _verify_generic_macro_structure(self, manifest): 'root_path': root_path, 'name': 'column_list', 'unique_id': 'macro.dbt.column_list', + 'created_at': ANY, 'tags': [], 'resource_type': 'macro', 'depends_on': {'macros': []}, @@ -1105,6 +1106,7 @@ def expected_seeded_manifest(self, model_database=None, quote_model=False): 'model.test.model': { 'compiled_path': Normalized('target/compiled/test/models/model.sql'), 'build_path': None, + 'created_at': ANY, 'name': 'model', 'root_path': self.test_root_realpath, 'relation_name': relation_name_node_format.format( @@ -1170,7 +1172,7 @@ def expected_seeded_manifest(self, model_database=None, quote_model=False): 'tags': [], }, }, - 'patch_path': model_schema_yml_path, + 'patch_path': 'test://' + model_schema_yml_path, 'docs': {'show': False}, 'compiled': True, 'compiled_sql': ANY, @@ -1182,6 +1184,7 @@ def expected_seeded_manifest(self, model_database=None, quote_model=False): 'model.test.second_model': { 'compiled_path': Normalized('target/compiled/test/models/second_model.sql'), 'build_path': None, + 'created_at': ANY, 'name': 'second_model', 'root_path': self.test_root_realpath, 'relation_name': relation_name_node_format.format( @@ -1248,7 +1251,7 @@ def expected_seeded_manifest(self, model_database=None, quote_model=False): 'tags': [], }, }, - 'patch_path': model_schema_yml_path, + 'patch_path': 'test://' + model_schema_yml_path, 'docs': {'show': False}, 'compiled': True, 'compiled_sql': ANY, @@ -1260,10 +1263,11 @@ def expected_seeded_manifest(self, model_database=None, quote_model=False): 'seed.test.seed': { 'compiled_path': None, 'build_path': None, + 'created_at': ANY, 'compiled': True, 'compiled_sql': '', 'config': seed_config, - 'patch_path': seed_schema_yml_path, + 'patch_path': 'test://' + seed_schema_yml_path, 'path': 'seed.csv', 'name': 'seed', 'root_path': self.test_root_realpath, @@ -1340,6 +1344,7 @@ def expected_seeded_manifest(self, model_database=None, quote_model=False): 'alias': 'not_null_model_id', 'compiled_path': Normalized('target/compiled/test/models/schema.yml/schema_test/not_null_model_id.sql'), 'build_path': None, + 'created_at': ANY, 'column_name': 'id', 'columns': {}, 'config': test_config, @@ -1386,6 +1391,7 @@ def expected_seeded_manifest(self, model_database=None, quote_model=False): 'alias': 'snapshot_seed', 'compiled_path': None, 'build_path': None, + 'created_at': ANY, 'checksum': self._checksum_file(snapshot_path), 'columns': {}, 'compiled': True, @@ -1429,6 +1435,7 @@ def expected_seeded_manifest(self, model_database=None, quote_model=False): 'alias': 'test_nothing_model_', 'compiled_path': Normalized('target/compiled/test/models/schema.yml/schema_test/test_nothing_model_.sql'), 'build_path': None, + 'created_at': ANY, 'column_name': None, 'columns': {}, 'config': test_config, @@ -1474,6 +1481,7 @@ def expected_seeded_manifest(self, model_database=None, quote_model=False): 'alias': 'unique_model_id', 'compiled_path': Normalized('target/compiled/test/models/schema.yml/schema_test/unique_model_id.sql'), 'build_path': None, + 'created_at': ANY, 'column_name': 'id', 'columns': {}, 'config': test_config, @@ -1519,6 +1527,7 @@ def expected_seeded_manifest(self, model_database=None, quote_model=False): }, 'sources': { 'source.test.my_source.my_table': { + 'created_at': ANY, 'columns': { 'id': { 'description': 'An ID field', @@ -1568,6 +1577,7 @@ def expected_seeded_manifest(self, model_database=None, quote_model=False): }, 'exposures': { 'exposure.test.notebook_exposure': { + 'created_at': ANY, 'depends_on': { 'macros': [], 'nodes': ['model.test.model', 'model.test.second_model'] @@ -1592,6 +1602,7 @@ def expected_seeded_manifest(self, model_database=None, quote_model=False): 'url': 'http://example.com/notebook/1' }, 'exposure.test.simple_exposure': { + 'created_at': ANY, 'depends_on': { 'macros': [], 'nodes': [ @@ -1679,6 +1690,7 @@ def expected_postgres_references_manifest(self, model_database=None): 'alias': 'ephemeral_copy', 'compiled_path': Normalized('target/compiled/test/ref_models/ephemeral_copy.sql'), 'build_path': None, + 'created_at': ANY, 'columns': {}, 'config': self.rendered_model_config(materialized='ephemeral'), 'sources': [['my_source', 'my_table']], @@ -1719,6 +1731,7 @@ def expected_postgres_references_manifest(self, model_database=None): 'alias': 'ephemeral_summary', 'compiled_path': Normalized('target/compiled/test/ref_models/ephemeral_summary.sql'), 'build_path': None, + 'created_at': ANY, 'columns': { 'first_name': { 'description': 'The first name being summarized', @@ -1750,7 +1763,7 @@ def expected_postgres_references_manifest(self, model_database=None): 'name': 'ephemeral_summary', 'original_file_path': ephemeral_summary_path, 'package_name': 'test', - 'patch_path': self.dir('ref_models/schema.yml'), + 'patch_path': 'test://' + self.dir('ref_models/schema.yml'), 'path': 'ephemeral_summary.sql', 'raw_sql': LineIndifferent( '{{\n config(\n materialized = "table"\n )\n}}\n\n' @@ -1780,6 +1793,7 @@ def expected_postgres_references_manifest(self, model_database=None): 'alias': 'view_summary', 'compiled_path': Normalized('target/compiled/test/ref_models/view_summary.sql'), 'build_path': None, + 'created_at': ANY, 'columns': { 'first_name': { 'description': 'The first name being summarized', @@ -1811,7 +1825,7 @@ def expected_postgres_references_manifest(self, model_database=None): 'name': 'view_summary', 'original_file_path': view_summary_path, 'package_name': 'test', - 'patch_path': self.dir('ref_models/schema.yml'), + 'patch_path': 'test://' + self.dir('ref_models/schema.yml'), 'path': 'view_summary.sql', 'raw_sql': LineIndifferent( '{{\n config(\n materialized = "view"\n )\n}}\n\n' @@ -1840,6 +1854,7 @@ def expected_postgres_references_manifest(self, model_database=None): 'alias': 'seed', 'compiled_path': None, 'build_path': None, + 'created_at': ANY, 'columns': { 'id': { 'name': 'id', @@ -1892,7 +1907,7 @@ def expected_postgres_references_manifest(self, model_database=None): 'name': 'seed', 'original_file_path': seed_path, 'package_name': 'test', - 'patch_path': self.dir('seed/schema.yml'), + 'patch_path': 'test://' + self.dir('seed/schema.yml'), 'path': 'seed.csv', 'raw_sql': '', 'refs': [], @@ -1917,6 +1932,7 @@ def expected_postgres_references_manifest(self, model_database=None): 'alias': 'snapshot_seed', 'compiled_path': None, 'build_path': None, + 'created_at': ANY, 'checksum': self._checksum_file(snapshot_path), 'columns': {}, 'compiled': True, @@ -1974,6 +1990,7 @@ def expected_postgres_references_manifest(self, model_database=None): 'identifier': True, 'column': None, }, + 'created_at': ANY, 'database': self.default_database, 'description': 'My table', 'external': None, @@ -2004,6 +2021,7 @@ def expected_postgres_references_manifest(self, model_database=None): }, 'exposures': { 'exposure.test.notebook_exposure': { + 'created_at': ANY, 'depends_on': { 'macros': [], 'nodes': ['model.test.view_summary'] @@ -2150,6 +2168,7 @@ def expected_postgres_references_manifest(self, model_database=None): 'macro.test.test_nothing': { 'name': 'test_nothing', 'depends_on': {'macros': []}, + 'created_at': ANY, 'description': 'My custom test that I wrote that does nothing', 'docs': {'show': True}, 'macro_sql': AnyStringWith('test nothing'), @@ -2159,7 +2178,7 @@ def expected_postgres_references_manifest(self, model_database=None): 'meta': { 'some_key': 100, }, - 'patch_path': self.dir('macros/schema.yml'), + 'patch_path': 'test://' + self.dir('macros/schema.yml'), 'resource_type': 'macro', 'unique_id': 'macro.test.test_nothing', 'tags': [], @@ -2201,6 +2220,7 @@ def expected_bigquery_complex_manifest(self): 'fqn': ['test', 'clustered'], 'compiled_path': Normalized('target/compiled/test/bq_models/clustered.sql'), 'build_path': None, + 'created_at': ANY, 'name': 'clustered', 'original_file_path': clustered_sql_path, 'package_name': 'test', @@ -2261,7 +2281,7 @@ def expected_bigquery_complex_manifest(self): }, 'deferred': False, 'description': 'A clustered and partitioned copy of the test model', - 'patch_path': self.dir('bq_models/schema.yml'), + 'patch_path': 'test://' + self.dir('bq_models/schema.yml'), 'docs': {'show': True}, 'compiled': True, 'compiled_sql': ANY, @@ -2279,6 +2299,7 @@ def expected_bigquery_complex_manifest(self): 'alias': 'multi_clustered', 'compiled_path': Normalized('target/compiled/test/bq_models/multi_clustered.sql'), 'build_path': None, + 'created_at': ANY, 'config': self.rendered_model_config( cluster_by=['first_name', 'email'], materialized='table', @@ -2348,7 +2369,7 @@ def expected_bigquery_complex_manifest(self): }, 'deferred': False, 'description': 'A clustered and partitioned copy of the test model, clustered on multiple columns', - 'patch_path': self.dir('bq_models/schema.yml'), + 'patch_path': 'test://' + self.dir('bq_models/schema.yml'), 'docs': {'show': True}, 'compiled': True, 'compiled_sql': ANY, @@ -2366,6 +2387,7 @@ def expected_bigquery_complex_manifest(self): 'alias': 'nested_view', 'compiled_path': Normalized('target/compiled/test/bq_models/nested_view.sql'), 'build_path': None, + 'created_at': ANY, 'config': self.rendered_model_config(), 'sources': [], 'depends_on': { @@ -2433,7 +2455,7 @@ def expected_bigquery_complex_manifest(self): }, 'deferred': False, 'description': 'The test model', - 'patch_path': self.dir('bq_models/schema.yml'), + 'patch_path': 'test://' + self.dir('bq_models/schema.yml'), 'docs': {'show': True}, 'compiled': True, 'compiled_sql': ANY, @@ -2446,6 +2468,7 @@ def expected_bigquery_complex_manifest(self): 'alias': 'nested_table', 'compiled_path': Normalized('target/compiled/test/bq_models/nested_table.sql'), 'build_path': None, + 'created_at': ANY, 'config': self.rendered_model_config(materialized='table'), 'sources': [], 'depends_on': { @@ -2484,7 +2507,8 @@ def expected_bigquery_complex_manifest(self): 'seed.test.seed': { 'compiled_path': None, 'build_path': None, - 'patch_path': self.dir('seed/schema.yml'), + 'created_at': ANY, + 'patch_path': 'test://' + self.dir('seed/schema.yml'), 'path': 'seed.csv', 'name': 'seed', 'root_path': self.test_root_realpath, @@ -2565,6 +2589,7 @@ def expected_bigquery_complex_manifest(self): 'alias': 'snapshot_seed', 'compiled_path': None, 'build_path': None, + 'created_at': ANY, 'checksum': self._checksum_file(snapshot_path), 'columns': {}, 'compiled': True, @@ -2677,6 +2702,7 @@ def expected_redshift_incremental_view_manifest(self): 'model.test.model': { 'compiled_path': Normalized('target/compiled/test/rs_models/model.sql'), 'build_path': None, + 'created_at': ANY, 'name': 'model', 'root_path': self.test_root_realpath, 'relation_name': '"{0}"."{1}".model'.format( @@ -2745,7 +2771,7 @@ def expected_redshift_incremental_view_manifest(self): 'tags': [], }, }, - 'patch_path': self.dir('rs_models/schema.yml'), + 'patch_path': 'test://' + self.dir('rs_models/schema.yml'), 'docs': {'show': True}, 'compiled': True, 'compiled_sql': ANY, @@ -2757,7 +2783,8 @@ def expected_redshift_incremental_view_manifest(self): 'seed.test.seed': { 'compiled_path': None, 'build_path': None, - 'patch_path': self.dir('seed/schema.yml'), + 'created_at': ANY, + 'patch_path': 'test://' + self.dir('seed/schema.yml'), 'path': 'seed.csv', 'name': 'seed', 'root_path': self.test_root_realpath, @@ -2838,6 +2865,7 @@ def expected_redshift_incremental_view_manifest(self): 'alias': 'snapshot_seed', 'compiled_path': None, 'build_path': None, + 'created_at': ANY, 'checksum': self._checksum_file(snapshot_path), 'columns': {}, 'compiled': True, diff --git a/test/integration/067_partial_parsing_tests/analysis/.gitignore b/test/integration/067_partial_parsing_tests/analysis/.gitignore new file mode 100644 index 00000000000..5e7d2734cfc --- /dev/null +++ b/test/integration/067_partial_parsing_tests/analysis/.gitignore @@ -0,0 +1,4 @@ +# Ignore everything in this directory +* +# Except this file +!.gitignore diff --git a/test/integration/067_partial_parsing_tests/extra-files/model_three.sql b/test/integration/067_partial_parsing_tests/extra-files/model_three.sql new file mode 100644 index 00000000000..45aa2b750f7 --- /dev/null +++ b/test/integration/067_partial_parsing_tests/extra-files/model_three.sql @@ -0,0 +1,12 @@ +{{ config(materialized='table') }} + +with source_data as ( + + select 1 as id + union all + select null as id + +) + +select * +from source_data diff --git a/test/integration/067_partial_parsing_tests/extra-files/model_two.sql b/test/integration/067_partial_parsing_tests/extra-files/model_two.sql new file mode 100644 index 00000000000..01f38b0698e --- /dev/null +++ b/test/integration/067_partial_parsing_tests/extra-files/model_two.sql @@ -0,0 +1 @@ +select 1 as notfun diff --git a/test/integration/067_partial_parsing_tests/extra-files/models-schema1.yml b/test/integration/067_partial_parsing_tests/extra-files/models-schema1.yml new file mode 100644 index 00000000000..36e5ce68a6e --- /dev/null +++ b/test/integration/067_partial_parsing_tests/extra-files/models-schema1.yml @@ -0,0 +1,5 @@ +version: 2 + +models: + - name: model_one + description: "The first model" diff --git a/test/integration/067_partial_parsing_tests/extra-files/models-schema2.yml b/test/integration/067_partial_parsing_tests/extra-files/models-schema2.yml new file mode 100644 index 00000000000..c758661136f --- /dev/null +++ b/test/integration/067_partial_parsing_tests/extra-files/models-schema2.yml @@ -0,0 +1,9 @@ +version: 2 + +models: + - name: model_one + description: "The first model" + - name: model_three + description: "The third model" + tests: + - unique diff --git a/test/integration/067_partial_parsing_tests/extra-files/raw_customers.csv b/test/integration/067_partial_parsing_tests/extra-files/raw_customers.csv new file mode 100644 index 00000000000..2315be73844 --- /dev/null +++ b/test/integration/067_partial_parsing_tests/extra-files/raw_customers.csv @@ -0,0 +1,11 @@ +id,first_name,last_name,email +1,Michael,Perez,mperez0@chronoengine.com +2,Shawn,Mccoy,smccoy1@reddit.com +3,Kathleen,Payne,kpayne2@cargocollective.com +4,Jimmy,Cooper,jcooper3@cargocollective.com +5,Katherine,Rice,krice4@typepad.com +6,Sarah,Ryan,sryan5@gnu.org +7,Martin,Mcdonald,mmcdonald6@opera.com +8,Frank,Robinson,frobinson7@wunderground.com +9,Jennifer,Franklin,jfranklin8@mail.ru +10,Henry,Welch,hwelch9@list-manage.com diff --git a/test/integration/067_partial_parsing_tests/local_dependency/data/seed.csv b/test/integration/067_partial_parsing_tests/local_dependency/data/seed.csv new file mode 100644 index 00000000000..3ff3deb87eb --- /dev/null +++ b/test/integration/067_partial_parsing_tests/local_dependency/data/seed.csv @@ -0,0 +1,2 @@ +id +1 diff --git a/test/integration/067_partial_parsing_tests/local_dependency/dbt_project.yml b/test/integration/067_partial_parsing_tests/local_dependency/dbt_project.yml new file mode 100644 index 00000000000..cbf108e353a --- /dev/null +++ b/test/integration/067_partial_parsing_tests/local_dependency/dbt_project.yml @@ -0,0 +1,23 @@ + +name: 'local_dep' +version: '1.0' +config-version: 2 + +profile: 'default' + +source-paths: ["models"] +analysis-paths: ["analysis"] +test-paths: ["tests"] +data-paths: ["data"] +macro-paths: ["macros"] + +require-dbt-version: '>=0.1.0' + +target-path: "target" # directory which will store compiled SQL files +clean-targets: # directories to be removed by `dbt clean` + - "target" + - "dbt_modules" + + +seeds: + quote_columns: False diff --git a/test/integration/067_partial_parsing_tests/local_dependency/macros/dep_macro.sql b/test/integration/067_partial_parsing_tests/local_dependency/macros/dep_macro.sql new file mode 100644 index 00000000000..81e9a0faeef --- /dev/null +++ b/test/integration/067_partial_parsing_tests/local_dependency/macros/dep_macro.sql @@ -0,0 +1,3 @@ +{% macro some_overridden_macro() -%} +100 +{%- endmacro %} diff --git a/test/integration/067_partial_parsing_tests/local_dependency/macros/generate_schema_name.sql b/test/integration/067_partial_parsing_tests/local_dependency/macros/generate_schema_name.sql new file mode 100644 index 00000000000..127ba8c5575 --- /dev/null +++ b/test/integration/067_partial_parsing_tests/local_dependency/macros/generate_schema_name.sql @@ -0,0 +1,15 @@ +{# This should be ignored as it's in a subpackage #} +{% macro generate_schema_name(custom_schema_name=none, node=none) -%} + {{ exceptions.raise_compiler_error('invalid', node=node) }} +{%- endmacro %} + +{# This should be ignored as it's in a subpackage #} +{% macro generate_database_name(custom_database_name=none, node=none) -%} + {{ exceptions.raise_compiler_error('invalid', node=node) }} +{%- endmacro %} + + +{# This should be ignored as it's in a subpackage #} +{% macro generate_alias_name(custom_alias_name=none, node=none) -%} + {{ exceptions.raise_compiler_error('invalid', node=node) }} +{%- endmacro %} diff --git a/test/integration/067_partial_parsing_tests/local_dependency/models/model_to_import.sql b/test/integration/067_partial_parsing_tests/local_dependency/models/model_to_import.sql new file mode 100644 index 00000000000..4b91aa0f2fa --- /dev/null +++ b/test/integration/067_partial_parsing_tests/local_dependency/models/model_to_import.sql @@ -0,0 +1 @@ +select * from {{ ref('seed') }} diff --git a/test/integration/067_partial_parsing_tests/local_dependency/models/schema.yml b/test/integration/067_partial_parsing_tests/local_dependency/models/schema.yml new file mode 100644 index 00000000000..0f942e08325 --- /dev/null +++ b/test/integration/067_partial_parsing_tests/local_dependency/models/schema.yml @@ -0,0 +1,11 @@ +version: 2 +sources: + - name: my_source + schema: invalid_schema + tables: + - name: my_table +sources: + - name: seed_source + schema: "{{ var('schema_override', target.schema) }}" + tables: + - name: "seed" diff --git a/test/integration/067_partial_parsing_tests/macros/.gitignore b/test/integration/067_partial_parsing_tests/macros/.gitignore new file mode 100644 index 00000000000..5e7d2734cfc --- /dev/null +++ b/test/integration/067_partial_parsing_tests/macros/.gitignore @@ -0,0 +1,4 @@ +# Ignore everything in this directory +* +# Except this file +!.gitignore diff --git a/test/integration/067_partial_parsing_tests/models/.gitignore b/test/integration/067_partial_parsing_tests/models/.gitignore new file mode 100644 index 00000000000..7aebca2016d --- /dev/null +++ b/test/integration/067_partial_parsing_tests/models/.gitignore @@ -0,0 +1 @@ +model_two.sql diff --git a/test/integration/067_partial_parsing_tests/models/model_one.sql b/test/integration/067_partial_parsing_tests/models/model_one.sql new file mode 100644 index 00000000000..3bd54a4c1b6 --- /dev/null +++ b/test/integration/067_partial_parsing_tests/models/model_one.sql @@ -0,0 +1 @@ +select 1 as fun diff --git a/test/integration/067_partial_parsing_tests/seed/.gitignore b/test/integration/067_partial_parsing_tests/seed/.gitignore new file mode 100644 index 00000000000..5e7d2734cfc --- /dev/null +++ b/test/integration/067_partial_parsing_tests/seed/.gitignore @@ -0,0 +1,4 @@ +# Ignore everything in this directory +* +# Except this file +!.gitignore diff --git a/test/integration/067_partial_parsing_tests/snapshots/.gitignore b/test/integration/067_partial_parsing_tests/snapshots/.gitignore new file mode 100644 index 00000000000..5e7d2734cfc --- /dev/null +++ b/test/integration/067_partial_parsing_tests/snapshots/.gitignore @@ -0,0 +1,4 @@ +# Ignore everything in this directory +* +# Except this file +!.gitignore diff --git a/test/integration/067_partial_parsing_tests/test_partial_parsing.py b/test/integration/067_partial_parsing_tests/test_partial_parsing.py new file mode 100644 index 00000000000..b87844051a5 --- /dev/null +++ b/test/integration/067_partial_parsing_tests/test_partial_parsing.py @@ -0,0 +1,141 @@ +from dbt.exceptions import CompilationException +from dbt.contracts.graph.manifest import Manifest +from dbt.contracts.files import ParseFileType +from test.integration.base import DBTIntegrationTest, use_profile, normalize +import shutil +import os + +def get_manifest(): + path = './target/partial_parse.msgpack' + if os.path.exists(path): + with open(path, 'rb') as fp: + manifest_mp = fp.read() + manifest: Manifest = Manifest.from_msgpack(manifest_mp) + return manifest + else: + return None + +class TestModels(DBTIntegrationTest): + + @property + def schema(self): + return "test_067" + + @property + def models(self): + return "models" + + + @use_profile('postgres') + def test_postgres_partial_parsing(self): + # initial run + self.run_dbt(['clean']) + results = self.run_dbt(["run"]) + self.assertEqual(len(results), 1) + + # add a model file + shutil.copyfile('extra-files/model_two.sql', 'models/model_two.sql') + results = self.run_dbt(["--partial-parse", "run"]) + self.assertEqual(len(results), 2) + + # add a schema file + shutil.copyfile('extra-files/models-schema1.yml', 'models/schema.yml') + results = self.run_dbt(["--partial-parse", "run"]) + self.assertEqual(len(results), 2) + manifest = get_manifest() + self.assertIn('model.test.model_one', manifest.nodes) + model_one_node = manifest.nodes['model.test.model_one'] + self.assertEqual(model_one_node.description, 'The first model') + self.assertEqual(model_one_node.patch_path, 'test://models/schema.yml') + + # add a model and a schema file (with a test) at the same time + shutil.copyfile('extra-files/models-schema2.yml', 'models/schema.yml') + shutil.copyfile('extra-files/model_three.sql', 'models/model_three.sql') + results = self.run_dbt(["--partial-parse", "run"]) + self.assertEqual(len(results), 3) + manifest = get_manifest() + self.assertEqual(len(manifest.files), 33) + self.assertIn('test://models/model_three.sql', manifest.files) + model_three_file = manifest.files['test://models/model_three.sql'] + self.assertEqual(model_three_file.parse_file_type, ParseFileType.Model) + self.assertEqual(type(model_three_file).__name__, 'SourceFile') + model_three_node = manifest.nodes[model_three_file.nodes[0]] + schema_file_id = 'test://models/schema.yml' + self.assertEqual(model_three_node.patch_path, schema_file_id) + self.assertEqual(model_three_node.description, 'The third model') + schema_file = manifest.files[schema_file_id] + self.assertEqual(type(schema_file).__name__, 'SchemaSourceFile') + self.assertEqual(len(schema_file.tests), 1) + + # go back to previous version of schema file, removing patch and test for model three + shutil.copyfile('extra-files/models-schema1.yml', 'models/schema.yml') + results = self.run_dbt(["--partial-parse", "run"]) + self.assertEqual(len(results), 3) + + # remove schema file, still have 3 models + os.remove(normalize('models/schema.yml')) + results = self.run_dbt(["--partial-parse", "run"]) + self.assertEqual(len(results), 3) + manifest = get_manifest() + self.assertNotIn('test://models/schema.yml', manifest.files) + self.assertEqual(len(manifest.files), 32) + + # Put schema file back and remove a model + # referred to in schema file + shutil.copyfile('extra-files/models-schema2.yml', 'models/schema.yml') + os.remove(normalize('models/model_three.sql')) + with self.assertRaises(CompilationException): + results = self.run_dbt(["--partial-parse", "run"]) + + # Put model back again + shutil.copyfile('extra-files/model_three.sql', 'models/model_three.sql') + results = self.run_dbt(["--partial-parse", "run"]) + self.assertEqual(len(results), 3) + + def tearDown(self): + if os.path.exists(normalize('models/model_two.sql')): + os.remove(normalize('models/model_two.sql')) + if os.path.exists(normalize('models/model_three.sql')): + os.remove(normalize('models/model_three.sql')) + if os.path.exists(normalize('models/schema.yml')): + os.remove(normalize('models/schema.yml')) + + +class TestSources(DBTIntegrationTest): + + @property + def schema(self): + return "test_067" + + @property + def models(self): + return "models" + + + +class TestPartialParsingDependency(DBTIntegrationTest): + + @property + def schema(self): + return "test_067" + + @property + def models(self): + return "models" + + @property + def packages_config(self): + return { + "packages": [ + { + 'local': 'local_dependency' + } + ] + } + + @use_profile("postgres") + def test_postgres_duplicate_model_enabled_across_packages(self): + self.run_dbt(["deps"]) +# self.run_dbt(["run"]) + + diff --git a/test/integration/067_partial_parsing_tests/tests/.gitignore b/test/integration/067_partial_parsing_tests/tests/.gitignore new file mode 100644 index 00000000000..5e7d2734cfc --- /dev/null +++ b/test/integration/067_partial_parsing_tests/tests/.gitignore @@ -0,0 +1,4 @@ +# Ignore everything in this directory +* +# Except this file +!.gitignore diff --git a/test/unit/test_contracts_graph_compiled.py b/test/unit/test_contracts_graph_compiled.py index 5d6e6a4d8d9..9f9376439b5 100644 --- a/test/unit/test_contracts_graph_compiled.py +++ b/test/unit/test_contracts_graph_compiled.py @@ -11,8 +11,8 @@ from dbt.node_types import NodeType from .utils import ( - assert_from_dict, assert_symmetric, + assert_from_dict, assert_fails_validation, dict_replace, replace_config, @@ -88,6 +88,7 @@ def minimal_uncompiled_dict(): return { 'name': 'foo', 'root_path': '/root/', + 'created_at': 1, 'resource_type': str(NodeType.Model), 'path': '/root/models/foo.sql', 'original_file_path': 'models/foo.sql', @@ -109,6 +110,7 @@ def basic_uncompiled_dict(): return { 'name': 'foo', 'root_path': '/root/', + 'created_at': 1, 'resource_type': str(NodeType.Model), 'path': '/root/models/foo.sql', 'original_file_path': 'models/foo.sql', @@ -152,6 +154,7 @@ def basic_compiled_dict(): return { 'name': 'foo', 'root_path': '/root/', + 'created_at': 1, 'resource_type': str(NodeType.Model), 'path': '/root/models/foo.sql', 'original_file_path': 'models/foo.sql', @@ -310,6 +313,7 @@ def minimal_schema_test_dict(): return { 'name': 'foo', 'root_path': '/root/', + 'created_at': 1, 'resource_type': str(NodeType.Test), 'path': '/root/x/path.sql', 'original_file_path': '/root/path.sql', @@ -402,6 +406,7 @@ def basic_uncompiled_schema_test_dict(): return { 'name': 'foo', 'root_path': '/root/', + 'created_at': 1, 'resource_type': str(NodeType.Test), 'path': '/root/x/path.sql', 'original_file_path': '/root/path.sql', @@ -450,6 +455,7 @@ def basic_compiled_schema_test_dict(): return { 'name': 'foo', 'root_path': '/root/', + 'created_at': 1, 'resource_type': str(NodeType.Test), 'path': '/root/x/path.sql', 'original_file_path': '/root/path.sql', diff --git a/test/unit/test_contracts_graph_parsed.py b/test/unit/test_contracts_graph_parsed.py index f2fb11b3abd..5007ae35533 100644 --- a/test/unit/test_contracts_graph_parsed.py +++ b/test/unit/test_contracts_graph_parsed.py @@ -42,7 +42,7 @@ from dbt import flags from dbt.dataclass_schema import ValidationError -from .utils import ContractTestCase, assert_symmetric, assert_from_dict, assert_to_dict, compare_dicts, assert_fails_validation, dict_replace, replace_config +from .utils import ContractTestCase, assert_symmetric, assert_from_dict, compare_dicts, assert_fails_validation, dict_replace, replace_config @pytest.fixture(autouse=True) @@ -128,6 +128,7 @@ def base_parsed_model_dict(): return { 'name': 'foo', 'root_path': '/root/', + 'created_at': 1, 'resource_type': str(NodeType.Model), 'path': '/root/x/path.sql', 'original_file_path': '/root/path.sql', @@ -194,6 +195,7 @@ def minimal_parsed_model_dict(): return { 'name': 'foo', 'root_path': '/root/', + 'created_at': 1, 'resource_type': str(NodeType.Model), 'path': '/root/x/path.sql', 'original_file_path': '/root/path.sql', @@ -214,6 +216,7 @@ def complex_parsed_model_dict(): return { 'name': 'foo', 'root_path': '/root/', + 'created_at': 1, 'resource_type': str(NodeType.Model), 'path': '/root/x/path.sql', 'original_file_path': '/root/path.sql', @@ -410,6 +413,7 @@ def basic_parsed_seed_dict(): return { 'name': 'foo', 'root_path': '/root/', + 'created_at': 1, 'resource_type': str(NodeType.Seed), 'path': '/root/seeds/seed.csv', 'original_file_path': 'seeds/seed.csv', @@ -481,6 +485,7 @@ def minimal_parsed_seed_dict(): return { 'name': 'foo', 'root_path': '/root/', + 'created_at': 1, 'resource_type': str(NodeType.Seed), 'path': '/root/seeds/seed.csv', 'original_file_path': 'seeds/seed.csv', @@ -500,6 +505,7 @@ def complex_parsed_seed_dict(): return { 'name': 'foo', 'root_path': '/root/', + 'created_at': 1, 'resource_type': str(NodeType.Seed), 'path': '/root/seeds/seed.csv', 'original_file_path': 'seeds/seed.csv', @@ -654,7 +660,7 @@ def basic_parsed_model_patch_dict(): return { 'name': 'foo', 'description': 'The foo model', - 'original_file_path': '/path/to/schema.yml', + 'original_file_path': 'path/to/schema.yml', 'docs': {'show': True}, 'meta': {}, 'yaml_key': 'models', @@ -677,7 +683,7 @@ def basic_parsed_model_patch_object(): yaml_key='models', package_name='test', description='The foo model', - original_file_path='/path/to/schema.yml', + original_file_path='path/to/schema.yml', columns={'a': ColumnInfo(name='a', description='a text field', meta={})}, docs=Docs(), meta={}, @@ -706,7 +712,7 @@ def patched_model_object(): tags=[], meta={}, config=NodeConfig(), - patch_path='/path/to/schema.yml', + patch_path='test://path/to/schema.yml', columns={'a': ColumnInfo(name='a', description='a text field', meta={})}, docs=Docs(), checksum=FileHash.from_contents(''), @@ -751,6 +757,7 @@ def base_parsed_hook_dict(): return { 'name': 'foo', 'root_path': '/root/', + 'created_at': 1, 'resource_type': str(NodeType.Operation), 'path': '/root/x/path.sql', 'original_file_path': '/root/path.sql', @@ -819,6 +826,7 @@ def complex_parsed_hook_dict(): return { 'name': 'foo', 'root_path': '/root/', + 'created_at': 1, 'resource_type': str(NodeType.Operation), 'path': '/root/x/path.sql', 'original_file_path': '/root/path.sql', @@ -935,6 +943,7 @@ def minimal_parsed_schema_test_dict(): return { 'name': 'foo', 'root_path': '/root/', + 'created_at': 1, 'resource_type': str(NodeType.Test), 'path': '/root/x/path.sql', 'original_file_path': '/root/path.sql', @@ -959,6 +968,7 @@ def basic_parsed_schema_test_dict(): return { 'name': 'foo', 'root_path': '/root/', + 'created_at': 1, 'resource_type': str(NodeType.Test), 'path': '/root/x/path.sql', 'original_file_path': '/root/path.sql', @@ -1031,6 +1041,7 @@ def complex_parsed_schema_test_dict(): return { 'name': 'foo', 'root_path': '/root/', + 'created_at': 1, 'resource_type': str(NodeType.Test), 'path': '/root/x/path.sql', 'original_file_path': '/root/path.sql', @@ -1356,6 +1367,7 @@ def basic_timestamp_snapshot_dict(): return { 'name': 'foo', 'root_path': '/root/', + 'created_at': 1, 'resource_type': str(NodeType.Snapshot), 'path': '/root/x/path.sql', 'original_file_path': '/root/path.sql', @@ -1486,6 +1498,7 @@ def basic_check_snapshot_dict(): return { 'name': 'foo', 'root_path': '/root/', + 'created_at': 1, 'resource_type': str(NodeType.Snapshot), 'path': '/root/x/path.sql', 'original_file_path': '/root/path.sql', @@ -1652,7 +1665,7 @@ def populated_parsed_node_patch_dict(): return { 'name': 'foo', 'description': 'The foo model', - 'original_file_path': '/path/to/schema.yml', + 'original_file_path': 'path/to/schema.yml', 'columns': { 'a': { 'name': 'a', @@ -1673,7 +1686,7 @@ def populated_parsed_node_patch_object(): return ParsedNodePatch( name='foo', description='The foo model', - original_file_path='/path/to/schema.yml', + original_file_path='path/to/schema.yml', columns={'a': ColumnInfo(name='a', description='a text field', meta={})}, meta={'key': ['value']}, yaml_key='models', @@ -1694,6 +1707,7 @@ def _ok_dict(self): 'name': 'foo', 'path': '/root/path.sql', 'original_file_path': '/root/path.sql', + 'created_at': 1, 'package_name': 'test', 'macro_sql': '{% macro foo() %}select 1 as id{% endmacro %}', 'root_path': '/root/', @@ -1724,7 +1738,7 @@ def test_ok(self): description='my macro description', arguments=[], ) - self.assert_symmetric(macro, macro_dict) + assert_symmetric(macro, macro_dict) self.assertEqual(macro.local_vars(), {}) pickle.loads(pickle.dumps(macro)) @@ -1785,6 +1799,7 @@ def minimum_parsed_source_definition_dict(): 'root_path': '/root', 'path': '/root/models/sources.yml', 'original_file_path': '/root/models/sources.yml', + 'created_at': 1, 'database': 'some_db', 'schema': 'some_schema', 'fqn': ['test', 'source', 'my_source', 'my_source_table'], @@ -1805,6 +1820,7 @@ def basic_parsed_source_definition_dict(): 'root_path': '/root', 'path': '/root/models/sources.yml', 'original_file_path': '/root/models/sources.yml', + 'created_at': 1, 'database': 'some_db', 'schema': 'some_schema', 'fqn': ['test', 'source', 'my_source', 'my_source_table'], @@ -1860,6 +1876,7 @@ def complex_parsed_source_definition_dict(): 'root_path': '/root', 'path': '/root/models/sources.yml', 'original_file_path': '/root/models/sources.yml', + 'created_at': 1, 'database': 'some_db', 'schema': 'some_schema', 'fqn': ['test', 'source', 'my_source', 'my_source_table'], @@ -2010,7 +2027,8 @@ def minimal_parsed_exposure_dict(): 'path': 'models/something.yml', 'root_path': '/usr/src/app', 'original_file_path': 'models/something.yml', - 'description': '' + 'description': '', + 'created_at': 1, } @@ -2035,7 +2053,8 @@ def basic_parsed_exposure_dict(): 'path': 'models/something.yml', 'root_path': '/usr/src/app', 'original_file_path': 'models/something.yml', - 'description': '' + 'description': '', + 'created_at': 1, } @@ -2060,6 +2079,7 @@ def complex_parsed_exposure_dict(): return { 'name': 'my_exposure', 'type': 'analysis', + 'created_at': 1, 'owner': { 'email': 'test@example.com', 'name': 'A Name', diff --git a/test/unit/test_docs_blocks.py b/test/unit/test_docs_blocks.py index 224e6c85300..fd244cf4547 100644 --- a/test/unit/test_docs_blocks.py +++ b/test/unit/test_docs_blocks.py @@ -144,7 +144,7 @@ def _build_file(self, contents, relative_path) -> FileBlock: def test_load_file(self): parser = docs.DocumentationParser( root_project=self.root_project_config, - manifest=Manifest({}, {}, {}, {}, {}, {}, [], {}), + manifest=Manifest(), project=self.subdir_project_config, ) @@ -169,7 +169,7 @@ def test_load_file_extras(self): parser = docs.DocumentationParser( root_project=self.root_project_config, - manifest=Manifest({}, {}, {}, {}, {}, {}, [], {}), + manifest=Manifest(), project=self.subdir_project_config, ) @@ -186,7 +186,7 @@ def test_load_file_extras(self): def test_multiple_raw_blocks(self): parser = docs.DocumentationParser( root_project=self.root_project_config, - manifest=Manifest({}, {}, {}, {}, {}, {}, [], {}), + manifest=Manifest(), project=self.subdir_project_config, ) diff --git a/test/unit/test_graph.py b/test/unit/test_graph.py index 21c18d1d95c..d81a001cb99 100644 --- a/test/unit/test_graph.py +++ b/test/unit/test_graph.py @@ -184,7 +184,6 @@ def load_manifest(self, config): loader = dbt.parser.manifest.ManifestLoader(config, {config.project_name: config}) loader.manifest.macros = self.macro_manifest.macros loader.load() - loader.update_manifest() return loader.manifest def test__single_model(self): diff --git a/test/unit/test_manifest.py b/test/unit/test_manifest.py index f38bf4080f5..8e9c6523712 100644 --- a/test/unit/test_manifest.py +++ b/test/unit/test_manifest.py @@ -41,7 +41,7 @@ 'depends_on', 'database', 'schema', 'name', 'resource_type', 'package_name', 'root_path', 'path', 'original_file_path', 'raw_sql', 'description', 'columns', 'fqn', 'build_path', 'compiled_path', 'patch_path', 'docs', - 'deferred', 'checksum', 'unrendered_config', + 'deferred', 'checksum', 'unrendered_config', 'created_at', }) REQUIRED_COMPILED_NODE_KEYS = frozenset(REQUIRED_PARSED_NODE_KEYS | { diff --git a/test/unit/test_parse_manifest.py b/test/unit/test_parse_manifest.py index edf950586ec..ec804d1737e 100644 --- a/test/unit/test_parse_manifest.py +++ b/test/unit/test_parse_manifest.py @@ -102,62 +102,7 @@ def _new_file(self, searched, name, match): ) return SourceFile(path=path, checksum=checksum) - def test_model_no_cache(self): - source_file = self._matching_file('models', 'model_1.sql') - self.parser.load_file.return_value = source_file - - self.loader.old_manifest = None - self.loader.parse_with_cache(FileBlock(source_file), self.parser) - # there was nothing in the cache, so parse_file should get called - # with a FileBlock that has the given source file in it self.parser.parse_file.assert_called_once_with(FileBlock(file=source_file)) - def test_model_cache_hit(self): - source_file = self._matching_file('models', 'model_1.sql') - self.parser.load_file.return_value = source_file - - source_file_dupe = self._matching_file('models', 'model_1.sql') - source_file_dupe.nodes.append('model.root.model_1') - - old_manifest = self._new_manifest() - old_manifest.files[source_file_dupe.path.search_key] = source_file_dupe - self.loader.old_manifest = old_manifest - self.loader.old_manifest.nodes = {'model.root.model_1': mock.MagicMock()} - - self.loader.parse_with_cache(FileBlock(source_file), self.parser) - # there was a cache hit, so parse_file should never have been called - self.parser.parse_file.assert_not_called() - - def test_model_cache_mismatch_checksum(self): - source_file = self._mismatched_file('models', 'model_1.sql') - self.parser.load_file.return_value = source_file - - source_file_dupe = self._mismatched_file('models', 'model_1.sql') - source_file_dupe.nodes.append('model.root.model_1') - - old_manifest = self._new_manifest() - old_manifest.files[source_file_dupe.path.search_key] = source_file_dupe - old_manifest.nodes = {'model.root.model_1': mock.MagicMock()} - self.loader.old_manifest = old_manifest - - self.loader.parse_with_cache(FileBlock(source_file), self.parser) - # there was a cache checksum mismatch, so parse_file should get called - # with a FileBlock that has the given source file in it - self.parser.parse_file.assert_called_once_with(FileBlock(file=source_file)) - - def test_model_cache_missing_file(self): - source_file = self._matching_file('models', 'model_1.sql') - self.parser.load_file.return_value = source_file - - source_file_different = self._matching_file('models', 'model_2.sql') - source_file_different.nodes.append('model.root.model_2') - - old_manifest = self._new_manifest() - old_manifest.files[source_file_different.path.search_key] = source_file_different - old_manifest.nodes = {'model.root.model_2': mock.MagicMock()} - - self.loader.old_manifest = old_manifest - self.loader.parse_with_cache(FileBlock(source_file), self.parser) - # the filename wasn't in the cache, so parse_file should get called - # with a FileBlock that has the given source file in it. - self.parser.parse_file.assert_called_once_with(FileBlock(file=source_file)) +# Note: none of the tests in this test case made sense with the removal +# of the old method of partial parsing. diff --git a/test/unit/test_parser.py b/test/unit/test_parser.py index 582888ae469..0fe89ad8865 100644 --- a/test/unit/test_parser.py +++ b/test/unit/test_parser.py @@ -17,10 +17,10 @@ ) from dbt.parser.search import FileBlock from dbt.parser.schema_test_builders import YamlBlock -from dbt.parser.manifest import process_docs, process_sources, process_refs +from dbt.parser.sources import SourcePatcher from dbt.node_types import NodeType -from dbt.contracts.files import SourceFile, FileHash, FilePath +from dbt.contracts.files import SourceFile, FileHash, FilePath, SchemaSourceFile from dbt.contracts.graph.manifest import Manifest from dbt.contracts.graph.model_config import ( NodeConfig, TestConfig, SnapshotConfig @@ -129,7 +129,6 @@ def setUp(self): self.manifest = Manifest( macros={m.unique_id: m for m in generate_name_macros('root')}, - nodes={}, sources={}, docs={}, exposures={}, selectors={}, disabled={}, files={}, ) def tearDown(self): @@ -144,21 +143,22 @@ def file_block_for(self, data: str, filename: str, searched: str): relative_path=filename, project_root=root_dir, ) - source_file = SourceFile( + sf_cls = SchemaSourceFile if filename.endswith('.yml') else SourceFile + source_file = sf_cls( path=path, checksum=FileHash.from_contents(data), + project_name='snowplow', ) source_file.contents = data return FileBlock(file=source_file) - def assert_has_manifest_lengths(self, manifest, files=1, macros=3, nodes=0, - sources=0, docs=0, patches=0, disabled=0): - self.assertEqual(len(manifest.files), files) + def assert_has_manifest_lengths(self, manifest, macros=3, nodes=0, + sources=0, docs=0, disabled=0): self.assertEqual(len(manifest.macros), macros) self.assertEqual(len(manifest.nodes), nodes) self.assertEqual(len(manifest.sources), sources) self.assertEqual(len(manifest.docs), docs) - self.assertEqual(sum(len(v) for v in manifest.disabled.values()), disabled) + self.assertEqual(len(manifest.disabled), disabled) @@ -228,6 +228,10 @@ def setUp(self): manifest=self.manifest, root_project=self.root_project_config, ) + self.source_patcher = SourcePatcher( + root_project=self.root_project_config, + manifest=self.manifest, + ) def file_block_for(self, data, filename): return super().file_block_for(data, filename, 'models') @@ -251,7 +255,6 @@ def test__read_basic_source(self): self.assertEqual(len(model_blocks), 0) self.assertEqual(len(source_blocks), 0) self.assertEqual(len(macro_blocks), 0) - self.assertEqual(len(list(self.parser.manifest.patches)), 0) self.assertEqual(len(list(self.parser.manifest.nodes)), 0) source_values = list(self.parser.manifest.sources.values()) self.assertEqual(len(source_values), 1) @@ -283,7 +286,6 @@ def test__read_basic_source_tests(self): self.assertEqual(len(source_tests), 0) self.assertEqual(len(macro_tests), 0) self.assertEqual(len(list(self.parser.manifest.nodes)), 0) - self.assertEqual(len(list(self.parser.manifest.patches)), 0) self.assertEqual(len(list(self.parser.manifest.source_patches)), 0) source_values = list(self.parser.manifest.sources.values()) self.assertEqual(len(source_values), 1) @@ -294,10 +296,10 @@ def test__read_basic_source_tests(self): def test__parse_basic_source_tests(self): block = self.file_block_for(SINGLE_TABLE_SOURCE_TESTS, 'test_one.yml') + self.parser.manifest.files[block.file.file_id] = block.file self.parser.parse_file(block) self.assertEqual(len(self.parser.manifest.nodes), 0) self.assertEqual(len(self.parser.manifest.sources), 1) - self.assertEqual(len(self.parser.manifest.patches), 0) src = list(self.parser.manifest.sources.values())[0] self.assertEqual(src.source.name, 'my_source') self.assertEqual(src.source.schema, None) @@ -305,7 +307,7 @@ def test__parse_basic_source_tests(self): self.assertEqual(src.table.description, 'A description of my table') tests = [ - self.parser.parse_source_test(src, test, col) + self.source_patcher.parse_source_test(src, test, col) for test, col in src.get_tests() ] tests.sort(key=lambda n: n.unique_id) @@ -321,13 +323,12 @@ def test__parse_basic_source_tests(self): self.assertEqual(tests[1].column_name, 'color') self.assertEqual(tests[1].fqn, ['snowplow', 'schema_test', tests[1].name]) - path = get_abs_os_path('./dbt_modules/snowplow/models/test_one.yml') - self.assertIn(path, self.parser.manifest.files) - self.assertEqual(self.parser.manifest.files[path].nodes, []) - self.assertIn(path, self.parser.manifest.files) - self.assertEqual(self.parser.manifest.files[path].sources, + file_id = 'snowplow://models/test_one.yml' + self.assertIn(file_id, self.parser.manifest.files) + self.assertEqual(self.parser.manifest.files[file_id].tests, []) + self.assertEqual(self.parser.manifest.files[file_id].sources, ['source.snowplow.my_source.my_table']) - self.assertEqual(self.parser.manifest.files[path].source_patches, []) + self.assertEqual(self.parser.manifest.files[file_id].source_patches, []) def test__read_source_patch(self): block = self.yaml_block_for(SINGLE_TABLE_SOURCE_PATCH, 'test_one.yml') @@ -340,7 +341,6 @@ def test__read_source_patch(self): self.assertEqual(len(source_tests), 0) self.assertEqual(len(macro_tests), 0) self.assertEqual(len(list(self.parser.manifest.nodes)), 0) - self.assertEqual(len(list(self.parser.manifest.patches)), 0) self.assertEqual(len(list(self.parser.manifest.sources)), 0) source_patches = list(self.parser.manifest.source_patches.values()) self.assertEqual(len(source_patches), 1) @@ -356,36 +356,44 @@ def test__read_source_patch(self): class SchemaParserModelsTest(SchemaParserTest): + def setUp(self): + super().setUp() + my_model_node = MockNode( + package='root', + name='my_model', + config=mock.MagicMock(enabled=True), + refs=[], + sources=[], + patch_path=None, + ) + nodes = {my_model_node.unique_id: my_model_node} + macros={m.unique_id: m for m in generate_name_macros('root')} + self.manifest = Manifest(nodes=nodes, macros=macros) + self.manifest.ref_lookup + self.parser = SchemaParser( + project=self.snowplow_project_config, + manifest=self.manifest, + root_project=self.root_project_config, + ) + def test__read_basic_model_tests(self): block = self.yaml_block_for(SINGLE_TABLE_MODEL_TESTS, 'test_one.yml') self.parser.parse_file(block) - self.assertEqual(len(list(self.parser.manifest.patches)), 1) self.assertEqual(len(list(self.parser.manifest.sources)), 0) - self.assertEqual(len(list(self.parser.manifest.nodes)), 3) + self.assertEqual(len(list(self.parser.manifest.nodes)), 4) def test__parse_basic_model_tests(self): block = self.file_block_for(SINGLE_TABLE_MODEL_TESTS, 'test_one.yml') + self.parser.manifest.files[block.file.file_id] = block.file self.parser.parse_file(block) - self.assert_has_manifest_lengths(self.parser.manifest, nodes=3) - self.assertEqual(len(self.parser.manifest.patches), 1) - - patch = list(self.parser.manifest.patches.values())[0] - self.assertEqual(len(patch.columns), 1) - self.assertEqual(patch.name, 'my_model') - self.assertEqual(patch.description, 'A description of my model') - expected_patch = ParsedNodePatch( - name='my_model', - description='A description of my model', - columns={'color': ColumnInfo(name='color', description='The color value')}, - original_file_path=normalize('models/test_one.yml'), - meta={}, - yaml_key='models', - package_name='snowplow', - docs=Docs(show=True), - ) - self.assertEqual(patch, expected_patch) - - tests = sorted(self.parser.manifest.nodes.values(), key=lambda n: n.unique_id) + self.assert_has_manifest_lengths(self.parser.manifest, nodes=4) + + all_nodes = sorted(self.parser.manifest.nodes.values(), key=lambda n: n.unique_id) + tests = [] + for node in all_nodes: + if node.resource_type != NodeType.Test: + continue + tests.append(node) self.assertEqual(tests[0].config.severity, 'ERROR') self.assertEqual(tests[0].tags, ['schema']) self.assertEqual(tests[0].refs, [['my_model']]) @@ -445,12 +453,11 @@ def test__parse_basic_model_tests(self): }, ) - path = get_abs_os_path('./dbt_modules/snowplow/models/test_one.yml') - self.assertIn(path, self.parser.manifest.files) - self.assertEqual(sorted(self.parser.manifest.files[path].nodes), + file_id = 'snowplow://models/test_one.yml' + self.assertIn(file_id, self.parser.manifest.files) + self.assertEqual(sorted(self.parser.manifest.files[file_id].tests), [t.unique_id for t in tests]) - self.assertIn(path, self.parser.manifest.files) - self.assertEqual(self.parser.manifest.files[path].patches, ['my_model']) + self.assertEqual(self.parser.manifest.files[file_id].node_patches, ['model.root.my_model']) class ModelParserTest(BaseParserTest): @@ -468,6 +475,7 @@ def file_block_for(self, data, filename): def test_basic(self): raw_sql = '{{ config(materialized="table") }}select 1 as id' block = self.file_block_for(raw_sql, 'nested/model_1.sql') + self.parser.manifest.files[block.file.file_id] = block.file self.parser.parse_file(block) self.assert_has_manifest_lengths(self.parser.manifest, nodes=1) node = list(self.parser.manifest.nodes.values())[0] @@ -489,15 +497,14 @@ def test_basic(self): unrendered_config={'materialized': 'table'}, ) self.assertEqual(node, expected) - path = get_abs_os_path('./dbt_modules/snowplow/models/nested/model_1.sql') - self.assertIn(path, self.parser.manifest.files) - self.assertEqual(self.parser.manifest.files[path].nodes, ['model.snowplow.model_1']) + file_id = 'snowplow://models/nested/model_1.sql' + self.assertIn(file_id, self.parser.manifest.files) + self.assertEqual(self.parser.manifest.files[file_id].nodes, ['model.snowplow.model_1']) def test_parse_error(self): block = self.file_block_for('{{ SYNTAX ERROR }}', 'nested/model_1.sql') with self.assertRaises(CompilationException): self.parser.parse_file(block) - self.assert_has_manifest_lengths(self.parser.manifest, files=0) class SnapshotParserTest(BaseParserTest): @@ -516,7 +523,6 @@ def test_parse_error(self): block = self.file_block_for('{% snapshot foo %}select 1 as id{%snapshot bar %}{% endsnapshot %}', 'nested/snap_1.sql') with self.assertRaises(CompilationException): self.parser.parse_file(block) - self.assert_has_manifest_lengths(self.parser.manifest, files=0) def test_single_block(self): raw_sql = '''{{ @@ -529,6 +535,7 @@ def test_single_block(self): {{% snapshot foo %}}{}{{% endsnapshot %}} '''.format(raw_sql) block = self.file_block_for(full_file, 'nested/snap_1.sql') + self.parser.manifest.files[block.file.file_id] = block.file self.parser.parse_file(block) self.assert_has_manifest_lengths(self.parser.manifest, nodes=1) node = list(self.parser.manifest.nodes.values())[0] @@ -563,10 +570,14 @@ def test_single_block(self): 'updated_at': 'last_update', }, ) - self.assertEqual(node, expected) - path = get_abs_os_path('./dbt_modules/snowplow/snapshots/nested/snap_1.sql') - self.assertIn(path, self.parser.manifest.files) - self.assertEqual(self.parser.manifest.files[path].nodes, ['snapshot.snowplow.foo']) + expected_dict = expected.to_dict() + del expected_dict['created_at'] + node_dict = node.to_dict() + del node_dict['created_at'] + self.assertEqual(node_dict, expected_dict) + file_id = 'snowplow://snapshots/nested/snap_1.sql' + self.assertIn(file_id, self.parser.manifest.files) + self.assertEqual(self.parser.manifest.files[file_id].nodes, ['snapshot.snowplow.foo']) def test_multi_block(self): raw_1 = ''' @@ -590,6 +601,7 @@ def test_multi_block(self): {{% snapshot bar %}}{}{{% endsnapshot %}} '''.format(raw_1, raw_2) block = self.file_block_for(full_file, 'nested/snap_1.sql') + self.parser.manifest.files[block.file.file_id] = block.file self.parser.parse_file(block) self.assert_has_manifest_lengths(self.parser.manifest, nodes=2) nodes = sorted(self.parser.manifest.nodes.values(), key=lambda n: n.name) @@ -655,9 +667,9 @@ def test_multi_block(self): ) self.assertEqual(nodes[0], expect_bar) self.assertEqual(nodes[1], expect_foo) - path = get_abs_os_path('./dbt_modules/snowplow/snapshots/nested/snap_1.sql') - self.assertIn(path, self.parser.manifest.files) - self.assertEqual(sorted(self.parser.manifest.files[path].nodes), + file_id = 'snowplow://snapshots/nested/snap_1.sql' + self.assertIn(file_id, self.parser.manifest.files) + self.assertEqual(sorted(self.parser.manifest.files[file_id].nodes), ['snapshot.snowplow.bar', 'snapshot.snowplow.foo']) @@ -666,7 +678,7 @@ def setUp(self): super().setUp() self.parser = MacroParser( project=self.snowplow_project_config, - manifest=Manifest({},{},{},{},{},{},{},{}) + manifest=Manifest() ) def file_block_for(self, data, filename): @@ -675,6 +687,7 @@ def file_block_for(self, data, filename): def test_single_block(self): raw_sql = '{% macro foo(a, b) %}a ~ b{% endmacro %}' block = self.file_block_for(raw_sql, 'macro.sql') + self.parser.manifest.files[block.file.file_id] = block.file self.parser.parse_file(block) self.assertEqual(len(self.parser.manifest.macros), 1) macro = list(self.parser.manifest.macros.values())[0] @@ -689,13 +702,15 @@ def test_single_block(self): macro_sql=raw_sql, ) self.assertEqual(macro, expected) - path = get_abs_os_path('./dbt_modules/snowplow/macros/macro.sql') - self.assertIn(path, self.parser.manifest.files) - self.assertEqual(self.parser.manifest.files[path].macros, ['macro.snowplow.foo']) + file_id = 'snowplow://macros/macro.sql' + self.assertIn(file_id, self.parser.manifest.files) + self.assertEqual(self.parser.manifest.files[file_id].macros, ['macro.snowplow.foo']) def test_multiple_blocks(self): raw_sql = '{% macro foo(a, b) %}a ~ b{% endmacro %}\n{% macro bar(c, d) %}c + d{% endmacro %}' block = self.file_block_for(raw_sql, 'macro.sql') + print(f"--- test_multiple_blocks block: {block}") + self.parser.manifest.files[block.file.file_id] = block.file self.parser.parse_file(block) self.assertEqual(len(self.parser.manifest.macros), 2) macros = sorted(self.parser.manifest.macros.values(), key=lambda m: m.name) @@ -720,10 +735,10 @@ def test_multiple_blocks(self): macro_sql='{% macro foo(a, b) %}a ~ b{% endmacro %}', ) self.assertEqual(macros, [expected_bar, expected_foo]) - path = get_abs_os_path('./dbt_modules/snowplow/macros/macro.sql') - self.assertIn(path, self.parser.manifest.files) + file_id = 'snowplow://macros/macro.sql' + self.assertIn(file_id, self.parser.manifest.files) self.assertEqual( - sorted(self.parser.manifest.files[path].macros), + sorted(self.parser.manifest.files[file_id].macros), ['macro.snowplow.bar', 'macro.snowplow.foo'], ) @@ -743,6 +758,7 @@ def file_block_for(self, data, filename): def test_basic(self): raw_sql = 'select * from {{ ref("blah") }} limit 0' block = self.file_block_for(raw_sql, 'test_1.sql') + self.manifest.files[block.file.file_id] = block.file self.parser.parse_file(block) self.assert_has_manifest_lengths(self.parser.manifest, nodes=1) node = list(self.parser.manifest.nodes.values())[0] @@ -766,9 +782,9 @@ def test_basic(self): unrendered_config={}, ) self.assertEqual(node, expected) - path = get_abs_os_path('./dbt_modules/snowplow/tests/test_1.sql') - self.assertIn(path, self.parser.manifest.files) - self.assertEqual(self.parser.manifest.files[path].nodes, ['test.snowplow.test_1']) + file_id = 'snowplow://tests/test_1.sql' + self.assertIn(file_id, self.parser.manifest.files) + self.assertEqual(self.parser.manifest.files[file_id].nodes, ['test.snowplow.test_1']) class AnalysisParserTest(BaseParserTest): @@ -786,6 +802,7 @@ def file_block_for(self, data, filename): def test_basic(self): raw_sql = 'select 1 as id' block = self.file_block_for(raw_sql, 'nested/analysis_1.sql') + self.manifest.files[block.file.file_id] = block.file self.parser.parse_file(block) self.assert_has_manifest_lengths(self.parser.manifest, nodes=1) node = list(self.parser.manifest.nodes.values())[0] @@ -808,69 +825,8 @@ def test_basic(self): unrendered_config={}, ) self.assertEqual(node, expected) - path = get_abs_os_path('./dbt_modules/snowplow/analyses/nested/analysis_1.sql') - self.assertIn(path, self.parser.manifest.files) - self.assertEqual(self.parser.manifest.files[path].nodes, ['analysis.snowplow.analysis_1']) - - -class ProcessingTest(BaseParserTest): - def setUp(self): - super().setUp() - x_depends_on = mock.MagicMock() - y_depends_on = mock.MagicMock() - self.x_node = MockNode( - package='project', - name='x', - config=mock.MagicMock(enabled=True), - refs=[], - sources=[['src', 'tbl']], - depends_on=x_depends_on, - description='other_project: {{ doc("otherproject", "my_doc") }}', - ) - self.y_node = MockNode( - package='otherproject', - name='y', - config=mock.MagicMock(enabled=True), - refs=[['x']], - sources=[], - depends_on=y_depends_on, - description='{{ doc("my_doc") }}', - ) - self.src_node = MockSource( - package='thirdproject', - source_name='src', - name='tbl', - config=mock.MagicMock(enabled=True), - ) - self.doc = MockDocumentation( - package='otherproject', - name='my_doc', - block_contents='some docs', - ) - nodes = { - self.x_node.unique_id: self.x_node, - self.y_node.unique_id: self.y_node, - } - sources = { - self.src_node.unique_id: self.src_node, - } - docs = { - self.doc.unique_id: self.doc, - } - self.manifest = Manifest( - nodes=nodes, sources=sources, macros={}, docs=docs, - disabled=[], files={}, exposures={}, selectors={}, - ) - - def test_process_docs(self): - process_docs(self.manifest, self.root_project_config) - self.assertEqual(self.x_node.description, 'other_project: some docs') - self.assertEqual(self.y_node.description, 'some docs') + file_id = 'snowplow://analyses/nested/analysis_1.sql' + self.assertIn(file_id, self.parser.manifest.files) + self.assertEqual(self.parser.manifest.files[file_id].nodes, ['analysis.snowplow.analysis_1']) - def test_process_sources(self): - process_sources(self.manifest, 'project') - self.x_node.depends_on.nodes.append.assert_called_once_with('source.thirdproject.src.tbl') - def test_process_refs(self): - process_refs(self.manifest, 'project') - self.y_node.depends_on.nodes.append.assert_called_once_with('model.project.x') diff --git a/test/unit/test_partial_parsing.py b/test/unit/test_partial_parsing.py new file mode 100644 index 00000000000..5ad35e2ea8d --- /dev/null +++ b/test/unit/test_partial_parsing.py @@ -0,0 +1,78 @@ +import unittest +from unittest import mock + +import dbt.exceptions +from dbt.parser.partial import PartialParsing +from dbt.contracts.graph.manifest import Manifest +from dbt.contracts.graph.parsed import ParsedModelNode +from dbt.contracts.files import ParseFileType, SourceFile, SchemaSourceFile, FilePath, FileHash +from dbt.node_types import NodeType + + +class TestPartialParsing(unittest.TestCase): + + def setUp(self): + + project_name = 'my_test' + project_root = '/users/root' + model_file = SourceFile( + path=FilePath(project_root=project_root, searched_path='models', relative_path='my_model.sql'), + checksum=FileHash.from_contents('abcdef'), + project_name=project_name, + parse_file_type=ParseFileType.Model, + nodes=['model.my_test.my_model'], + ) + schema_file = SchemaSourceFile( + path=FilePath(project_root=project_root, searched_path='models', relative_path='schema.yml'), + checksum=FileHash.from_contents('ghijkl'), + project_name=project_name, + parse_file_type=ParseFileType.Schema, + dfy={'version': 2, 'models': [{'name': 'my_model', 'description': 'Test model'}]}, + ndp=['model.my_test.my_model'], + ) + self.saved_files = {model_file.file_id: model_file, schema_file.file_id: schema_file} + model_node = self.get_model('my_model') + nodes = { model_node.unique_id: model_node } + self.saved_manifest = Manifest(files=self.saved_files, nodes=nodes) + self.new_files = { + model_file.file_id: SourceFile.from_dict(model_file.to_dict()), + schema_file.file_id: SchemaSourceFile.from_dict(schema_file.to_dict()), + } + + self.partial_parsing = PartialParsing(self.saved_manifest, self.new_files) + + def get_model(self, name): + return ParsedModelNode( + package_name='my_test', + root_path='/users/root/', + path=f'{name}.sql', + original_file_path=f'models/{name}.sql', + raw_sql='select * from wherever', + name=name, + resource_type=NodeType.Model, + unique_id=f'model.my_test.{name}', + fqn=['my_test', 'models', name], + database='test_db', + schema='test_schema', + alias='bar', + checksum=FileHash.from_contents(''), + patch_path='my_test://models/schema.yml', + ) + + def test_simple(self): + + # Nothing has changed + self.assertIsNotNone(self.partial_parsing) + self.assertTrue(self.partial_parsing.skip_parsing()) + + # Change a model file + self.partial_parsing.new_files['my_test://models/my_model.sql'].checksum = FileHash.from_contents('xyzabc') + self.partial_parsing.build_file_diff() + self.assertFalse(self.partial_parsing.skip_parsing()) + pp_files = self.partial_parsing.get_parsing_files() + # models has 'patch_path' so we expect to see a SchemaParser file listed + expected_pp_files = {'my_test': {'ModelParser': ['my_test://models/my_model.sql'], 'SchemaParser': ['my_test://models/schema.yml']}} + self.assertEqual(pp_files, expected_pp_files) + expected_pp_dict = {'version': 2, 'models': [{'name': 'my_model', 'description': 'Test model'}]} + schema_file = self.saved_files['my_test://models/schema.yml'] + self.assertEqual(schema_file.pp_dict, expected_pp_dict) diff --git a/test/unit/utils.py b/test/unit/utils.py index 118768d68af..01c737e1b63 100644 --- a/test/unit/utils.py +++ b/test/unit/utils.py @@ -184,15 +184,24 @@ def compare_dicts(dict1, dict2): print("--- Found no differences in dictionaries") -def assert_to_dict(obj, dct): - assert obj.to_dict(omit_none=True) == dct - - def assert_from_dict(obj, dct, cls=None): if cls is None: cls = obj.__class__ cls.validate(dct) - assert cls.from_dict(dct) == obj + obj_from_dict = cls.from_dict(dct) + if hasattr(obj, 'created_at'): + obj_from_dict.created_at = 1 + obj.created_at = 1 + assert obj_from_dict == obj + + +def assert_to_dict(obj, dct): + obj_to_dict = obj.to_dict(omit_none=True) + if 'created_at' in obj_to_dict: + obj_to_dict['created_at'] = 1 + if 'created_at' in dct: + dct['created_at'] = 1 + assert obj_to_dict == dct def assert_symmetric(obj, dct, cls=None):