Compare commits

...

8 Commits

Author SHA1 Message Date
Mike Alfare
1686186dca merging main into feature 2023-07-25 20:22:20 -04:00
Mike Alfare
a02483dc27 Merge branch 'main' into feature/materialized-views/refactor-prep
# Conflicts:
#	core/dbt/include/global_project/macros/adapters/metadata.sql
2023-07-25 20:15:54 -04:00
Mike Alfare
c895a085c0 aligning with proposal discussion on GH 2023-07-20 19:25:05 -04:00
Mike Alfare
3d719ce63b Merge branch 'main' into feature/materialized-views/refactor-prep 2023-07-20 14:15:06 -04:00
Mike Alfare
e3570f4e64 Merge branch 'main' into feature/materialized-views/refactor-prep 2023-07-18 20:22:26 -04:00
Mike Alfare
67d0238e64 changie 2023-07-18 20:10:00 -04:00
Mike Alfare
1da16059c5 fixed inconsistencies from renaming 2023-07-18 19:58:02 -04:00
Mike Alfare
488b5a80bd refactoring first draft 2023-07-18 11:16:56 -04:00
132 changed files with 970 additions and 971 deletions

View File

@@ -0,0 +1,6 @@
kind: Under the Hood
body: Refactor jinja templates and relation models in support of new relation framework
time: 2023-07-18T20:09:42.008424-04:00
custom:
Author: mikealfare
Issue: "8128"

View File

@@ -10,3 +10,5 @@ ignore =
E741
E501 # long line checking is done in black
exclude = test/
per-file-ignores =
*/__init__.py: F401

View File

