diff --git a/CHANGELOG.md b/CHANGELOG.md index dd4e8cecde0..45b98820875 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,9 +1,15 @@ ## dbt 0.18.0 (Release TBD) +### Breaking changes +- Previously, dbt put macros from all installed plugins into the namespace. This version of dbt will not include adapter plugin macros unless they are from the currently-in-use adapter or one of its dependencies [#2590](https://github.com/fishtown-analytics/dbt/pull/2590) + ### Features - Added support for Snowflake query tags at the connection and model level ([#1030](https://github.com/fishtown-analytics/dbt/issues/1030), [#2555](https://github.com/fishtown-analytics/dbt/pull/2555/)) - Added option to specify profile when connecting to Redshift via IAM ([#2437](https://github.com/fishtown-analytics/dbt/issues/2437), [#2581](https://github.com/fishtown-analytics/dbt/pull/2581)) +### Fixes +- Adapter plugins can once again override plugins defined in core ([#2548](https://github.com/fishtown-analytics/dbt/issues/2548), [#2590](https://github.com/fishtown-analytics/dbt/pull/2590)) + Contributors: - [@brunomurino](https://github.com/brunomurino) ([#2437](https://github.com/fishtown-analytics/dbt/pull/2581)) - [@DrMcTaco](https://github.com/DrMcTaco) ([#1030](https://github.com/fishtown-analytics/dbt/issues/1030)),[#2555](https://github.com/fishtown-analytics/dbt/pull/2555/)) diff --git a/core/dbt/adapters/base/connections.py b/core/dbt/adapters/base/connections.py index 07088dc0083..e2df9c3ffde 100644 --- a/core/dbt/adapters/base/connections.py +++ b/core/dbt/adapters/base/connections.py @@ -10,7 +10,6 @@ import agate import dbt.exceptions -import dbt.flags from dbt.contracts.connection import ( Connection, Identifier, ConnectionState, AdapterRequiredConfig, LazyHandle ) @@ -19,6 +18,7 @@ MacroQueryStringSetter, ) from dbt.logger import GLOBAL_LOGGER as logger +from dbt import flags class BaseConnectionManager(metaclass=abc.ABCMeta): @@ -39,7 +39,7 @@ class BaseConnectionManager(metaclass=abc.ABCMeta): def __init__(self, profile: AdapterRequiredConfig): self.profile = profile self.thread_connections: Dict[Hashable, Connection] = {} - self.lock: RLock = dbt.flags.MP_CONTEXT.RLock() + self.lock: RLock = flags.MP_CONTEXT.RLock() self.query_header: Optional[MacroQueryStringSetter] = None def set_query_header(self, manifest: Manifest) -> None: @@ -60,7 +60,7 @@ def get_thread_connection(self) -> Connection: ) return self.thread_connections[key] - def set_thread_connection(self, conn): + def set_thread_connection(self, conn: Connection) -> None: key = self.get_thread_identifier() if key in self.thread_connections: raise dbt.exceptions.InternalException( @@ -235,7 +235,7 @@ def _close_handle(cls, connection: Connection) -> None: @classmethod def _rollback(cls, connection: Connection) -> None: """Roll back the given connection.""" - if dbt.flags.STRICT_MODE: + if flags.STRICT_MODE: if not isinstance(connection, Connection): raise dbt.exceptions.CompilerException( f'In _rollback, got {connection} - not a Connection!' @@ -253,7 +253,7 @@ def _rollback(cls, connection: Connection) -> None: @classmethod def close(cls, connection: Connection) -> Connection: - if dbt.flags.STRICT_MODE: + if flags.STRICT_MODE: if not isinstance(connection, Connection): raise dbt.exceptions.CompilerException( f'In close, got {connection} - not a Connection!' diff --git a/core/dbt/adapters/base/impl.py b/core/dbt/adapters/base/impl.py index 6af38b5264b..f4defc204b3 100644 --- a/core/dbt/adapters/base/impl.py +++ b/core/dbt/adapters/base/impl.py @@ -1,7 +1,6 @@ import abc from concurrent.futures import as_completed, Future from contextlib import contextmanager -from dataclasses import dataclass from datetime import datetime from itertools import chain from typing import ( @@ -17,21 +16,24 @@ get_relation_returned_multiple_results, InternalException, NotImplementedException, RuntimeException, ) -import dbt.flags +from dbt import flags from dbt import deprecations +from dbt.adapters.protocol import ( + AdapterConfig, + ConnectionManagerProtocol, +) from dbt.clients.agate_helper import empty_table, merge_tables, table_from_rows from dbt.clients.jinja import MacroGenerator from dbt.contracts.graph.compiled import CompileResultNode, CompiledSeedNode from dbt.contracts.graph.manifest import Manifest from dbt.contracts.graph.parsed import ParsedSeedNode -from dbt.contracts.graph.model_config import BaseConfig from dbt.exceptions import warn_or_error from dbt.node_types import NodeType from dbt.logger import GLOBAL_LOGGER as logger from dbt.utils import filter_null_values, executor -from dbt.adapters.base.connections import BaseConnectionManager, Connection +from dbt.adapters.base.connections import Connection from dbt.adapters.base.meta import AdapterMeta, available from dbt.adapters.base.relation import ( ComponentName, BaseRelation, InformationSchema, SchemaSearchMap @@ -108,11 +110,6 @@ def _relation_name(rel: Optional[BaseRelation]) -> str: return str(rel) -@dataclass -class AdapterConfig(BaseConfig): - pass - - class BaseAdapter(metaclass=AdapterMeta): """The BaseAdapter provides an abstract base class for adapters. @@ -151,7 +148,7 @@ class BaseAdapter(metaclass=AdapterMeta): """ Relation: Type[BaseRelation] = BaseRelation Column: Type[BaseColumn] = BaseColumn - ConnectionManager: Type[BaseConnectionManager] + ConnectionManager: Type[ConnectionManagerProtocol] # A set of clobber config fields accepted by this adapter # for use in materializations @@ -267,7 +264,7 @@ def load_internal_manifest(self) -> Manifest: def _schema_is_cached(self, database: Optional[str], schema: str) -> bool: """Check if the schema is cached, and by default logs if it is not.""" - if dbt.flags.USE_CACHE is False: + if flags.USE_CACHE is False: return False elif (database, schema) not in self.cache: logger.debug( @@ -323,7 +320,7 @@ def _relations_cache_for_schemas(self, manifest: Manifest) -> None: """Populate the relations cache for the given schemas. Returns an iterable of the schemas populated, as strings. """ - if not dbt.flags.USE_CACHE: + if not flags.USE_CACHE: return cache_schemas = self._get_cache_schemas(manifest) @@ -352,7 +349,7 @@ def set_relations_cache( """Run a query that gets a populated cache of the relations in the database and set the cache on this adapter. """ - if not dbt.flags.USE_CACHE: + if not flags.USE_CACHE: return with self.cache.lock: @@ -368,7 +365,7 @@ def cache_added(self, relation: Optional[BaseRelation]) -> str: raise_compiler_error( 'Attempted to cache a null relation for {}'.format(name) ) - if dbt.flags.USE_CACHE: + if flags.USE_CACHE: self.cache.add(relation) # so jinja doesn't render things return '' @@ -383,7 +380,7 @@ def cache_dropped(self, relation: Optional[BaseRelation]) -> str: raise_compiler_error( 'Attempted to drop a null relation for {}'.format(name) ) - if dbt.flags.USE_CACHE: + if flags.USE_CACHE: self.cache.drop(relation) return '' @@ -405,7 +402,7 @@ def cache_renamed( .format(src_name, dst_name, name) ) - if dbt.flags.USE_CACHE: + if flags.USE_CACHE: self.cache.rename(from_relation, to_relation) return '' diff --git a/core/dbt/adapters/base/plugin.py b/core/dbt/adapters/base/plugin.py index 4085d991248..c87b2a26a91 100644 --- a/core/dbt/adapters/base/plugin.py +++ b/core/dbt/adapters/base/plugin.py @@ -1,7 +1,19 @@ from typing import List, Optional, Type -from dbt.adapters.base import BaseAdapter, Credentials +from dbt.adapters.base import Credentials from dbt.exceptions import CompilationException +from dbt.adapters.protocol import AdapterProtocol + + +def project_name_from_path(include_path: str) -> str: + # avoid an import cycle + from dbt.config.project import Project + partial = Project.partial_load(include_path) + if partial.project_name is None: + raise CompilationException( + f'Invalid project at {include_path}: name not set!' + ) + return partial.project_name class AdapterPlugin: @@ -13,23 +25,16 @@ class AdapterPlugin: """ def __init__( self, - adapter: Type[BaseAdapter], + adapter: Type[AdapterProtocol], credentials: Type[Credentials], include_path: str, dependencies: Optional[List[str]] = None ): - # avoid an import cycle - from dbt.config.project import Project - self.adapter: Type[BaseAdapter] = adapter + self.adapter: Type[AdapterProtocol] = adapter self.credentials: Type[Credentials] = credentials self.include_path: str = include_path - partial = Project.partial_load(include_path) - if partial.project_name is None: - raise CompilationException( - f'Invalid project at {include_path}: name not set!' - ) - self.project_name: str = partial.project_name + self.project_name: str = project_name_from_path(include_path) self.dependencies: List[str] if dependencies is None: self.dependencies = [] diff --git a/core/dbt/adapters/base/relation.py b/core/dbt/adapters/base/relation.py index 44654bfb342..10dfc69677a 100644 --- a/core/dbt/adapters/base/relation.py +++ b/core/dbt/adapters/base/relation.py @@ -1,132 +1,19 @@ -from dbt.utils import filter_null_values, deep_merge, classproperty -from dbt.node_types import NodeType - -import dbt.exceptions - -from collections.abc import Mapping, Hashable -from dataclasses import dataclass, fields +from collections.abc import Hashable +from dataclasses import dataclass from typing import ( - Optional, TypeVar, Generic, Any, Type, Dict, Union, Iterator, Tuple, - Set + Optional, TypeVar, Any, Type, Dict, Union, Iterator, Tuple, Set ) -from typing_extensions import Protocol - -from hologram import JsonSchemaMixin -from hologram.helpers import StrEnum -from dbt.contracts.util import Replaceable from dbt.contracts.graph.compiled import CompiledNode from dbt.contracts.graph.parsed import ParsedSourceDefinition, ParsedNode +from dbt.contracts.relation import ( + RelationType, ComponentName, HasQuoting, FakeAPIObject, Policy, Path +) from dbt.exceptions import InternalException -from dbt import deprecations - - -class RelationType(StrEnum): - Table = 'table' - View = 'view' - CTE = 'cte' - MaterializedView = 'materializedview' - External = 'external' - - -class ComponentName(StrEnum): - Database = 'database' - Schema = 'schema' - Identifier = 'identifier' - - -class HasQuoting(Protocol): - quoting: Dict[str, bool] - - -class FakeAPIObject(JsonSchemaMixin, Replaceable, Mapping): - # override the mapping truthiness, len is always >1 - def __bool__(self): - return True - - def __getitem__(self, key): - try: - return getattr(self, key) - except AttributeError: - raise KeyError(key) from None - - def __iter__(self): - deprecations.warn('not-a-dictionary', obj=self) - for _, name in self._get_fields(): - yield name - - def __len__(self): - deprecations.warn('not-a-dictionary', obj=self) - return len(fields(self.__class__)) - - def incorporate(self, **kwargs): - value = self.to_dict() - value = deep_merge(value, kwargs) - return self.from_dict(value) - - -T = TypeVar('T') - - -@dataclass -class _ComponentObject(FakeAPIObject, Generic[T]): - database: T - schema: T - identifier: T - - def get_part(self, key: ComponentName) -> T: - if key == ComponentName.Database: - return self.database - elif key == ComponentName.Schema: - return self.schema - elif key == ComponentName.Identifier: - return self.identifier - else: - raise ValueError( - 'Got a key of {}, expected one of {}' - .format(key, list(ComponentName)) - ) - - def replace_dict(self, dct: Dict[ComponentName, T]): - kwargs: Dict[str, T] = {} - for k, v in dct.items(): - kwargs[str(k)] = v - return self.replace(**kwargs) - - -@dataclass -class Policy(_ComponentObject[bool]): - database: bool = True - schema: bool = True - identifier: bool = True - - -@dataclass -class Path(_ComponentObject[Optional[str]]): - database: Optional[str] - schema: Optional[str] - identifier: Optional[str] - - def __post_init__(self): - # handle pesky jinja2.Undefined sneaking in here and messing up render - if not isinstance(self.database, (type(None), str)): - raise dbt.exceptions.CompilationException( - 'Got an invalid path database: {}'.format(self.database) - ) - if not isinstance(self.schema, (type(None), str)): - raise dbt.exceptions.CompilationException( - 'Got an invalid path schema: {}'.format(self.schema) - ) - if not isinstance(self.identifier, (type(None), str)): - raise dbt.exceptions.CompilationException( - 'Got an invalid path identifier: {}'.format(self.identifier) - ) +from dbt.node_types import NodeType +from dbt.utils import filter_null_values, deep_merge, classproperty - def get_lowered_part(self, key: ComponentName) -> Optional[str]: - part = self.get_part(key) - if part is not None: - part = part.lower() - return part +import dbt.exceptions Self = TypeVar('Self', bound='BaseRelation') @@ -161,7 +48,7 @@ def __eq__(self, other): return self.to_dict() == other.to_dict() @classmethod - def get_default_quote_policy(cls: Type[Self]) -> Policy: + def get_default_quote_policy(cls) -> Policy: return cls._get_field_named('quote_policy').default def get(self, key, default=None): diff --git a/core/dbt/adapters/factory.py b/core/dbt/adapters/factory.py index 8cc8d20ae39..c648c4819ee 100644 --- a/core/dbt/adapters/factory.py +++ b/core/dbt/adapters/factory.py @@ -1,42 +1,52 @@ import threading +from pathlib import Path from importlib import import_module -from typing import Type, Dict, Any +from typing import Type, Dict, Any, List, Optional -from dbt.exceptions import RuntimeException -from dbt.include.global_project import PACKAGES +from dbt.exceptions import RuntimeException, InternalException +from dbt.include.global_project import ( + PACKAGE_PATH as GLOBAL_PROJECT_PATH, + PROJECT_NAME as GLOBAL_PROJECT_NAME, +) from dbt.logger import GLOBAL_LOGGER as logger from dbt.contracts.connection import Credentials, AdapterRequiredConfig -from dbt.adapters.base.impl import BaseAdapter, AdapterConfig -from dbt.adapters.base.plugin import AdapterPlugin - -# TODO: we can't import these because they cause an import cycle. -# Profile has to call into load_plugin to get credentials, so adapter/relation -# don't work -BaseRelation = Any +from dbt.adapters.protocol import ( + AdapterProtocol, + AdapterConfig, + RelationProtocol, +) +from dbt.adapters.base.plugin import AdapterPlugin -Adapter = BaseAdapter +Adapter = AdapterProtocol -class AdpaterContainer: +class AdapterContainer: def __init__(self): self.lock = threading.Lock() self.adapters: Dict[str, Adapter] = {} - self.adapter_types: Dict[str, Type[Adapter]] = {} + self.plugins: Dict[str, AdapterPlugin] = {} + # map package names to their include paths + self.packages: Dict[str, Path] = { + GLOBAL_PROJECT_NAME: Path(GLOBAL_PROJECT_PATH), + } - def get_adapter_class_by_name(self, name: str) -> Type[Adapter]: + def get_plugin_by_name(self, name: str) -> AdapterPlugin: with self.lock: - if name in self.adapter_types: - return self.adapter_types[name] - - names = ", ".join(self.adapter_types.keys()) + if name in self.plugins: + return self.plugins[name] + names = ", ".join(self.plugins.keys()) message = f"Invalid adapter type {name}! Must be one of {names}" raise RuntimeException(message) - def get_relation_class_by_name(self, name: str) -> Type[BaseRelation]: + def get_adapter_class_by_name(self, name: str) -> Type[Adapter]: + plugin = self.get_plugin_by_name(name) + return plugin.adapter + + def get_relation_class_by_name(self, name: str) -> Type[RelationProtocol]: adapter = self.get_adapter_class_by_name(name) return adapter.Relation @@ -47,7 +57,7 @@ def get_config_class_by_name( return adapter.AdapterSpecificConfigs def load_plugin(self, name: str) -> Type[Credentials]: - # this doesn't need a lock: in the worst case we'll overwrite PACKAGES + # this doesn't need a lock: in the worst case we'll overwrite packages # and adapter_type entries with the same value, as they're all # singletons try: @@ -74,9 +84,9 @@ def load_plugin(self, name: str) -> Type[Credentials]: with self.lock: # things do hold the lock to iterate over it so we need it to add - self.adapter_types[name] = plugin.adapter + self.plugins[name] = plugin - PACKAGES[plugin.project_name] = plugin.include_path + self.packages[plugin.project_name] = Path(plugin.include_path) for dep in plugin.dependencies: self.load_plugin(dep) @@ -114,8 +124,48 @@ def cleanup_connections(self): for adapter in self.adapters.values(): adapter.cleanup_connections() - -FACTORY: AdpaterContainer = AdpaterContainer() + def get_adapter_package_names(self, name: Optional[str]) -> List[str]: + if name is None: + # the important thing is that the global project is last. + return sorted( + self.packages, + key=lambda k: k == GLOBAL_PROJECT_NAME + ) + package_names: List[str] = [] + # slice into a list instead of using a set + pop(), to preserve order + plugin_names: List[str] = [name] + while plugin_names: + plugin_name = plugin_names[0] + plugin_names = plugin_names[1:] + try: + plugin = self.plugins[plugin_name] + except KeyError: + raise InternalException( + f'No plugin found for {plugin_name}' + ) from None + package_names.append(plugin.project_name) + if plugin.dependencies is None: + continue + for dep in plugin.dependencies: + if dep not in package_names: + plugin_names.append(dep) + package_names.append(GLOBAL_PROJECT_NAME) + return package_names + + def get_include_paths(self, name: Optional[str]) -> List[Path]: + paths = [] + for package_name in self.get_adapter_package_names(name): + try: + path = self.packages[package_name] + except KeyError: + raise InternalException( + f'No internal package listing found for {package_name}' + ) + paths.append(path) + return paths + + +FACTORY: AdapterContainer = AdapterContainer() def register_adapter(config: AdapterRequiredConfig) -> None: @@ -139,7 +189,7 @@ def cleanup_connections(): FACTORY.cleanup_connections() -def get_adapter_class_by_name(name: str) -> Type[BaseAdapter]: +def get_adapter_class_by_name(name: str) -> Type[AdapterProtocol]: return FACTORY.get_adapter_class_by_name(name) @@ -147,9 +197,17 @@ def get_config_class_by_name(name: str) -> Type[AdapterConfig]: return FACTORY.get_config_class_by_name(name) -def get_relation_class_by_name(name: str) -> Type[BaseRelation]: +def get_relation_class_by_name(name: str) -> Type[RelationProtocol]: return FACTORY.get_relation_class_by_name(name) def load_plugin(name: str) -> Type[Credentials]: return FACTORY.load_plugin(name) + + +def get_include_paths(name: Optional[str]) -> List[Path]: + return FACTORY.get_include_paths(name) + + +def get_adapter_package_names(name: Optional[str]) -> List[str]: + return FACTORY.get_adapter_package_names(name) diff --git a/core/dbt/adapters/protocol.py b/core/dbt/adapters/protocol.py new file mode 100644 index 00000000000..68809b77283 --- /dev/null +++ b/core/dbt/adapters/protocol.py @@ -0,0 +1,134 @@ +from dataclasses import dataclass +from typing import ( + Type, Hashable, Optional, ContextManager, List, Generic, TypeVar, ClassVar, + Tuple, Union +) +from typing_extensions import Protocol + +import agate + +from dbt.contracts.connection import Connection, AdapterRequiredConfig +from dbt.contracts.graph.compiled import CompiledNode +from dbt.contracts.graph.parsed import ParsedNode, ParsedSourceDefinition +from dbt.contracts.graph.model_config import BaseConfig +from dbt.contracts.graph.manifest import Manifest +from dbt.contracts.relation import Policy, HasQuoting + + +@dataclass +class AdapterConfig(BaseConfig): + pass + + +class ConnectionManagerProtocol(Protocol): + TYPE: str + + +class ColumnProtocol(Protocol): + pass + + +Self = TypeVar('Self', bound='RelationProtocol') + + +class RelationProtocol(Protocol): + @classmethod + def get_default_quote_policy(cls) -> Policy: + ... + + @classmethod + def create_from( + cls: Type[Self], + config: HasQuoting, + node: Union[CompiledNode, ParsedNode, ParsedSourceDefinition], + ) -> Self: + ... + + +AdapterConfig_T = TypeVar( + 'AdapterConfig_T', bound=AdapterConfig +) +ConnectionManager_T = TypeVar( + 'ConnectionManager_T', bound=ConnectionManagerProtocol +) +Relation_T = TypeVar( + 'Relation_T', bound=RelationProtocol +) +Column_T = TypeVar( + 'Column_T', bound=ColumnProtocol +) + + +class AdapterProtocol( + Protocol, + Generic[AdapterConfig_T, ConnectionManager_T, Relation_T, Column_T] +): + AdapterSpecificConfigs: ClassVar[Type[AdapterConfig_T]] + Column: ClassVar[Type[Column_T]] + Relation: ClassVar[Type[Relation_T]] + ConnectionManager: ClassVar[Type[ConnectionManager_T]] + connections: ConnectionManager_T + + def __init__(self, config: AdapterRequiredConfig): + ... + + @classmethod + def type(cls) -> str: + pass + + def set_query_header(self, manifest: Manifest) -> None: + ... + + @staticmethod + def get_thread_identifier() -> Hashable: + ... + + def get_thread_connection(self) -> Connection: + ... + + def set_thread_connection(self, conn: Connection) -> None: + ... + + def get_if_exists(self) -> Optional[Connection]: + ... + + def clear_thread_connection(self) -> None: + ... + + def clear_transaction(self) -> None: + ... + + def exception_handler(self, sql: str) -> ContextManager: + ... + + def set_connection_name(self, name: Optional[str] = None) -> Connection: + ... + + def cancel_open(self) -> Optional[List[str]]: + ... + + def open(cls, connection: Connection) -> Connection: + ... + + def release(self) -> None: + ... + + def cleanup_all(self) -> None: + ... + + def begin(self) -> None: + ... + + def commit(self) -> None: + ... + + def close(cls, connection: Connection) -> Connection: + ... + + def commit_if_has_connection(self) -> None: + ... + + def execute( + self, sql: str, auto_begin: bool = False, fetch: bool = False + ) -> Tuple[str, agate.Table]: + ... diff --git a/core/dbt/adapters/sql/connections.py b/core/dbt/adapters/sql/connections.py index 3fe1dad2ec7..91ebe4e0d85 100644 --- a/core/dbt/adapters/sql/connections.py +++ b/core/dbt/adapters/sql/connections.py @@ -6,9 +6,10 @@ import dbt.clients.agate_helper import dbt.exceptions -from dbt.contracts.connection import Connection from dbt.adapters.base import BaseConnectionManager +from dbt.contracts.connection import Connection from dbt.logger import GLOBAL_LOGGER as logger +from dbt import flags class SQLConnectionManager(BaseConnectionManager): @@ -133,7 +134,7 @@ def add_commit_query(self): def begin(self): connection = self.get_thread_connection() - if dbt.flags.STRICT_MODE: + if flags.STRICT_MODE: if not isinstance(connection, Connection): raise dbt.exceptions.CompilerException( f'In begin, got {connection} - not a Connection!' @@ -151,7 +152,7 @@ def begin(self): def commit(self): connection = self.get_thread_connection() - if dbt.flags.STRICT_MODE: + if flags.STRICT_MODE: if not isinstance(connection, Connection): raise dbt.exceptions.CompilerException( f'In commit, got {connection} - not a Connection!' diff --git a/core/dbt/adapters/sql/impl.py b/core/dbt/adapters/sql/impl.py index 84268b34d16..3377453d451 100644 --- a/core/dbt/adapters/sql/impl.py +++ b/core/dbt/adapters/sql/impl.py @@ -4,12 +4,11 @@ import dbt.clients.agate_helper from dbt.contracts.connection import Connection import dbt.exceptions -import dbt.flags from dbt.adapters.base import BaseAdapter, available from dbt.adapters.sql import SQLConnectionManager from dbt.logger import GLOBAL_LOGGER as logger -from dbt.adapters.factory import BaseRelation +from dbt.adapters.base.relation import BaseRelation LIST_RELATIONS_MACRO_NAME = 'list_relations_without_caching' GET_COLUMNS_IN_RELATION_MACRO_NAME = 'get_columns_in_relation' diff --git a/core/dbt/clients/jinja.py b/core/dbt/clients/jinja.py index 6138e8902c8..07a344338b7 100644 --- a/core/dbt/clients/jinja.py +++ b/core/dbt/clients/jinja.py @@ -30,7 +30,7 @@ InternalException, raise_compiler_error, CompilationException, invalid_materialization_argument, MacroReturn ) -from dbt.flags import MACRO_DEBUGGING +from dbt import flags from dbt.logger import GLOBAL_LOGGER as logger # noqa @@ -93,8 +93,8 @@ def _compile(self, source, filename): If the value is 'write', also write the files to disk. WARNING: This can write a ton of data if you aren't careful. """ - if filename == '