@@ -1,5 +1,5 @@
import abc
from concurrent.futures import as_completed, Future
from concurrent.futures import Future, as_completed
from contextlib import contextmanager
from datetime import datetime
from enum import Enum
@@ -20,11 +20,36 @@ from typing import (
Union,
)
from dbt.contracts.graph.nodes import ColumnLevelConstraint, ConstraintType, ModelLevelConstraint
import agate
import pytz
from dbt import deprecations
from dbt.adapters.base import Column as BaseColumn, Credentials
from dbt.adapters.base.connections import AdapterResponse, Connection
from dbt.adapters.base.meta import AdapterMeta, available
from dbt.adapters.base.relation import BaseRelation, InformationSchema, SchemaSearchMap
from dbt.adapters.cache import RelationsCache, _make_ref_key_dict
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.manifest import Manifest, MacroManifest
from dbt.contracts.graph.nodes import (
ColumnLevelConstraint,
ConstraintType,
ModelLevelConstraint,
ResultNode,
)
from dbt.contracts.relation import ComponentName
from dbt.events.functions import fire_event, warn_or_error
from dbt.events.types import (
CacheMiss,
CatalogGenerationError,
CodeExecution,
CodeExecutionStatus,
ConstraintNotEnforced,
ConstraintNotSupported,
ListRelations,
)
from dbt.exceptions import (
DbtInternalError,
DbtRuntimeError,
@@ -42,36 +67,8 @@ from dbt.exceptions import (
UnexpectedNonTimestampError,
UnexpectedNullError,
)
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.manifest import Manifest, MacroManifest
from dbt.contracts.graph.nodes import ResultNode
from dbt.events.functions import fire_event, warn_or_error
from dbt.events.types import (
CacheMiss,
ListRelations,
CodeExecution,
CodeExecutionStatus,
CatalogGenerationError,
ConstraintNotSupported,
ConstraintNotEnforced,
)
from dbt.utils import filter_null_values, executor, cast_to_str, AttrDict
from dbt.adapters.base.connections import Connection, AdapterResponse
from dbt.adapters.base.meta import AdapterMeta, available
from dbt.adapters.base.relation import (
ComponentName,
BaseRelation,
InformationSchema,
SchemaSearchMap,
)
from dbt.adapters.base import Column as BaseColumn
from dbt.adapters.base import Credentials
from dbt.adapters.cache import RelationsCache, _make_ref_key_dict
from dbt import deprecations
GET_CATALOG_MACRO_NAME = "get_catalog"
FRESHNESS_MACRO_NAME = "collect_freshness"
@@ -857,7 +854,7 @@ class BaseAdapter(metaclass=AdapterMeta):
@available
def quote_as_configured(self, identifier: str, quote_key: str) -> str:
"""Quote or do not quote the given identifer as configured in the
"""Quote or do not quote the given identifier as configured in the
project config for the quote key.
The quote key should be one of 'database' (on bigquery, 'profile'),

View File

@@ -1,25 +1,24 @@
from collections.abc import Hashable
from dataclasses import dataclass, field
from typing import Optional, TypeVar, Any, Type, Dict, Iterator, Tuple, Set
from typing import Any, Dict, Iterator, Optional, Set, Tuple, Type, TypeVar
from dbt.contracts.graph.nodes import SourceDefinition, ManifestNode, ResultNode, ParsedNode
from dbt.contracts.graph.nodes import ManifestNode, ParsedNode, ResultNode, SourceDefinition
from dbt.contracts.relation import (
RelationType,
ComponentName,
HasQuoting,
FakeAPIObject,
Policy,
HasQuoting,
Path,
Policy,
RelationType,
)
import dbt.exceptions
from dbt.exceptions import (
ApproximateMatchError,
DbtInternalError,
MultipleDatabasesNotAllowedError,
)
from dbt.node_types import NodeType
from dbt.utils import filter_null_values, deep_merge, classproperty
import dbt.exceptions
from dbt.utils import classproperty, deep_merge, filter_null_values
Self = TypeVar("Self", bound="BaseRelation")
@@ -31,7 +30,8 @@ class BaseRelation(FakeAPIObject, Hashable):
type: Optional[RelationType] = None
quote_character: str = '"'
# Python 3.11 requires that these use default_factory instead of simple default
# ValueError: mutable default <class 'dbt.contracts.relation.Policy'> for field include_policy is not allowed: use default_factory
# ValueError: mutable default <class 'dbt.contracts.relation.Policy'> for field include_policy is not allowed:
# use default_factory
include_policy: Policy = field(default_factory=lambda: Policy())
quote_policy: Policy = field(default_factory=lambda: Policy())
dbt_created: bool = False
@@ -170,13 +170,14 @@ class BaseRelation(FakeAPIObject, Hashable):
def _render_iterator(self) -> Iterator[Tuple[Optional[ComponentName], Optional[str]]]:
for key in ComponentName:
for member in ComponentName:
path_part: Optional[str] = None
if self.include_policy.get_part(key):
path_part = self.path.get_part(key)
if path_part is not None and self.quote_policy.get_part(key):
component = ComponentName(member)
if self.include_policy.get_part(component):
path_part = self.path.get_part(component)
if path_part is not None and self.quote_policy.get_part(component):
path_part = self.quoted(path_part)
yield key, path_part
yield component, path_part
def render(self) -> str:
# if there is nothing set, this will return the empty string.
@@ -259,7 +260,7 @@ class BaseRelation(FakeAPIObject, Hashable):
return cls.create_from_source(node, **kwargs)
else:
# Can't use ManifestNode here because of parameterized generics
if not isinstance(node, (ParsedNode)):
if not isinstance(node, ParsedNode):
raise DbtInternalError(
f"type mismatch, expected ManifestNode but got {type(node)}"
)

View File

View File

@@ -0,0 +1,8 @@
from dbt.adapters.relation.models._relation import (
DescribeRelationResults,
RelationComponent,
)
from dbt.adapters.relation.models._change import (
RelationChangeAction,
RelationChange,
)

View File

@@ -2,19 +2,19 @@ from abc import ABC, abstractmethod
from dataclasses import dataclass
from typing import Hashable
from dbt.adapters.relation_configs.config_base import RelationConfigBase
from dbt.adapters.relation.models._relation import RelationComponent
from dbt.dataclass_schema import StrEnum
class RelationConfigChangeAction(StrEnum):
class RelationChangeAction(StrEnum):
alter = "alter"
create = "create"
drop = "drop"
@dataclass(frozen=True, eq=True, unsafe_hash=True)
class RelationConfigChange(RelationConfigBase, ABC):
action: RelationConfigChangeAction
class RelationChange(RelationComponent, ABC):
action: RelationChangeAction
context: Hashable # this is usually a RelationConfig, e.g. IndexConfig, but shouldn't be limited
@property

View File

@@ -1,7 +1,8 @@
from dataclasses import dataclass
from typing import Union, Dict
from typing import Any, Dict, Union
import agate
from dbt.utils import filter_null_values
@@ -18,24 +19,24 @@ like name. But it also can have multiple indexes, which needs to be a separate q
])
}
"""
RelationResults = Dict[str, Union[agate.Row, agate.Table]]
DescribeRelationResults = Dict[str, Union[agate.Row, agate.Table]]
@dataclass(frozen=True)
class RelationConfigBase:
class RelationComponent:
@classmethod
def from_dict(cls, kwargs_dict) -> "RelationConfigBase":
def from_dict(cls, config_dict: Dict[str, Any]) -> "RelationComponent":
"""
This assumes the subclass of `RelationConfigBase` is flat, in the sense that no attribute is
itself another subclass of `RelationConfigBase`. If that's not the case, this should be overriden
to manually manage that complexity.
Args:
kwargs_dict: the dict representation of this instance
config_dict: the dict representation of this instance
Returns: the `RelationConfigBase` representation associated with the provided dict
"""
return cls(**filter_null_values(kwargs_dict)) # type: ignore
return cls(**filter_null_values(config_dict)) # type: ignore
@classmethod
def _not_implemented_error(cls) -> NotImplementedError:

View File

@@ -1,12 +0,0 @@
from dbt.adapters.relation_configs.config_base import ( # noqa: F401
RelationConfigBase,
RelationResults,
)
from dbt.adapters.relation_configs.config_change import ( # noqa: F401
RelationConfigChangeAction,
RelationConfigChange,
)
from dbt.adapters.relation_configs.config_validation import ( # noqa: F401
RelationConfigValidationMixin,
RelationConfigValidationRule,
)

View File

@@ -1,17 +1,16 @@
import agate
from typing import Any, Optional, Tuple, Type, List
from typing import Any, List, Optional, Tuple, Type
from dbt.contracts.connection import Connection, AdapterResponse
from dbt.exceptions import RelationTypeNullError
from dbt.adapters.base import BaseAdapter, available
from dbt.adapters.base.relation import BaseRelation
from dbt.adapters.cache import _make_ref_key_dict
from dbt.adapters.sql import SQLConnectionManager
from dbt.contracts.connection import AdapterResponse, Connection
from dbt.events.functions import fire_event
from dbt.events.types import ColTypeChange, SchemaCreation, SchemaDrop
from dbt.exceptions import RelationTypeNullError
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"
LIST_SCHEMAS_MACRO_NAME = "list_schemas"
@@ -222,14 +221,14 @@ class SQLAdapter(BaseAdapter):
def validate_sql(self, sql: str) -> AdapterResponse:
"""Submit the given SQL to the engine for validation, but not execution.
By default we simply prefix the query with the explain keyword and allow the
By default, we simply prefix the query with the explain keyword and allow the
exceptions thrown by the underlying engine on invalid SQL inputs to bubble up
to the exception handler. For adjustments to the explain statement - such as
for adapters that have different mechanisms for hinting at query validation
or dry-run - callers may be able to override the validate_sql_query macro with
the addition of an <adapter>__validate_sql implementation.
:param sql str: The sql to validate
:param sql: str The sql to validate
"""
kwargs = {
"sql": sql,

View File

@@ -5,7 +5,7 @@ from dbt.exceptions import DbtRuntimeError
@dataclass(frozen=True, eq=True, unsafe_hash=True)
class RelationConfigValidationRule:
class ValidationRule:
validation_check: bool
validation_error: Optional[DbtRuntimeError]
@@ -18,12 +18,12 @@ class RelationConfigValidationRule:
@dataclass(frozen=True)
class RelationConfigValidationMixin:
class ValidationMixin:
def __post_init__(self):
self.run_validation_rules()
@property
def validation_rules(self) -> Set[RelationConfigValidationRule]:
def validation_rules(self) -> Set[ValidationRule]:
"""
A set of validation rules to run against the object upon creation.

View File

@@ -1,132 +0,0 @@
{% macro get_columns_in_relation(relation) -%}
{{ return(adapter.dispatch('get_columns_in_relation', 'dbt')(relation)) }}
{% endmacro %}
{% macro default__get_columns_in_relation(relation) -%}
{{ exceptions.raise_not_implemented(
'get_columns_in_relation macro not implemented for adapter '+adapter.type()) }}
{% endmacro %}
{# helper for adapter-specific implementations of get_columns_in_relation #}
{% macro sql_convert_columns_in_relation(table) -%}
{% set columns = [] %}
{% for row in table %}
{% do columns.append(api.Column(*row)) %}
{% endfor %}
{{ return(columns) }}
{% endmacro %}
{% macro get_empty_subquery_sql(select_sql, select_sql_header=none) -%}
{{ return(adapter.dispatch('get_empty_subquery_sql', 'dbt')(select_sql, select_sql_header)) }}
{% endmacro %}
{#
Builds a query that results in the same schema as the given select_sql statement, without necessitating a data scan.
Useful for running a query in a 'pre-flight' context, such as model contract enforcement (assert_columns_equivalent macro).
#}
{% macro default__get_empty_subquery_sql(select_sql, select_sql_header=none) %}
{%- if select_sql_header is not none -%}
{{ select_sql_header }}
{%- endif -%}
select * from (
{{ select_sql }}
) as __dbt_sbq
where false
limit 0
{% endmacro %}
{% macro get_empty_schema_sql(columns) -%}
{{ return(adapter.dispatch('get_empty_schema_sql', 'dbt')(columns)) }}
{% endmacro %}
{% macro default__get_empty_schema_sql(columns) %}
{%- set col_err = [] -%}
select
{% for i in columns %}
{%- set col = columns[i] -%}
{%- if col['data_type'] is not defined -%}
{{ col_err.append(col['name']) }}
{%- endif -%}
{% set col_name = adapter.quote(col['name']) if col.get('quote') else col['name'] %}
cast(null as {{ col['data_type'] }}) as {{ col_name }}{{ ", " if not loop.last }}
{%- endfor -%}
{%- if (col_err | length) > 0 -%}
{{ exceptions.column_type_missing(column_names=col_err) }}
{%- endif -%}
{% endmacro %}
{% macro get_column_schema_from_query(select_sql, select_sql_header=none) -%}
{% set columns = [] %}
{# -- Using an 'empty subquery' here to get the same schema as the given select_sql statement, without necessitating a data scan.#}
{% set sql = get_empty_subquery_sql(select_sql, select_sql_header) %}
{% set column_schema = adapter.get_column_schema_from_query(sql) %}
{{ return(column_schema) }}
{% endmacro %}
-- here for back compat
{% macro get_columns_in_query(select_sql) -%}
{{ return(adapter.dispatch('get_columns_in_query', 'dbt')(select_sql)) }}
{% endmacro %}
{% macro default__get_columns_in_query(select_sql) %}
{% call statement('get_columns_in_query', fetch_result=True, auto_begin=False) -%}
{{ get_empty_subquery_sql(select_sql) }}
{% endcall %}
{{ return(load_result('get_columns_in_query').table.columns | map(attribute='name') | list) }}
{% endmacro %}
{% macro alter_column_type(relation, column_name, new_column_type) -%}
{{ return(adapter.dispatch('alter_column_type', 'dbt')(relation, column_name, new_column_type)) }}
{% endmacro %}
{% macro default__alter_column_type(relation, column_name, new_column_type) -%}
{#
1. Create a new column (w/ temp name and correct type)
2. Copy data over to it
3. Drop the existing column (cascade!)
4. Rename the new column to existing column
#}
{%- set tmp_column = column_name + "__dbt_alter" -%}
{% call statement('alter_column_type') %}
alter table {{ relation }} add column {{ adapter.quote(tmp_column) }} {{ new_column_type }};
update {{ relation }} set {{ adapter.quote(tmp_column) }} = {{ adapter.quote(column_name) }};
alter table {{ relation }} drop column {{ adapter.quote(column_name) }} cascade;
alter table {{ relation }} rename column {{ adapter.quote(tmp_column) }} to {{ adapter.quote(column_name) }}
{% endcall %}
{% endmacro %}
{% macro alter_relation_add_remove_columns(relation, add_columns = none, remove_columns = none) -%}
{{ return(adapter.dispatch('alter_relation_add_remove_columns', 'dbt')(relation, add_columns, remove_columns)) }}
{% endmacro %}
{% macro default__alter_relation_add_remove_columns(relation, add_columns, remove_columns) %}
{% if add_columns is none %}
{% set add_columns = [] %}
{% endif %}
{% if remove_columns is none %}
{% set remove_columns = [] %}
{% endif %}
{% set sql -%}
alter {{ relation.type }} {{ relation }}
{% for column in add_columns %}
add column {{ column.name }} {{ column.data_type }}{{ ',' if not loop.last }}
{% endfor %}{{ ',' if add_columns and remove_columns }}
{% for column in remove_columns %}
drop column {{ column.name }}{{ ',' if not loop.last }}
{% endfor %}
{%- endset -%}
{% do run_query(sql) %}
{% endmacro %}

View File

@@ -1,44 +0,0 @@
{% macro drop_relation(relation) -%}
{{ return(adapter.dispatch('drop_relation', 'dbt')(relation)) }}
{% endmacro %}
{% macro default__drop_relation(relation) -%}
{% call statement('drop_relation', auto_begin=False) -%}
{%- if relation.is_table -%}
{{- drop_table(relation) -}}
{%- elif relation.is_view -%}
{{- drop_view(relation) -}}
{%- elif relation.is_materialized_view -%}
{{- drop_materialized_view(relation) -}}
{%- else -%}
drop {{ relation.type }} if exists {{ relation }} cascade
{%- endif -%}
{%- endcall %}
{% endmacro %}
{% macro drop_table(relation) -%}
{{ return(adapter.dispatch('drop_table', 'dbt')(relation)) }}
{%- endmacro %}
{% macro default__drop_table(relation) -%}
drop table if exists {{ relation }} cascade
{%- endmacro %}
{% macro drop_view(relation) -%}
{{ return(adapter.dispatch('drop_view', 'dbt')(relation)) }}
{%- endmacro %}
{% macro default__drop_view(relation) -%}
drop view if exists {{ relation }} cascade
{%- endmacro %}
{% macro drop_materialized_view(relation) -%}
{{ return(adapter.dispatch('drop_materialized_view', 'dbt')(relation)) }}
{%- endmacro %}
{% macro default__drop_materialized_view(relation) -%}
drop materialized view if exists {{ relation }} cascade
{%- endmacro %}

View File

@@ -1,74 +0,0 @@
{% macro get_catalog(information_schema, schemas) -%}
{{ return(adapter.dispatch('get_catalog', 'dbt')(information_schema, schemas)) }}
{%- endmacro %}
{% macro default__get_catalog(information_schema, schemas) -%}
{% set typename = adapter.type() %}
{% set msg -%}
get_catalog not implemented for {{ typename }}
{%- endset %}
{{ exceptions.raise_compiler_error(msg) }}
{% endmacro %}
{% macro information_schema_name(database) %}
{{ return(adapter.dispatch('information_schema_name', 'dbt')(database)) }}
{% endmacro %}
{% macro default__information_schema_name(database) -%}
{%- if database -%}
{{ database }}.INFORMATION_SCHEMA
{%- else -%}
INFORMATION_SCHEMA
{%- endif -%}
{%- endmacro %}
{% macro list_schemas(database) -%}
{{ return(adapter.dispatch('list_schemas', 'dbt')(database)) }}
{% endmacro %}
{% macro default__list_schemas(database) -%}
{% set sql %}
select distinct schema_name
from {{ information_schema_name(database) }}.SCHEMATA
where catalog_name ilike '{{ database }}'
{% endset %}
{{ return(run_query(sql)) }}
{% endmacro %}
{% macro check_schema_exists(information_schema, schema) -%}
{{ return(adapter.dispatch('check_schema_exists', 'dbt')(information_schema, schema)) }}
{% endmacro %}
{% macro default__check_schema_exists(information_schema, schema) -%}
{% set sql -%}
select count(*)
from {{ information_schema.replace(information_schema_view='SCHEMATA') }}
where catalog_name='{{ information_schema.database }}'
and schema_name='{{ schema }}'
{%- endset %}
{{ return(run_query(sql)) }}
{% endmacro %}
{% macro list_relations_without_caching(schema_relation) %}
{{ return(adapter.dispatch('list_relations_without_caching', 'dbt')(schema_relation)) }}
{% endmacro %}
{% macro default__list_relations_without_caching(schema_relation) %}
{{ exceptions.raise_not_implemented(
'list_relations_without_caching macro not implemented for adapter '+adapter.type()) }}
{% endmacro %}
{% macro get_relations() %}
{{ return(adapter.dispatch('get_relations', 'dbt')()) }}
{% endmacro %}
{% macro default__get_relations() %}
{{ exceptions.raise_not_implemented(
'get_relations macro not implemented for adapter '+adapter.type()) }}
{% endmacro %}

View File

@@ -0,0 +1,14 @@
{% macro get_catalog(information_schema, schemas) -%}
{{ return(adapter.dispatch('get_catalog', 'dbt')(information_schema, schemas)) }}
{%- endmacro %}
{% macro default__get_catalog(information_schema, schemas) -%}
{% set typename = adapter.type() %}
{% set msg -%}
get_catalog not implemented for {{ typename }}
{%- endset %}
{{ exceptions.raise_compiler_error(msg) }}
{% endmacro %}

View File

@@ -0,0 +1,9 @@
{% macro get_relations() %}
{{ return(adapter.dispatch('get_relations', 'dbt')()) }}
{% endmacro %}
{% macro default__get_relations() %}
{{ exceptions.raise_not_implemented(
'get_relations macro not implemented for adapter '+adapter.type()) }}
{% endmacro %}

View File

@@ -0,0 +1,12 @@
{% macro information_schema_name(database) %}
{{ return(adapter.dispatch('information_schema_name', 'dbt')(database)) }}
{% endmacro %}
{% macro default__information_schema_name(database) -%}
{%- if database -%}
{{ database }}.INFORMATION_SCHEMA
{%- else -%}
INFORMATION_SCHEMA
{%- endif -%}
{%- endmacro %}

View File

@@ -0,0 +1,9 @@
{% macro list_relations_without_caching(schema_relation) %}
{{ return(adapter.dispatch('list_relations_without_caching', 'dbt')(schema_relation)) }}
{% endmacro %}
{% macro default__list_relations_without_caching(schema_relation) %}
{{ exceptions.raise_not_implemented(
'list_relations_without_caching macro not implemented for adapter '+adapter.type()) }}
{% endmacro %}

View File

@@ -1,33 +0,0 @@
{% macro alter_column_comment(relation, column_dict) -%}
{{ return(adapter.dispatch('alter_column_comment', 'dbt')(relation, column_dict)) }}
{% endmacro %}
{% macro default__alter_column_comment(relation, column_dict) -%}
{{ exceptions.raise_not_implemented(
'alter_column_comment macro not implemented for adapter '+adapter.type()) }}
{% endmacro %}
{% macro alter_relation_comment(relation, relation_comment) -%}
{{ return(adapter.dispatch('alter_relation_comment', 'dbt')(relation, relation_comment)) }}
{% endmacro %}
{% macro default__alter_relation_comment(relation, relation_comment) -%}
{{ exceptions.raise_not_implemented(
'alter_relation_comment macro not implemented for adapter '+adapter.type()) }}
{% endmacro %}
{% macro persist_docs(relation, model, for_relation=true, for_columns=true) -%}
{{ return(adapter.dispatch('persist_docs', 'dbt')(relation, model, for_relation, for_columns)) }}
{% endmacro %}
{% macro default__persist_docs(relation, model, for_relation, for_columns) -%}
{% if for_relation and config.persist_relation_docs() and model.description %}
{% do run_query(alter_relation_comment(relation, model.description)) %}
{% endif %}
{% if for_columns and config.persist_column_docs() and model.columns %}
{% do run_query(alter_column_comment(relation, model.columns)) %}
{% endif %}
{% endmacro %}

View File

@@ -32,28 +32,6 @@
{% endmacro %}
{% macro truncate_relation(relation) -%}
{{ return(adapter.dispatch('truncate_relation', 'dbt')(relation)) }}
{% endmacro %}
{% macro default__truncate_relation(relation) -%}
{% call statement('truncate_relation') -%}
truncate table {{ relation }}
{%- endcall %}
{% endmacro %}
{% macro rename_relation(from_relation, to_relation) -%}
{{ return(adapter.dispatch('rename_relation', 'dbt')(from_relation, to_relation)) }}
{% endmacro %}
{% macro default__rename_relation(from_relation, to_relation) -%}
{% set target_name = adapter.quote_as_configured(to_relation.identifier, 'identifier') %}
{% call statement('rename_relation') -%}
alter table {{ from_relation }} rename to {{ target_name }}
{%- endcall %}
{% endmacro %}
{% macro get_or_create_relation(database, schema, identifier, type) -%}
{{ return(adapter.dispatch('get_or_create_relation', 'dbt')(database, schema, identifier, type)) }}
@@ -89,10 +67,3 @@
{% macro load_relation(relation) %}
{{ return(load_cached_relation(relation)) }}
{% endmacro %}
{% macro drop_relation_if_exists(relation) %}
{% if relation is not none %}
{{ adapter.drop_relation(relation) }}
{% endif %}
{% endmacro %}

View File

@@ -0,0 +1,55 @@
{% macro alter_column_type(relation, column_name, new_column_type) -%}
{{ return(adapter.dispatch('alter_column_type', 'dbt')(relation, column_name, new_column_type)) }}
{% endmacro %}
{% macro default__alter_column_type(relation, column_name, new_column_type) -%}
{#
1. Create a new column (w/ temp name and correct type)
2. Copy data over to it
3. Drop the existing column (cascade!)
4. Rename the new column to existing column
#}
{%- set tmp_column = column_name + "__dbt_alter" -%}
{% call statement('alter_column_type') %}
alter table {{ relation }} add column {{ adapter.quote(tmp_column) }} {{ new_column_type }};
update {{ relation }} set {{ adapter.quote(tmp_column) }} = {{ adapter.quote(column_name) }};
alter table {{ relation }} drop column {{ adapter.quote(column_name) }} cascade;
alter table {{ relation }} rename column {{ adapter.quote(tmp_column) }} to {{ adapter.quote(column_name) }}
{% endcall %}
{% endmacro %}
{% macro alter_relation_add_remove_columns(relation, add_columns = none, remove_columns = none) -%}
{{ return(adapter.dispatch('alter_relation_add_remove_columns', 'dbt')(relation, add_columns, remove_columns)) }}
{% endmacro %}
{% macro default__alter_relation_add_remove_columns(relation, add_columns, remove_columns) %}
{% if add_columns is none %}
{% set add_columns = [] %}
{% endif %}
{% if remove_columns is none %}
{% set remove_columns = [] %}
{% endif %}
{% set sql -%}
alter {{ relation.type }} {{ relation }}
{% for column in add_columns %}
add column {{ column.name }} {{ column.data_type }}{{ ',' if not loop.last }}
{% endfor %}{{ ',' if add_columns and remove_columns }}
{% for column in remove_columns %}
drop column {{ column.name }}{{ ',' if not loop.last }}
{% endfor %}
{%- endset -%}
{% do run_query(sql) %}
{% endmacro %}

View File

@@ -0,0 +1,42 @@
{% macro get_column_schema_from_query(select_sql, select_sql_header=none) -%}
{% set columns = [] %}
{# -- Using an 'empty subquery' here to get the same schema as the given select_sql statement, without necessitating a data scan.#}
{% set sql = get_empty_subquery_sql(select_sql, select_sql_header) %}
{% set column_schema = adapter.get_column_schema_from_query(sql) %}
{{ return(column_schema) }}
{% endmacro %}
{% macro get_empty_subquery_sql(select_sql, select_sql_header=none) -%}
{{ return(adapter.dispatch('get_empty_subquery_sql', 'dbt')(select_sql, select_sql_header)) }}
{% endmacro %}
{#
Builds a query that results in the same schema as the given select_sql statement, without necessitating a data scan.
Useful for running a query in a 'pre-flight' context, such as model contract enforcement (assert_columns_equivalent macro).
#}
{% macro default__get_empty_subquery_sql(select_sql, select_sql_header=none) %}
{%- if select_sql_header is not none -%}
{{ select_sql_header }}
{%- endif -%}
select * from (
{{ select_sql }}
) as __dbt_sbq
where false
limit 0
{% endmacro %}
-- here for back compat
{% macro get_columns_in_query(select_sql) -%}
{{ return(adapter.dispatch('get_columns_in_query', 'dbt')(select_sql)) }}
{% endmacro %}
{% macro default__get_columns_in_query(select_sql) %}
{% call statement('get_columns_in_query', fetch_result=True, auto_begin=False) -%}
{{ get_empty_subquery_sql(select_sql) }}
{% endcall %}
{{ return(load_result('get_columns_in_query').table.columns | map(attribute='name') | list) }}
{% endmacro %}

View File

@@ -0,0 +1,19 @@
{% macro get_columns_in_relation(relation) -%}
{{ return(adapter.dispatch('get_columns_in_relation', 'dbt')(relation)) }}
{% endmacro %}
{% macro default__get_columns_in_relation(relation) -%}
{{ exceptions.raise_not_implemented(
'get_columns_in_relation macro not implemented for adapter '+adapter.type()) }}
{% endmacro %}
{# helper for adapter-specific implementations of get_columns_in_relation #}
{% macro sql_convert_columns_in_relation(table) -%}
{% set columns = [] %}
{% for row in table %}
{% do columns.append(api.Column(*row)) %}
{% endfor %}
{{ return(columns) }}
{% endmacro %}

View File

@@ -0,0 +1,20 @@
{% macro get_empty_schema_sql(columns) -%}
{{ return(adapter.dispatch('get_empty_schema_sql', 'dbt')(columns)) }}
{% endmacro %}
{% macro default__get_empty_schema_sql(columns) %}
{%- set col_err = [] -%}
select
{% for i in columns %}
{%- set col = columns[i] -%}
{%- if col['data_type'] is not defined -%}
{{ col_err.append(col['name']) }}
{%- endif -%}
{% set col_name = adapter.quote(col['name']) if col.get('quote') else col['name'] %}
cast(null as {{ col['data_type'] }}) as {{ col_name }}{{ ", " if not loop.last }}
{%- endfor -%}
{%- if (col_err | length) > 0 -%}
{{ exceptions.column_type_missing(column_names=col_err) }}
{%- endif -%}
{% endmacro %}

View File

@@ -0,0 +1,9 @@
{% macro alter_column_comment(relation, column_dict) -%}
{{ return(adapter.dispatch('alter_column_comment', 'dbt')(relation, column_dict)) }}
{% endmacro %}
{% macro default__alter_column_comment(relation, column_dict) -%}
{{ exceptions.raise_not_implemented(
'alter_column_comment macro not implemented for adapter '+adapter.type()) }}
{% endmacro %}

View File

@@ -0,0 +1,9 @@
{% macro alter_relation_comment(relation, relation_comment) -%}
{{ return(adapter.dispatch('alter_relation_comment', 'dbt')(relation, relation_comment)) }}
{% endmacro %}
{% macro default__alter_relation_comment(relation, relation_comment) -%}
{{ exceptions.raise_not_implemented(
'alter_relation_comment macro not implemented for adapter '+adapter.type()) }}
{% endmacro %}

View File

@@ -1,24 +1,6 @@
{# ------- BOOLEAN MACROS --------- #}
{#
-- COPY GRANTS
-- When a relational object (view or table) is replaced in this database,
-- do previous grants carry over to the new object? This may depend on:
-- whether we use alter-rename-swap versus CREATE OR REPLACE
-- user-supplied configuration (e.g. copy_grants on Snowflake)
-- By default, play it safe, assume TRUE: that grants ARE copied over.
-- This means dbt will first "show" current grants and then calculate diffs.
-- It may require an additional query than is strictly necessary,
-- but better safe than sorry.
#}
{% macro copy_grants() %}
{{ return(adapter.dispatch('copy_grants', 'dbt')()) }}
{% endmacro %}
{% macro default__copy_grants() %}
{{ return(True) }}
{% endmacro %}
{#
@@ -39,49 +21,6 @@
{%- endmacro -%}
{% macro should_revoke(existing_relation, full_refresh_mode=True) %}
{% if not existing_relation %}
{#-- The table doesn't already exist, so no grants to copy over --#}
{{ return(False) }}
{% elif full_refresh_mode %}
{#-- The object is being REPLACED -- whether grants are copied over depends on the value of user config --#}
{{ return(copy_grants()) }}
{% else %}
{#-- The table is being merged/upserted/inserted -- grants will be carried over --#}
{{ return(True) }}
{% endif %}
{% endmacro %}
{# ------- DCL STATEMENT TEMPLATES --------- #}
{% macro get_show_grant_sql(relation) %}
{{ return(adapter.dispatch("get_show_grant_sql", "dbt")(relation)) }}
{% endmacro %}
{% macro default__get_show_grant_sql(relation) %}
show grants on {{ relation }}
{% endmacro %}
{% macro get_grant_sql(relation, privilege, grantees) %}
{{ return(adapter.dispatch('get_grant_sql', 'dbt')(relation, privilege, grantees)) }}
{% endmacro %}
{%- macro default__get_grant_sql(relation, privilege, grantees) -%}
grant {{ privilege }} on {{ relation }} to {{ grantees | join(', ') }}
{%- endmacro -%}
{% macro get_revoke_sql(relation, privilege, grantees) %}
{{ return(adapter.dispatch('get_revoke_sql', 'dbt')(relation, privilege, grantees)) }}
{% endmacro %}
{%- macro default__get_revoke_sql(relation, privilege, grantees) -%}
revoke {{ privilege }} on {{ relation }} from {{ grantees | join(', ') }}
{%- endmacro -%}
{# ------- RUNTIME APPLICATION --------- #}

View File

@@ -0,0 +1,20 @@
{#
-- COPY GRANTS
-- When a relational object (view or table) is replaced in this database,
-- do previous grants carry over to the new object? This may depend on:
-- whether we use alter-rename-swap versus CREATE OR REPLACE
-- user-supplied configuration (e.g. copy_grants on Snowflake)
-- By default, play it safe, assume TRUE: that grants ARE copied over.
-- This means dbt will first "show" current grants and then calculate diffs.
-- It may require an additional query than is strictly necessary,
-- but better safe than sorry.
#}
{% macro copy_grants() %}
{{ return(adapter.dispatch('copy_grants', 'dbt')()) }}
{% endmacro %}
{% macro default__copy_grants() %}
{{ return(True) }}
{% endmacro %}

View File

@@ -0,0 +1,8 @@
{% macro get_grant_sql(relation, privilege, grantees) %}
{{ return(adapter.dispatch('get_grant_sql', 'dbt')(relation, privilege, grantees)) }}
{% endmacro %}
{%- macro default__get_grant_sql(relation, privilege, grantees) -%}
grant {{ privilege }} on {{ relation }} to {{ grantees | join(', ') }}
{%- endmacro -%}

View File

@@ -0,0 +1,24 @@
{% macro get_revoke_sql(relation, privilege, grantees) %}
{{ return(adapter.dispatch('get_revoke_sql', 'dbt')(relation, privilege, grantees)) }}
{% endmacro %}
{%- macro default__get_revoke_sql(relation, privilege, grantees) -%}
revoke {{ privilege }} on {{ relation }} from {{ grantees | join(', ') }}
{%- endmacro -%}
{% macro should_revoke(existing_relation, full_refresh_mode=True) %}
{% if not existing_relation %}
{#-- The table doesn't already exist, so no grants to copy over --#}
{{ return(False) }}
{% elif full_refresh_mode %}
{#-- The object is being REPLACED -- whether grants are copied over depends on the value of user config --#}
{{ return(copy_grants()) }}
{% else %}
{#-- The table is being merged/upserted/inserted -- grants will be carried over --#}
{{ return(True) }}
{% endif %}
{% endmacro %}

View File

@@ -0,0 +1,8 @@
{% macro get_show_grant_sql(relation) %}
{{ return(adapter.dispatch("get_show_grant_sql", "dbt")(relation)) }}
{% endmacro %}
{% macro default__get_show_grant_sql(relation) %}
show grants on {{ relation }}
{% endmacro %}

View File

@@ -2,6 +2,7 @@
{{ return(adapter.dispatch('get_create_index_sql', 'dbt')(relation, index_dict)) }}
{% endmacro %}
{% macro default__get_create_index_sql(relation, index_dict) -%}
{% do return(None) %}
{% endmacro %}
@@ -11,6 +12,7 @@
{{ adapter.dispatch('create_indexes', 'dbt')(relation) }}
{%- endmacro %}
{% macro default__create_indexes(relation) -%}
{%- set _indexes = config.get('indexes', default=[]) -%}
@@ -21,21 +23,3 @@
{% endif %}
{% endfor %}
{% endmacro %}
{% macro get_drop_index_sql(relation, index_name) -%}
{{ adapter.dispatch('get_drop_index_sql', 'dbt')(relation, index_name) }}
{%- endmacro %}
{% macro default__get_drop_index_sql(relation, index_name) -%}
{{ exceptions.raise_compiler_error("`get_drop_index_sql has not been implemented for this adapter.") }}
{%- endmacro %}
{% macro get_show_indexes_sql(relation) -%}
{{ adapter.dispatch('get_show_indexes_sql', 'dbt')(relation) }}
{%- endmacro %}
{% macro default__get_show_indexes_sql(relation) -%}
{{ exceptions.raise_compiler_error("`get_show_indexes_sql has not been implemented for this adapter.") }}
{%- endmacro %}

View File

@@ -0,0 +1,8 @@
{% macro get_drop_index_sql(relation, index_name) -%}
{{ adapter.dispatch('get_drop_index_sql', 'dbt')(relation, index_name) }}
{%- endmacro %}
{% macro default__get_drop_index_sql(relation, index_name) -%}
{{ exceptions.raise_compiler_error("`get_drop_index_sql has not been implemented for this adapter.") }}
{%- endmacro %}

View File

@@ -0,0 +1,8 @@
{% macro get_show_indexes_sql(relation) -%}
{{ adapter.dispatch('get_show_indexes_sql', 'dbt')(relation) }}
{%- endmacro %}
{% macro default__get_show_indexes_sql(relation) -%}
{{ exceptions.raise_compiler_error("`get_show_indexes_sql has not been implemented for this adapter.") }}
{%- endmacro %}

View File

@@ -2,19 +2,9 @@
{{ adapter.dispatch('create_schema', 'dbt')(relation) }}
{% endmacro %}
{% macro default__create_schema(relation) -%}
{%- call statement('create_schema') -%}
create schema if not exists {{ relation.without_identifier() }}
{% endcall %}
{% endmacro %}
{% macro drop_schema(relation) -%}
{{ adapter.dispatch('drop_schema', 'dbt')(relation) }}
{% endmacro %}
{% macro default__drop_schema(relation) -%}
{%- call statement('drop_schema') -%}
drop schema if exists {{ relation.without_identifier() }} cascade
{% endcall %}
{% endmacro %}

View File

@@ -0,0 +1,10 @@
{% macro drop_schema(relation) -%}
{{ adapter.dispatch('drop_schema', 'dbt')(relation) }}
{% endmacro %}
{% macro default__drop_schema(relation) -%}
{%- call statement('drop_schema') -%}
drop schema if exists {{ relation.without_identifier() }} cascade
{% endcall %}
{% endmacro %}

View File

@@ -0,0 +1,29 @@
{% macro list_schemas(database) -%}
{{ return(adapter.dispatch('list_schemas', 'dbt')(database)) }}
{% endmacro %}
{% macro default__list_schemas(database) -%}
{% set sql %}
select distinct schema_name
from {{ information_schema_name(database) }}.SCHEMATA
where catalog_name ilike '{{ database }}'
{% endset %}
{{ return(run_query(sql)) }}
{% endmacro %}
{% macro check_schema_exists(information_schema, schema) -%}
{{ return(adapter.dispatch('check_schema_exists', 'dbt')(information_schema, schema)) }}
{% endmacro %}
{% macro default__check_schema_exists(information_schema, schema) -%}
{% set sql -%}
select count(*)
from {{ information_schema.replace(information_schema_view='SCHEMATA') }}
where catalog_name='{{ information_schema.database }}'
and schema_name='{{ schema }}'
{%- endset %}
{{ return(run_query(sql)) }}
{% endmacro %}

View File

@@ -0,0 +1,8 @@
{% macro drop_materialized_view(relation) -%}
{{ return(adapter.dispatch('drop_materialized_view', 'dbt')(relation)) }}
{%- endmacro %}
{% macro default__drop_materialized_view(relation) -%}
drop materialized view if exists {{ relation }} cascade
{%- endmacro %}

View File

@@ -0,0 +1,8 @@
{% macro drop_table(relation) -%}
{{ return(adapter.dispatch('drop_table', 'dbt')(relation)) }}
{%- endmacro %}
{% macro default__drop_table(relation) -%}
drop table if exists {{ relation }} cascade
{%- endmacro %}

View File

@@ -42,3 +42,32 @@
{{ return({'relations': [target_relation]}) }}
{% endmacro %}
{% macro get_create_view_as_sql(relation, sql) -%}
{{ adapter.dispatch('get_create_view_as_sql', 'dbt')(relation, sql) }}
{%- endmacro %}
{% macro default__get_create_view_as_sql(relation, sql) -%}
{{ return(create_view_as(relation, sql)) }}
{% endmacro %}
/* {# keep logic under old name for backwards compatibility #} */
{% macro create_view_as(relation, sql) -%}
{{ adapter.dispatch('create_view_as', 'dbt')(relation, sql) }}
{%- endmacro %}
{% macro default__create_view_as(relation, sql) -%}
{%- set sql_header = config.get('sql_header', none) -%}
{{ sql_header if sql_header is not none }}
create view {{ relation }}
{% set contract_config = config.get('contract') %}
{% if contract_config.enforced %}
{{ get_assert_columns_equivalent(sql) }}
{%- endif %}
as (
{{ sql }}
);
{%- endmacro %}

View File

@@ -0,0 +1,8 @@
{% macro drop_view(relation) -%}
{{ return(adapter.dispatch('drop_view', 'dbt')(relation)) }}
{%- endmacro %}
{% macro default__drop_view(relation) -%}
drop view if exists {{ relation }} cascade
{%- endmacro %}

View File

@@ -0,0 +1,14 @@
{% macro persist_docs(relation, model, for_relation=true, for_columns=true) -%}
{{ return(adapter.dispatch('persist_docs', 'dbt')(relation, model, for_relation, for_columns)) }}
{% endmacro %}
{% macro default__persist_docs(relation, model, for_relation, for_columns) -%}
{% if for_relation and config.persist_relation_docs() and model.description %}
{% do run_query(alter_relation_comment(relation, model.description)) %}
{% endif %}
{% if for_columns and config.persist_column_docs() and model.columns %}
{% do run_query(alter_column_comment(relation, model.columns)) %}
{% endif %}
{% endmacro %}

View File

@@ -22,33 +22,6 @@
{% endmaterialization %}
{% macro materialized_view_setup(backup_relation, intermediate_relation, pre_hooks) %}
-- backup_relation and intermediate_relation should not already exist in the database
-- it's possible these exist because of a previous run that exited unexpectedly
{% set preexisting_backup_relation = load_cached_relation(backup_relation) %}
{% set preexisting_intermediate_relation = load_cached_relation(intermediate_relation) %}
-- drop the temp relations if they exist already in the database
{{ drop_relation_if_exists(preexisting_backup_relation) }}
{{ drop_relation_if_exists(preexisting_intermediate_relation) }}
{{ run_hooks(pre_hooks, inside_transaction=False) }}
{% endmacro %}
{% macro materialized_view_teardown(backup_relation, intermediate_relation, post_hooks) %}
-- drop the temp relations if they exist to leave the database clean for the next run
{{ drop_relation_if_exists(backup_relation) }}
{{ drop_relation_if_exists(intermediate_relation) }}
{{ run_hooks(post_hooks, inside_transaction=False) }}
{% endmacro %}
{% macro materialized_view_get_build_sql(existing_relation, target_relation, backup_relation, intermediate_relation) %}
{% set full_refresh_mode = should_full_refresh() %}
@@ -88,6 +61,33 @@
{% endmacro %}
{% macro materialized_view_setup(backup_relation, intermediate_relation, pre_hooks) %}
-- backup_relation and intermediate_relation should not already exist in the database
-- it's possible these exist because of a previous run that exited unexpectedly
{% set preexisting_backup_relation = load_cached_relation(backup_relation) %}
{% set preexisting_intermediate_relation = load_cached_relation(intermediate_relation) %}
-- drop the temp relations if they exist already in the database
{{ drop_relation_if_exists(preexisting_backup_relation) }}
{{ drop_relation_if_exists(preexisting_intermediate_relation) }}
{{ run_hooks(pre_hooks, inside_transaction=False) }}
{% endmacro %}
{% macro materialized_view_teardown(backup_relation, intermediate_relation, post_hooks) %}
-- drop the temp relations if they exist to leave the database clean for the next run
{{ drop_relation_if_exists(backup_relation) }}
{{ drop_relation_if_exists(intermediate_relation) }}
{{ run_hooks(post_hooks, inside_transaction=False) }}
{% endmacro %}
{% macro materialized_view_execute_no_op(target_relation) %}
{% do store_raw_result(
name="main",

View File

@@ -1,27 +0,0 @@
{% macro get_create_view_as_sql(relation, sql) -%}
{{ adapter.dispatch('get_create_view_as_sql', 'dbt')(relation, sql) }}
{%- endmacro %}
{% macro default__get_create_view_as_sql(relation, sql) -%}
{{ return(create_view_as(relation, sql)) }}
{% endmacro %}
/* {# keep logic under old name for backwards compatibility #} */
{% macro create_view_as(relation, sql) -%}
{{ adapter.dispatch('create_view_as', 'dbt')(relation, sql) }}
{%- endmacro %}
{% macro default__create_view_as(relation, sql) -%}
{%- set sql_header = config.get('sql_header', none) -%}
{{ sql_header if sql_header is not none }}
create view {{ relation }}
{% set contract_config = config.get('contract') %}
{% if contract_config.enforced %}
{{ get_assert_columns_equivalent(sql) }}
{%- endif %}
as (
{{ sql }}
);
{%- endmacro %}

Some files were not shown because too many files have changed in this diff Show More