From 1be16c1ad7abfca6e595fc473fa55da18bf750c4 Mon Sep 17 00:00:00 2001 From: Tatiana Al-Chueyr Date: Tue, 18 Jul 2023 01:15:20 +0100 Subject: [PATCH] Refactor dbt project parsing and Airflow DAG generation (#360) This is a major refactor related to how we parse and render `dbt` projects in Airflow. It encourages users to move away from custom Cosmos implementation to parse and filter `dbt` projects. It promotes parsing the dbt project using `dbt ls` or the `dbt` `manifest` file. Some of the changes: * Create a standard class (`DbtToAirflowConverter`) to define the shared interface between `DbtDag` and `DbtTaskGroup` * Define data classes to exchange information between the different steps of the Cosmos (parse dbt project, validate arguments, filter, create Airflow DAG) * Split the logic of parsing `dbt` projects from the actual Airflow DAG generation * Add support to parse dbt projects using `dbt ls` * Add support to parse dbt projects using their `manifest.json` Breaking changes: * `from cosmos.dag import DbtDag` is now `from cosmos.airflow.dag import DbtDag` or `from cosmos import DbtDag` * `from cosmos.dag import DbtTaskGroup` is now `from cosmos.airflow.dag import DbtTaskGroup` or `from cosmos import DbtTaskGroup` * `select` and `exclude` arguments, passed to `DbtDag` or `DbtTaskGroup` used to be dictionaries, now they are lists - so we are compatible with the representation in dbt **DISCLAIMER**: This PR was not tested with the following: * validate with K8s operators * validate with docker operators This change relates to ticket: #360 --- cosmos/__init__.py | 4 +- cosmos/airflow/__init__.py | 0 cosmos/airflow/dag.py | 24 + cosmos/airflow/graph.py | 196 + cosmos/airflow/task_group.py | 24 + cosmos/converter.py | 180 + cosmos/core/airflow.py | 81 +- cosmos/dag.py | 95 - cosmos/dbt/executable.py | 8 + cosmos/dbt/graph.py | 243 + cosmos/dbt/project.py | 47 + cosmos/dbt/selector.py | 146 + cosmos/operators/local.py | 1 - cosmos/render.py | 277 - cosmos/task_group.py | 98 - dev/dags/basic_cosmos_dag.py | 2 +- dev/dags/basic_cosmos_task_group.py | 2 +- dev/dags/dbt/jaffle_shop/profiles.yml | 8 +- dev/dags/example_virtualenv.py | 2 +- docs/contributing.rst | 18 + pyproject.toml | 4 +- tests/airflow/test_graph.py | 258 + tests/dbt/test_graph.py | 176 + tests/dbt/test_selector.py | 94 + tests/sample/manifest.json | 10142 ++++++++++++++++++++++++ tests/test_airflow.py | 258 - tests/test_converter.py | 17 + tests/test_render.py | 136 - 28 files changed, 11586 insertions(+), 955 deletions(-) create mode 100644 cosmos/airflow/__init__.py create mode 100644 cosmos/airflow/dag.py create mode 100644 cosmos/airflow/graph.py create mode 100644 cosmos/airflow/task_group.py create mode 100644 cosmos/converter.py delete mode 100644 cosmos/dag.py create mode 100644 cosmos/dbt/executable.py create mode 100644 cosmos/dbt/graph.py create mode 100644 cosmos/dbt/project.py create mode 100644 cosmos/dbt/selector.py delete mode 100644 cosmos/render.py delete mode 100644 cosmos/task_group.py create mode 100644 tests/airflow/test_graph.py create mode 100644 tests/dbt/test_graph.py create mode 100644 tests/dbt/test_selector.py create mode 100644 tests/sample/manifest.json delete mode 100644 tests/test_airflow.py create mode 100644 tests/test_converter.py delete mode 100644 tests/test_render.py diff --git a/cosmos/__init__.py b/cosmos/__init__.py index 13b36d2e1..ef5d4c45d 100644 --- a/cosmos/__init__.py +++ b/cosmos/__init__.py @@ -9,8 +9,8 @@ from cosmos.dataset import get_dbt_dataset # re-export the dag and task group -from cosmos.dag import DbtDag -from cosmos.task_group import DbtTaskGroup +from cosmos.airflow.dag import DbtDag +from cosmos.airflow.task_group import DbtTaskGroup # re-export the operators from cosmos.operators.local import ( diff --git a/cosmos/airflow/__init__.py b/cosmos/airflow/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/cosmos/airflow/dag.py b/cosmos/airflow/dag.py new file mode 100644 index 000000000..8ab955a1f --- /dev/null +++ b/cosmos/airflow/dag.py @@ -0,0 +1,24 @@ +""" +This module contains a function to render a dbt project as an Airflow DAG. +""" +from __future__ import annotations + +from typing import Any + +from airflow.models.dag import DAG + +from cosmos.converter import airflow_kwargs, specific_kwargs, DbtToAirflowConverter + + +class DbtDag(DAG, DbtToAirflowConverter): + """ + Render a dbt project as an Airflow DAG. + """ + + def __init__( + self, + *args: Any, + **kwargs: Any, + ) -> None: + DAG.__init__(self, *args, **airflow_kwargs(**kwargs)) + DbtToAirflowConverter.__init__(self, *args, dag=self, **specific_kwargs(**kwargs)) diff --git a/cosmos/airflow/graph.py b/cosmos/airflow/graph.py new file mode 100644 index 000000000..85e41424d --- /dev/null +++ b/cosmos/airflow/graph.py @@ -0,0 +1,196 @@ +from __future__ import annotations + +import logging +from typing import Callable + + +from airflow.models.dag import DAG +from airflow.utils.task_group import TaskGroup + +from cosmos.core.airflow import get_airflow_task as create_airflow_task +from cosmos.core.graph.entities import Task as TaskMetadata +from cosmos.dataset import get_dbt_dataset +from cosmos.dbt.graph import DbtNode + + +logger = logging.getLogger(__name__) + + +def calculate_operator_class( + execution_mode: str, + dbt_class: str, +) -> str: + """ + Given an execution mode and dbt class, return the operator class path to use. + + :param execution_mode: Cosmos execution mode (e.g. local, virtualenv, docker, kubernetes) + :param dbt_class: The dbt command being used (e.g. DbtSnapshot, DbtRun, DbtSeed) + :returns: path string to the correspondent Cosmos Airflow operator + (e.g. cosmos.operators.localDbtSnapshotLocalOperator) + """ + return f"cosmos.operators.{execution_mode}.{dbt_class}{execution_mode.capitalize()}Operator" + + +def calculate_leaves(tasks_ids: list[str], nodes: dict[str, DbtNode]) -> list[str]: + """ + Return a list of unique_ids for nodes that are not parents (don't have dependencies on other tasks). + + :param tasks_ids: Node/task IDs which are materialized in the Airflow DAG + :param nodes: Dictionary mapping dbt nodes (node.unique_id to node) + :returns: List of unique_ids for the nodes that are graph leaves + """ + parents = [] + leaves = [] + materialized_nodes = [node for node in nodes.values() if node.unique_id in tasks_ids] + [parents.extend(node.depends_on) for node in materialized_nodes] + parents_ids = set(parents) + for node in materialized_nodes: + if node.unique_id not in parents_ids: + leaves.append(node.unique_id) + return leaves + + +def create_task_metadata(node: DbtNode, execution_mode: str, args: dict) -> TaskMetadata: + """ + Create the metadata that will be used to instantiate the Airflow Task used to run the Dbt node. + + :param node: The dbt node which we desired to convert into an Airflow Task + :param execution_mode: The Cosmos execution mode we're aiming to run the dbt task at (e.g. local) + :param args: Arguments to be used to instantiate an Airflow Task + :returns: The metadata necessary to instantiate the source dbt node as an Airflow task. + """ + dbt_resource_to_class = {"model": "DbtRun", "snapshot": "DbtSnapshot", "seed": "DbtSeed", "test": "DbtTest"} + args = {**args, **{"models": node.name}} + task_id_suffix = "run" if node.resource_type == "model" else node.resource_type + + if node.resource_type in dbt_resource_to_class: + task_metadata = TaskMetadata( + id=f"{node.name}_{task_id_suffix}", + operator_class=calculate_operator_class( + execution_mode=execution_mode, dbt_class=dbt_resource_to_class[node.resource_type] + ), + arguments=args, + ) + return task_metadata + else: + logger.error(f"Unsupported resource type {node.resource_type} (node {node.unique_id}).") + + +def create_test_task_metadata( + test_task_name: str, + execution_mode: str, + task_args: dict, + on_warning_callback: callable, + model_name: str | None = None, +) -> TaskMetadata: + """ + Create the metadata that will be used to instantiate the Airflow Task that will be used to run the Dbt test node. + + :param test_task_name: Name of the Airflow task to be created + :param execution_mode: The Cosmos execution mode we're aiming to run the dbt task at (e.g. local) + :param task_args: Arguments to be used to instantiate an Airflow Task + :param on_warning_callback: A callback function called on warnings with additional Context variables “test_names” + and “test_results” of type List. + :param model_name: If the test relates to a specific model, the name of the model it relates to + :returns: The metadata necessary to instantiate the source dbt node as an Airflow task. + """ + task_args = dict(task_args) + task_args["on_warning_callback"] = on_warning_callback + if model_name is not None: + task_args["models"] = model_name + return TaskMetadata( + id=test_task_name, + operator_class=calculate_operator_class( + execution_mode=execution_mode, + dbt_class="DbtTest", + ), + arguments=task_args, + ) + + +def build_airflow_graph( + nodes: dict[str, DbtNode], + dag: DAG, # Airflow-specific - parent DAG where to associate tasks and (optional) task groups + execution_mode: str, # Cosmos-specific - decide what which class to use + task_args: dict[str, str], # Cosmos/DBT - used to instantiate tasks + test_behavior: str | None, # Cosmos-specific: how to inject tests to Airflow DAG + dbt_project_name: str, # DBT / Cosmos - used to name test task if mode is after_all, + conn_id: str, # Cosmos, dataset URI + task_group: TaskGroup | None = None, + on_warning_callback: Callable | None = None, # argument specific to the DBT test command + emit_datasets: bool = True, # Cosmos +) -> None: + """ + Instantiate dbt `nodes` as Airflow tasks within the given `task_group` (optional) or `dag` (mandatory). + + The following arguments affect how each airflow task is instantiated: + * `execution_mode` + * `task_args` + + The parameter `test_behavior` influences how many and where test nodes will be added, while the argument + `on_warning_callback` allows users to set a callback function to be called depending on the test result. + If the `test_behavior` is None, no test nodes are added. Otherwise, if the `test_behaviour` is `after_all`, + a single test task will be added after the Cosmos leave tasks, and it is named using `dbt_project_name`. + Finally, if the `test_behaviour` is `after_each`, a test will be added after each model. + + If `emit_datasets` is True, tasks will create outlets using: + * `dbt_project_name` + * `conn_id` + + :param nodes: Dictionary mapping dbt nodes (node.unique_id to node) + :param dag: Airflow DAG instance + :param execution_mode: The Cosmos execution mode we're aiming to run the dbt task at (e.g. local) + :param task_args: Arguments to be used to instantiate an Airflow Task + :param test_behavior: Defines how many test dbt nodes and where they will be added + :param dbt_project_name: Name of the dbt pipeline of interest + :param conn_id: Airflow connection ID + :param task_group: Airflow Task Group instance + :param on_warning_callback: A callback function called on warnings with additional Context variables “test_names” + and “test_results” of type List. + :param emit_datasets: Decides if Cosmos should add outlets to model classes or not. + """ + tasks_map = {} + + # In most cases, we'll map one DBT node to one Airflow task + # The exception are the test nodes, since it would be too slow to run test tasks individually. + # If test_behaviour=="after_each", each model task will be bundled with a test task, using TaskGroup + for node_id, node in nodes.items(): + task_meta = create_task_metadata(node=node, execution_mode=execution_mode, args=task_args) + if emit_datasets: + task_args["outlets"] = [get_dbt_dataset(conn_id, dbt_project_name, node.name)] + if task_meta and node.resource_type != "test": + if node.resource_type == "model" and test_behavior == "after_each": + with TaskGroup(dag=dag, group_id=node.name, parent_group=task_group) as model_task_group: + task = create_airflow_task(task_meta, dag, task_group=model_task_group) + test_meta = create_test_task_metadata( + f"{node.name}_test", + execution_mode, + task_args=task_args, + model_name=node.name, + on_warning_callback=on_warning_callback, + ) + test_task = create_airflow_task(test_meta, dag, task_group=model_task_group) + task >> test_task + task_or_group = model_task_group + else: + task_or_group = create_airflow_task(task_meta, dag, task_group=task_group) + tasks_map[node_id] = task_or_group + + # If test_behaviour=="after_all", there will be one test task, run "by the end" of the DAG + # The end of a DAG is defined by the DAG leaf tasks (tasks which do not have downstream tasks) + if test_behavior == "after_all": + task_args.pop("outlets", None) + test_meta = create_test_task_metadata( + f"{dbt_project_name}_test", execution_mode, task_args=task_args, on_warning_callback=on_warning_callback + ) + test_task = create_airflow_task(test_meta, dag, task_group=task_group) + leaves_ids = calculate_leaves(tasks_ids=tasks_map.keys(), nodes=nodes) + for leaf_node_id in leaves_ids: + tasks_map[leaf_node_id] >> test_task + + # Create the Airflow task dependencies between non-test nodes + for node_id, node in nodes.items(): + for parent_node_id in node.depends_on: + # depending on the node type, it will not have mapped 1:1 to tasks_map + if (node_id in tasks_map) and (parent_node_id in tasks_map): + tasks_map[parent_node_id] >> tasks_map[node_id] diff --git a/cosmos/airflow/task_group.py b/cosmos/airflow/task_group.py new file mode 100644 index 000000000..67746a9bc --- /dev/null +++ b/cosmos/airflow/task_group.py @@ -0,0 +1,24 @@ +""" +This module contains a function to render a dbt project as an Airflow Task Group. +""" +from __future__ import annotations +from typing import Any + +from airflow.utils.task_group import TaskGroup + +from cosmos.converter import airflow_kwargs, specific_kwargs, DbtToAirflowConverter + + +class DbtTaskGroup(TaskGroup, DbtToAirflowConverter): + """ + Render a dbt project as an Airflow Task Group. + """ + + def __init__( + self, + *args: Any, + **kwargs: Any, + ) -> None: + group_id = kwargs.get("group_id", kwargs.get("dbt_project_name", "dbt_task_group")) + TaskGroup.__init__(self, group_id, *args, **airflow_kwargs(**kwargs)) + DbtToAirflowConverter.__init__(self, *args, task_group=self, **specific_kwargs(**kwargs)) diff --git a/cosmos/converter.py b/cosmos/converter.py new file mode 100644 index 000000000..de289d03c --- /dev/null +++ b/cosmos/converter.py @@ -0,0 +1,180 @@ +from __future__ import annotations + +import inspect +import logging +import pathlib +from typing import Any, Callable, Optional + +try: + from typing import Literal +except ImportError: + from typing_extensions import Literal + +from airflow.exceptions import AirflowException +from airflow.models.dag import DAG +from airflow.utils.task_group import TaskGroup + +from cosmos.airflow.graph import build_airflow_graph +from cosmos.dbt.executable import get_system_dbt +from cosmos.dbt.graph import DbtGraph, LoadMode +from cosmos.dbt.project import DbtProject +from cosmos.dbt.selector import retrieve_by_label + + +logger = logging.getLogger(__name__) + + +def specific_kwargs(**kwargs: dict[str, Any]) -> dict[str, Any]: + """ + Extract kwargs specific to the cosmos.converter.DbtToAirflowConverter class initialization method. + + :param kwargs: kwargs which can contain DbtToAirflowConverter and non DbtToAirflowConverter kwargs. + """ + new_kwargs = {} + specific_args_keys = inspect.getfullargspec(DbtToAirflowConverter.__init__).args + for arg_key, arg_value in kwargs.items(): + if arg_key in specific_args_keys: + new_kwargs[arg_key] = arg_value + return new_kwargs + + +def airflow_kwargs(**kwargs: dict[str, Any]) -> dict[str, Any]: + """ + Extract kwargs specific to the Airflow DAG or TaskGroup class initialization method. + + :param kwargs: kwargs which can contain Airflow DAG or TaskGroup and cosmos.converter.DbtToAirflowConverter kwargs. + """ + new_kwargs = {} + non_airflow_kwargs = specific_kwargs(**kwargs) + for arg_key, arg_value in kwargs.items(): + if arg_key not in non_airflow_kwargs: + new_kwargs[arg_key] = arg_value + return new_kwargs + + +def validate_arguments( + select: list[str], exclude: list[str], profile_args: dict[str, Any], task_args: dict[str, Any] +) -> None: + """ + Validate that mutually exclusive selectors filters have not been given. + Validate deprecated arguments. + + :param select: A list of dbt select arguments (e.g. 'config.materialized:incremental') + :param exclude: A list of dbt exclude arguments (e.g. 'tag:nightly') + :param profile_args: Arguments to pass to the dbt profile + :param task_args: Arguments to be used to instantiate an Airflow Task + """ + for field in ("tags", "paths"): + select_items = retrieve_by_label(select, field) + exclude_items = retrieve_by_label(exclude, field) + intersection = {str(item) for item in set(select_items).intersection(exclude_items)} + if intersection: + raise AirflowException(f"Can't specify the same {field[:-1]} in `select` and `exclude`: " f"{intersection}") + + # if task_args has a schema, add it to the profile args and add a deprecated warning + if "schema" in task_args: + profile_args["schema"] = task_args["schema"] + logger.warning("Specifying a schema in the `task_args` is deprecated. Please use the `profile_args` instead.") + + +class DbtToAirflowConverter: + """ + Logic common to build an Airflow DbtDag and DbtTaskGroup from a DBT project. + + :param dag: Airflow DAG to be populated + :param task_group (optional): Airflow Task Group to be populated + :param dbt_project_name: The name of the dbt project + :param dbt_root_path: The path to the dbt root directory + :param dbt_models_dir: The path to the dbt models directory within the project + :param dbt_seeds_dir: The path to the dbt seeds directory within the project + :param conn_id: The Airflow connection ID to use for the dbt profile + :param profile_args: Arguments to pass to the dbt profile + :param profile_name_override: A name to use for the dbt profile. If not provided, and no profile target is found + in your project's dbt_project.yml, "cosmos_profile" is used. + :param target_name_override: A name to use for the dbt target. If not provided, "cosmos_target" is used. + :param dbt_args: Parameters to pass to the underlying dbt operators, can include dbt_executable_path to utilize venv + :param operator_args: Parameters to pass to the underlying operators, can include KubernetesPodOperator + or DockerOperator parameters + :param emit_datasets: If enabled test nodes emit Airflow Datasets for downstream cross-DAG dependencies + :param test_behavior: The behavior for running tests. Options are "none", "after_each", and "after_all". + Defaults to "after_each" + :param select: A list of dbt select arguments (e.g. 'config.materialized:incremental') + :param exclude: A list of dbt exclude arguments (e.g. 'tag:nightly') + :param execution_mode: How Cosmos should run each dbt node (local, virtualenv, docker, k8s) + Options are "local", "virtualenv", "docker", and "kubernetes". + Defaults to "local" + :param on_warning_callback: A callback function called on warnings with additional Context variables "test_names" + and "test_results" of type `List`. Each index in "test_names" corresponds to the same index in "test_results". + """ + + def __init__( + self, + dbt_project_name: str, + conn_id: str, + dag: DAG | None = None, + task_group: TaskGroup | None = None, + profile_args: dict[str, str] = {}, + dbt_args: dict[str, Any] = {}, + profile_name_override: str | None = None, + target_name_override: str | None = None, + operator_args: dict[str, Any] = {}, + emit_datasets: bool = True, + dbt_root_path: str = "/usr/local/airflow/dags/dbt", + dbt_models_dir: str | None = None, + dbt_seeds_dir: str | None = None, + dbt_snapshots_dir: str | None = None, + test_behavior: Literal["none", "after_each", "after_all"] = "after_each", + select: list[str] | None = None, + exclude: list[str] | None = None, + execution_mode: Literal["local", "docker", "kubernetes", "virtualenv"] = "local", + load_mode: LoadMode = LoadMode.AUTOMATIC, + manifest_path: str | pathlib.Path | None = None, + on_warning_callback: Optional[Callable] = None, + *args: Any, + **kwargs: Any, + ) -> None: + select = select or [] + exclude = exclude or [] + + dbt_project = DbtProject( + name=dbt_project_name, + root_dir=dbt_root_path, + models_dir=dbt_models_dir, + seeds_dir=dbt_seeds_dir, + snapshots_dir=dbt_snapshots_dir, + manifest_path=manifest_path, + ) + + dbt_graph = DbtGraph( + project=dbt_project, + exclude=exclude, + select=select, + dbt_cmd=dbt_args.get("dbt_executable_path", get_system_dbt()), + ) + dbt_graph.load(method=load_mode, execution_mode=execution_mode) + + task_args = { + **dbt_args, + **operator_args, + "profile_args": profile_args, + "profile_name": profile_name_override, + "target_name": target_name_override, + # the following args may be only needed for local / venv: + "project_dir": dbt_project.dir, + "conn_id": conn_id, + } + + validate_arguments(select, exclude, profile_args, task_args) + + build_airflow_graph( + nodes=dbt_graph.nodes, + dag=dag or (task_group and task_group.dag), + task_group=task_group, + execution_mode=execution_mode, + task_args=task_args, + test_behavior=test_behavior, + dbt_project_name=dbt_project.name, + conn_id=conn_id, + on_warning_callback=on_warning_callback, + emit_datasets=emit_datasets, + ) diff --git a/cosmos/core/airflow.py b/cosmos/core/airflow.py index 42bbd3178..f88139ff0 100644 --- a/cosmos/core/airflow.py +++ b/cosmos/core/airflow.py @@ -1,93 +1,16 @@ import importlib import logging -from typing import Any, Dict, Optional +from typing import Optional from airflow.models import BaseOperator from airflow.models.dag import DAG from airflow.utils.task_group import TaskGroup -from cosmos.core.graph.entities import Group, Task +from cosmos.core.graph.entities import Task logger = logging.getLogger(__name__) -class CosmosDag(DAG): - """ - Render a Group as an Airflow DAG. Subclass of Airflow DAG. - """ - - def __init__( - self, - cosmos_group: Group, - *args: Any, - **kwargs: Any, - ) -> None: - # if the user doesn't specify a dag_id, use the entity id - if "dag_id" not in kwargs: - kwargs["dag_id"] = cosmos_group.id - - super().__init__(*args, **kwargs) - - entities: Dict[str, Any] = {} - - # render all the entities in the group - for ent in cosmos_group.entities: - if isinstance(ent, Group): - entities[ent.id] = CosmosTaskGroup(cosmos_group=ent, dag=self) - elif isinstance(ent, Task): - entities[ent.id] = get_airflow_task(task=ent, dag=self) - - # add dependencies - for ent in cosmos_group.entities: - for upstream_id in ent.upstream_entity_ids: - entities[upstream_id] >> entities[ent.id] - - -class CosmosTaskGroup(TaskGroup): - """ - Render a Group as an Airflow TaskGroup. Subclass of Airflow TaskGroup. - """ - - def __init__( - self, - cosmos_group: Group, - dag: Optional[DAG] = None, - *args: Any, - **kwargs: Any, - ) -> None: - # if the user doesn't specify a group_id, use the entity id - if "group_id" not in kwargs: - kwargs["group_id"] = cosmos_group.id - - # add dag back to kwargs and call the airflow constructor - kwargs["dag"] = dag - super().__init__(*args, **kwargs) - - entities: Dict[str, Any] = {} - - # render all the entities in the group - for ent in cosmos_group.entities: - if isinstance(ent, Group): - entities[ent.id] = CosmosTaskGroup(cosmos_group=ent, dag=dag, parent_group=self) - elif isinstance(ent, Task): - entities[ent.id] = get_airflow_task( - task=ent, - dag=dag, - task_group=self, - ) - - # add dependencies - for ent in cosmos_group.entities: - for upstream_id in ent.upstream_entity_ids: - if upstream_id not in entities: - raise ValueError(f"Entity {upstream_id} is not in the group {cosmos_group.id}") - - if ent.id not in entities: - raise ValueError(f"Entity {ent.id} is not in the group {cosmos_group.id}") - - entities[upstream_id] >> entities[ent.id] - - def get_airflow_task(task: Task, dag: DAG, task_group: Optional[TaskGroup] = None) -> BaseOperator: """ Get the Airflow Operator class for a Task. diff --git a/cosmos/dag.py b/cosmos/dag.py deleted file mode 100644 index 9d56c7572..000000000 --- a/cosmos/dag.py +++ /dev/null @@ -1,95 +0,0 @@ -""" -This module contains a function to render a dbt project as an Airflow DAG. -""" -from __future__ import annotations - -try: - from typing import Literal -except ImportError: - from typing_extensions import Literal - -from typing import Any, Callable, Dict, List, Optional - -from cosmos.core.airflow import CosmosDag - -from .render import render_project - - -class DbtDag(CosmosDag): - """ - Render a dbt project as an Airflow DAG. Overrides the Airflow DAG model to allow - for additional configs to be passed. - - :param dbt_project_name: The name of the dbt project - :param dbt_root_path: The path to the dbt root directory - :param dbt_models_dir: The path to the dbt models directory within the project - :param dbt_seeds_dir: The path to the dbt seeds directory within the project - :param conn_id: The Airflow connection ID to use for the dbt profile - :param profile_args: Arguments to pass to the dbt profile - :param profile_name_override: A name to use for the dbt profile. If not provided, and no profile target is found - in your project's dbt_project.yml, "cosmos_profile" is used. - :param target_name_override: A name to use for the dbt target. If not provided, "cosmos_target" is used. - :param dbt_args: Parameters to pass to the underlying dbt operators, can include dbt_executable_path to utilize venv - :param operator_args: Parameters to pass to the underlying operators, can include KubernetesPodOperator - or DockerOperator parameters - :param emit_datasets: If enabled test nodes emit Airflow Datasets for downstream cross-DAG dependencies - :param test_behavior: The behavior for running tests. Options are "none", "after_each", and "after_all". - Defaults to "after_each" - :param select: A dict of dbt selector arguments (i.e., {"tags": ["tag_1", "tag_2"]}) - :param exclude: A dict of dbt exclude arguments (i.e., {"tags": ["tag_1", "tag_2"]}) - :param execution_mode: The execution mode in which the dbt project should be run. - Options are "local", "virtualenv", "docker", and "kubernetes". - Defaults to "local" - :param on_warning_callback: A callback function called on warnings with additional Context variables "test_names" - and "test_results" of type `List`. Each index in "test_names" corresponds to the same index in "test_results". - """ - - def __init__( - self, - dbt_project_name: str, - conn_id: str, - profile_args: Dict[str, str] = {}, - dbt_args: Dict[str, Any] = {}, - profile_name_override: str | None = None, - target_name_override: str | None = None, - operator_args: Dict[str, Any] = {}, - emit_datasets: bool = True, - dbt_root_path: str = "/usr/local/airflow/dags/dbt", - dbt_models_dir: str = "models", - dbt_seeds_dir: str = "seeds", - test_behavior: Literal["none", "after_each", "after_all"] = "after_each", - select: Dict[str, List[str]] = {}, - exclude: Dict[str, List[str]] = {}, - execution_mode: Literal["local", "docker", "kubernetes", "virtualenv"] = "local", - on_warning_callback: Optional[Callable] = None, - *args: Any, - **kwargs: Any, - ) -> None: - # add additional args to the dbt_args - dbt_args = { - **dbt_args, - "conn_id": conn_id, - } - - # get the group of the dbt project - group = render_project( - dbt_project_name=dbt_project_name, - dbt_root_path=dbt_root_path, - dbt_models_dir=dbt_models_dir, - dbt_seeds_dir=dbt_seeds_dir, - task_args=dbt_args, - operator_args=operator_args, - test_behavior=test_behavior, - emit_datasets=emit_datasets, - conn_id=conn_id, - profile_args=profile_args, - profile_name=profile_name_override, - target_name=target_name_override, - select=select, - exclude=exclude, - execution_mode=execution_mode, - on_warning_callback=on_warning_callback, - ) - - # call the airflow DAG constructor - super().__init__(group, *args, **kwargs) diff --git a/cosmos/dbt/executable.py b/cosmos/dbt/executable.py new file mode 100644 index 000000000..7a1b670ae --- /dev/null +++ b/cosmos/dbt/executable.py @@ -0,0 +1,8 @@ +import shutil + + +def get_system_dbt() -> str: + """ + Tries to identify which is the path to the dbt executable, return "dbt" otherwise. + """ + return shutil.which("dbt-ol") or shutil.which("dbt") or "dbt" diff --git a/cosmos/dbt/graph.py b/cosmos/dbt/graph.py new file mode 100644 index 000000000..87739a081 --- /dev/null +++ b/cosmos/dbt/graph.py @@ -0,0 +1,243 @@ +from __future__ import annotations +import itertools +import json +import logging +from dataclasses import dataclass, field +from enum import Enum +from subprocess import Popen, PIPE +from typing import Any + +from cosmos.dbt.executable import get_system_dbt +from cosmos.dbt.parser.project import DbtProject as LegacyDbtProject +from cosmos.dbt.project import DbtProject +from cosmos.dbt.selector import select_nodes + +logger = logging.getLogger(__name__) + +# TODO replace inline constants + + +class CosmosLoadDbtException(Exception): + """ + Exception raised while trying to load a `dbt` project as a `DbtGraph` instance. + """ + + pass + + +class LoadMode(Enum): + """ + Supported ways to load a `dbt` project into a `DbtGraph` instance. + """ + + AUTOMATIC = "automatic" + CUSTOM = "custom" + DBT_LS = "dbt_ls" + DBT_MANIFEST = "dbt_manifest" + + +@dataclass +class DbtNode: + """ + Metadata related to a dbt node (e.g. model, seed, snapshot). + """ + + name: str + unique_id: str + resource_type: str + depends_on: list[str] + file_path: str + tags: list[str] = field(default_factory=lambda: []) + config: dict[str, Any] = field(default_factory=lambda: {}) + + +class DbtGraph: + """ + A dbt project graph (represented by `nodes` and `filtered_nodes`). + Supports different ways of loading the `dbt` project into this representation. + + Different loading methods can result in different `nodes` and `filtered_nodes`. + + Example of how to use: + + dbt_graph = DbtGraph( + project=DbtProject(name="jaffle_shop", root_dir=DBT_PROJECTS_ROOT_DIR), + exclude=["*orders*"], + select=[], + dbt_cmd="/usr/local/bin/dbt", + ) + dbt_graph.load(method=LoadMode.DBT_LS, execution_mode="local") + """ + + nodes: dict[str, DbtNode] = dict() + filtered_nodes: dict[str, DbtNode] = dict() + + def __init__( + self, + project: DbtProject, + exclude: list[str] | None = None, + select: list[str] = None, + dbt_cmd: str = get_system_dbt(), + ): + self.project = project + self.exclude = exclude or [] + self.select = select or [] + + # specific to loading using ls + self.dbt_cmd = dbt_cmd + + def load(self, method: LoadMode = LoadMode.AUTOMATIC, execution_mode: str = "local") -> None: + """ + Load a `dbt` project into a `DbtGraph`, setting `nodes` and `filtered_nodes` accordingly. + + :param method: How to load `nodes` from a `dbt` project (automatically, using custom parser, using dbt manifest + or dbt ls) + :param execution_mode: How Cosmos should run each dbt node (local, virtualenv, docker, k8s) + """ + load_method = { + LoadMode.CUSTOM: self.load_via_custom_parser, + LoadMode.DBT_LS: self.load_via_dbt_ls, + LoadMode.DBT_MANIFEST: self.load_from_dbt_manifest, + } + if method == LoadMode.AUTOMATIC: + if self.project.is_manifest_available(): + self.load_from_dbt_manifest() + return + elif execution_mode in ("local", "virtualenv") and self.project.is_profile_yml_available(): + try: + self.load_via_dbt_ls() + return + except FileNotFoundError: + self.load_via_custom_parser() + return + else: + self.load_via_custom_parser() + return + + if method == LoadMode.DBT_MANIFEST and not self.project.is_manifest_available(): + raise CosmosLoadDbtException(f"Unable to load manifest using {self.project.manifest_path}") + + load_method[method]() + + def load_via_dbt_ls(self): + """ + This is the most accurate way of loading `dbt` projects and filtering them out, since it uses the `dbt` command + line for both parsing and filtering the nodes. + + Updates in-place: + * self.nodes + * self.filtered_nodes + """ + logger.info("Trying to parse the dbt project using dbt ls...") + command = [self.dbt_cmd, "ls", "--output", "json", "--profiles-dir", self.project.dir] + if self.exclude: + command.extend(["--exclude", *self.exclude]) + if self.select: + command.extend(["--select", *self.select]) + logger.info(f"Running command: {command}") + try: + process = Popen(command, stdout=PIPE, stderr=PIPE, cwd=self.project.dir, universal_newlines=True) + except FileNotFoundError as exception: + raise CosmosLoadDbtException(f"Unable to run the command due to the error:\n{exception}") + + stdout, stderr = process.communicate() + + logger.debug(f"Output: {stdout}") + + if stderr or "Runtime Error" in stdout: + details = stderr or stdout + raise CosmosLoadDbtException(f"Unable to run the command due to the error:\n{details}") + + nodes = {} + for line in stdout.split("\n"): + try: + node_dict = json.loads(line.strip()) + except json.decoder.JSONDecodeError: + logger.info("Skipping line: %s", line) + else: + node = DbtNode( + name=node_dict["name"], + unique_id=node_dict["unique_id"], + resource_type=node_dict["resource_type"], + depends_on=node_dict["depends_on"].get("nodes", []), + file_path=self.project.dir / node_dict["original_file_path"], + tags=node_dict["tags"], + config=node_dict["config"], + ) + nodes[node.unique_id] = node + + self.nodes = nodes + self.filtered_nodes = nodes + + def load_via_custom_parser(self): + """ + This is the least accurate way of loading `dbt` projects and filtering them out, since it uses custom Cosmos + logic, which is usually a subset of what is available in `dbt`. + + Internally, it uses the legacy Cosmos DbtProject representation and converts it to the current + nodes list representation. + + Updates in-place: + * self.nodes + * self.filtered_nodes + """ + logger.info("Trying to parse the dbt project using a custom Cosmos method...") + project = LegacyDbtProject( + dbt_root_path=self.project.root_dir, + dbt_models_dir=self.project.models_dir.stem, + dbt_snapshots_dir=self.project.snapshots_dir.stem, + dbt_seeds_dir=self.project.seeds_dir.stem, + project_name=self.project.name, + ) + nodes = {} + models = itertools.chain(project.models.items(), project.snapshots.items(), project.seeds.items()) + for model_name, model in models: + config = {item.split(":")[0]: item.split(":")[-1] for item in model.config.config_selectors} + node = DbtNode( + name=model_name, + unique_id=model_name, + resource_type=model.type, + depends_on=model.config.upstream_models, + file_path=model.path, + tags=[], + config=config, + ) + nodes[model_name] = node + + self.nodes = nodes + self.filtered_nodes = select_nodes( + project_dir=self.project.dir, nodes=nodes, select=self.select, exclude=self.exclude + ) + + def load_from_dbt_manifest(self): + """ + This approach accurately loads `dbt` projects using the `manifest.yml` file. + + However, since the Manifest does not represent filters, it relies on the Custom Cosmos implementation + to filter out the nodes relevant to the user (based on self.exclude and self.select). + + Updates in-place: + * self.nodes + * self.filtered_nodes + """ + logger.info("Trying to parse the dbt project using a dbt manifest...") + nodes = {} + with open(self.project.manifest_path) as fp: + manifest = json.load(fp) + + for unique_id, node_dict in manifest.get("nodes", {}).items(): + node = DbtNode( + name=node_dict["name"], + unique_id=unique_id, + resource_type=node_dict["resource_type"], + depends_on=node_dict["depends_on"].get("nodes", []), + file_path=self.project.dir / node_dict["original_file_path"], + tags=node_dict["tags"], + config=node_dict["config"], + ) + nodes[node.unique_id] = node + + self.nodes = nodes + self.filtered_nodes = select_nodes( + project_dir=self.project.dir, nodes=nodes, select=self.select, exclude=self.exclude + ) diff --git a/cosmos/dbt/project.py b/cosmos/dbt/project.py new file mode 100644 index 000000000..df3a8bcaa --- /dev/null +++ b/cosmos/dbt/project.py @@ -0,0 +1,47 @@ +from __future__ import annotations +from dataclasses import dataclass +from pathlib import Path + + +DEFAULT_PROFILE_FILE_NAME = "profiles.yml" + + +@dataclass +class DbtProject: + name: str + root_dir: Path + models_dir: Path | None = None + seeds_dir: Path | None = None + snapshots_dir: Path | None = None + manifest_path: Path | None = None + profile_path: Path | None = None + _cosmos_created_profile_file: bool = False + + def __post_init__(self): + if self.models_dir is None: + self.models_dir = self.dir / "models" + if self.seeds_dir is None: + self.seeds_dir = self.dir / "seeds" + if self.snapshots_dir is None: + self.snapshots_dir = self.dir / "snapshots" + if self.profile_path is None: + self.profile_path = self.dir / "profiles.yml" + + @property + def dir(self) -> Path: + """ + Path to dbt pipeline, defined by self.root_dir and self.name. + """ + return self.root_dir / self.name + + def is_manifest_available(self) -> bool: + """ + Check if the `dbt` project manifest is set and if the file exists. + """ + return self.manifest_path and Path(self.manifest_path).exists() + + def is_profile_yml_available(self) -> bool: + """ + Check if the `dbt` profiles.yml file exists. + """ + return Path(self.profile_path).exists() diff --git a/cosmos/dbt/selector.py b/cosmos/dbt/selector.py new file mode 100644 index 000000000..e0098acea --- /dev/null +++ b/cosmos/dbt/selector.py @@ -0,0 +1,146 @@ +from __future__ import annotations +import logging +from pathlib import Path + + +SUPPORTED_CONFIG = ["materialized", "schema", "tags"] +PATH_SELECTOR = "path:" +TAG_SELECTOR = "tag:" +CONFIG_SELECTOR = "config." + + +logger = logging.getLogger(__name__) + + +class SelectorConfig: + """ + Represents a select/exclude statement. + Supports to load it from a string. + """ + + def __init__(self, project_dir: Path, statement: str): + """ + Create a selector config file. + + :param project_dir: Directory to a dbt project + :param statement: dbt statement as passed within select and exclude arguments + + References: + https://docs.getdbt.com/reference/node-selection/syntax + https://docs.getdbt.com/reference/node-selection/yaml-selectors + """ + self.project_dir = project_dir + self.paths: list[str] = [] + self.tags: list[str] = [] + self.config: dict[str, str] = {} + self.other: list[str] = [] + self.load_from_statement(statement) + + def load_from_statement(self, statement: str): + """ + Load in-place select parameters. + Raises an exception if they are not yet implemented in Cosmos. + + :param statement: dbt statement as passed within select and exclude arguments + + References: + https://docs.getdbt.com/reference/node-selection/syntax + https://docs.getdbt.com/reference/node-selection/yaml-selectors + """ + items = statement.split(",") + for item in items: + if item.startswith(PATH_SELECTOR): + index = len(PATH_SELECTOR) + self.paths.append(self.project_dir / item[index:]) + elif item.startswith(TAG_SELECTOR): + index = len(TAG_SELECTOR) + self.tags.append(item[index:]) + elif item.startswith(CONFIG_SELECTOR): + index = len(CONFIG_SELECTOR) + key, value = item[index:].split(":") + if key in SUPPORTED_CONFIG: + self.config[key] = value + else: + self.other.append(item) + logger.warning("Unsupported select statement: %s", item) + + +def select_nodes_ids_by_intersection(nodes: dict, config: SelectorConfig) -> list[str]: + """ + Return a list of node ids which matches the configuration defined in config. + + :param nodes: Dictionary mapping dbt nodes (node.unique_id to node) + :param config: User-defined select statements + + References: + https://docs.getdbt.com/reference/node-selection/syntax + https://docs.getdbt.com/reference/node-selection/yaml-selectors + """ + selected_nodes = set() + for node_id, node in nodes.items(): + if config.tags and not (sorted(node.tags) == sorted(config.tags)): + continue + + supported_node_config = {key: value for key, value in node.config.items() if key in SUPPORTED_CONFIG} + if config.config and not (config.config.items() <= supported_node_config.items()): + continue + + if config.paths and not (set(config.paths).issubset(set(node.file_path.parents))): + continue + + selected_nodes.add(node_id) + + return selected_nodes + + +def retrieve_by_label(statement_list: list[str], label: str) -> set: + """ + Return a set of values associated with a label. + + Example: + >>> values = retrieve_by_label(["path:/tmp,tag:a", "tag:b,path:/home"]) + >>> values + {"a", "b"} + """ + label_values = set() + for statement in statement_list: + config = SelectorConfig(Path(), statement) + item_values = getattr(config, label) + label_values = label_values.union(item_values) + return label_values + + +def select_nodes( + project_dir: Path, nodes: dict[str, str], select: list[str] | None = None, exclude: list[str] | None = None +) -> dict[str, str]: + """ + Given a group of nodes within a project, apply select and exclude filters using + dbt node selection. + + References: + https://docs.getdbt.com/reference/node-selection/syntax + https://docs.getdbt.com/reference/node-selection/yaml-selectors + """ + select = select or [] + exclude = exclude or [] + if not select and not exclude: + return nodes + + subset_ids = set() + + for statement in select: + config = SelectorConfig(project_dir, statement) + select_ids = select_nodes_ids_by_intersection(nodes, config) + subset_ids = subset_ids.union(set(select_ids)) + + if select: + nodes = {id_: nodes[id_] for id_ in subset_ids} + + nodes_ids = set(nodes.keys()) + + for statement in exclude: + config = SelectorConfig(project_dir, statement) + exclude_ids = select_nodes_ids_by_intersection(nodes, config) + subset_ids = set(nodes_ids) - set(exclude_ids) + + return {id_: nodes[id_] for id_ in subset_ids} diff --git a/cosmos/operators/local.py b/cosmos/operators/local.py index 72cd4f373..8461bd07b 100644 --- a/cosmos/operators/local.py +++ b/cosmos/operators/local.py @@ -194,7 +194,6 @@ def run_command( self.project_dir, tmp_project_dir, ) - profile_name = self.get_profile_name(tmp_project_dir) target_name = self.get_target_name() diff --git a/cosmos/render.py b/cosmos/render.py deleted file mode 100644 index 36ffa2b86..000000000 --- a/cosmos/render.py +++ /dev/null @@ -1,277 +0,0 @@ -""" -This module contains a function to render a dbt project into Cosmos entities. -""" -from __future__ import annotations - -import itertools -import logging - -try: - from typing import Literal -except ImportError: - from typing_extensions import Literal - -from typing import Any, Callable, Dict, List, Optional - -from airflow.exceptions import AirflowException - -from cosmos.core.graph.entities import CosmosEntity, Group, Task -from cosmos.dataset import get_dbt_dataset -from cosmos.dbt.parser.project import DbtModelType, DbtProject - -logger = logging.getLogger(__name__) - - -def calculate_operator_class( - execution_mode: str, - dbt_class: str, -) -> str: - "Given an execution mode and dbt class, return the operator class to use" - return f"cosmos.operators.{execution_mode}.{dbt_class}{execution_mode.capitalize()}Operator" - - -def render_project( - dbt_project_name: str, - dbt_root_path: str = "/usr/local/airflow/dags/dbt", - dbt_models_dir: str = "models", - dbt_snapshots_dir: str = "snapshots", - dbt_seeds_dir: str = "seeds", - task_args: Dict[str, Any] = {}, - operator_args: Dict[str, Any] = {}, - test_behavior: Literal["none", "after_each", "after_all"] = "after_each", - emit_datasets: bool = True, - conn_id: str = "default_conn_id", - profile_args: Dict[str, str] = {}, - profile_name: str | None = None, - target_name: str | None = None, - select: Dict[str, List[str]] = {}, - exclude: Dict[str, List[str]] = {}, - execution_mode: Literal["local", "docker", "kubernetes"] = "local", - on_warning_callback: Optional[Callable] = None, -) -> Group: - """ - Turn a dbt project into a Group - - :param dbt_project_name: The name of the dbt project - :param dbt_root_path: The root path to your dbt folder. Defaults to /usr/local/airflow/dags/dbt - :param task_args: Arguments to pass to the underlying dbt operators - :param operator_args: Parameters to pass to the underlying operators, can include KubernetesPodOperator - or DockerOperator parameters - :param test_behavior: The behavior for running tests. Options are "none", "after_each", and "after_all". - Defaults to "after_each" - :param emit_datasets: If enabled test nodes emit Airflow Datasets for downstream cross-DAG dependencies - :param conn_id: The Airflow connection ID to use - :param profile_args: Arguments to pass to the dbt profile - :param profile_name: A name to use for the dbt profile. If not provided, and no profile target is found - in your project's dbt_project.yml, "cosmos_profile" is used. - :param target_name: A name to use for the dbt target. If not provided, "cosmos_target" is used. - :param select: A dict of dbt selector arguments (i.e., {"tags": ["tag_1", "tag_2"]}) - :param exclude: A dict of dbt exclude arguments (i.e., {"tags": ["tag_1", "tag_2]}}) - :param execution_mode: The execution mode in which the dbt project should be run. - Options are "local", "docker", and "kubernetes". - Defaults to "local" - :param on_warning_callback: A callback function called on warnings with additional Context variables "test_names" - and "test_results" of type `List`. Each index in "test_names" corresponds to the same index in "test_results". - """ - # first, get the dbt project - project = DbtProject( - dbt_root_path=dbt_root_path, - dbt_models_dir=dbt_models_dir, - dbt_snapshots_dir=dbt_snapshots_dir, - dbt_seeds_dir=dbt_seeds_dir, - project_name=dbt_project_name, - ) - - # this is the group that will be returned - base_group = Group(id=dbt_project_name) - entities: Dict[str, CosmosEntity] = {} # this is a dict of all the entities we create - - # add project_dir arg to task_args - if execution_mode == "local": - task_args["project_dir"] = project.project_dir - - # ensures the same tag isn't in select & exclude - if "tags" in select and "tags" in exclude: - if set(select["tags"]).intersection(exclude["tags"]): - raise AirflowException( - f"Can't specify the same tag in `select` and `include`: " - f"{set(select['tags']).intersection(exclude['tags'])}" - ) - - if "paths" in select and "paths" in exclude: - if set(select["paths"]).intersection(exclude["paths"]): - raise AirflowException( - f"Can't specify the same path in `select` and `include`: " - f"{set(select['paths']).intersection(exclude['paths'])}" - ) - - # if task_args has a schema, add it to the profile args and add a deprecated warning - if "schema" in task_args: - profile_args["schema"] = task_args["schema"] - logger.warning("Specifying a schema in the task_args is deprecated. Please use the profile_args instead.") - - # iterate over each model once to create the initial tasks - for model_name, model in itertools.chain(project.models.items(), project.snapshots.items(), project.seeds.items()): - # filters down to a path within the project_dir - if "paths" in select: - root_directories = [project.project_dir / path.strip("/") for path in select.get("paths", [])] - if not set(root_directories).intersection(model.path.parents): - continue - - # filters out any specified paths - if "paths" in exclude: - root_directories = [project.project_dir / path.strip("/") for path in exclude.get("paths")] - if set(root_directories).intersection(model.path.parents): - continue - - if "configs" in select: - # TODO: coverme - if not set(select["configs"]).intersection(model.config.config_selectors): - continue - - if "configs" in exclude: - # TODO: coverme - if set(exclude["configs"]).intersection(model.config.config_selectors): - continue - - run_args: Dict[str, Any] = { - **task_args, - **operator_args, - "models": model_name, - "profile_args": profile_args, - "profile_name": profile_name, - "target_name": target_name, - } - test_args: Dict[str, Any] = { - **task_args, - **operator_args, - "models": model_name, - "profile_args": profile_args, - "profile_name": profile_name, - "target_name": target_name, - } - # DbtTestOperator specific arg - test_args["on_warning_callback"] = on_warning_callback - if emit_datasets: - outlets = [get_dbt_dataset(conn_id, dbt_project_name, model_name)] - - if test_behavior == "after_each": - test_args["outlets"] = outlets - else: - # TODO: coverme - run_args["outlets"] = outlets - - if model.type == DbtModelType.DBT_MODEL: - # make the run task for model - run_task = Task( - id=f"{model_name}_run", - operator_class=calculate_operator_class( - execution_mode=execution_mode, - dbt_class="DbtRun", - ), - arguments=run_args, - ) - elif model.type == DbtModelType.DBT_SNAPSHOT: - # make the run task for snapshot - run_task = Task( - id=f"{model_name}_snapshot", - operator_class=calculate_operator_class( - execution_mode=execution_mode, - dbt_class="DbtSnapshot", - ), - arguments=run_args, - ) - elif model.type == DbtModelType.DBT_SEED: - # make the run task for snapshot - run_task = Task( - id=f"{model_name}_seed", - operator_class=calculate_operator_class( - execution_mode=execution_mode, - dbt_class="DbtSeed", - ), - arguments=run_args, - ) - else: - # TODO: coverme - logger.error("Unknown dbt type.") - continue - - # if test_behavior isn't "after_each", we can just add the task to the - # base group and do nothing else for now - if test_behavior != "after_each": - entities[model_name] = run_task - base_group.add_entity(entity=run_task) - continue - - # otherwise, we need to make a test task after run tasks and turn them into a group - entities[run_task.id] = run_task - - if model.type == DbtModelType.DBT_MODEL: - test_task = Task( - id=f"{model_name}_test", - operator_class=calculate_operator_class( - execution_mode=execution_mode, - dbt_class="DbtTest", - ), - upstream_entity_ids=[run_task.id], - arguments=test_args, - ) - entities[test_task.id] = test_task - # make the group - model_group = Group( - id=f"{model_name}", - entities=[run_task, test_task], - ) - entities[model_group.id] = model_group - base_group.add_entity(entity=model_group) - - # all other non-run tasks don't need to be grouped with test tasks - else: - entities[model_name] = run_task - base_group.add_entity(entity=run_task) - - # add dependencies now that we have all the entities - for model_name, model in itertools.chain(project.models.items(), project.snapshots.items(), project.seeds.items()): - upstream_deps = model.config.upstream_models - for upstream_model_name in upstream_deps: - try: - dep_task = entities[upstream_model_name] - entities[model_name].add_upstream(dep_task) - except KeyError: - logger.error(f"Dependency {upstream_model_name} not found for model {model}") - if test_behavior == "after_all": - # make a test task - test_task = Task( - id=f"{dbt_project_name}_test", - operator_class=calculate_operator_class( - execution_mode=execution_mode, - dbt_class="DbtTest", - ), - arguments={**task_args, **operator_args}, - ) - entities[test_task.id] = test_task - - # add it to the base group - base_group.add_entity(test_task) - - # add it as an upstream to all the models that don't have downstream tasks - # since we don't have downstream info readily available, we have to iterate - # start with all models, and remove them as we find downstream tasks - models_with_no_downstream_tasks = [model_name for model_name, model in project.models.items()] - - # iterate over all models - for model_name, model in project.models.items(): - # iterate over all upstream models - for upstream_model_name in model.config.upstream_models: - # remove the upstream model from the list of models with no downstream tasks - try: - models_with_no_downstream_tasks.remove(upstream_model_name) - except ValueError: - pass - - # add the test task as an upstream to all models with no downstream tasks - for model_name in models_with_no_downstream_tasks: - if model_name in entities: - test_task.add_upstream(entity=entities[model_name]) - - return base_group diff --git a/cosmos/task_group.py b/cosmos/task_group.py deleted file mode 100644 index b1db10d6d..000000000 --- a/cosmos/task_group.py +++ /dev/null @@ -1,98 +0,0 @@ -""" -This module contains a function to render a dbt project as an Airflow Task Group. -""" -from __future__ import annotations - -try: - from typing import Literal -except ImportError: - from typing_extensions import Literal - -from typing import Any, Callable, Dict, List, Optional - -from cosmos.core.airflow import CosmosTaskGroup - -from .render import render_project - - -class DbtTaskGroup(CosmosTaskGroup): - """ - Render a dbt project as an Airflow Task Group. Overrides the Airflow Task Group model to allow - for additional configs to be passed. - - :param dbt_project_name: The name of the dbt project - :param dbt_root_path: The path to the dbt root directory - :param dbt_models_dir: The path to the dbt models directory within the project - :param dbt_snapshots_dir: The path to the dbt snapshots directory within the project - :param dbt_seeds_dir: The path to the dbt seeds directory within the project - :param conn_id: The Airflow connection ID to use for the dbt profile - :param profile_args: Arguments to pass to the dbt profile - :param profile_name_override: A name to use for the dbt profile. If not provided, and no profile target is found - in your project's dbt_project.yml, "cosmos_profile" is used. - :param target_name_override: A name to use for the dbt target. If not provided, "cosmos_target" is used. - :param dbt_args: Parameters to pass to the underlying dbt operators, can include dbt_executable_path to utilize venv - :param operator_args: Parameters to pass to the underlying operators, can include KubernetesPodOperator - or DockerOperator parameters - :param emit_datasets: If enabled test nodes emit Airflow Datasets for downstream cross-DAG dependencies - :param test_behavior: The behavior for running tests. Options are "none", "after_each", and "after_all". - Defaults to "after_each" - :param select: A dict of dbt selector arguments (i.e., {"tags": ["tag_1", "tag_2"]}) - :param exclude: A dict of dbt exclude arguments (i.e., {"tags": ["tag_1", "tag_2"]}) - :param execution_mode: The execution mode in which the dbt project should be run. - Options are "local", "virtualenv", "docker", and "kubernetes". - Defaults to "local" - :param on_warning_callback: A callback function called on warnings with additional Context variables "test_names" - and "test_results" of type `List`. Each index in "test_names" corresponds to the same index in "test_results". - """ - - def __init__( - self, - dbt_project_name: str, - conn_id: str, - profile_args: Dict[str, str] = {}, - profile_name_override: Optional[str] = None, - target_name_override: Optional[str] = None, - dbt_args: Dict[str, Any] = {}, - operator_args: Dict[str, Any] = {}, - emit_datasets: bool = True, - dbt_root_path: str = "/usr/local/airflow/dags/dbt", - dbt_models_dir: str = "models", - dbt_snapshots_dir: str = "snapshots", - dbt_seeds_dir: str = "seeds", - test_behavior: Literal["none", "after_each", "after_all"] = "after_each", - select: Dict[str, List[str]] = {}, - exclude: Dict[str, List[str]] = {}, - execution_mode: Literal["local", "docker", "kubernetes", "virtualenv"] = "local", - on_warning_callback: Optional[Callable] = None, - *args: Any, - **kwargs: Any, - ) -> None: - # add additional args to the dbt_args - dbt_args = { - **dbt_args, - "conn_id": conn_id, - } - - # get the group of the dbt project - group = render_project( - dbt_project_name=dbt_project_name, - dbt_root_path=dbt_root_path, - dbt_models_dir=dbt_models_dir, - dbt_snapshots_dir=dbt_snapshots_dir, - dbt_seeds_dir=dbt_seeds_dir, - task_args=dbt_args, - operator_args=operator_args, - test_behavior=test_behavior, - emit_datasets=emit_datasets, - conn_id=conn_id, - profile_args=profile_args, - profile_name=profile_name_override, - target_name=target_name_override, - select=select, - exclude=exclude, - execution_mode=execution_mode, - on_warning_callback=on_warning_callback, - ) - - # call the airflow constructor - super().__init__(group, *args, **kwargs) diff --git a/dev/dags/basic_cosmos_dag.py b/dev/dags/basic_cosmos_dag.py index 62ecce7f9..843069496 100644 --- a/dev/dags/basic_cosmos_dag.py +++ b/dev/dags/basic_cosmos_dag.py @@ -6,7 +6,7 @@ from datetime import datetime from pathlib import Path -from cosmos.dag import DbtDag +from cosmos import DbtDag DEFAULT_DBT_ROOT_PATH = Path(__file__).parent / "dbt" DBT_ROOT_PATH = os.getenv("DBT_ROOT_PATH", DEFAULT_DBT_ROOT_PATH) diff --git a/dev/dags/basic_cosmos_task_group.py b/dev/dags/basic_cosmos_task_group.py index 92514c92f..7d80dc767 100644 --- a/dev/dags/basic_cosmos_task_group.py +++ b/dev/dags/basic_cosmos_task_group.py @@ -8,7 +8,7 @@ from airflow.decorators import dag from airflow.operators.empty import EmptyOperator -from cosmos.task_group import DbtTaskGroup +from cosmos import DbtTaskGroup DEFAULT_DBT_ROOT_PATH = Path(__file__).parent / "dbt" DBT_ROOT_PATH = os.getenv("DBT_ROOT_PATH", DEFAULT_DBT_ROOT_PATH) diff --git a/dev/dags/dbt/jaffle_shop/profiles.yml b/dev/dags/dbt/jaffle_shop/profiles.yml index 76268aba8..a963d3eb4 100644 --- a/dev/dags/dbt/jaffle_shop/profiles.yml +++ b/dev/dags/dbt/jaffle_shop/profiles.yml @@ -5,9 +5,9 @@ jaffle_shop: dev: type: postgres host: localhost - user: alice - password: + user: postgres + password: postgres port: 5432 - dbname: jaffle_shop - schema: dbt_alice + dbname: public + schema: dbt_example threads: 4 diff --git a/dev/dags/example_virtualenv.py b/dev/dags/example_virtualenv.py index f6433a4b8..b8a81d9a3 100644 --- a/dev/dags/example_virtualenv.py +++ b/dev/dags/example_virtualenv.py @@ -5,7 +5,7 @@ from datetime import datetime from pathlib import Path -from cosmos.dag import DbtDag +from cosmos import DbtDag DEFAULT_DBT_ROOT_PATH = Path(__file__).parent / "dbt" DBT_ROOT_PATH = Path(os.getenv("DBT_ROOT_PATH", DEFAULT_DBT_ROOT_PATH)) diff --git a/docs/contributing.rst b/docs/contributing.rst index eab0a0ca0..74f689a18 100644 --- a/docs/contributing.rst +++ b/docs/contributing.rst @@ -53,6 +53,24 @@ To run the integration tests for the first time, use: If testing for the same Airflow and Python version, next runs of the integration tests can be: +.. code-block:: bash + hatch run tests.py3.8-2.5:test-integration + +.. code-block:: bash + + docker run --name postgres -p 5432:5432 -p 5433:5433 -e POSTGRES_PASSWORD=postgres postgres + +To run the integration tests for the first time, use: + +.. code-block:: bash + + export AIRFLOW_HOME=`pwd` + export AIRFLOW_CONN_AIRFLOW_DB=postgres://postgres:postgres@0.0.0.0:5432/postgres + hatch run tests.py3.8-2.5:test-integration-setup + hatch run tests.py3.8-2.5:test-integration + +If testing for the same Airflow and Python version, next runs of the integration tests can be: + .. code-block:: bash hatch run tests.py3.8-2.5:test-integration diff --git a/pyproject.toml b/pyproject.toml index 85ec275d9..9c8701193 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -149,8 +149,8 @@ matrix.airflow.dependencies = [ [tool.hatch.envs.tests.scripts] freeze = "pip freeze" -test = 'pytest -vv --durations=0 . -m "not integration"' -test-cov = 'pytest -vv --cov=cosmos --cov-report=term-missing --cov-report=xml --durations=0 -m "not integration"' +test = 'pytest -vv --durations=0 . -m "not integration" --ignore=tests/test_example_dags.py' +test-cov = 'pytest -vv --cov=cosmos --cov-report=term-missing --cov-report=xml --durations=0 -m "not integration" --ignore=tests/test_example_dags.py' # we install using the following workaround to overcome installation conflicts, such as: # apache-airflow 2.3.0 and dbt-core [0.13.0 - 1.5.2] and jinja2>=3.0.0 because these package versions have conflicting dependencies test-integration-setup = """pip uninstall dbt-postgres; \ diff --git a/tests/airflow/test_graph.py b/tests/airflow/test_graph.py new file mode 100644 index 000000000..67edee12c --- /dev/null +++ b/tests/airflow/test_graph.py @@ -0,0 +1,258 @@ +from pathlib import Path +from datetime import datetime + +import pytest +from airflow import __version__ as airflow_version +from airflow.models import DAG +from packaging import version + +from cosmos.airflow.graph import ( + build_airflow_graph, + calculate_leaves, + create_task_metadata, + create_test_task_metadata, + calculate_operator_class, +) +from cosmos.dbt.graph import DbtNode + + +SAMPLE_PROJ_PATH = Path("/home/user/path/dbt-proj/") + +parent_seed = DbtNode( + name="seed_parent", + unique_id="seed_parent", + resource_type="seed", + depends_on=[], + file_path="", +) +parent_node = DbtNode( + name="parent", + unique_id="parent", + resource_type="model", + depends_on=["seed_parent"], + file_path=SAMPLE_PROJ_PATH / "gen2/models/parent.sql", + tags=["has_child"], + config={"materialized": "view"}, +) +test_parent_node = DbtNode( + name="test_parent", unique_id="test_parent", resource_type="test", depends_on=["parent"], file_path="" +) +child_node = DbtNode( + name="child", + unique_id="child", + resource_type="model", + depends_on=["parent"], + file_path=SAMPLE_PROJ_PATH / "gen3/models/child.sql", + tags=["nightly"], + config={"materialized": "table"}, +) +test_child_node = DbtNode( + name="test_child", + unique_id="test_child", + resource_type="test", + depends_on=["child"], + file_path="", +) + +sample_nodes_list = [parent_seed, parent_node, test_parent_node, child_node, test_child_node] +sample_nodes = {node.unique_id: node for node in sample_nodes_list} + + +@pytest.mark.skipif( + version.parse(airflow_version) < version.parse("2.4"), + reason="Airflow DAG did not have task_group_dict until the 2.4 release", +) +@pytest.mark.integration +def test_build_airflow_graph_with_after_each(): + with DAG("test-id", start_date=datetime(2022, 1, 1)) as dag: + task_args = { + "project_dir": SAMPLE_PROJ_PATH, + "conn_id": "fake_conn", + } + build_airflow_graph( + nodes=sample_nodes, + dag=dag, + execution_mode="local", + task_args=task_args, + test_behavior="after_each", + dbt_project_name="astro_shop", + conn_id="fake_conn", + ) + topological_sort = [task.task_id for task in dag.topological_sort()] + expected_sort = [ + "seed_parent_seed", + "parent.parent_run", + "parent.parent_test", + "child.child_run", + "child.child_test", + ] + assert topological_sort == expected_sort + task_groups = dag.task_group_dict + assert len(task_groups) == 2 + + assert task_groups["parent"].upstream_task_ids == {"seed_parent_seed"} + assert list(task_groups["parent"].children.keys()) == ["parent.parent_run", "parent.parent_test"] + + assert task_groups["child"].upstream_task_ids == {"parent.parent_test"} + assert list(task_groups["child"].children.keys()) == ["child.child_run", "child.child_test"] + + assert len(dag.leaves) == 1 + assert dag.leaves[0].task_id == "child.child_test" + + +@pytest.mark.skipif( + version.parse(airflow_version) < version.parse("2.4"), + reason="Airflow DAG did not have task_group_dict until the 2.4 release", +) +@pytest.mark.integration +def test_build_airflow_graph_with_after_all(): + with DAG("test-id", start_date=datetime(2022, 1, 1)) as dag: + task_args = { + "project_dir": SAMPLE_PROJ_PATH, + "conn_id": "fake_conn", + } + build_airflow_graph( + nodes=sample_nodes, + dag=dag, + execution_mode="local", + task_args=task_args, + test_behavior="after_all", + dbt_project_name="astro_shop", + conn_id="fake_conn", + ) + topological_sort = [task.task_id for task in dag.topological_sort()] + expected_sort = ["seed_parent_seed", "parent_run", "child_run", "astro_shop_test"] + assert topological_sort == expected_sort + + task_groups = dag.task_group_dict + assert len(task_groups) == 0 + + assert len(dag.leaves) == 1 + assert dag.leaves[0].task_id == "astro_shop_test" + + +def test_calculate_operator_class(): + class_module_import_path = calculate_operator_class(execution_mode="kubernetes", dbt_class="Seed") + assert class_module_import_path == "cosmos.operators.kubernetes.SeedKubernetesOperator" + + +def test_calculate_leaves(): + grandparent_node = DbtNode( + name="grandparent", + unique_id="grandparent", + resource_type="model", + depends_on=[], + file_path="", + tags=[], + config={}, + ) + parent1_node = DbtNode( + name="parent1", + unique_id="parent1", + resource_type="model", + depends_on=["grandparent"], + file_path="", + tags=[], + config={}, + ) + parent2_node = DbtNode( + name="parent2", + unique_id="parent2", + resource_type="model", + depends_on=["grandparent"], + file_path="", + tags=[], + config={}, + ) + child_node = DbtNode( + name="child", + unique_id="child", + resource_type="model", + depends_on=["parent1", "parent2"], + file_path="", + tags=[], + config={}, + ) + + nodes_list = [grandparent_node, parent1_node, parent2_node, child_node] + nodes = {node.unique_id: node for node in nodes_list} + + leaves = calculate_leaves(nodes.keys(), nodes) + assert leaves == ["child"] + + +def test_create_task_metadata_unsupported(caplog): + child_node = DbtNode( + name="unsupported", + unique_id="unsupported", + resource_type="unsupported", + depends_on=[], + file_path="", + tags=[], + config={}, + ) + response = create_task_metadata(child_node, execution_mode="", args={}) + assert response is None + expected_msg = "Unsupported resource type unsupported (node unsupported)." + assert caplog.messages[0] == expected_msg + + +def test_create_task_metadata_model(caplog): + child_node = DbtNode( + name="my_model", + unique_id="my_folder.my_model", + resource_type="model", + depends_on=[], + file_path="", + tags=[], + config={}, + ) + metadata = create_task_metadata(child_node, execution_mode="local", args={}) + assert metadata.id == "my_model_run" + assert metadata.operator_class == "cosmos.operators.local.DbtRunLocalOperator" + assert metadata.arguments == {"models": "my_model"} + + +def test_create_task_metadata_seed(caplog): + sample_node = DbtNode( + name="my_seed", + unique_id="my_folder.my_seed", + resource_type="seed", + depends_on=[], + file_path="", + tags=[], + config={}, + ) + metadata = create_task_metadata(sample_node, execution_mode="docker", args={}) + assert metadata.id == "my_seed_seed" + assert metadata.operator_class == "cosmos.operators.docker.DbtSeedDockerOperator" + assert metadata.arguments == {"models": "my_seed"} + + +def test_create_task_metadata_snapshot(caplog): + sample_node = DbtNode( + name="my_snapshot", + unique_id="my_folder.my_snapshot", + resource_type="snapshot", + depends_on=[], + file_path="", + tags=[], + config={}, + ) + metadata = create_task_metadata(sample_node, execution_mode="kubernetes", args={}) + assert metadata.id == "my_snapshot_snapshot" + assert metadata.operator_class == "cosmos.operators.kubernetes.DbtSnapshotKubernetesOperator" + assert metadata.arguments == {"models": "my_snapshot"} + + +def test_create_test_task_metadata(): + metadata = create_test_task_metadata( + test_task_name="test_no_nulls", + execution_mode="local", + task_args={"task_arg": "value"}, + on_warning_callback=True, + model_name="my_model", + ) + assert metadata.id == "test_no_nulls" + assert metadata.operator_class == "cosmos.operators.local.DbtTestLocalOperator" + assert metadata.arguments == {"task_arg": "value", "on_warning_callback": True, "models": "my_model"} diff --git a/tests/dbt/test_graph.py b/tests/dbt/test_graph.py new file mode 100644 index 000000000..f5f1c6244 --- /dev/null +++ b/tests/dbt/test_graph.py @@ -0,0 +1,176 @@ +from pathlib import Path +from unittest.mock import patch + +import pytest + +from cosmos.dbt.graph import DbtGraph, LoadMode, CosmosLoadDbtException +from cosmos.dbt.project import DbtProject + +DBT_PROJECTS_ROOT_DIR = Path(__file__).parent.parent.parent / "dev/dags/dbt" +SAMPLE_MANIFEST = Path(__file__).parent.parent / "sample/manifest.json" + + +def test_load_via_manifest_with_exclude(): + dbt_project = DbtProject(name="jaffle_shop", root_dir=DBT_PROJECTS_ROOT_DIR, manifest_path=SAMPLE_MANIFEST) + dbt_graph = DbtGraph(project=dbt_project, exclude=["config.materialized:table"]) + dbt_graph.load_from_dbt_manifest() + + assert len(dbt_graph.nodes) == 28 + assert len(dbt_graph.filtered_nodes) == 26 + assert "model.jaffle_shop.orders" not in dbt_graph.filtered_nodes + + sample_node = dbt_graph.nodes["model.jaffle_shop.customers"] + assert sample_node.name == "customers" + assert sample_node.unique_id == "model.jaffle_shop.customers" + assert sample_node.resource_type == "model" + assert sample_node.depends_on == [ + "model.jaffle_shop.stg_customers", + "model.jaffle_shop.stg_orders", + "model.jaffle_shop.stg_payments", + ] + assert sample_node.file_path == DBT_PROJECTS_ROOT_DIR / "jaffle_shop/models/customers.sql" + + +@patch("cosmos.dbt.graph.DbtGraph.load_from_dbt_manifest", return_value=None) +def test_load_automatic_manifest_is_available(mock_load_from_dbt_manifest): + dbt_project = DbtProject(name="jaffle_shop", root_dir=DBT_PROJECTS_ROOT_DIR, manifest_path=SAMPLE_MANIFEST) + dbt_graph = DbtGraph(project=dbt_project) + dbt_graph.load(execution_mode="local") + assert mock_load_from_dbt_manifest.called + + +@patch("cosmos.dbt.graph.DbtGraph.load_via_custom_parser", side_effect=FileNotFoundError()) +@patch("cosmos.dbt.graph.DbtGraph.load_via_dbt_ls", return_value=None) +def test_load_automatic_without_manifest(mock_load_via_dbt_ls, mock_load_via_custom_parser): + dbt_project = DbtProject(name="jaffle_shop", root_dir=DBT_PROJECTS_ROOT_DIR, manifest_path="/tmp/manifest.json") + dbt_graph = DbtGraph(project=dbt_project) + dbt_graph.load(execution_mode="local") + assert mock_load_via_dbt_ls.called + assert not mock_load_via_custom_parser.called + + +@patch("cosmos.dbt.graph.DbtGraph.load_via_custom_parser", return_value=None) +@patch("cosmos.dbt.graph.DbtGraph.load_via_dbt_ls", side_effect=FileNotFoundError()) +def test_load_automatic_without_manifest_and_without_dbt_cmd(mock_load_via_dbt_ls, mock_load_via_custom_parser): + dbt_project = DbtProject(name="jaffle_shop", root_dir=DBT_PROJECTS_ROOT_DIR) + dbt_graph = DbtGraph(project=dbt_project) + dbt_graph.load(execution_mode="local", method=LoadMode.AUTOMATIC) + assert mock_load_via_dbt_ls.called + assert mock_load_via_custom_parser.called + + +def test_load_manifest_without_manifest(): + dbt_project = DbtProject(name="jaffle_shop", root_dir=DBT_PROJECTS_ROOT_DIR) + dbt_graph = DbtGraph(project=dbt_project) + with pytest.raises(CosmosLoadDbtException) as err_info: + dbt_graph.load(execution_mode="local", method=LoadMode.DBT_MANIFEST) + assert err_info.value.args[0] == "Unable to load manifest using None" + + +@patch("cosmos.dbt.graph.DbtGraph.load_from_dbt_manifest", return_value=None) +def test_load_manifest_with_manifest(mock_load_from_dbt_manifest): + dbt_project = DbtProject(name="jaffle_shop", root_dir=DBT_PROJECTS_ROOT_DIR, manifest_path=SAMPLE_MANIFEST) + dbt_graph = DbtGraph(project=dbt_project) + dbt_graph.load(execution_mode="local", method=LoadMode.DBT_MANIFEST) + assert mock_load_from_dbt_manifest.called + + +@pytest.mark.parametrize( + "exec_mode,method,expected_function", + [ + ("local", LoadMode.AUTOMATIC, "mock_load_via_dbt_ls"), + ("virtualenv", LoadMode.AUTOMATIC, "mock_load_via_dbt_ls"), + ("kubernetes", LoadMode.AUTOMATIC, "mock_load_via_custom_parser"), + ("docker", LoadMode.AUTOMATIC, "mock_load_via_custom_parser"), + ("local", LoadMode.DBT_LS, "mock_load_via_dbt_ls"), + ("local", LoadMode.CUSTOM, "mock_load_via_custom_parser"), + ], +) +@patch("cosmos.dbt.graph.DbtGraph.load_via_custom_parser", return_value=None) +@patch("cosmos.dbt.graph.DbtGraph.load_via_dbt_ls", return_value=None) +@patch("cosmos.dbt.graph.DbtGraph.load_from_dbt_manifest", return_value=None) +def test_load( + mock_load_from_dbt_manifest, mock_load_via_dbt_ls, mock_load_via_custom_parser, exec_mode, method, expected_function +): + dbt_project = DbtProject(name="jaffle_shop", root_dir=DBT_PROJECTS_ROOT_DIR) + dbt_graph = DbtGraph(project=dbt_project) + + dbt_graph.load(method=method, execution_mode=exec_mode) + load_function = locals()[expected_function] + assert load_function.called + + +@pytest.mark.integration +def test_load_via_dbt_ls_with_exclude(): + dbt_project = DbtProject(name="jaffle_shop", root_dir=DBT_PROJECTS_ROOT_DIR) + dbt_graph = DbtGraph(project=dbt_project, select=["*customers*"], exclude=["*orders*"]) + dbt_graph.load_via_dbt_ls() + assert dbt_graph.nodes == dbt_graph.filtered_nodes + assert len(dbt_graph.nodes) == 7 + expected_keys = [ + "model.jaffle_shop.customers", + "model.jaffle_shop.stg_customers", + "seed.jaffle_shop.raw_customers", + "test.jaffle_shop.not_null_customers_customer_id.5c9bf9911d", + "test.jaffle_shop.not_null_stg_customers_customer_id.e2cfb1f9aa", + "test.jaffle_shop.unique_customers_customer_id.c5af1ff4b1", + "test.jaffle_shop.unique_stg_customers_customer_id.c7614daada", + ] + assert list(dbt_graph.nodes.keys()) == expected_keys + + sample_node = dbt_graph.nodes["model.jaffle_shop.customers"] + assert sample_node.name == "customers" + assert sample_node.unique_id == "model.jaffle_shop.customers" + assert sample_node.resource_type == "model" + assert sample_node.depends_on == [ + "model.jaffle_shop.stg_customers", + "model.jaffle_shop.stg_orders", + "model.jaffle_shop.stg_payments", + ] + assert sample_node.file_path == DBT_PROJECTS_ROOT_DIR / "jaffle_shop/models/customers.sql" + + +@pytest.mark.integration +def test_load_via_dbt_ls_without_exclude(): + dbt_project = DbtProject(name="jaffle_shop", root_dir=DBT_PROJECTS_ROOT_DIR) + dbt_graph = DbtGraph(project=dbt_project) + dbt_graph.load_via_dbt_ls() + + assert dbt_graph.nodes == dbt_graph.filtered_nodes + assert len(dbt_graph.nodes) == 28 + + +def test_load_via_dbt_ls_with_invalid_dbt_path(): + dbt_project = DbtProject(name="jaffle_shop", root_dir=DBT_PROJECTS_ROOT_DIR) + dbt_graph = DbtGraph(dbt_cmd="/inexistent/dbt", project=dbt_project) + with pytest.raises(CosmosLoadDbtException) as err_info: + dbt_graph.load_via_dbt_ls() + expected = "Unable to run the command due to the error:\n[Errno 2] No such file or directory: '/inexistent/dbt'" + assert err_info.value.args[0].startswith(expected) + + +@pytest.mark.integration +@patch("cosmos.dbt.graph.Popen.communicate", return_value=("Some Runtime Error", "")) +def test_load_via_dbt_ls_with_runtime_error_in_stdout(mock_popen_communicate): + # It may seem strange, but at least until dbt 1.6.0, there are circumstances when it outputs errors to stdout + dbt_project = DbtProject(name="jaffle_shop", root_dir=DBT_PROJECTS_ROOT_DIR) + dbt_graph = DbtGraph(project=dbt_project) + with pytest.raises(CosmosLoadDbtException) as err_info: + dbt_graph.load_via_dbt_ls() + expected = "Unable to run the command due to the error:\nSome Runtime Error" + assert err_info.value.args[0] == expected + mock_popen_communicate.assert_called_once() + + +def test_load_via_load_via_custom_parser(): + dbt_project = DbtProject( + name="jaffle_shop", + root_dir=DBT_PROJECTS_ROOT_DIR, + ) + dbt_graph = DbtGraph(project=dbt_project) + + dbt_graph.load_via_custom_parser() + + assert dbt_graph.nodes == dbt_graph.filtered_nodes + # the custom parser does not add dbt test nodes + assert len(dbt_graph.nodes) == 8 diff --git a/tests/dbt/test_selector.py b/tests/dbt/test_selector.py new file mode 100644 index 000000000..4056fc77d --- /dev/null +++ b/tests/dbt/test_selector.py @@ -0,0 +1,94 @@ +from pathlib import Path + +from cosmos.dbt.graph import DbtNode +from cosmos.dbt.selector import select_nodes + +SAMPLE_PROJ_PATH = Path("/home/user/path/dbt-proj/") + +grandparent_node = DbtNode( + name="grandparent", + unique_id="grandparent", + resource_type="model", + depends_on=[], + file_path=SAMPLE_PROJ_PATH / "gen1/models/grandparent.sql", + tags=["has_child"], + config={"materialized": "view"}, +) +parent_node = DbtNode( + name="parent", + unique_id="parent", + resource_type="model", + depends_on=["grandparent"], + file_path=SAMPLE_PROJ_PATH / "gen2/models/parent.sql", + tags=["has_child"], + config={"materialized": "view"}, +) +child_node = DbtNode( + name="child", + unique_id="child", + resource_type="model", + depends_on=["parent"], + file_path=SAMPLE_PROJ_PATH / "gen3/models/child.sql", + tags=["nightly"], + config={"materialized": "table"}, +) + +sample_nodes = { + grandparent_node.unique_id: grandparent_node, + parent_node.unique_id: parent_node, + child_node.unique_id: child_node, +} + + +def test_select_nodes_by_select_tag(): + selected = select_nodes(project_dir=SAMPLE_PROJ_PATH, nodes=sample_nodes, select=["tag:has_child"]) + expected = {grandparent_node.unique_id: grandparent_node, parent_node.unique_id: parent_node} + assert selected == expected + + +def test_select_nodes_by_select_config(): + selected = select_nodes(project_dir=SAMPLE_PROJ_PATH, nodes=sample_nodes, select=["config.materialized:table"]) + expected = {child_node.unique_id: child_node} + assert selected == expected + + +def test_select_nodes_by_select_path(): + selected = select_nodes(project_dir=SAMPLE_PROJ_PATH, nodes=sample_nodes, select=["path:gen2/models"]) + expected = { + parent_node.unique_id: parent_node, + } + assert selected == expected + + +def test_select_nodes_by_select_union(): + selected = select_nodes(project_dir=SAMPLE_PROJ_PATH, nodes=sample_nodes, select=["tag:has_child", "tag:nightly"]) + expected = { + grandparent_node.unique_id: grandparent_node, + parent_node.unique_id: parent_node, + child_node.unique_id: child_node, + } + assert selected == expected + + +def test_select_nodes_by_select_intersection(): + selected = select_nodes(project_dir=SAMPLE_PROJ_PATH, nodes=sample_nodes, select=["tag:has_child,tag:nightly"]) + assert selected == {} + + +def test_select_nodes_by_exclude_tag(): + selected = select_nodes(project_dir=SAMPLE_PROJ_PATH, nodes=sample_nodes, exclude=["tag:has_child"]) + expected = {child_node.unique_id: child_node} + assert selected == expected + + +def test_select_nodes_by_exclude_unsupported_selector(caplog): + select_nodes(project_dir=SAMPLE_PROJ_PATH, nodes=sample_nodes, exclude=["unsupported:filter"]) + assert "Unsupported select statement: unsupported:filter" in caplog.messages + + +def test_select_nodes_by_select_union_exclude_tags(): + selected = select_nodes( + project_dir=SAMPLE_PROJ_PATH, nodes=sample_nodes, select=["config.materialized:view"], exclude=["tag:has_child"] + ) + expected = {} + assert selected == expected diff --git a/tests/sample/manifest.json b/tests/sample/manifest.json new file mode 100644 index 000000000..4150234b8 --- /dev/null +++ b/tests/sample/manifest.json @@ -0,0 +1,10142 @@ +{ + "child_map": { + "model.jaffle_shop.customers": [ + "test.jaffle_shop.not_null_customers_customer_id.5c9bf9911d", + "test.jaffle_shop.relationships_orders_customer_id__customer_id__ref_customers_.c6ec7f58f2", + "test.jaffle_shop.unique_customers_customer_id.c5af1ff4b1" + ], + "model.jaffle_shop.orders": [ + "test.jaffle_shop.accepted_values_orders_status__placed__shipped__completed__return_pending__returned.be6b5b5ec3", + "test.jaffle_shop.not_null_orders_amount.106140f9fd", + "test.jaffle_shop.not_null_orders_bank_transfer_amount.7743500c49", + "test.jaffle_shop.not_null_orders_coupon_amount.ab90c90625", + "test.jaffle_shop.not_null_orders_credit_card_amount.d3ca593b59", + "test.jaffle_shop.not_null_orders_customer_id.c5f02694af", + "test.jaffle_shop.not_null_orders_gift_card_amount.413a0d2d7a", + "test.jaffle_shop.not_null_orders_order_id.cf6c17daed", + "test.jaffle_shop.relationships_orders_customer_id__customer_id__ref_customers_.c6ec7f58f2", + "test.jaffle_shop.unique_orders_order_id.fed79b3a6e" + ], + "model.jaffle_shop.stg_customers": [ + "model.jaffle_shop.customers", + "test.jaffle_shop.not_null_stg_customers_customer_id.e2cfb1f9aa", + "test.jaffle_shop.unique_stg_customers_customer_id.c7614daada" + ], + "model.jaffle_shop.stg_orders": [ + "model.jaffle_shop.customers", + "model.jaffle_shop.orders", + "test.jaffle_shop.accepted_values_stg_orders_status__placed__shipped__completed__return_pending__returned.080fb20aad", + "test.jaffle_shop.not_null_stg_orders_order_id.81cfe2fe64", + "test.jaffle_shop.unique_stg_orders_order_id.e3b841c71a" + ], + "model.jaffle_shop.stg_payments": [ + "model.jaffle_shop.customers", + "model.jaffle_shop.orders", + "test.jaffle_shop.accepted_values_stg_payments_payment_method__credit_card__coupon__bank_transfer__gift_card.3c3820f278", + "test.jaffle_shop.not_null_stg_payments_payment_id.c19cc50075", + "test.jaffle_shop.unique_stg_payments_payment_id.3744510712" + ], + "seed.jaffle_shop.raw_customers": [ + "model.jaffle_shop.stg_customers" + ], + "seed.jaffle_shop.raw_orders": [ + "model.jaffle_shop.stg_orders" + ], + "seed.jaffle_shop.raw_payments": [ + "model.jaffle_shop.stg_payments" + ], + "test.jaffle_shop.accepted_values_orders_status__placed__shipped__completed__return_pending__returned.be6b5b5ec3": [], + "test.jaffle_shop.accepted_values_stg_orders_status__placed__shipped__completed__return_pending__returned.080fb20aad": [], + "test.jaffle_shop.accepted_values_stg_payments_payment_method__credit_card__coupon__bank_transfer__gift_card.3c3820f278": [], + "test.jaffle_shop.not_null_customers_customer_id.5c9bf9911d": [], + "test.jaffle_shop.not_null_orders_amount.106140f9fd": [], + "test.jaffle_shop.not_null_orders_bank_transfer_amount.7743500c49": [], + "test.jaffle_shop.not_null_orders_coupon_amount.ab90c90625": [], + "test.jaffle_shop.not_null_orders_credit_card_amount.d3ca593b59": [], + "test.jaffle_shop.not_null_orders_customer_id.c5f02694af": [], + "test.jaffle_shop.not_null_orders_gift_card_amount.413a0d2d7a": [], + "test.jaffle_shop.not_null_orders_order_id.cf6c17daed": [], + "test.jaffle_shop.not_null_stg_customers_customer_id.e2cfb1f9aa": [], + "test.jaffle_shop.not_null_stg_orders_order_id.81cfe2fe64": [], + "test.jaffle_shop.not_null_stg_payments_payment_id.c19cc50075": [], + "test.jaffle_shop.relationships_orders_customer_id__customer_id__ref_customers_.c6ec7f58f2": [], + "test.jaffle_shop.unique_customers_customer_id.c5af1ff4b1": [], + "test.jaffle_shop.unique_orders_order_id.fed79b3a6e": [], + "test.jaffle_shop.unique_stg_customers_customer_id.c7614daada": [], + "test.jaffle_shop.unique_stg_orders_order_id.e3b841c71a": [], + "test.jaffle_shop.unique_stg_payments_payment_id.3744510712": [] + }, + "disabled": {}, + "docs": { + "doc.dbt.__overview__": { + "block_contents": "### Welcome!\n\nWelcome to the auto-generated documentation for your dbt project!\n\n### Navigation\n\nYou can use the `Project` and `Database` navigation tabs on the left side of the window to explore the models\nin your project.\n\n#### Project Tab\nThe `Project` tab mirrors the directory structure of your dbt project. In this tab, you can see all of the\nmodels defined in your dbt project, as well as models imported from dbt packages.\n\n#### Database Tab\nThe `Database` tab also exposes your models, but in a format that looks more like a database explorer. This view\nshows relations (tables and views) grouped into database schemas. Note that ephemeral models are _not_ shown\nin this interface, as they do not exist in the database.\n\n### Graph Exploration\nYou can click the blue icon on the bottom-right corner of the page to view the lineage graph of your models.\n\nOn model pages, you'll see the immediate parents and children of the model you're exploring. By clicking the `Expand`\nbutton at the top-right of this lineage pane, you'll be able to see all of the models that are used to build,\nor are built from, the model you're exploring.\n\nOnce expanded, you'll be able to use the `--select` and `--exclude` model selection syntax to filter the\nmodels in the graph. For more information on model selection, check out the [dbt docs](https://docs.getdbt.com/docs/model-selection-syntax).\n\nNote that you can also right-click on models to interactively filter and explore the graph.\n\n---\n\n### More information\n\n- [What is dbt](https://docs.getdbt.com/docs/introduction)?\n- Read the [dbt viewpoint](https://docs.getdbt.com/docs/viewpoint)\n- [Installation](https://docs.getdbt.com/docs/installation)\n- Join the [dbt Community](https://www.getdbt.com/community/) for questions and discussion", + "name": "__overview__", + "original_file_path": "docs/overview.md", + "package_name": "dbt", + "path": "overview.md", + "resource_type": "doc", + "unique_id": "doc.dbt.__overview__" + }, + "doc.jaffle_shop.__overview__": { + "block_contents": "## Data Documentation for Jaffle Shop\n\n`jaffle_shop` is a fictional ecommerce store.\n\nThis [dbt](https://www.getdbt.com/) project is for testing out code.\n\nThe source code can be found [here](https://github.com/clrcrl/jaffle_shop).", + "name": "__overview__", + "original_file_path": "models/overview.md", + "package_name": "jaffle_shop", + "path": "overview.md", + "resource_type": "doc", + "unique_id": "doc.jaffle_shop.__overview__" + }, + "doc.jaffle_shop.orders_status": { + "block_contents": "Orders can be one of the following statuses:\n\n| status | description |\n|----------------|------------------------------------------------------------------------------------------------------------------------|\n| placed | The order has been placed but has not yet left the warehouse |\n| shipped | The order has ben shipped to the customer and is currently in transit |\n| completed | The order has been received by the customer |\n| return_pending | The customer has indicated that they would like to return the order, but it has not yet been received at the warehouse |\n| returned | The order has been returned by the customer and received at the warehouse |", + "name": "orders_status", + "original_file_path": "models/docs.md", + "package_name": "jaffle_shop", + "path": "docs.md", + "resource_type": "doc", + "unique_id": "doc.jaffle_shop.orders_status" + } + }, + "exposures": {}, + "group_map": {}, + "groups": {}, + "macros": { + "macro.dbt._split_part_negative": { + "arguments": [], + "created_at": 1687942823.00502, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro _split_part_negative(string_text, delimiter_text, part_number) %}\n\n split_part(\n {{ string_text }},\n {{ delimiter_text }},\n length({{ string_text }})\n - length(\n replace({{ string_text }}, {{ delimiter_text }}, '')\n ) + 2 {{ part_number }}\n )\n\n{% endmacro %}", + "meta": {}, + "name": "_split_part_negative", + "original_file_path": "macros/utils/split_part.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/split_part.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt._split_part_negative" + }, + "macro.dbt.after_commit": { + "arguments": [], + "created_at": 1687942822.871392, + "depends_on": { + "macros": [ + "macro.dbt.make_hook_config" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro after_commit(sql) %}\n {{ make_hook_config(sql, inside_transaction=False) }}\n{% endmacro %}", + "meta": {}, + "name": "after_commit", + "original_file_path": "macros/materializations/hooks.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/hooks.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.after_commit" + }, + "macro.dbt.alter_column_comment": { + "arguments": [], + "created_at": 1687942823.028165, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres__alter_column_comment" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro alter_column_comment(relation, column_dict) -%}\n {{ return(adapter.dispatch('alter_column_comment', 'dbt')(relation, column_dict)) }}\n{% endmacro %}", + "meta": {}, + "name": "alter_column_comment", + "original_file_path": "macros/adapters/persist_docs.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/persist_docs.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.alter_column_comment" + }, + "macro.dbt.alter_column_type": { + "arguments": [], + "created_at": 1687942823.039871, + "depends_on": { + "macros": [ + "macro.dbt.default__alter_column_type" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro alter_column_type(relation, column_name, new_column_type) -%}\n {{ return(adapter.dispatch('alter_column_type', 'dbt')(relation, column_name, new_column_type)) }}\n{% endmacro %}", + "meta": {}, + "name": "alter_column_type", + "original_file_path": "macros/adapters/columns.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/columns.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.alter_column_type" + }, + "macro.dbt.alter_relation_add_remove_columns": { + "arguments": [], + "created_at": 1687942823.040888, + "depends_on": { + "macros": [ + "macro.dbt.default__alter_relation_add_remove_columns" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro alter_relation_add_remove_columns(relation, add_columns = none, remove_columns = none) -%}\n {{ return(adapter.dispatch('alter_relation_add_remove_columns', 'dbt')(relation, add_columns, remove_columns)) }}\n{% endmacro %}", + "meta": {}, + "name": "alter_relation_add_remove_columns", + "original_file_path": "macros/adapters/columns.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/columns.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.alter_relation_add_remove_columns" + }, + "macro.dbt.alter_relation_comment": { + "arguments": [], + "created_at": 1687942823.0285828, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres__alter_relation_comment" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro alter_relation_comment(relation, relation_comment) -%}\n {{ return(adapter.dispatch('alter_relation_comment', 'dbt')(relation, relation_comment)) }}\n{% endmacro %}", + "meta": {}, + "name": "alter_relation_comment", + "original_file_path": "macros/adapters/persist_docs.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/persist_docs.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.alter_relation_comment" + }, + "macro.dbt.any_value": { + "arguments": [], + "created_at": 1687942822.997621, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres__any_value" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro any_value(expression) -%}\n {{ return(adapter.dispatch('any_value', 'dbt') (expression)) }}\n{% endmacro %}", + "meta": {}, + "name": "any_value", + "original_file_path": "macros/utils/any_value.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/any_value.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.any_value" + }, + "macro.dbt.apply_grants": { + "arguments": [], + "created_at": 1687942823.025873, + "depends_on": { + "macros": [ + "macro.dbt.default__apply_grants" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro apply_grants(relation, grant_config, should_revoke) %}\n {{ return(adapter.dispatch(\"apply_grants\", \"dbt\")(relation, grant_config, should_revoke)) }}\n{% endmacro %}", + "meta": {}, + "name": "apply_grants", + "original_file_path": "macros/adapters/apply_grants.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/apply_grants.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.apply_grants" + }, + "macro.dbt.array_append": { + "arguments": [], + "created_at": 1687942823.0065181, + "depends_on": { + "macros": [ + "macro.dbt.default__array_append" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro array_append(array, new_element) -%}\n {{ return(adapter.dispatch('array_append', 'dbt')(array, new_element)) }}\n{%- endmacro %}", + "meta": {}, + "name": "array_append", + "original_file_path": "macros/utils/array_append.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/array_append.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.array_append" + }, + "macro.dbt.array_concat": { + "arguments": [], + "created_at": 1687942823.002666, + "depends_on": { + "macros": [ + "macro.dbt.default__array_concat" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro array_concat(array_1, array_2) -%}\n {{ return(adapter.dispatch('array_concat', 'dbt')(array_1, array_2)) }}\n{%- endmacro %}", + "meta": {}, + "name": "array_concat", + "original_file_path": "macros/utils/array_concat.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/array_concat.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.array_concat" + }, + "macro.dbt.array_construct": { + "arguments": [], + "created_at": 1687942823.00587, + "depends_on": { + "macros": [ + "macro.dbt.default__array_construct" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro array_construct(inputs=[], data_type=api.Column.translate_type('integer')) -%}\n {{ return(adapter.dispatch('array_construct', 'dbt')(inputs, data_type)) }}\n{%- endmacro %}", + "meta": {}, + "name": "array_construct", + "original_file_path": "macros/utils/array_construct.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/array_construct.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.array_construct" + }, + "macro.dbt.assert_columns_equivalent": { + "arguments": [], + "created_at": 1687942822.948245, + "depends_on": { + "macros": [ + "macro.dbt.get_column_schema_from_query", + "macro.dbt.get_empty_schema_sql", + "macro.dbt.format_columns" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro assert_columns_equivalent(sql) %}\n {#-- Obtain the column schema provided by sql file. #}\n {%- set sql_file_provided_columns = get_column_schema_from_query(sql, config.get('sql_header', none)) -%}\n {#--Obtain the column schema provided by the schema file by generating an 'empty schema' query from the model's columns. #}\n {%- set schema_file_provided_columns = get_column_schema_from_query(get_empty_schema_sql(model['columns'])) -%}\n\n {#-- create dictionaries with name and formatted data type and strings for exception #}\n {%- set sql_columns = format_columns(sql_file_provided_columns) -%}\n {%- set yaml_columns = format_columns(schema_file_provided_columns) -%}\n\n {%- if sql_columns|length != yaml_columns|length -%}\n {%- do exceptions.raise_contract_error(yaml_columns, sql_columns) -%}\n {%- endif -%}\n\n {%- for sql_col in sql_columns -%}\n {%- set yaml_col = [] -%}\n {%- for this_col in yaml_columns -%}\n {%- if this_col['name'] == sql_col['name'] -%}\n {%- do yaml_col.append(this_col) -%}\n {%- break -%}\n {%- endif -%}\n {%- endfor -%}\n {%- if not yaml_col -%}\n {#-- Column with name not found in yaml #}\n {%- do exceptions.raise_contract_error(yaml_columns, sql_columns) -%}\n {%- endif -%}\n {%- if sql_col['formatted'] != yaml_col[0]['formatted'] -%}\n {#-- Column data types don't match #}\n {%- do exceptions.raise_contract_error(yaml_columns, sql_columns) -%}\n {%- endif -%}\n {%- endfor -%}\n\n{% endmacro %}", + "meta": {}, + "name": "assert_columns_equivalent", + "original_file_path": "macros/materializations/models/table/columns_spec_ddl.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/table/columns_spec_ddl.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.assert_columns_equivalent" + }, + "macro.dbt.before_begin": { + "arguments": [], + "created_at": 1687942822.871079, + "depends_on": { + "macros": [ + "macro.dbt.make_hook_config" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro before_begin(sql) %}\n {{ make_hook_config(sql, inside_transaction=False) }}\n{% endmacro %}", + "meta": {}, + "name": "before_begin", + "original_file_path": "macros/materializations/hooks.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/hooks.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.before_begin" + }, + "macro.dbt.bool_or": { + "arguments": [], + "created_at": 1687942823.00308, + "depends_on": { + "macros": [ + "macro.dbt.default__bool_or" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro bool_or(expression) -%}\n {{ return(adapter.dispatch('bool_or', 'dbt') (expression)) }}\n{% endmacro %}", + "meta": {}, + "name": "bool_or", + "original_file_path": "macros/utils/bool_or.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/bool_or.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.bool_or" + }, + "macro.dbt.build_config_dict": { + "arguments": [], + "created_at": 1687942823.045846, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro build_config_dict(model) %}\n {%- set config_dict = {} -%}\n {% set config_dbt_used = zip(model.config.config_keys_used, model.config.config_keys_defaults) | list %}\n {%- for key, default in config_dbt_used -%}\n {# weird type testing with enum, would be much easier to write this logic in Python! #}\n {%- if key == \"language\" -%}\n {%- set value = \"python\" -%}\n {%- endif -%}\n {%- set value = model.config.get(key, default) -%}\n {%- do config_dict.update({key: value}) -%}\n {%- endfor -%}\nconfig_dict = {{ config_dict }}\n{% endmacro %}", + "meta": {}, + "name": "build_config_dict", + "original_file_path": "macros/python_model/python.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/python_model/python.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.build_config_dict" + }, + "macro.dbt.build_ref_function": { + "arguments": [], + "created_at": 1687942823.0447218, + "depends_on": { + "macros": [ + "macro.dbt.resolve_model_name" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro build_ref_function(model) %}\n\n {%- set ref_dict = {} -%}\n {%- for _ref in model.refs -%}\n {% set _ref_args = [_ref.get('package'), _ref['name']] if _ref.get('package') else [_ref['name'],] %}\n {%- set resolved = ref(*_ref_args, v=_ref.get('version')) -%}\n {%- if _ref.get('version') -%}\n {% do _ref_args.extend([\"v\" ~ _ref['version']]) %}\n {%- endif -%}\n {%- do ref_dict.update({_ref_args | join('.'): resolve_model_name(resolved)}) -%}\n {%- endfor -%}\n\ndef ref(*args, **kwargs):\n refs = {{ ref_dict | tojson }}\n key = '.'.join(args)\n version = kwargs.get(\"v\") or kwargs.get(\"version\")\n if version:\n key += f\".v{version}\"\n dbt_load_df_function = kwargs.get(\"dbt_load_df_function\")\n return dbt_load_df_function(refs[key])\n\n{% endmacro %}", + "meta": {}, + "name": "build_ref_function", + "original_file_path": "macros/python_model/python.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/python_model/python.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.build_ref_function" + }, + "macro.dbt.build_snapshot_staging_table": { + "arguments": [], + "created_at": 1687942822.892981, + "depends_on": { + "macros": [ + "macro.dbt.make_temp_relation", + "macro.dbt.snapshot_staging_table", + "macro.dbt.statement", + "macro.dbt.create_table_as" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro build_snapshot_staging_table(strategy, sql, target_relation) %}\n {% set temp_relation = make_temp_relation(target_relation) %}\n\n {% set select = snapshot_staging_table(strategy, sql, target_relation) %}\n\n {% call statement('build_snapshot_staging_relation') %}\n {{ create_table_as(True, temp_relation, select) }}\n {% endcall %}\n\n {% do return(temp_relation) %}\n{% endmacro %}", + "meta": {}, + "name": "build_snapshot_staging_table", + "original_file_path": "macros/materializations/snapshots/helpers.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/snapshots/helpers.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.build_snapshot_staging_table" + }, + "macro.dbt.build_snapshot_table": { + "arguments": [], + "created_at": 1687942822.892051, + "depends_on": { + "macros": [ + "macro.dbt.default__build_snapshot_table" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro build_snapshot_table(strategy, sql) -%}\n {{ adapter.dispatch('build_snapshot_table', 'dbt')(strategy, sql) }}\n{% endmacro %}", + "meta": {}, + "name": "build_snapshot_table", + "original_file_path": "macros/materializations/snapshots/helpers.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/snapshots/helpers.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.build_snapshot_table" + }, + "macro.dbt.build_source_function": { + "arguments": [], + "created_at": 1687942823.045177, + "depends_on": { + "macros": [ + "macro.dbt.resolve_model_name" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro build_source_function(model) %}\n\n {%- set source_dict = {} -%}\n {%- for _source in model.sources -%}\n {%- set resolved = source(*_source) -%}\n {%- do source_dict.update({_source | join('.'): resolve_model_name(resolved)}) -%}\n {%- endfor -%}\n\ndef source(*args, dbt_load_df_function):\n sources = {{ source_dict | tojson }}\n key = '.'.join(args)\n return dbt_load_df_function(sources[key])\n\n{% endmacro %}", + "meta": {}, + "name": "build_source_function", + "original_file_path": "macros/python_model/python.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/python_model/python.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.build_source_function" + }, + "macro.dbt.call_dcl_statements": { + "arguments": [], + "created_at": 1687942823.0253298, + "depends_on": { + "macros": [ + "macro.dbt.default__call_dcl_statements" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro call_dcl_statements(dcl_statement_list) %}\n {{ return(adapter.dispatch(\"call_dcl_statements\", \"dbt\")(dcl_statement_list)) }}\n{% endmacro %}", + "meta": {}, + "name": "call_dcl_statements", + "original_file_path": "macros/adapters/apply_grants.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/apply_grants.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.call_dcl_statements" + }, + "macro.dbt.cast_bool_to_text": { + "arguments": [], + "created_at": 1687942822.9970849, + "depends_on": { + "macros": [ + "macro.dbt.default__cast_bool_to_text" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro cast_bool_to_text(field) %}\n {{ adapter.dispatch('cast_bool_to_text', 'dbt') (field) }}\n{% endmacro %}", + "meta": {}, + "name": "cast_bool_to_text", + "original_file_path": "macros/utils/cast_bool_to_text.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/cast_bool_to_text.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.cast_bool_to_text" + }, + "macro.dbt.check_for_schema_changes": { + "arguments": [], + "created_at": 1687942822.9420822, + "depends_on": { + "macros": [ + "macro.dbt.diff_columns", + "macro.dbt.diff_column_data_types" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro check_for_schema_changes(source_relation, target_relation) %}\n\n {% set schema_changed = False %}\n\n {%- set source_columns = adapter.get_columns_in_relation(source_relation) -%}\n {%- set target_columns = adapter.get_columns_in_relation(target_relation) -%}\n {%- set source_not_in_target = diff_columns(source_columns, target_columns) -%}\n {%- set target_not_in_source = diff_columns(target_columns, source_columns) -%}\n\n {% set new_target_types = diff_column_data_types(source_columns, target_columns) %}\n\n {% if source_not_in_target != [] %}\n {% set schema_changed = True %}\n {% elif target_not_in_source != [] or new_target_types != [] %}\n {% set schema_changed = True %}\n {% elif new_target_types != [] %}\n {% set schema_changed = True %}\n {% endif %}\n\n {% set changes_dict = {\n 'schema_changed': schema_changed,\n 'source_not_in_target': source_not_in_target,\n 'target_not_in_source': target_not_in_source,\n 'source_columns': source_columns,\n 'target_columns': target_columns,\n 'new_target_types': new_target_types\n } %}\n\n {% set msg %}\n In {{ target_relation }}:\n Schema changed: {{ schema_changed }}\n Source columns not in target: {{ source_not_in_target }}\n Target columns not in source: {{ target_not_in_source }}\n New column types: {{ new_target_types }}\n {% endset %}\n\n {% do log(msg) %}\n\n {{ return(changes_dict) }}\n\n{% endmacro %}", + "meta": {}, + "name": "check_for_schema_changes", + "original_file_path": "macros/materializations/models/incremental/on_schema_change.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/incremental/on_schema_change.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.check_for_schema_changes" + }, + "macro.dbt.check_schema_exists": { + "arguments": [], + "created_at": 1687942823.032896, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres__check_schema_exists" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro check_schema_exists(information_schema, schema) -%}\n {{ return(adapter.dispatch('check_schema_exists', 'dbt')(information_schema, schema)) }}\n{% endmacro %}", + "meta": {}, + "name": "check_schema_exists", + "original_file_path": "macros/adapters/metadata.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/metadata.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.check_schema_exists" + }, + "macro.dbt.collect_freshness": { + "arguments": [], + "created_at": 1687942823.019376, + "depends_on": { + "macros": [ + "macro.dbt.default__collect_freshness" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro collect_freshness(source, loaded_at_field, filter) %}\n {{ return(adapter.dispatch('collect_freshness', 'dbt')(source, loaded_at_field, filter))}}\n{% endmacro %}", + "meta": {}, + "name": "collect_freshness", + "original_file_path": "macros/adapters/freshness.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/freshness.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.collect_freshness" + }, + "macro.dbt.concat": { + "arguments": [], + "created_at": 1687942822.991219, + "depends_on": { + "macros": [ + "macro.dbt.default__concat" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro concat(fields) -%}\n {{ return(adapter.dispatch('concat', 'dbt')(fields)) }}\n{%- endmacro %}", + "meta": {}, + "name": "concat", + "original_file_path": "macros/utils/concat.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/concat.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.concat" + }, + "macro.dbt.convert_datetime": { + "arguments": [], + "created_at": 1687942822.9874282, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro convert_datetime(date_str, date_fmt) %}\n\n {% set error_msg -%}\n The provided partition date '{{ date_str }}' does not match the expected format '{{ date_fmt }}'\n {%- endset %}\n\n {% set res = try_or_compiler_error(error_msg, modules.datetime.datetime.strptime, date_str.strip(), date_fmt) %}\n {{ return(res) }}\n\n{% endmacro %}", + "meta": {}, + "name": "convert_datetime", + "original_file_path": "macros/etc/datetime.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/etc/datetime.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.convert_datetime" + }, + "macro.dbt.copy_grants": { + "arguments": [], + "created_at": 1687942823.0217059, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres__copy_grants" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro copy_grants() %}\n {{ return(adapter.dispatch('copy_grants', 'dbt')()) }}\n{% endmacro %}", + "meta": {}, + "name": "copy_grants", + "original_file_path": "macros/adapters/apply_grants.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/apply_grants.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.copy_grants" + }, + "macro.dbt.create_columns": { + "arguments": [], + "created_at": 1687942822.88876, + "depends_on": { + "macros": [ + "macro.dbt.default__create_columns" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro create_columns(relation, columns) %}\n {{ adapter.dispatch('create_columns', 'dbt')(relation, columns) }}\n{% endmacro %}", + "meta": {}, + "name": "create_columns", + "original_file_path": "macros/materializations/snapshots/helpers.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/snapshots/helpers.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.create_columns" + }, + "macro.dbt.create_csv_table": { + "arguments": [], + "created_at": 1687942822.9726899, + "depends_on": { + "macros": [ + "macro.dbt.default__create_csv_table" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro create_csv_table(model, agate_table) -%}\n {{ adapter.dispatch('create_csv_table', 'dbt')(model, agate_table) }}\n{%- endmacro %}", + "meta": {}, + "name": "create_csv_table", + "original_file_path": "macros/materializations/seeds/helpers.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/seeds/helpers.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.create_csv_table" + }, + "macro.dbt.create_indexes": { + "arguments": [], + "created_at": 1687942823.010463, + "depends_on": { + "macros": [ + "macro.dbt.default__create_indexes" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro create_indexes(relation) -%}\n {{ adapter.dispatch('create_indexes', 'dbt')(relation) }}\n{%- endmacro %}", + "meta": {}, + "name": "create_indexes", + "original_file_path": "macros/adapters/indexes.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/indexes.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.create_indexes" + }, + "macro.dbt.create_or_replace_view": { + "arguments": [], + "created_at": 1687942822.961272, + "depends_on": { + "macros": [ + "macro.dbt.run_hooks", + "macro.dbt.handle_existing_table", + "macro.dbt.should_full_refresh", + "macro.dbt.statement", + "macro.dbt.get_create_view_as_sql", + "macro.dbt.should_revoke", + "macro.dbt.apply_grants" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro create_or_replace_view() %}\n {%- set identifier = model['alias'] -%}\n\n {%- set old_relation = adapter.get_relation(database=database, schema=schema, identifier=identifier) -%}\n {%- set exists_as_view = (old_relation is not none and old_relation.is_view) -%}\n\n {%- set target_relation = api.Relation.create(\n identifier=identifier, schema=schema, database=database,\n type='view') -%}\n {% set grant_config = config.get('grants') %}\n\n {{ run_hooks(pre_hooks) }}\n\n -- If there's a table with the same name and we weren't told to full refresh,\n -- that's an error. If we were told to full refresh, drop it. This behavior differs\n -- for Snowflake and BigQuery, so multiple dispatch is used.\n {%- if old_relation is not none and old_relation.is_table -%}\n {{ handle_existing_table(should_full_refresh(), old_relation) }}\n {%- endif -%}\n\n -- build model\n {% call statement('main') -%}\n {{ get_create_view_as_sql(target_relation, sql) }}\n {%- endcall %}\n\n {% set should_revoke = should_revoke(exists_as_view, full_refresh_mode=True) %}\n {% do apply_grants(target_relation, grant_config, should_revoke=should_revoke) %}\n\n {{ run_hooks(post_hooks) }}\n\n {{ return({'relations': [target_relation]}) }}\n\n{% endmacro %}", + "meta": {}, + "name": "create_or_replace_view", + "original_file_path": "macros/materializations/models/view/create_or_replace_view.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/view/create_or_replace_view.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.create_or_replace_view" + }, + "macro.dbt.create_schema": { + "arguments": [], + "created_at": 1687942823.007079, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres__create_schema" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro create_schema(relation) -%}\n {{ adapter.dispatch('create_schema', 'dbt')(relation) }}\n{% endmacro %}", + "meta": {}, + "name": "create_schema", + "original_file_path": "macros/adapters/schema.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/schema.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.create_schema" + }, + "macro.dbt.create_table_as": { + "arguments": [], + "created_at": 1687942822.954165, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres__create_table_as" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro create_table_as(temporary, relation, compiled_code, language='sql') -%}\n {# backward compatibility for create_table_as that does not support language #}\n {% if language == \"sql\" %}\n {{ adapter.dispatch('create_table_as', 'dbt')(temporary, relation, compiled_code)}}\n {% else %}\n {{ adapter.dispatch('create_table_as', 'dbt')(temporary, relation, compiled_code, language) }}\n {% endif %}\n\n{%- endmacro %}", + "meta": {}, + "name": "create_table_as", + "original_file_path": "macros/materializations/models/table/create_table_as.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/table/create_table_as.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.create_table_as" + }, + "macro.dbt.create_view_as": { + "arguments": [], + "created_at": 1687942822.962175, + "depends_on": { + "macros": [ + "macro.dbt.default__create_view_as" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro create_view_as(relation, sql) -%}\n {{ adapter.dispatch('create_view_as', 'dbt')(relation, sql) }}\n{%- endmacro %}", + "meta": {}, + "name": "create_view_as", + "original_file_path": "macros/materializations/models/view/create_view_as.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/view/create_view_as.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.create_view_as" + }, + "macro.dbt.current_timestamp": { + "arguments": [], + "created_at": 1687942823.008286, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres__current_timestamp" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{%- macro current_timestamp() -%}\n {{ adapter.dispatch('current_timestamp', 'dbt')() }}\n{%- endmacro -%}\n\n", + "meta": {}, + "name": "current_timestamp", + "original_file_path": "macros/adapters/timestamps.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/timestamps.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.current_timestamp" + }, + "macro.dbt.current_timestamp_backcompat": { + "arguments": [], + "created_at": 1687942823.008908, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres__current_timestamp_backcompat" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro current_timestamp_backcompat() %}\n {{ return(adapter.dispatch('current_timestamp_backcompat', 'dbt')()) }}\n{% endmacro %}", + "meta": {}, + "name": "current_timestamp_backcompat", + "original_file_path": "macros/adapters/timestamps.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/timestamps.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.current_timestamp_backcompat" + }, + "macro.dbt.current_timestamp_in_utc_backcompat": { + "arguments": [], + "created_at": 1687942823.009182, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres__current_timestamp_in_utc_backcompat" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro current_timestamp_in_utc_backcompat() %}\n {{ return(adapter.dispatch('current_timestamp_in_utc_backcompat', 'dbt')()) }}\n{% endmacro %}", + "meta": {}, + "name": "current_timestamp_in_utc_backcompat", + "original_file_path": "macros/adapters/timestamps.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/timestamps.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.current_timestamp_in_utc_backcompat" + }, + "macro.dbt.date_trunc": { + "arguments": [], + "created_at": 1687942823.005321, + "depends_on": { + "macros": [ + "macro.dbt.default__date_trunc" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro date_trunc(datepart, date) -%}\n {{ return(adapter.dispatch('date_trunc', 'dbt') (datepart, date)) }}\n{%- endmacro %}", + "meta": {}, + "name": "date_trunc", + "original_file_path": "macros/utils/date_trunc.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/date_trunc.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.date_trunc" + }, + "macro.dbt.dateadd": { + "arguments": [], + "created_at": 1687942822.992304, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres__dateadd" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro dateadd(datepart, interval, from_date_or_timestamp) %}\n {{ return(adapter.dispatch('dateadd', 'dbt')(datepart, interval, from_date_or_timestamp)) }}\n{% endmacro %}", + "meta": {}, + "name": "dateadd", + "original_file_path": "macros/utils/dateadd.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/dateadd.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.dateadd" + }, + "macro.dbt.datediff": { + "arguments": [], + "created_at": 1687942822.995577, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres__datediff" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro datediff(first_date, second_date, datepart) %}\n {{ return(adapter.dispatch('datediff', 'dbt')(first_date, second_date, datepart)) }}\n{% endmacro %}", + "meta": {}, + "name": "datediff", + "original_file_path": "macros/utils/datediff.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/datediff.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.datediff" + }, + "macro.dbt.dates_in_range": { + "arguments": [], + "created_at": 1687942822.9888482, + "depends_on": { + "macros": [ + "macro.dbt.convert_datetime" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro dates_in_range(start_date_str, end_date_str=none, in_fmt=\"%Y%m%d\", out_fmt=\"%Y%m%d\") %}\n {% set end_date_str = start_date_str if end_date_str is none else end_date_str %}\n\n {% set start_date = convert_datetime(start_date_str, in_fmt) %}\n {% set end_date = convert_datetime(end_date_str, in_fmt) %}\n\n {% set day_count = (end_date - start_date).days %}\n {% if day_count < 0 %}\n {% set msg -%}\n Partition start date is after the end date ({{ start_date }}, {{ end_date }})\n {%- endset %}\n\n {{ exceptions.raise_compiler_error(msg, model) }}\n {% endif %}\n\n {% set date_list = [] %}\n {% for i in range(0, day_count + 1) %}\n {% set the_date = (modules.datetime.timedelta(days=i) + start_date) %}\n {% if not out_fmt %}\n {% set _ = date_list.append(the_date) %}\n {% else %}\n {% set _ = date_list.append(the_date.strftime(out_fmt)) %}\n {% endif %}\n {% endfor %}\n\n {{ return(date_list) }}\n{% endmacro %}", + "meta": {}, + "name": "dates_in_range", + "original_file_path": "macros/etc/datetime.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/etc/datetime.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.dates_in_range" + }, + "macro.dbt.default__alter_column_comment": { + "arguments": [], + "created_at": 1687942823.028358, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__alter_column_comment(relation, column_dict) -%}\n {{ exceptions.raise_not_implemented(\n 'alter_column_comment macro not implemented for adapter '+adapter.type()) }}\n{% endmacro %}", + "meta": {}, + "name": "default__alter_column_comment", + "original_file_path": "macros/adapters/persist_docs.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/persist_docs.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__alter_column_comment" + }, + "macro.dbt.default__alter_column_type": { + "arguments": [], + "created_at": 1687942823.040595, + "depends_on": { + "macros": [ + "macro.dbt.statement" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__alter_column_type(relation, column_name, new_column_type) -%}\n {#\n 1. Create a new column (w/ temp name and correct type)\n 2. Copy data over to it\n 3. Drop the existing column (cascade!)\n 4. Rename the new column to existing column\n #}\n {%- set tmp_column = column_name + \"__dbt_alter\" -%}\n\n {% call statement('alter_column_type') %}\n alter table {{ relation }} add column {{ adapter.quote(tmp_column) }} {{ new_column_type }};\n update {{ relation }} set {{ adapter.quote(tmp_column) }} = {{ adapter.quote(column_name) }};\n alter table {{ relation }} drop column {{ adapter.quote(column_name) }} cascade;\n alter table {{ relation }} rename column {{ adapter.quote(tmp_column) }} to {{ adapter.quote(column_name) }}\n {% endcall %}\n\n{% endmacro %}", + "meta": {}, + "name": "default__alter_column_type", + "original_file_path": "macros/adapters/columns.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/columns.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__alter_column_type" + }, + "macro.dbt.default__alter_relation_add_remove_columns": { + "arguments": [], + "created_at": 1687942823.041811, + "depends_on": { + "macros": [ + "macro.dbt.run_query" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__alter_relation_add_remove_columns(relation, add_columns, remove_columns) %}\n\n {% if add_columns is none %}\n {% set add_columns = [] %}\n {% endif %}\n {% if remove_columns is none %}\n {% set remove_columns = [] %}\n {% endif %}\n\n {% set sql -%}\n\n alter {{ relation.type }} {{ relation }}\n\n {% for column in add_columns %}\n add column {{ column.name }} {{ column.data_type }}{{ ',' if not loop.last }}\n {% endfor %}{{ ',' if add_columns and remove_columns }}\n\n {% for column in remove_columns %}\n drop column {{ column.name }}{{ ',' if not loop.last }}\n {% endfor %}\n\n {%- endset -%}\n\n {% do run_query(sql) %}\n\n{% endmacro %}", + "meta": {}, + "name": "default__alter_relation_add_remove_columns", + "original_file_path": "macros/adapters/columns.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/columns.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__alter_relation_add_remove_columns" + }, + "macro.dbt.default__alter_relation_comment": { + "arguments": [], + "created_at": 1687942823.02877, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__alter_relation_comment(relation, relation_comment) -%}\n {{ exceptions.raise_not_implemented(\n 'alter_relation_comment macro not implemented for adapter '+adapter.type()) }}\n{% endmacro %}", + "meta": {}, + "name": "default__alter_relation_comment", + "original_file_path": "macros/adapters/persist_docs.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/persist_docs.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__alter_relation_comment" + }, + "macro.dbt.default__any_value": { + "arguments": [], + "created_at": 1687942822.997757, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__any_value(expression) -%}\n\n any_value({{ expression }})\n\n{%- endmacro %}", + "meta": {}, + "name": "default__any_value", + "original_file_path": "macros/utils/any_value.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/any_value.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__any_value" + }, + "macro.dbt.default__apply_grants": { + "arguments": [], + "created_at": 1687942823.027256, + "depends_on": { + "macros": [ + "macro.dbt.run_query", + "macro.dbt.get_show_grant_sql", + "macro.dbt.get_dcl_statement_list", + "macro.dbt.call_dcl_statements" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__apply_grants(relation, grant_config, should_revoke=True) %}\n {#-- If grant_config is {} or None, this is a no-op --#}\n {% if grant_config %}\n {% if should_revoke %}\n {#-- We think previous grants may have carried over --#}\n {#-- Show current grants and calculate diffs --#}\n {% set current_grants_table = run_query(get_show_grant_sql(relation)) %}\n {% set current_grants_dict = adapter.standardize_grants_dict(current_grants_table) %}\n {% set needs_granting = diff_of_two_dicts(grant_config, current_grants_dict) %}\n {% set needs_revoking = diff_of_two_dicts(current_grants_dict, grant_config) %}\n {% if not (needs_granting or needs_revoking) %}\n {{ log('On ' ~ relation ~': All grants are in place, no revocation or granting needed.')}}\n {% endif %}\n {% else %}\n {#-- We don't think there's any chance of previous grants having carried over. --#}\n {#-- Jump straight to granting what the user has configured. --#}\n {% set needs_revoking = {} %}\n {% set needs_granting = grant_config %}\n {% endif %}\n {% if needs_granting or needs_revoking %}\n {% set revoke_statement_list = get_dcl_statement_list(relation, needs_revoking, get_revoke_sql) %}\n {% set grant_statement_list = get_dcl_statement_list(relation, needs_granting, get_grant_sql) %}\n {% set dcl_statement_list = revoke_statement_list + grant_statement_list %}\n {% if dcl_statement_list %}\n {{ call_dcl_statements(dcl_statement_list) }}\n {% endif %}\n {% endif %}\n {% endif %}\n{% endmacro %}", + "meta": {}, + "name": "default__apply_grants", + "original_file_path": "macros/adapters/apply_grants.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/apply_grants.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__apply_grants" + }, + "macro.dbt.default__array_append": { + "arguments": [], + "created_at": 1687942823.006666, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__array_append(array, new_element) -%}\n array_append({{ array }}, {{ new_element }})\n{%- endmacro %}", + "meta": {}, + "name": "default__array_append", + "original_file_path": "macros/utils/array_append.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/array_append.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__array_append" + }, + "macro.dbt.default__array_concat": { + "arguments": [], + "created_at": 1687942823.002811, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__array_concat(array_1, array_2) -%}\n array_cat({{ array_1 }}, {{ array_2 }})\n{%- endmacro %}", + "meta": {}, + "name": "default__array_concat", + "original_file_path": "macros/utils/array_concat.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/array_concat.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__array_concat" + }, + "macro.dbt.default__array_construct": { + "arguments": [], + "created_at": 1687942823.006149, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__array_construct(inputs, data_type) -%}\n {% if inputs|length > 0 %}\n array[ {{ inputs|join(' , ') }} ]\n {% else %}\n array[]::{{data_type}}[]\n {% endif %}\n{%- endmacro %}", + "meta": {}, + "name": "default__array_construct", + "original_file_path": "macros/utils/array_construct.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/array_construct.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__array_construct" + }, + "macro.dbt.default__bool_or": { + "arguments": [], + "created_at": 1687942823.003194, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__bool_or(expression) -%}\n\n bool_or({{ expression }})\n\n{%- endmacro %}", + "meta": {}, + "name": "default__bool_or", + "original_file_path": "macros/utils/bool_or.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/bool_or.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__bool_or" + }, + "macro.dbt.default__build_snapshot_table": { + "arguments": [], + "created_at": 1687942822.8924742, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__build_snapshot_table(strategy, sql) %}\n\n select *,\n {{ strategy.scd_id }} as dbt_scd_id,\n {{ strategy.updated_at }} as dbt_updated_at,\n {{ strategy.updated_at }} as dbt_valid_from,\n nullif({{ strategy.updated_at }}, {{ strategy.updated_at }}) as dbt_valid_to\n from (\n {{ sql }}\n ) sbq\n\n{% endmacro %}", + "meta": {}, + "name": "default__build_snapshot_table", + "original_file_path": "macros/materializations/snapshots/helpers.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/snapshots/helpers.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__build_snapshot_table" + }, + "macro.dbt.default__call_dcl_statements": { + "arguments": [], + "created_at": 1687942823.02561, + "depends_on": { + "macros": [ + "macro.dbt.statement" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__call_dcl_statements(dcl_statement_list) %}\n {#\n -- By default, supply all grant + revoke statements in a single semicolon-separated block,\n -- so that they're all processed together.\n\n -- Some databases do not support this. Those adapters will need to override this macro\n -- to run each statement individually.\n #}\n {% call statement('grants') %}\n {% for dcl_statement in dcl_statement_list %}\n {{ dcl_statement }};\n {% endfor %}\n {% endcall %}\n{% endmacro %}", + "meta": {}, + "name": "default__call_dcl_statements", + "original_file_path": "macros/adapters/apply_grants.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/apply_grants.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__call_dcl_statements" + }, + "macro.dbt.default__cast_bool_to_text": { + "arguments": [], + "created_at": 1687942822.9972851, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__cast_bool_to_text(field) %}\n cast({{ field }} as {{ api.Column.translate_type('string') }})\n{% endmacro %}", + "meta": {}, + "name": "default__cast_bool_to_text", + "original_file_path": "macros/utils/cast_bool_to_text.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/cast_bool_to_text.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__cast_bool_to_text" + }, + "macro.dbt.default__check_schema_exists": { + "arguments": [], + "created_at": 1687942823.033236, + "depends_on": { + "macros": [ + "macro.dbt.replace", + "macro.dbt.run_query" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__check_schema_exists(information_schema, schema) -%}\n {% set sql -%}\n select count(*)\n from {{ information_schema.replace(information_schema_view='SCHEMATA') }}\n where catalog_name='{{ information_schema.database }}'\n and schema_name='{{ schema }}'\n {%- endset %}\n {{ return(run_query(sql)) }}\n{% endmacro %}", + "meta": {}, + "name": "default__check_schema_exists", + "original_file_path": "macros/adapters/metadata.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/metadata.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__check_schema_exists" + }, + "macro.dbt.default__collect_freshness": { + "arguments": [], + "created_at": 1687942823.019836, + "depends_on": { + "macros": [ + "macro.dbt.statement", + "macro.dbt.current_timestamp" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__collect_freshness(source, loaded_at_field, filter) %}\n {% call statement('collect_freshness', fetch_result=True, auto_begin=False) -%}\n select\n max({{ loaded_at_field }}) as max_loaded_at,\n {{ current_timestamp() }} as snapshotted_at\n from {{ source }}\n {% if filter %}\n where {{ filter }}\n {% endif %}\n {% endcall %}\n {{ return(load_result('collect_freshness')) }}\n{% endmacro %}", + "meta": {}, + "name": "default__collect_freshness", + "original_file_path": "macros/adapters/freshness.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/freshness.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__collect_freshness" + }, + "macro.dbt.default__concat": { + "arguments": [], + "created_at": 1687942822.9913611, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__concat(fields) -%}\n {{ fields|join(' || ') }}\n{%- endmacro %}", + "meta": {}, + "name": "default__concat", + "original_file_path": "macros/utils/concat.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/concat.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__concat" + }, + "macro.dbt.default__copy_grants": { + "arguments": [], + "created_at": 1687942823.021837, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__copy_grants() %}\n {{ return(True) }}\n{% endmacro %}", + "meta": {}, + "name": "default__copy_grants", + "original_file_path": "macros/adapters/apply_grants.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/apply_grants.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__copy_grants" + }, + "macro.dbt.default__create_columns": { + "arguments": [], + "created_at": 1687942822.8890939, + "depends_on": { + "macros": [ + "macro.dbt.statement" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__create_columns(relation, columns) %}\n {% for column in columns %}\n {% call statement() %}\n alter table {{ relation }} add column \"{{ column.name }}\" {{ column.data_type }};\n {% endcall %}\n {% endfor %}\n{% endmacro %}", + "meta": {}, + "name": "default__create_columns", + "original_file_path": "macros/materializations/snapshots/helpers.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/snapshots/helpers.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__create_columns" + }, + "macro.dbt.default__create_csv_table": { + "arguments": [], + "created_at": 1687942822.973732, + "depends_on": { + "macros": [ + "macro.dbt.statement" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__create_csv_table(model, agate_table) %}\n {%- set column_override = model['config'].get('column_types', {}) -%}\n {%- set quote_seed_column = model['config'].get('quote_columns', None) -%}\n\n {% set sql %}\n create table {{ this.render() }} (\n {%- for col_name in agate_table.column_names -%}\n {%- set inferred_type = adapter.convert_type(agate_table, loop.index0) -%}\n {%- set type = column_override.get(col_name, inferred_type) -%}\n {%- set column_name = (col_name | string) -%}\n {{ adapter.quote_seed_column(column_name, quote_seed_column) }} {{ type }} {%- if not loop.last -%}, {%- endif -%}\n {%- endfor -%}\n )\n {% endset %}\n\n {% call statement('_') -%}\n {{ sql }}\n {%- endcall %}\n\n {{ return(sql) }}\n{% endmacro %}", + "meta": {}, + "name": "default__create_csv_table", + "original_file_path": "macros/materializations/seeds/helpers.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/seeds/helpers.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__create_csv_table" + }, + "macro.dbt.default__create_indexes": { + "arguments": [], + "created_at": 1687942823.0109131, + "depends_on": { + "macros": [ + "macro.dbt.get_create_index_sql", + "macro.dbt.run_query" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__create_indexes(relation) -%}\n {%- set _indexes = config.get('indexes', default=[]) -%}\n\n {% for _index_dict in _indexes %}\n {% set create_index_sql = get_create_index_sql(relation, _index_dict) %}\n {% if create_index_sql %}\n {% do run_query(create_index_sql) %}\n {% endif %}\n {% endfor %}\n{% endmacro %}", + "meta": {}, + "name": "default__create_indexes", + "original_file_path": "macros/adapters/indexes.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/indexes.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__create_indexes" + }, + "macro.dbt.default__create_schema": { + "arguments": [], + "created_at": 1687942823.0072918, + "depends_on": { + "macros": [ + "macro.dbt.statement" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__create_schema(relation) -%}\n {%- call statement('create_schema') -%}\n create schema if not exists {{ relation.without_identifier() }}\n {% endcall %}\n{% endmacro %}", + "meta": {}, + "name": "default__create_schema", + "original_file_path": "macros/adapters/schema.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/schema.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__create_schema" + }, + "macro.dbt.default__create_table_as": { + "arguments": [], + "created_at": 1687942822.954907, + "depends_on": { + "macros": [ + "macro.dbt.get_assert_columns_equivalent", + "macro.dbt.get_table_columns_and_constraints", + "macro.dbt.get_select_subquery" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__create_table_as(temporary, relation, sql) -%}\n {%- set sql_header = config.get('sql_header', none) -%}\n\n {{ sql_header if sql_header is not none }}\n\n create {% if temporary: -%}temporary{%- endif %} table\n {{ relation.include(database=(not temporary), schema=(not temporary)) }}\n {% set contract_config = config.get('contract') %}\n {% if contract_config.enforced %}\n {{ get_assert_columns_equivalent(sql) }}\n {{ get_table_columns_and_constraints() }}\n {%- set sql = get_select_subquery(sql) %}\n {% endif %}\n as (\n {{ sql }}\n );\n{%- endmacro %}", + "meta": {}, + "name": "default__create_table_as", + "original_file_path": "macros/materializations/models/table/create_table_as.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/table/create_table_as.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__create_table_as" + }, + "macro.dbt.default__create_view_as": { + "arguments": [], + "created_at": 1687942822.962664, + "depends_on": { + "macros": [ + "macro.dbt.get_assert_columns_equivalent" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__create_view_as(relation, sql) -%}\n {%- set sql_header = config.get('sql_header', none) -%}\n\n {{ sql_header if sql_header is not none }}\n create view {{ relation }}\n {% set contract_config = config.get('contract') %}\n {% if contract_config.enforced %}\n {{ get_assert_columns_equivalent(sql) }}\n {%- endif %}\n as (\n {{ sql }}\n );\n{%- endmacro %}", + "meta": {}, + "name": "default__create_view_as", + "original_file_path": "macros/materializations/models/view/create_view_as.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/view/create_view_as.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__create_view_as" + }, + "macro.dbt.default__current_timestamp": { + "arguments": [], + "created_at": 1687942823.0084531, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__current_timestamp() -%}\n {{ exceptions.raise_not_implemented(\n 'current_timestamp macro not implemented for adapter ' + adapter.type()) }}\n{%- endmacro %}", + "meta": {}, + "name": "default__current_timestamp", + "original_file_path": "macros/adapters/timestamps.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/timestamps.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__current_timestamp" + }, + "macro.dbt.default__current_timestamp_backcompat": { + "arguments": [], + "created_at": 1687942823.008993, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__current_timestamp_backcompat() %}\n current_timestamp::timestamp\n{% endmacro %}", + "meta": {}, + "name": "default__current_timestamp_backcompat", + "original_file_path": "macros/adapters/timestamps.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/timestamps.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__current_timestamp_backcompat" + }, + "macro.dbt.default__current_timestamp_in_utc_backcompat": { + "arguments": [], + "created_at": 1687942823.0093691, + "depends_on": { + "macros": [ + "macro.dbt.current_timestamp_backcompat", + "macro.dbt_postgres.postgres__current_timestamp_backcompat" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__current_timestamp_in_utc_backcompat() %}\n {{ return(adapter.dispatch('current_timestamp_backcompat', 'dbt')()) }}\n{% endmacro %}", + "meta": {}, + "name": "default__current_timestamp_in_utc_backcompat", + "original_file_path": "macros/adapters/timestamps.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/timestamps.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__current_timestamp_in_utc_backcompat" + }, + "macro.dbt.default__date_trunc": { + "arguments": [], + "created_at": 1687942823.005462, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__date_trunc(datepart, date) -%}\n date_trunc('{{datepart}}', {{date}})\n{%- endmacro %}", + "meta": {}, + "name": "default__date_trunc", + "original_file_path": "macros/utils/date_trunc.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/date_trunc.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__date_trunc" + }, + "macro.dbt.default__dateadd": { + "arguments": [], + "created_at": 1687942822.992487, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__dateadd(datepart, interval, from_date_or_timestamp) %}\n\n dateadd(\n {{ datepart }},\n {{ interval }},\n {{ from_date_or_timestamp }}\n )\n\n{% endmacro %}", + "meta": {}, + "name": "default__dateadd", + "original_file_path": "macros/utils/dateadd.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/dateadd.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__dateadd" + }, + "macro.dbt.default__datediff": { + "arguments": [], + "created_at": 1687942822.9957619, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__datediff(first_date, second_date, datepart) -%}\n\n datediff(\n {{ datepart }},\n {{ first_date }},\n {{ second_date }}\n )\n\n{%- endmacro %}", + "meta": {}, + "name": "default__datediff", + "original_file_path": "macros/utils/datediff.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/datediff.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__datediff" + }, + "macro.dbt.default__drop_relation": { + "arguments": [], + "created_at": 1687942823.016405, + "depends_on": { + "macros": [ + "macro.dbt.statement" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__drop_relation(relation) -%}\n {% call statement('drop_relation', auto_begin=False) -%}\n drop {{ relation.type }} if exists {{ relation }} cascade\n {%- endcall %}\n{% endmacro %}", + "meta": {}, + "name": "default__drop_relation", + "original_file_path": "macros/adapters/relation.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/relation.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__drop_relation" + }, + "macro.dbt.default__drop_schema": { + "arguments": [], + "created_at": 1687942823.0077598, + "depends_on": { + "macros": [ + "macro.dbt.statement" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__drop_schema(relation) -%}\n {%- call statement('drop_schema') -%}\n drop schema if exists {{ relation.without_identifier() }} cascade\n {% endcall %}\n{% endmacro %}", + "meta": {}, + "name": "default__drop_schema", + "original_file_path": "macros/adapters/schema.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/schema.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__drop_schema" + }, + "macro.dbt.default__escape_single_quotes": { + "arguments": [], + "created_at": 1687942822.9933708, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__escape_single_quotes(expression) -%}\n{{ expression | replace(\"'\",\"''\") }}\n{%- endmacro %}", + "meta": {}, + "name": "default__escape_single_quotes", + "original_file_path": "macros/utils/escape_single_quotes.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/escape_single_quotes.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__escape_single_quotes" + }, + "macro.dbt.default__except": { + "arguments": [], + "created_at": 1687942822.990392, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__except() %}\n\n except\n\n{% endmacro %}", + "meta": {}, + "name": "default__except", + "original_file_path": "macros/utils/except.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/except.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__except" + }, + "macro.dbt.default__format_column": { + "arguments": [], + "created_at": 1687942822.949109, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__format_column(column) -%}\n {% set data_type = column.dtype %}\n {% set formatted = column.column.lower() ~ \" \" ~ data_type %}\n {{ return({'name': column.name, 'data_type': data_type, 'formatted': formatted}) }}\n{%- endmacro -%}", + "meta": {}, + "name": "default__format_column", + "original_file_path": "macros/materializations/models/table/columns_spec_ddl.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/table/columns_spec_ddl.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__format_column" + }, + "macro.dbt.default__generate_alias_name": { + "arguments": [], + "created_at": 1687942822.979058, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__generate_alias_name(custom_alias_name=none, node=none) -%}\n\n {%- if custom_alias_name -%}\n\n {{ custom_alias_name | trim }}\n\n {%- elif node.version -%}\n\n {{ return(node.name ~ \"_v\" ~ (node.version | replace(\".\", \"_\"))) }}\n\n {%- else -%}\n\n {{ node.name }}\n\n {%- endif -%}\n\n{%- endmacro %}", + "meta": {}, + "name": "default__generate_alias_name", + "original_file_path": "macros/get_custom_name/get_custom_alias.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/get_custom_name/get_custom_alias.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__generate_alias_name" + }, + "macro.dbt.default__generate_database_name": { + "arguments": [], + "created_at": 1687942822.981025, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__generate_database_name(custom_database_name=none, node=none) -%}\n {%- set default_database = target.database -%}\n {%- if custom_database_name is none -%}\n\n {{ default_database }}\n\n {%- else -%}\n\n {{ custom_database_name }}\n\n {%- endif -%}\n\n{%- endmacro %}", + "meta": {}, + "name": "default__generate_database_name", + "original_file_path": "macros/get_custom_name/get_custom_database.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/get_custom_name/get_custom_database.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__generate_database_name" + }, + "macro.dbt.default__generate_schema_name": { + "arguments": [], + "created_at": 1687942822.979957, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__generate_schema_name(custom_schema_name, node) -%}\n\n {%- set default_schema = target.schema -%}\n {%- if custom_schema_name is none -%}\n\n {{ default_schema }}\n\n {%- else -%}\n\n {{ default_schema }}_{{ custom_schema_name | trim }}\n\n {%- endif -%}\n\n{%- endmacro %}", + "meta": {}, + "name": "default__generate_schema_name", + "original_file_path": "macros/get_custom_name/get_custom_schema.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/get_custom_name/get_custom_schema.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__generate_schema_name" + }, + "macro.dbt.default__get_assert_columns_equivalent": { + "arguments": [], + "created_at": 1687942822.946909, + "depends_on": { + "macros": [ + "macro.dbt.assert_columns_equivalent" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__get_assert_columns_equivalent(sql) -%}\n {{ return(assert_columns_equivalent(sql)) }}\n{%- endmacro %}", + "meta": {}, + "name": "default__get_assert_columns_equivalent", + "original_file_path": "macros/materializations/models/table/columns_spec_ddl.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/table/columns_spec_ddl.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__get_assert_columns_equivalent" + }, + "macro.dbt.default__get_batch_size": { + "arguments": [], + "created_at": 1687942822.975518, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__get_batch_size() %}\n {{ return(10000) }}\n{% endmacro %}", + "meta": {}, + "name": "default__get_batch_size", + "original_file_path": "macros/materializations/seeds/helpers.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/seeds/helpers.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__get_batch_size" + }, + "macro.dbt.default__get_binding_char": { + "arguments": [], + "created_at": 1687942822.975202, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__get_binding_char() %}\n {{ return('%s') }}\n{% endmacro %}", + "meta": {}, + "name": "default__get_binding_char", + "original_file_path": "macros/materializations/seeds/helpers.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/seeds/helpers.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__get_binding_char" + }, + "macro.dbt.default__get_catalog": { + "arguments": [], + "created_at": 1687942823.031789, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__get_catalog(information_schema, schemas) -%}\n\n {% set typename = adapter.type() %}\n {% set msg -%}\n get_catalog not implemented for {{ typename }}\n {%- endset %}\n\n {{ exceptions.raise_compiler_error(msg) }}\n{% endmacro %}", + "meta": {}, + "name": "default__get_catalog", + "original_file_path": "macros/adapters/metadata.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/metadata.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__get_catalog" + }, + "macro.dbt.default__get_column_names": { + "arguments": [], + "created_at": 1687942822.95544, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__get_column_names() %}\n {#- loop through user_provided_columns to get column names -#}\n {%- set user_provided_columns = model['columns'] -%}\n {%- for i in user_provided_columns %}\n {%- set col = user_provided_columns[i] -%}\n {%- set col_name = adapter.quote(col['name']) if col.get('quote') else col['name'] -%}\n {{ col_name }}{{ \", \" if not loop.last }}\n {%- endfor -%}\n{% endmacro %}", + "meta": {}, + "name": "default__get_column_names", + "original_file_path": "macros/materializations/models/table/create_table_as.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/table/create_table_as.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__get_column_names" + }, + "macro.dbt.default__get_columns_in_query": { + "arguments": [], + "created_at": 1687942823.039603, + "depends_on": { + "macros": [ + "macro.dbt.statement", + "macro.dbt.get_empty_subquery_sql" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__get_columns_in_query(select_sql) %}\n {% call statement('get_columns_in_query', fetch_result=True, auto_begin=False) -%}\n {{ get_empty_subquery_sql(select_sql) }}\n {% endcall %}\n {{ return(load_result('get_columns_in_query').table.columns | map(attribute='name') | list) }}\n{% endmacro %}", + "meta": {}, + "name": "default__get_columns_in_query", + "original_file_path": "macros/adapters/columns.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/columns.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__get_columns_in_query" + }, + "macro.dbt.default__get_columns_in_relation": { + "arguments": [], + "created_at": 1687942823.036652, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__get_columns_in_relation(relation) -%}\n {{ exceptions.raise_not_implemented(\n 'get_columns_in_relation macro not implemented for adapter '+adapter.type()) }}\n{% endmacro %}", + "meta": {}, + "name": "default__get_columns_in_relation", + "original_file_path": "macros/adapters/columns.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/columns.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__get_columns_in_relation" + }, + "macro.dbt.default__get_create_index_sql": { + "arguments": [], + "created_at": 1687942823.010286, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__get_create_index_sql(relation, index_dict) -%}\n {% do return(None) %}\n{% endmacro %}", + "meta": {}, + "name": "default__get_create_index_sql", + "original_file_path": "macros/adapters/indexes.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/indexes.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__get_create_index_sql" + }, + "macro.dbt.default__get_create_table_as_sql": { + "arguments": [], + "created_at": 1687942822.953666, + "depends_on": { + "macros": [ + "macro.dbt.create_table_as" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__get_create_table_as_sql(temporary, relation, sql) -%}\n {{ return(create_table_as(temporary, relation, sql)) }}\n{% endmacro %}", + "meta": {}, + "name": "default__get_create_table_as_sql", + "original_file_path": "macros/materializations/models/table/create_table_as.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/table/create_table_as.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__get_create_table_as_sql" + }, + "macro.dbt.default__get_create_view_as_sql": { + "arguments": [], + "created_at": 1687942822.961925, + "depends_on": { + "macros": [ + "macro.dbt.create_view_as" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__get_create_view_as_sql(relation, sql) -%}\n {{ return(create_view_as(relation, sql)) }}\n{% endmacro %}", + "meta": {}, + "name": "default__get_create_view_as_sql", + "original_file_path": "macros/materializations/models/view/create_view_as.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/view/create_view_as.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__get_create_view_as_sql" + }, + "macro.dbt.default__get_csv_sql": { + "arguments": [], + "created_at": 1687942822.9749131, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__get_csv_sql(create_or_truncate_sql, insert_sql) %}\n {{ create_or_truncate_sql }};\n -- dbt seed --\n {{ insert_sql }}\n{% endmacro %}", + "meta": {}, + "name": "default__get_csv_sql", + "original_file_path": "macros/materializations/seeds/helpers.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/seeds/helpers.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__get_csv_sql" + }, + "macro.dbt.default__get_dcl_statement_list": { + "arguments": [], + "created_at": 1687942823.0251, + "depends_on": { + "macros": [ + "macro.dbt.support_multiple_grantees_per_dcl_statement" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "\n\n{%- macro default__get_dcl_statement_list(relation, grant_config, get_dcl_macro) -%}\n {#\n -- Unpack grant_config into specific privileges and the set of users who need them granted/revoked.\n -- Depending on whether this database supports multiple grantees per statement, pass in the list of\n -- all grantees per privilege, or (if not) template one statement per privilege-grantee pair.\n -- `get_dcl_macro` will be either `get_grant_sql` or `get_revoke_sql`\n #}\n {%- set dcl_statements = [] -%}\n {%- for privilege, grantees in grant_config.items() %}\n {%- if support_multiple_grantees_per_dcl_statement() and grantees -%}\n {%- set dcl = get_dcl_macro(relation, privilege, grantees) -%}\n {%- do dcl_statements.append(dcl) -%}\n {%- else -%}\n {%- for grantee in grantees -%}\n {% set dcl = get_dcl_macro(relation, privilege, [grantee]) %}\n {%- do dcl_statements.append(dcl) -%}\n {% endfor -%}\n {%- endif -%}\n {%- endfor -%}\n {{ return(dcl_statements) }}\n{%- endmacro %}", + "meta": {}, + "name": "default__get_dcl_statement_list", + "original_file_path": "macros/adapters/apply_grants.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/apply_grants.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__get_dcl_statement_list" + }, + "macro.dbt.default__get_delete_insert_merge_sql": { + "arguments": [], + "created_at": 1687942822.922552, + "depends_on": { + "macros": [ + "macro.dbt.get_quoted_csv" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__get_delete_insert_merge_sql(target, source, unique_key, dest_columns, incremental_predicates) -%}\n\n {%- set dest_cols_csv = get_quoted_csv(dest_columns | map(attribute=\"name\")) -%}\n\n {% if unique_key %}\n {% if unique_key is sequence and unique_key is not string %}\n delete from {{target }}\n using {{ source }}\n where (\n {% for key in unique_key %}\n {{ source }}.{{ key }} = {{ target }}.{{ key }}\n {{ \"and \" if not loop.last}}\n {% endfor %}\n {% if incremental_predicates %}\n {% for predicate in incremental_predicates %}\n and {{ predicate }}\n {% endfor %}\n {% endif %}\n );\n {% else %}\n delete from {{ target }}\n where (\n {{ unique_key }}) in (\n select ({{ unique_key }})\n from {{ source }}\n )\n {%- if incremental_predicates %}\n {% for predicate in incremental_predicates %}\n and {{ predicate }}\n {% endfor %}\n {%- endif -%};\n\n {% endif %}\n {% endif %}\n\n insert into {{ target }} ({{ dest_cols_csv }})\n (\n select {{ dest_cols_csv }}\n from {{ source }}\n )\n\n{%- endmacro %}", + "meta": {}, + "name": "default__get_delete_insert_merge_sql", + "original_file_path": "macros/materializations/models/incremental/merge.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/incremental/merge.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__get_delete_insert_merge_sql" + }, + "macro.dbt.default__get_empty_schema_sql": { + "arguments": [], + "created_at": 1687942823.038543, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__get_empty_schema_sql(columns) %}\n {%- set col_err = [] -%}\n select\n {% for i in columns %}\n {%- set col = columns[i] -%}\n {%- if col['data_type'] is not defined -%}\n {{ col_err.append(col['name']) }}\n {%- endif -%}\n {% set col_name = adapter.quote(col['name']) if col.get('quote') else col['name'] %}\n cast(null as {{ col['data_type'] }}) as {{ col_name }}{{ \", \" if not loop.last }}\n {%- endfor -%}\n {%- if (col_err | length) > 0 -%}\n {{ exceptions.column_type_missing(column_names=col_err) }}\n {%- endif -%}\n{% endmacro %}", + "meta": {}, + "name": "default__get_empty_schema_sql", + "original_file_path": "macros/adapters/columns.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/columns.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__get_empty_schema_sql" + }, + "macro.dbt.default__get_empty_subquery_sql": { + "arguments": [], + "created_at": 1687942823.037494, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__get_empty_subquery_sql(select_sql, select_sql_header=none) %}\n {%- if select_sql_header is not none -%}\n {{ select_sql_header }}\n {%- endif -%}\n select * from (\n {{ select_sql }}\n ) as __dbt_sbq\n where false\n limit 0\n{% endmacro %}", + "meta": {}, + "name": "default__get_empty_subquery_sql", + "original_file_path": "macros/adapters/columns.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/columns.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__get_empty_subquery_sql" + }, + "macro.dbt.default__get_grant_sql": { + "arguments": [], + "created_at": 1687942823.0234182, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "\n\n{%- macro default__get_grant_sql(relation, privilege, grantees) -%}\n grant {{ privilege }} on {{ relation }} to {{ grantees | join(', ') }}\n{%- endmacro -%}\n\n\n", + "meta": {}, + "name": "default__get_grant_sql", + "original_file_path": "macros/adapters/apply_grants.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/apply_grants.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__get_grant_sql" + }, + "macro.dbt.default__get_incremental_append_sql": { + "arguments": [], + "created_at": 1687942822.9257421, + "depends_on": { + "macros": [ + "macro.dbt.get_insert_into_sql" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__get_incremental_append_sql(arg_dict) %}\n\n {% do return(get_insert_into_sql(arg_dict[\"target_relation\"], arg_dict[\"temp_relation\"], arg_dict[\"dest_columns\"])) %}\n\n{% endmacro %}", + "meta": {}, + "name": "default__get_incremental_append_sql", + "original_file_path": "macros/materializations/models/incremental/strategies.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/incremental/strategies.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__get_incremental_append_sql" + }, + "macro.dbt.default__get_incremental_default_sql": { + "arguments": [], + "created_at": 1687942822.927765, + "depends_on": { + "macros": [ + "macro.dbt.get_incremental_append_sql" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__get_incremental_default_sql(arg_dict) %}\n\n {% do return(get_incremental_append_sql(arg_dict)) %}\n\n{% endmacro %}", + "meta": {}, + "name": "default__get_incremental_default_sql", + "original_file_path": "macros/materializations/models/incremental/strategies.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/incremental/strategies.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__get_incremental_default_sql" + }, + "macro.dbt.default__get_incremental_delete_insert_sql": { + "arguments": [], + "created_at": 1687942822.926282, + "depends_on": { + "macros": [ + "macro.dbt.get_delete_insert_merge_sql" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__get_incremental_delete_insert_sql(arg_dict) %}\n\n {% do return(get_delete_insert_merge_sql(arg_dict[\"target_relation\"], arg_dict[\"temp_relation\"], arg_dict[\"unique_key\"], arg_dict[\"dest_columns\"], arg_dict[\"incremental_predicates\"])) %}\n\n{% endmacro %}", + "meta": {}, + "name": "default__get_incremental_delete_insert_sql", + "original_file_path": "macros/materializations/models/incremental/strategies.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/incremental/strategies.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__get_incremental_delete_insert_sql" + }, + "macro.dbt.default__get_incremental_insert_overwrite_sql": { + "arguments": [], + "created_at": 1687942822.9273689, + "depends_on": { + "macros": [ + "macro.dbt.get_insert_overwrite_merge_sql" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__get_incremental_insert_overwrite_sql(arg_dict) %}\n\n {% do return(get_insert_overwrite_merge_sql(arg_dict[\"target_relation\"], arg_dict[\"temp_relation\"], arg_dict[\"dest_columns\"], arg_dict[\"incremental_predicates\"])) %}\n\n{% endmacro %}", + "meta": {}, + "name": "default__get_incremental_insert_overwrite_sql", + "original_file_path": "macros/materializations/models/incremental/strategies.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/incremental/strategies.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__get_incremental_insert_overwrite_sql" + }, + "macro.dbt.default__get_incremental_merge_sql": { + "arguments": [], + "created_at": 1687942822.926823, + "depends_on": { + "macros": [ + "macro.dbt.get_merge_sql" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__get_incremental_merge_sql(arg_dict) %}\n\n {% do return(get_merge_sql(arg_dict[\"target_relation\"], arg_dict[\"temp_relation\"], arg_dict[\"unique_key\"], arg_dict[\"dest_columns\"], arg_dict[\"incremental_predicates\"])) %}\n\n{% endmacro %}", + "meta": {}, + "name": "default__get_incremental_merge_sql", + "original_file_path": "macros/materializations/models/incremental/strategies.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/incremental/strategies.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__get_incremental_merge_sql" + }, + "macro.dbt.default__get_insert_overwrite_merge_sql": { + "arguments": [], + "created_at": 1687942822.923584, + "depends_on": { + "macros": [ + "macro.dbt.get_quoted_csv" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__get_insert_overwrite_merge_sql(target, source, dest_columns, predicates, include_sql_header) -%}\n {#-- The only time include_sql_header is True: --#}\n {#-- BigQuery + insert_overwrite strategy + \"static\" partitions config --#}\n {#-- We should consider including the sql header at the materialization level instead --#}\n\n {%- set predicates = [] if predicates is none else [] + predicates -%}\n {%- set dest_cols_csv = get_quoted_csv(dest_columns | map(attribute=\"name\")) -%}\n {%- set sql_header = config.get('sql_header', none) -%}\n\n {{ sql_header if sql_header is not none and include_sql_header }}\n\n merge into {{ target }} as DBT_INTERNAL_DEST\n using {{ source }} as DBT_INTERNAL_SOURCE\n on FALSE\n\n when not matched by source\n {% if predicates %} and {{ predicates | join(' and ') }} {% endif %}\n then delete\n\n when not matched then insert\n ({{ dest_cols_csv }})\n values\n ({{ dest_cols_csv }})\n\n{% endmacro %}", + "meta": {}, + "name": "default__get_insert_overwrite_merge_sql", + "original_file_path": "macros/materializations/models/incremental/merge.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/incremental/merge.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__get_insert_overwrite_merge_sql" + }, + "macro.dbt.default__get_merge_sql": { + "arguments": [], + "created_at": 1687942822.921111, + "depends_on": { + "macros": [ + "macro.dbt.get_quoted_csv", + "macro.dbt.get_merge_update_columns" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__get_merge_sql(target, source, unique_key, dest_columns, incremental_predicates=none) -%}\n {%- set predicates = [] if incremental_predicates is none else [] + incremental_predicates -%}\n {%- set dest_cols_csv = get_quoted_csv(dest_columns | map(attribute=\"name\")) -%}\n {%- set merge_update_columns = config.get('merge_update_columns') -%}\n {%- set merge_exclude_columns = config.get('merge_exclude_columns') -%}\n {%- set update_columns = get_merge_update_columns(merge_update_columns, merge_exclude_columns, dest_columns) -%}\n {%- set sql_header = config.get('sql_header', none) -%}\n\n {% if unique_key %}\n {% if unique_key is sequence and unique_key is not mapping and unique_key is not string %}\n {% for key in unique_key %}\n {% set this_key_match %}\n DBT_INTERNAL_SOURCE.{{ key }} = DBT_INTERNAL_DEST.{{ key }}\n {% endset %}\n {% do predicates.append(this_key_match) %}\n {% endfor %}\n {% else %}\n {% set unique_key_match %}\n DBT_INTERNAL_SOURCE.{{ unique_key }} = DBT_INTERNAL_DEST.{{ unique_key }}\n {% endset %}\n {% do predicates.append(unique_key_match) %}\n {% endif %}\n {% else %}\n {% do predicates.append('FALSE') %}\n {% endif %}\n\n {{ sql_header if sql_header is not none }}\n\n merge into {{ target }} as DBT_INTERNAL_DEST\n using {{ source }} as DBT_INTERNAL_SOURCE\n on {{\"(\" ~ predicates | join(\") and (\") ~ \")\"}}\n\n {% if unique_key %}\n when matched then update set\n {% for column_name in update_columns -%}\n {{ column_name }} = DBT_INTERNAL_SOURCE.{{ column_name }}\n {%- if not loop.last %}, {%- endif %}\n {%- endfor %}\n {% endif %}\n\n when not matched then insert\n ({{ dest_cols_csv }})\n values\n ({{ dest_cols_csv }})\n\n{% endmacro %}", + "meta": {}, + "name": "default__get_merge_sql", + "original_file_path": "macros/materializations/models/incremental/merge.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/incremental/merge.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__get_merge_sql" + }, + "macro.dbt.default__get_merge_update_columns": { + "arguments": [], + "created_at": 1687942822.911655, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__get_merge_update_columns(merge_update_columns, merge_exclude_columns, dest_columns) %}\n {%- set default_cols = dest_columns | map(attribute=\"quoted\") | list -%}\n\n {%- if merge_update_columns and merge_exclude_columns -%}\n {{ exceptions.raise_compiler_error(\n 'Model cannot specify merge_update_columns and merge_exclude_columns. Please update model to use only one config'\n )}}\n {%- elif merge_update_columns -%}\n {%- set update_columns = merge_update_columns -%}\n {%- elif merge_exclude_columns -%}\n {%- set update_columns = [] -%}\n {%- for column in dest_columns -%}\n {% if column.column | lower not in merge_exclude_columns | map(\"lower\") | list %}\n {%- do update_columns.append(column.quoted) -%}\n {% endif %}\n {%- endfor -%}\n {%- else -%}\n {%- set update_columns = default_cols -%}\n {%- endif -%}\n\n {{ return(update_columns) }}\n\n{% endmacro %}", + "meta": {}, + "name": "default__get_merge_update_columns", + "original_file_path": "macros/materializations/models/incremental/column_helpers.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/incremental/column_helpers.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__get_merge_update_columns" + }, + "macro.dbt.default__get_or_create_relation": { + "arguments": [], + "created_at": 1687942823.018283, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__get_or_create_relation(database, schema, identifier, type) %}\n {%- set target_relation = adapter.get_relation(database=database, schema=schema, identifier=identifier) %}\n\n {% if target_relation %}\n {% do return([true, target_relation]) %}\n {% endif %}\n\n {%- set new_relation = api.Relation.create(\n database=database,\n schema=schema,\n identifier=identifier,\n type=type\n ) -%}\n {% do return([false, new_relation]) %}\n{% endmacro %}", + "meta": {}, + "name": "default__get_or_create_relation", + "original_file_path": "macros/adapters/relation.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/relation.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__get_or_create_relation" + }, + "macro.dbt.default__get_revoke_sql": { + "arguments": [], + "created_at": 1687942823.023883, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "\n\n{%- macro default__get_revoke_sql(relation, privilege, grantees) -%}\n revoke {{ privilege }} on {{ relation }} from {{ grantees | join(', ') }}\n{%- endmacro -%}\n\n\n", + "meta": {}, + "name": "default__get_revoke_sql", + "original_file_path": "macros/adapters/apply_grants.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/apply_grants.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__get_revoke_sql" + }, + "macro.dbt.default__get_select_subquery": { + "arguments": [], + "created_at": 1687942822.955848, + "depends_on": { + "macros": [ + "macro.dbt.default__get_column_names" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__get_select_subquery(sql) %}\n select {{ adapter.dispatch('get_column_names', 'dbt')() }}\n from (\n {{ sql }}\n ) as model_subq\n{%- endmacro %}", + "meta": {}, + "name": "default__get_select_subquery", + "original_file_path": "macros/materializations/models/table/create_table_as.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/table/create_table_as.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__get_select_subquery" + }, + "macro.dbt.default__get_show_grant_sql": { + "arguments": [], + "created_at": 1687942823.02295, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__get_show_grant_sql(relation) %}\n show grants on {{ relation }}\n{% endmacro %}", + "meta": {}, + "name": "default__get_show_grant_sql", + "original_file_path": "macros/adapters/apply_grants.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/apply_grants.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__get_show_grant_sql" + }, + "macro.dbt.default__get_table_columns_and_constraints": { + "arguments": [], + "created_at": 1687942822.945963, + "depends_on": { + "macros": [ + "macro.dbt.table_columns_and_constraints" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__get_table_columns_and_constraints() -%}\n {{ return(table_columns_and_constraints()) }}\n{%- endmacro %}", + "meta": {}, + "name": "default__get_table_columns_and_constraints", + "original_file_path": "macros/materializations/models/table/columns_spec_ddl.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/table/columns_spec_ddl.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__get_table_columns_and_constraints" + }, + "macro.dbt.default__get_test_sql": { + "arguments": [], + "created_at": 1687942822.904812, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__get_test_sql(main_sql, fail_calc, warn_if, error_if, limit) -%}\n select\n {{ fail_calc }} as failures,\n {{ fail_calc }} {{ warn_if }} as should_warn,\n {{ fail_calc }} {{ error_if }} as should_error\n from (\n {{ main_sql }}\n {{ \"limit \" ~ limit if limit != none }}\n ) dbt_internal_test\n{%- endmacro %}", + "meta": {}, + "name": "default__get_test_sql", + "original_file_path": "macros/materializations/tests/helpers.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/tests/helpers.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__get_test_sql" + }, + "macro.dbt.default__get_true_sql": { + "arguments": [], + "created_at": 1687942822.890411, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__get_true_sql() %}\n {{ return('TRUE') }}\n{% endmacro %}", + "meta": {}, + "name": "default__get_true_sql", + "original_file_path": "macros/materializations/snapshots/helpers.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/snapshots/helpers.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__get_true_sql" + }, + "macro.dbt.default__get_where_subquery": { + "arguments": [], + "created_at": 1687942822.905828, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__get_where_subquery(relation) -%}\n {% set where = config.get('where', '') %}\n {% if where %}\n {%- set filtered -%}\n (select * from {{ relation }} where {{ where }}) dbt_subquery\n {%- endset -%}\n {% do return(filtered) %}\n {%- else -%}\n {% do return(relation) %}\n {%- endif -%}\n{%- endmacro %}", + "meta": {}, + "name": "default__get_where_subquery", + "original_file_path": "macros/materializations/tests/where_subquery.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/tests/where_subquery.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__get_where_subquery" + }, + "macro.dbt.default__handle_existing_table": { + "arguments": [], + "created_at": 1687942822.959599, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__handle_existing_table(full_refresh, old_relation) %}\n {{ log(\"Dropping relation \" ~ old_relation ~ \" because it is of type \" ~ old_relation.type) }}\n {{ adapter.drop_relation(old_relation) }}\n{% endmacro %}", + "meta": {}, + "name": "default__handle_existing_table", + "original_file_path": "macros/materializations/models/view/helpers.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/view/helpers.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__handle_existing_table" + }, + "macro.dbt.default__hash": { + "arguments": [], + "created_at": 1687942822.996808, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__hash(field) -%}\n md5(cast({{ field }} as {{ api.Column.translate_type('string') }}))\n{%- endmacro %}", + "meta": {}, + "name": "default__hash", + "original_file_path": "macros/utils/hash.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/hash.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__hash" + }, + "macro.dbt.default__information_schema_name": { + "arguments": [], + "created_at": 1687942823.032175, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__information_schema_name(database) -%}\n {%- if database -%}\n {{ database }}.INFORMATION_SCHEMA\n {%- else -%}\n INFORMATION_SCHEMA\n {%- endif -%}\n{%- endmacro %}", + "meta": {}, + "name": "default__information_schema_name", + "original_file_path": "macros/adapters/metadata.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/metadata.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__information_schema_name" + }, + "macro.dbt.default__intersect": { + "arguments": [], + "created_at": 1687942822.992857, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__intersect() %}\n\n intersect\n\n{% endmacro %}", + "meta": {}, + "name": "default__intersect", + "original_file_path": "macros/utils/intersect.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/intersect.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__intersect" + }, + "macro.dbt.default__last_day": { + "arguments": [], + "created_at": 1687942823.004024, + "depends_on": { + "macros": [ + "macro.dbt.default_last_day" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__last_day(date, datepart) -%}\n {{dbt.default_last_day(date, datepart)}}\n{%- endmacro %}", + "meta": {}, + "name": "default__last_day", + "original_file_path": "macros/utils/last_day.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/last_day.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__last_day" + }, + "macro.dbt.default__length": { + "arguments": [], + "created_at": 1687942822.9919202, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__length(expression) %}\n\n length(\n {{ expression }}\n )\n\n{%- endmacro -%}", + "meta": {}, + "name": "default__length", + "original_file_path": "macros/utils/length.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/length.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__length" + }, + "macro.dbt.default__list_relations_without_caching": { + "arguments": [], + "created_at": 1687942823.033619, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__list_relations_without_caching(schema_relation) %}\n {{ exceptions.raise_not_implemented(\n 'list_relations_without_caching macro not implemented for adapter '+adapter.type()) }}\n{% endmacro %}", + "meta": {}, + "name": "default__list_relations_without_caching", + "original_file_path": "macros/adapters/metadata.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/metadata.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__list_relations_without_caching" + }, + "macro.dbt.default__list_schemas": { + "arguments": [], + "created_at": 1687942823.032667, + "depends_on": { + "macros": [ + "macro.dbt.information_schema_name", + "macro.dbt.run_query" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__list_schemas(database) -%}\n {% set sql %}\n select distinct schema_name\n from {{ information_schema_name(database) }}.SCHEMATA\n where catalog_name ilike '{{ database }}'\n {% endset %}\n {{ return(run_query(sql)) }}\n{% endmacro %}", + "meta": {}, + "name": "default__list_schemas", + "original_file_path": "macros/adapters/metadata.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/metadata.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__list_schemas" + }, + "macro.dbt.default__listagg": { + "arguments": [], + "created_at": 1687942822.995195, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__listagg(measure, delimiter_text, order_by_clause, limit_num) -%}\n\n {% if limit_num -%}\n array_to_string(\n array_slice(\n array_agg(\n {{ measure }}\n ){% if order_by_clause -%}\n within group ({{ order_by_clause }})\n {%- endif %}\n ,0\n ,{{ limit_num }}\n ),\n {{ delimiter_text }}\n )\n {%- else %}\n listagg(\n {{ measure }},\n {{ delimiter_text }}\n )\n {% if order_by_clause -%}\n within group ({{ order_by_clause }})\n {%- endif %}\n {%- endif %}\n\n{%- endmacro %}", + "meta": {}, + "name": "default__listagg", + "original_file_path": "macros/utils/listagg.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/listagg.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__listagg" + }, + "macro.dbt.default__load_csv_rows": { + "arguments": [], + "created_at": 1687942822.978139, + "depends_on": { + "macros": [ + "macro.dbt.get_batch_size", + "macro.dbt.get_seed_column_quoted_csv", + "macro.dbt.get_binding_char" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__load_csv_rows(model, agate_table) %}\n\n {% set batch_size = get_batch_size() %}\n\n {% set cols_sql = get_seed_column_quoted_csv(model, agate_table.column_names) %}\n {% set bindings = [] %}\n\n {% set statements = [] %}\n\n {% for chunk in agate_table.rows | batch(batch_size) %}\n {% set bindings = [] %}\n\n {% for row in chunk %}\n {% do bindings.extend(row) %}\n {% endfor %}\n\n {% set sql %}\n insert into {{ this.render() }} ({{ cols_sql }}) values\n {% for row in chunk -%}\n ({%- for column in agate_table.column_names -%}\n {{ get_binding_char() }}\n {%- if not loop.last%},{%- endif %}\n {%- endfor -%})\n {%- if not loop.last%},{%- endif %}\n {%- endfor %}\n {% endset %}\n\n {% do adapter.add_query(sql, bindings=bindings, abridge_sql_log=True) %}\n\n {% if loop.index0 == 0 %}\n {% do statements.append(sql) %}\n {% endif %}\n {% endfor %}\n\n {# Return SQL so we can render it out into the compiled files #}\n {{ return(statements[0]) }}\n{% endmacro %}", + "meta": {}, + "name": "default__load_csv_rows", + "original_file_path": "macros/materializations/seeds/helpers.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/seeds/helpers.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__load_csv_rows" + }, + "macro.dbt.default__make_backup_relation": { + "arguments": [], + "created_at": 1687942823.01597, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__make_backup_relation(base_relation, backup_relation_type, suffix) %}\n {%- set backup_identifier = base_relation.identifier ~ suffix -%}\n {%- set backup_relation = base_relation.incorporate(\n path={\"identifier\": backup_identifier},\n type=backup_relation_type\n ) -%}\n {{ return(backup_relation) }}\n{% endmacro %}", + "meta": {}, + "name": "default__make_backup_relation", + "original_file_path": "macros/adapters/relation.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/relation.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__make_backup_relation" + }, + "macro.dbt.default__make_intermediate_relation": { + "arguments": [], + "created_at": 1687942823.014782, + "depends_on": { + "macros": [ + "macro.dbt.default__make_temp_relation" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__make_intermediate_relation(base_relation, suffix) %}\n {{ return(default__make_temp_relation(base_relation, suffix)) }}\n{% endmacro %}", + "meta": {}, + "name": "default__make_intermediate_relation", + "original_file_path": "macros/adapters/relation.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/relation.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__make_intermediate_relation" + }, + "macro.dbt.default__make_temp_relation": { + "arguments": [], + "created_at": 1687942823.015351, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__make_temp_relation(base_relation, suffix) %}\n {%- set temp_identifier = base_relation.identifier ~ suffix -%}\n {%- set temp_relation = base_relation.incorporate(\n path={\"identifier\": temp_identifier}) -%}\n\n {{ return(temp_relation) }}\n{% endmacro %}", + "meta": {}, + "name": "default__make_temp_relation", + "original_file_path": "macros/adapters/relation.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/relation.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__make_temp_relation" + }, + "macro.dbt.default__persist_docs": { + "arguments": [], + "created_at": 1687942823.029631, + "depends_on": { + "macros": [ + "macro.dbt.run_query", + "macro.dbt.alter_relation_comment", + "macro.dbt.alter_column_comment" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__persist_docs(relation, model, for_relation, for_columns) -%}\n {% if for_relation and config.persist_relation_docs() and model.description %}\n {% do run_query(alter_relation_comment(relation, model.description)) %}\n {% endif %}\n\n {% if for_columns and config.persist_column_docs() and model.columns %}\n {% do run_query(alter_column_comment(relation, model.columns)) %}\n {% endif %}\n{% endmacro %}", + "meta": {}, + "name": "default__persist_docs", + "original_file_path": "macros/adapters/persist_docs.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/persist_docs.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__persist_docs" + }, + "macro.dbt.default__position": { + "arguments": [], + "created_at": 1687942822.99832, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__position(substring_text, string_text) %}\n\n position(\n {{ substring_text }} in {{ string_text }}\n )\n\n{%- endmacro -%}", + "meta": {}, + "name": "default__position", + "original_file_path": "macros/utils/position.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/position.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__position" + }, + "macro.dbt.default__post_snapshot": { + "arguments": [], + "created_at": 1687942822.8900359, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__post_snapshot(staging_relation) %}\n {# no-op #}\n{% endmacro %}", + "meta": {}, + "name": "default__post_snapshot", + "original_file_path": "macros/materializations/snapshots/helpers.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/snapshots/helpers.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__post_snapshot" + }, + "macro.dbt.default__rename_relation": { + "arguments": [], + "created_at": 1687942823.0173938, + "depends_on": { + "macros": [ + "macro.dbt.statement" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__rename_relation(from_relation, to_relation) -%}\n {% set target_name = adapter.quote_as_configured(to_relation.identifier, 'identifier') %}\n {% call statement('rename_relation') -%}\n alter table {{ from_relation }} rename to {{ target_name }}\n {%- endcall %}\n{% endmacro %}", + "meta": {}, + "name": "default__rename_relation", + "original_file_path": "macros/adapters/relation.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/relation.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__rename_relation" + }, + "macro.dbt.default__replace": { + "arguments": [], + "created_at": 1687942822.990945, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__replace(field, old_chars, new_chars) %}\n\n replace(\n {{ field }},\n {{ old_chars }},\n {{ new_chars }}\n )\n\n\n{% endmacro %}", + "meta": {}, + "name": "default__replace", + "original_file_path": "macros/utils/replace.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/replace.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__replace" + }, + "macro.dbt.default__reset_csv_table": { + "arguments": [], + "created_at": 1687942822.9745421, + "depends_on": { + "macros": [ + "macro.dbt.create_csv_table" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__reset_csv_table(model, full_refresh, old_relation, agate_table) %}\n {% set sql = \"\" %}\n {% if full_refresh %}\n {{ adapter.drop_relation(old_relation) }}\n {% set sql = create_csv_table(model, agate_table) %}\n {% else %}\n {{ adapter.truncate_relation(old_relation) }}\n {% set sql = \"truncate table \" ~ old_relation %}\n {% endif %}\n\n {{ return(sql) }}\n{% endmacro %}", + "meta": {}, + "name": "default__reset_csv_table", + "original_file_path": "macros/materializations/seeds/helpers.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/seeds/helpers.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__reset_csv_table" + }, + "macro.dbt.default__resolve_model_name": { + "arguments": [], + "created_at": 1687942823.043811, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "\n\n{%- macro default__resolve_model_name(input_model_name) -%}\n {{ input_model_name | string | replace('\"', '\\\"') }}\n{%- endmacro -%}\n\n", + "meta": {}, + "name": "default__resolve_model_name", + "original_file_path": "macros/python_model/python.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/python_model/python.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__resolve_model_name" + }, + "macro.dbt.default__right": { + "arguments": [], + "created_at": 1687942822.993966, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__right(string_text, length_expression) %}\n\n right(\n {{ string_text }},\n {{ length_expression }}\n )\n\n{%- endmacro -%}", + "meta": {}, + "name": "default__right", + "original_file_path": "macros/utils/right.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/right.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__right" + }, + "macro.dbt.default__safe_cast": { + "arguments": [], + "created_at": 1687942822.996346, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__safe_cast(field, type) %}\n {# most databases don't support this function yet\n so we just need to use cast #}\n cast({{field}} as {{type}})\n{% endmacro %}", + "meta": {}, + "name": "default__safe_cast", + "original_file_path": "macros/utils/safe_cast.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/safe_cast.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__safe_cast" + }, + "macro.dbt.default__snapshot_get_time": { + "arguments": [], + "created_at": 1687942823.008725, + "depends_on": { + "macros": [ + "macro.dbt.current_timestamp" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__snapshot_get_time() %}\n {{ current_timestamp() }}\n{% endmacro %}", + "meta": {}, + "name": "default__snapshot_get_time", + "original_file_path": "macros/adapters/timestamps.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/timestamps.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__snapshot_get_time" + }, + "macro.dbt.default__snapshot_hash_arguments": { + "arguments": [], + "created_at": 1687942822.878295, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__snapshot_hash_arguments(args) -%}\n md5({%- for arg in args -%}\n coalesce(cast({{ arg }} as varchar ), '')\n {% if not loop.last %} || '|' || {% endif %}\n {%- endfor -%})\n{%- endmacro %}", + "meta": {}, + "name": "default__snapshot_hash_arguments", + "original_file_path": "macros/materializations/snapshots/strategies.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/snapshots/strategies.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__snapshot_hash_arguments" + }, + "macro.dbt.default__snapshot_merge_sql": { + "arguments": [], + "created_at": 1687942822.873353, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__snapshot_merge_sql(target, source, insert_cols) -%}\n {%- set insert_cols_csv = insert_cols | join(', ') -%}\n\n merge into {{ target }} as DBT_INTERNAL_DEST\n using {{ source }} as DBT_INTERNAL_SOURCE\n on DBT_INTERNAL_SOURCE.dbt_scd_id = DBT_INTERNAL_DEST.dbt_scd_id\n\n when matched\n and DBT_INTERNAL_DEST.dbt_valid_to is null\n and DBT_INTERNAL_SOURCE.dbt_change_type in ('update', 'delete')\n then update\n set dbt_valid_to = DBT_INTERNAL_SOURCE.dbt_valid_to\n\n when not matched\n and DBT_INTERNAL_SOURCE.dbt_change_type = 'insert'\n then insert ({{ insert_cols_csv }})\n values ({{ insert_cols_csv }})\n\n{% endmacro %}", + "meta": {}, + "name": "default__snapshot_merge_sql", + "original_file_path": "macros/materializations/snapshots/snapshot_merge.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/snapshots/snapshot_merge.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__snapshot_merge_sql" + }, + "macro.dbt.default__snapshot_staging_table": { + "arguments": [], + "created_at": 1687942822.891789, + "depends_on": { + "macros": [ + "macro.dbt.snapshot_get_time" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__snapshot_staging_table(strategy, source_sql, target_relation) -%}\n\n with snapshot_query as (\n\n {{ source_sql }}\n\n ),\n\n snapshotted_data as (\n\n select *,\n {{ strategy.unique_key }} as dbt_unique_key\n\n from {{ target_relation }}\n where dbt_valid_to is null\n\n ),\n\n insertions_source_data as (\n\n select\n *,\n {{ strategy.unique_key }} as dbt_unique_key,\n {{ strategy.updated_at }} as dbt_updated_at,\n {{ strategy.updated_at }} as dbt_valid_from,\n nullif({{ strategy.updated_at }}, {{ strategy.updated_at }}) as dbt_valid_to,\n {{ strategy.scd_id }} as dbt_scd_id\n\n from snapshot_query\n ),\n\n updates_source_data as (\n\n select\n *,\n {{ strategy.unique_key }} as dbt_unique_key,\n {{ strategy.updated_at }} as dbt_updated_at,\n {{ strategy.updated_at }} as dbt_valid_from,\n {{ strategy.updated_at }} as dbt_valid_to\n\n from snapshot_query\n ),\n\n {%- if strategy.invalidate_hard_deletes %}\n\n deletes_source_data as (\n\n select\n *,\n {{ strategy.unique_key }} as dbt_unique_key\n from snapshot_query\n ),\n {% endif %}\n\n insertions as (\n\n select\n 'insert' as dbt_change_type,\n source_data.*\n\n from insertions_source_data as source_data\n left outer join snapshotted_data on snapshotted_data.dbt_unique_key = source_data.dbt_unique_key\n where snapshotted_data.dbt_unique_key is null\n or (\n snapshotted_data.dbt_unique_key is not null\n and (\n {{ strategy.row_changed }}\n )\n )\n\n ),\n\n updates as (\n\n select\n 'update' as dbt_change_type,\n source_data.*,\n snapshotted_data.dbt_scd_id\n\n from updates_source_data as source_data\n join snapshotted_data on snapshotted_data.dbt_unique_key = source_data.dbt_unique_key\n where (\n {{ strategy.row_changed }}\n )\n )\n\n {%- if strategy.invalidate_hard_deletes -%}\n ,\n\n deletes as (\n\n select\n 'delete' as dbt_change_type,\n source_data.*,\n {{ snapshot_get_time() }} as dbt_valid_from,\n {{ snapshot_get_time() }} as dbt_updated_at,\n {{ snapshot_get_time() }} as dbt_valid_to,\n snapshotted_data.dbt_scd_id\n\n from snapshotted_data\n left join deletes_source_data as source_data on snapshotted_data.dbt_unique_key = source_data.dbt_unique_key\n where source_data.dbt_unique_key is null\n )\n {%- endif %}\n\n select * from insertions\n union all\n select * from updates\n {%- if strategy.invalidate_hard_deletes %}\n union all\n select * from deletes\n {%- endif %}\n\n{%- endmacro %}", + "meta": {}, + "name": "default__snapshot_staging_table", + "original_file_path": "macros/materializations/snapshots/helpers.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/snapshots/helpers.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__snapshot_staging_table" + }, + "macro.dbt.default__snapshot_string_as_time": { + "arguments": [], + "created_at": 1687942822.879877, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__snapshot_string_as_time(timestamp) %}\n {% do exceptions.raise_not_implemented(\n 'snapshot_string_as_time macro not implemented for adapter '+adapter.type()\n ) %}\n{% endmacro %}", + "meta": {}, + "name": "default__snapshot_string_as_time", + "original_file_path": "macros/materializations/snapshots/strategies.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/snapshots/strategies.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__snapshot_string_as_time" + }, + "macro.dbt.default__split_part": { + "arguments": [], + "created_at": 1687942823.004779, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__split_part(string_text, delimiter_text, part_number) %}\n\n split_part(\n {{ string_text }},\n {{ delimiter_text }},\n {{ part_number }}\n )\n\n{% endmacro %}", + "meta": {}, + "name": "default__split_part", + "original_file_path": "macros/utils/split_part.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/split_part.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__split_part" + }, + "macro.dbt.default__string_literal": { + "arguments": [], + "created_at": 1687942822.998917, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__string_literal(value) -%}\n '{{ value }}'\n{%- endmacro %}", + "meta": {}, + "name": "default__string_literal", + "original_file_path": "macros/utils/literal.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/literal.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__string_literal" + }, + "macro.dbt.default__support_multiple_grantees_per_dcl_statement": { + "arguments": [], + "created_at": 1687942823.0221472, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "\n\n{%- macro default__support_multiple_grantees_per_dcl_statement() -%}\n {{ return(True) }}\n{%- endmacro -%}\n\n\n", + "meta": {}, + "name": "default__support_multiple_grantees_per_dcl_statement", + "original_file_path": "macros/adapters/apply_grants.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/apply_grants.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__support_multiple_grantees_per_dcl_statement" + }, + "macro.dbt.default__test_accepted_values": { + "arguments": [], + "created_at": 1687942822.982643, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__test_accepted_values(model, column_name, values, quote=True) %}\n\nwith all_values as (\n\n select\n {{ column_name }} as value_field,\n count(*) as n_records\n\n from {{ model }}\n group by {{ column_name }}\n\n)\n\nselect *\nfrom all_values\nwhere value_field not in (\n {% for value in values -%}\n {% if quote -%}\n '{{ value }}'\n {%- else -%}\n {{ value }}\n {%- endif -%}\n {%- if not loop.last -%},{%- endif %}\n {%- endfor %}\n)\n\n{% endmacro %}", + "meta": {}, + "name": "default__test_accepted_values", + "original_file_path": "macros/generic_test_sql/accepted_values.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/generic_test_sql/accepted_values.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__test_accepted_values" + }, + "macro.dbt.default__test_not_null": { + "arguments": [], + "created_at": 1687942822.98174, + "depends_on": { + "macros": [ + "macro.dbt.should_store_failures" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__test_not_null(model, column_name) %}\n\n{% set column_list = '*' if should_store_failures() else column_name %}\n\nselect {{ column_list }}\nfrom {{ model }}\nwhere {{ column_name }} is null\n\n{% endmacro %}", + "meta": {}, + "name": "default__test_not_null", + "original_file_path": "macros/generic_test_sql/not_null.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/generic_test_sql/not_null.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__test_not_null" + }, + "macro.dbt.default__test_relationships": { + "arguments": [], + "created_at": 1687942822.981408, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__test_relationships(model, column_name, to, field) %}\n\nwith child as (\n select {{ column_name }} as from_field\n from {{ model }}\n where {{ column_name }} is not null\n),\n\nparent as (\n select {{ field }} as to_field\n from {{ to }}\n)\n\nselect\n from_field\n\nfrom child\nleft join parent\n on child.from_field = parent.to_field\n\nwhere parent.to_field is null\n\n{% endmacro %}", + "meta": {}, + "name": "default__test_relationships", + "original_file_path": "macros/generic_test_sql/relationships.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/generic_test_sql/relationships.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__test_relationships" + }, + "macro.dbt.default__test_unique": { + "arguments": [], + "created_at": 1687942822.9820108, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__test_unique(model, column_name) %}\n\nselect\n {{ column_name }} as unique_field,\n count(*) as n_records\n\nfrom {{ model }}\nwhere {{ column_name }} is not null\ngroup by {{ column_name }}\nhaving count(*) > 1\n\n{% endmacro %}", + "meta": {}, + "name": "default__test_unique", + "original_file_path": "macros/generic_test_sql/unique.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/generic_test_sql/unique.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__test_unique" + }, + "macro.dbt.default__truncate_relation": { + "arguments": [], + "created_at": 1687942823.016861, + "depends_on": { + "macros": [ + "macro.dbt.statement" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__truncate_relation(relation) -%}\n {% call statement('truncate_relation') -%}\n truncate table {{ relation }}\n {%- endcall %}\n{% endmacro %}", + "meta": {}, + "name": "default__truncate_relation", + "original_file_path": "macros/adapters/relation.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/relation.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__truncate_relation" + }, + "macro.dbt.default__type_bigint": { + "arguments": [], + "created_at": 1687942823.0016088, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__type_bigint() %}\n {{ return(api.Column.translate_type(\"bigint\")) }}\n{% endmacro %}", + "meta": {}, + "name": "default__type_bigint", + "original_file_path": "macros/utils/data_types.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/data_types.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__type_bigint" + }, + "macro.dbt.default__type_boolean": { + "arguments": [], + "created_at": 1687942823.0023592, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{%- macro default__type_boolean() -%}\n {{ return(api.Column.translate_type(\"boolean\")) }}\n{%- endmacro -%}\n\n", + "meta": {}, + "name": "default__type_boolean", + "original_file_path": "macros/utils/data_types.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/data_types.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__type_boolean" + }, + "macro.dbt.default__type_float": { + "arguments": [], + "created_at": 1687942823.0008872, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__type_float() %}\n {{ return(api.Column.translate_type(\"float\")) }}\n{% endmacro %}", + "meta": {}, + "name": "default__type_float", + "original_file_path": "macros/utils/data_types.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/data_types.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__type_float" + }, + "macro.dbt.default__type_int": { + "arguments": [], + "created_at": 1687942823.002025, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{%- macro default__type_int() -%}\n {{ return(api.Column.translate_type(\"integer\")) }}\n{%- endmacro -%}\n\n", + "meta": {}, + "name": "default__type_int", + "original_file_path": "macros/utils/data_types.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/data_types.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__type_int" + }, + "macro.dbt.default__type_numeric": { + "arguments": [], + "created_at": 1687942823.001258, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__type_numeric() %}\n {{ return(api.Column.numeric_type(\"numeric\", 28, 6)) }}\n{% endmacro %}", + "meta": {}, + "name": "default__type_numeric", + "original_file_path": "macros/utils/data_types.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/data_types.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__type_numeric" + }, + "macro.dbt.default__type_string": { + "arguments": [], + "created_at": 1687942823.000193, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__type_string() %}\n {{ return(api.Column.translate_type(\"string\")) }}\n{% endmacro %}", + "meta": {}, + "name": "default__type_string", + "original_file_path": "macros/utils/data_types.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/data_types.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__type_string" + }, + "macro.dbt.default__type_timestamp": { + "arguments": [], + "created_at": 1687942823.000544, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro default__type_timestamp() %}\n {{ return(api.Column.translate_type(\"timestamp\")) }}\n{% endmacro %}", + "meta": {}, + "name": "default__type_timestamp", + "original_file_path": "macros/utils/data_types.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/data_types.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default__type_timestamp" + }, + "macro.dbt.default_last_day": { + "arguments": [], + "created_at": 1687942823.00386, + "depends_on": { + "macros": [ + "macro.dbt.dateadd", + "macro.dbt.date_trunc" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "\n\n{%- macro default_last_day(date, datepart) -%}\n cast(\n {{dbt.dateadd('day', '-1',\n dbt.dateadd(datepart, '1', dbt.date_trunc(datepart, date))\n )}}\n as date)\n{%- endmacro -%}\n\n", + "meta": {}, + "name": "default_last_day", + "original_file_path": "macros/utils/last_day.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/last_day.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.default_last_day" + }, + "macro.dbt.diff_column_data_types": { + "arguments": [], + "created_at": 1687942822.910557, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro diff_column_data_types(source_columns, target_columns) %}\n\n {% set result = [] %}\n {% for sc in source_columns %}\n {% set tc = target_columns | selectattr(\"name\", \"equalto\", sc.name) | list | first %}\n {% if tc %}\n {% if sc.data_type != tc.data_type and not sc.can_expand_to(other_column=tc) %}\n {{ result.append( { 'column_name': tc.name, 'new_type': sc.data_type } ) }}\n {% endif %}\n {% endif %}\n {% endfor %}\n\n {{ return(result) }}\n\n{% endmacro %}", + "meta": {}, + "name": "diff_column_data_types", + "original_file_path": "macros/materializations/models/incremental/column_helpers.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/incremental/column_helpers.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.diff_column_data_types" + }, + "macro.dbt.diff_columns": { + "arguments": [], + "created_at": 1687942822.908992, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro diff_columns(source_columns, target_columns) %}\n\n {% set result = [] %}\n {% set source_names = source_columns | map(attribute = 'column') | list %}\n {% set target_names = target_columns | map(attribute = 'column') | list %}\n\n {# --check whether the name attribute exists in the target - this does not perform a data type check #}\n {% for sc in source_columns %}\n {% if sc.name not in target_names %}\n {{ result.append(sc) }}\n {% endif %}\n {% endfor %}\n\n {{ return(result) }}\n\n{% endmacro %}", + "meta": {}, + "name": "diff_columns", + "original_file_path": "macros/materializations/models/incremental/column_helpers.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/incremental/column_helpers.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.diff_columns" + }, + "macro.dbt.drop_relation": { + "arguments": [], + "created_at": 1687942823.016174, + "depends_on": { + "macros": [ + "macro.dbt.default__drop_relation" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro drop_relation(relation) -%}\n {{ return(adapter.dispatch('drop_relation', 'dbt')(relation)) }}\n{% endmacro %}", + "meta": {}, + "name": "drop_relation", + "original_file_path": "macros/adapters/relation.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/relation.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.drop_relation" + }, + "macro.dbt.drop_relation_if_exists": { + "arguments": [], + "created_at": 1687942823.018911, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro drop_relation_if_exists(relation) %}\n {% if relation is not none %}\n {{ adapter.drop_relation(relation) }}\n {% endif %}\n{% endmacro %}", + "meta": {}, + "name": "drop_relation_if_exists", + "original_file_path": "macros/adapters/relation.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/relation.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.drop_relation_if_exists" + }, + "macro.dbt.drop_schema": { + "arguments": [], + "created_at": 1687942823.007472, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres__drop_schema" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro drop_schema(relation) -%}\n {{ adapter.dispatch('drop_schema', 'dbt')(relation) }}\n{% endmacro %}", + "meta": {}, + "name": "drop_schema", + "original_file_path": "macros/adapters/schema.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/schema.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.drop_schema" + }, + "macro.dbt.escape_single_quotes": { + "arguments": [], + "created_at": 1687942822.993175, + "depends_on": { + "macros": [ + "macro.dbt.default__escape_single_quotes" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro escape_single_quotes(expression) %}\n {{ return(adapter.dispatch('escape_single_quotes', 'dbt') (expression)) }}\n{% endmacro %}", + "meta": {}, + "name": "escape_single_quotes", + "original_file_path": "macros/utils/escape_single_quotes.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/escape_single_quotes.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.escape_single_quotes" + }, + "macro.dbt.except": { + "arguments": [], + "created_at": 1687942822.990304, + "depends_on": { + "macros": [ + "macro.dbt.default__except" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro except() %}\n {{ return(adapter.dispatch('except', 'dbt')()) }}\n{% endmacro %}", + "meta": {}, + "name": "except", + "original_file_path": "macros/utils/except.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/except.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.except" + }, + "macro.dbt.format_columns": { + "arguments": [], + "created_at": 1687942822.9486878, + "depends_on": { + "macros": [ + "macro.dbt.default__format_column" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro format_columns(columns) %}\n {% set formatted_columns = [] %}\n {% for column in columns %}\n {%- set formatted_column = adapter.dispatch('format_column', 'dbt')(column) -%}\n {%- do formatted_columns.append(formatted_column) -%}\n {% endfor %}\n {{ return(formatted_columns) }}\n{% endmacro %}", + "meta": {}, + "name": "format_columns", + "original_file_path": "macros/materializations/models/table/columns_spec_ddl.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/table/columns_spec_ddl.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.format_columns" + }, + "macro.dbt.generate_alias_name": { + "arguments": [], + "created_at": 1687942822.9786382, + "depends_on": { + "macros": [ + "macro.dbt.default__generate_alias_name" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro generate_alias_name(custom_alias_name=none, node=none) -%}\n {% do return(adapter.dispatch('generate_alias_name', 'dbt')(custom_alias_name, node)) %}\n{%- endmacro %}", + "meta": {}, + "name": "generate_alias_name", + "original_file_path": "macros/get_custom_name/get_custom_alias.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/get_custom_name/get_custom_alias.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.generate_alias_name" + }, + "macro.dbt.generate_database_name": { + "arguments": [], + "created_at": 1687942822.980743, + "depends_on": { + "macros": [ + "macro.dbt.default__generate_database_name" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro generate_database_name(custom_database_name=none, node=none) -%}\n {% do return(adapter.dispatch('generate_database_name', 'dbt')(custom_database_name, node)) %}\n{%- endmacro %}", + "meta": {}, + "name": "generate_database_name", + "original_file_path": "macros/get_custom_name/get_custom_database.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/get_custom_name/get_custom_database.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.generate_database_name" + }, + "macro.dbt.generate_schema_name": { + "arguments": [], + "created_at": 1687942822.97963, + "depends_on": { + "macros": [ + "macro.dbt.default__generate_schema_name" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro generate_schema_name(custom_schema_name=none, node=none) -%}\n {{ return(adapter.dispatch('generate_schema_name', 'dbt')(custom_schema_name, node)) }}\n{% endmacro %}", + "meta": {}, + "name": "generate_schema_name", + "original_file_path": "macros/get_custom_name/get_custom_schema.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/get_custom_name/get_custom_schema.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.generate_schema_name" + }, + "macro.dbt.generate_schema_name_for_env": { + "arguments": [], + "created_at": 1687942822.9802911, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro generate_schema_name_for_env(custom_schema_name, node) -%}\n\n {%- set default_schema = target.schema -%}\n {%- if target.name == 'prod' and custom_schema_name is not none -%}\n\n {{ custom_schema_name | trim }}\n\n {%- else -%}\n\n {{ default_schema }}\n\n {%- endif -%}\n\n{%- endmacro %}", + "meta": {}, + "name": "generate_schema_name_for_env", + "original_file_path": "macros/get_custom_name/get_custom_schema.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/get_custom_name/get_custom_schema.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.generate_schema_name_for_env" + }, + "macro.dbt.get_assert_columns_equivalent": { + "arguments": [], + "created_at": 1687942822.946754, + "depends_on": { + "macros": [ + "macro.dbt.default__get_assert_columns_equivalent" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "\n\n{%- macro get_assert_columns_equivalent(sql) -%}\n {{ adapter.dispatch('get_assert_columns_equivalent', 'dbt')(sql) }}\n{%- endmacro -%}\n\n", + "meta": {}, + "name": "get_assert_columns_equivalent", + "original_file_path": "macros/materializations/models/table/columns_spec_ddl.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/table/columns_spec_ddl.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_assert_columns_equivalent" + }, + "macro.dbt.get_batch_size": { + "arguments": [], + "created_at": 1687942822.975379, + "depends_on": { + "macros": [ + "macro.dbt.default__get_batch_size" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro get_batch_size() -%}\n {{ return(adapter.dispatch('get_batch_size', 'dbt')()) }}\n{%- endmacro %}", + "meta": {}, + "name": "get_batch_size", + "original_file_path": "macros/materializations/seeds/helpers.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/seeds/helpers.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_batch_size" + }, + "macro.dbt.get_binding_char": { + "arguments": [], + "created_at": 1687942822.975074, + "depends_on": { + "macros": [ + "macro.dbt.default__get_binding_char" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro get_binding_char() -%}\n {{ adapter.dispatch('get_binding_char', 'dbt')() }}\n{%- endmacro %}", + "meta": {}, + "name": "get_binding_char", + "original_file_path": "macros/materializations/seeds/helpers.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/seeds/helpers.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_binding_char" + }, + "macro.dbt.get_catalog": { + "arguments": [], + "created_at": 1687942823.031494, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres__get_catalog" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro get_catalog(information_schema, schemas) -%}\n {{ return(adapter.dispatch('get_catalog', 'dbt')(information_schema, schemas)) }}\n{%- endmacro %}", + "meta": {}, + "name": "get_catalog", + "original_file_path": "macros/adapters/metadata.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/metadata.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_catalog" + }, + "macro.dbt.get_column_schema_from_query": { + "arguments": [], + "created_at": 1687942823.0389261, + "depends_on": { + "macros": [ + "macro.dbt.get_empty_subquery_sql" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro get_column_schema_from_query(select_sql, select_sql_header=none) -%}\n {% set columns = [] %}\n {# -- Using an 'empty subquery' here to get the same schema as the given select_sql statement, without necessitating a data scan.#}\n {% set sql = get_empty_subquery_sql(select_sql, select_sql_header) %}\n {% set column_schema = adapter.get_column_schema_from_query(sql) %}\n {{ return(column_schema) }}\n{% endmacro %}", + "meta": {}, + "name": "get_column_schema_from_query", + "original_file_path": "macros/adapters/columns.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/columns.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_column_schema_from_query" + }, + "macro.dbt.get_columns_in_query": { + "arguments": [], + "created_at": 1687942823.0391302, + "depends_on": { + "macros": [ + "macro.dbt.default__get_columns_in_query" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro get_columns_in_query(select_sql) -%}\n {{ return(adapter.dispatch('get_columns_in_query', 'dbt')(select_sql)) }}\n{% endmacro %}", + "meta": {}, + "name": "get_columns_in_query", + "original_file_path": "macros/adapters/columns.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/columns.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_columns_in_query" + }, + "macro.dbt.get_columns_in_relation": { + "arguments": [], + "created_at": 1687942823.036205, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres__get_columns_in_relation" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro get_columns_in_relation(relation) -%}\n {{ return(adapter.dispatch('get_columns_in_relation', 'dbt')(relation)) }}\n{% endmacro %}", + "meta": {}, + "name": "get_columns_in_relation", + "original_file_path": "macros/adapters/columns.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/columns.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_columns_in_relation" + }, + "macro.dbt.get_create_index_sql": { + "arguments": [], + "created_at": 1687942823.0101209, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres__get_create_index_sql" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro get_create_index_sql(relation, index_dict) -%}\n {{ return(adapter.dispatch('get_create_index_sql', 'dbt')(relation, index_dict)) }}\n{% endmacro %}", + "meta": {}, + "name": "get_create_index_sql", + "original_file_path": "macros/adapters/indexes.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/indexes.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_create_index_sql" + }, + "macro.dbt.get_create_table_as_sql": { + "arguments": [], + "created_at": 1687942822.95344, + "depends_on": { + "macros": [ + "macro.dbt.default__get_create_table_as_sql" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro get_create_table_as_sql(temporary, relation, sql) -%}\n {{ adapter.dispatch('get_create_table_as_sql', 'dbt')(temporary, relation, sql) }}\n{%- endmacro %}", + "meta": {}, + "name": "get_create_table_as_sql", + "original_file_path": "macros/materializations/models/table/create_table_as.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/table/create_table_as.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_create_table_as_sql" + }, + "macro.dbt.get_create_view_as_sql": { + "arguments": [], + "created_at": 1687942822.961741, + "depends_on": { + "macros": [ + "macro.dbt.default__get_create_view_as_sql" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro get_create_view_as_sql(relation, sql) -%}\n {{ adapter.dispatch('get_create_view_as_sql', 'dbt')(relation, sql) }}\n{%- endmacro %}", + "meta": {}, + "name": "get_create_view_as_sql", + "original_file_path": "macros/materializations/models/view/create_view_as.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/view/create_view_as.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_create_view_as_sql" + }, + "macro.dbt.get_csv_sql": { + "arguments": [], + "created_at": 1687942822.974763, + "depends_on": { + "macros": [ + "macro.dbt.default__get_csv_sql" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro get_csv_sql(create_or_truncate_sql, insert_sql) %}\n {{ adapter.dispatch('get_csv_sql', 'dbt')(create_or_truncate_sql, insert_sql) }}\n{% endmacro %}", + "meta": {}, + "name": "get_csv_sql", + "original_file_path": "macros/materializations/seeds/helpers.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/seeds/helpers.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_csv_sql" + }, + "macro.dbt.get_dcl_statement_list": { + "arguments": [], + "created_at": 1687942823.024218, + "depends_on": { + "macros": [ + "macro.dbt.default__get_dcl_statement_list" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro get_dcl_statement_list(relation, grant_config, get_dcl_macro) %}\n {{ return(adapter.dispatch('get_dcl_statement_list', 'dbt')(relation, grant_config, get_dcl_macro)) }}\n{% endmacro %}", + "meta": {}, + "name": "get_dcl_statement_list", + "original_file_path": "macros/adapters/apply_grants.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/apply_grants.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_dcl_statement_list" + }, + "macro.dbt.get_delete_insert_merge_sql": { + "arguments": [], + "created_at": 1687942822.9214232, + "depends_on": { + "macros": [ + "macro.dbt.default__get_delete_insert_merge_sql" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro get_delete_insert_merge_sql(target, source, unique_key, dest_columns, incremental_predicates) -%}\n {{ adapter.dispatch('get_delete_insert_merge_sql', 'dbt')(target, source, unique_key, dest_columns, incremental_predicates) }}\n{%- endmacro %}", + "meta": {}, + "name": "get_delete_insert_merge_sql", + "original_file_path": "macros/materializations/models/incremental/merge.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/incremental/merge.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_delete_insert_merge_sql" + }, + "macro.dbt.get_empty_schema_sql": { + "arguments": [], + "created_at": 1687942823.037695, + "depends_on": { + "macros": [ + "macro.dbt.default__get_empty_schema_sql" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro get_empty_schema_sql(columns) -%}\n {{ return(adapter.dispatch('get_empty_schema_sql', 'dbt')(columns)) }}\n{% endmacro %}", + "meta": {}, + "name": "get_empty_schema_sql", + "original_file_path": "macros/adapters/columns.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/columns.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_empty_schema_sql" + }, + "macro.dbt.get_empty_subquery_sql": { + "arguments": [], + "created_at": 1687942823.0372648, + "depends_on": { + "macros": [ + "macro.dbt.default__get_empty_subquery_sql" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro get_empty_subquery_sql(select_sql, select_sql_header=none) -%}\n {{ return(adapter.dispatch('get_empty_subquery_sql', 'dbt')(select_sql, select_sql_header)) }}\n{% endmacro %}", + "meta": {}, + "name": "get_empty_subquery_sql", + "original_file_path": "macros/adapters/columns.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/columns.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_empty_subquery_sql" + }, + "macro.dbt.get_grant_sql": { + "arguments": [], + "created_at": 1687942823.023207, + "depends_on": { + "macros": [ + "macro.dbt.default__get_grant_sql" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro get_grant_sql(relation, privilege, grantees) %}\n {{ return(adapter.dispatch('get_grant_sql', 'dbt')(relation, privilege, grantees)) }}\n{% endmacro %}", + "meta": {}, + "name": "get_grant_sql", + "original_file_path": "macros/adapters/apply_grants.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/apply_grants.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_grant_sql" + }, + "macro.dbt.get_incremental_append_sql": { + "arguments": [], + "created_at": 1687942822.9254642, + "depends_on": { + "macros": [ + "macro.dbt.default__get_incremental_append_sql" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro get_incremental_append_sql(arg_dict) %}\n\n {{ return(adapter.dispatch('get_incremental_append_sql', 'dbt')(arg_dict)) }}\n\n{% endmacro %}", + "meta": {}, + "name": "get_incremental_append_sql", + "original_file_path": "macros/materializations/models/incremental/strategies.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/incremental/strategies.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_incremental_append_sql" + }, + "macro.dbt.get_incremental_default_sql": { + "arguments": [], + "created_at": 1687942822.927587, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres__get_incremental_default_sql" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro get_incremental_default_sql(arg_dict) %}\n\n {{ return(adapter.dispatch('get_incremental_default_sql', 'dbt')(arg_dict)) }}\n\n{% endmacro %}", + "meta": {}, + "name": "get_incremental_default_sql", + "original_file_path": "macros/materializations/models/incremental/strategies.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/incremental/strategies.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_incremental_default_sql" + }, + "macro.dbt.get_incremental_delete_insert_sql": { + "arguments": [], + "created_at": 1687942822.925951, + "depends_on": { + "macros": [ + "macro.dbt.default__get_incremental_delete_insert_sql" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro get_incremental_delete_insert_sql(arg_dict) %}\n\n {{ return(adapter.dispatch('get_incremental_delete_insert_sql', 'dbt')(arg_dict)) }}\n\n{% endmacro %}", + "meta": {}, + "name": "get_incremental_delete_insert_sql", + "original_file_path": "macros/materializations/models/incremental/strategies.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/incremental/strategies.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_incremental_delete_insert_sql" + }, + "macro.dbt.get_incremental_insert_overwrite_sql": { + "arguments": [], + "created_at": 1687942822.9270568, + "depends_on": { + "macros": [ + "macro.dbt.default__get_incremental_insert_overwrite_sql" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro get_incremental_insert_overwrite_sql(arg_dict) %}\n\n {{ return(adapter.dispatch('get_incremental_insert_overwrite_sql', 'dbt')(arg_dict)) }}\n\n{% endmacro %}", + "meta": {}, + "name": "get_incremental_insert_overwrite_sql", + "original_file_path": "macros/materializations/models/incremental/strategies.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/incremental/strategies.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_incremental_insert_overwrite_sql" + }, + "macro.dbt.get_incremental_merge_sql": { + "arguments": [], + "created_at": 1687942822.92649, + "depends_on": { + "macros": [ + "macro.dbt.default__get_incremental_merge_sql" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro get_incremental_merge_sql(arg_dict) %}\n\n {{ return(adapter.dispatch('get_incremental_merge_sql', 'dbt')(arg_dict)) }}\n\n{% endmacro %}", + "meta": {}, + "name": "get_incremental_merge_sql", + "original_file_path": "macros/materializations/models/incremental/strategies.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/incremental/strategies.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_incremental_merge_sql" + }, + "macro.dbt.get_insert_into_sql": { + "arguments": [], + "created_at": 1687942822.928073, + "depends_on": { + "macros": [ + "macro.dbt.get_quoted_csv" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro get_insert_into_sql(target_relation, temp_relation, dest_columns) %}\n\n {%- set dest_cols_csv = get_quoted_csv(dest_columns | map(attribute=\"name\")) -%}\n\n insert into {{ target_relation }} ({{ dest_cols_csv }})\n (\n select {{ dest_cols_csv }}\n from {{ temp_relation }}\n )\n\n{% endmacro %}", + "meta": {}, + "name": "get_insert_into_sql", + "original_file_path": "macros/materializations/models/incremental/strategies.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/incremental/strategies.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_insert_into_sql" + }, + "macro.dbt.get_insert_overwrite_merge_sql": { + "arguments": [], + "created_at": 1687942822.922858, + "depends_on": { + "macros": [ + "macro.dbt.default__get_insert_overwrite_merge_sql" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro get_insert_overwrite_merge_sql(target, source, dest_columns, predicates, include_sql_header=false) -%}\n {{ adapter.dispatch('get_insert_overwrite_merge_sql', 'dbt')(target, source, dest_columns, predicates, include_sql_header) }}\n{%- endmacro %}", + "meta": {}, + "name": "get_insert_overwrite_merge_sql", + "original_file_path": "macros/materializations/models/incremental/merge.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/incremental/merge.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_insert_overwrite_merge_sql" + }, + "macro.dbt.get_merge_sql": { + "arguments": [], + "created_at": 1687942822.919136, + "depends_on": { + "macros": [ + "macro.dbt.default__get_merge_sql" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro get_merge_sql(target, source, unique_key, dest_columns, incremental_predicates=none) -%}\n -- back compat for old kwarg name\n {% set incremental_predicates = kwargs.get('predicates', incremental_predicates) %}\n {{ adapter.dispatch('get_merge_sql', 'dbt')(target, source, unique_key, dest_columns, incremental_predicates) }}\n{%- endmacro %}", + "meta": {}, + "name": "get_merge_sql", + "original_file_path": "macros/materializations/models/incremental/merge.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/incremental/merge.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_merge_sql" + }, + "macro.dbt.get_merge_update_columns": { + "arguments": [], + "created_at": 1687942822.9108438, + "depends_on": { + "macros": [ + "macro.dbt.default__get_merge_update_columns" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro get_merge_update_columns(merge_update_columns, merge_exclude_columns, dest_columns) %}\n {{ return(adapter.dispatch('get_merge_update_columns', 'dbt')(merge_update_columns, merge_exclude_columns, dest_columns)) }}\n{% endmacro %}", + "meta": {}, + "name": "get_merge_update_columns", + "original_file_path": "macros/materializations/models/incremental/column_helpers.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/incremental/column_helpers.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_merge_update_columns" + }, + "macro.dbt.get_or_create_relation": { + "arguments": [], + "created_at": 1687942823.017668, + "depends_on": { + "macros": [ + "macro.dbt.default__get_or_create_relation" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro get_or_create_relation(database, schema, identifier, type) -%}\n {{ return(adapter.dispatch('get_or_create_relation', 'dbt')(database, schema, identifier, type)) }}\n{% endmacro %}", + "meta": {}, + "name": "get_or_create_relation", + "original_file_path": "macros/adapters/relation.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/relation.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_or_create_relation" + }, + "macro.dbt.get_quoted_csv": { + "arguments": [], + "created_at": 1687942822.908025, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro get_quoted_csv(column_names) %}\n\n {% set quoted = [] %}\n {% for col in column_names -%}\n {%- do quoted.append(adapter.quote(col)) -%}\n {%- endfor %}\n\n {%- set dest_cols_csv = quoted | join(', ') -%}\n {{ return(dest_cols_csv) }}\n\n{% endmacro %}", + "meta": {}, + "name": "get_quoted_csv", + "original_file_path": "macros/materializations/models/incremental/column_helpers.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/incremental/column_helpers.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_quoted_csv" + }, + "macro.dbt.get_revoke_sql": { + "arguments": [], + "created_at": 1687942823.023674, + "depends_on": { + "macros": [ + "macro.dbt.default__get_revoke_sql" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro get_revoke_sql(relation, privilege, grantees) %}\n {{ return(adapter.dispatch('get_revoke_sql', 'dbt')(relation, privilege, grantees)) }}\n{% endmacro %}", + "meta": {}, + "name": "get_revoke_sql", + "original_file_path": "macros/adapters/apply_grants.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/apply_grants.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_revoke_sql" + }, + "macro.dbt.get_seed_column_quoted_csv": { + "arguments": [], + "created_at": 1687942822.976216, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro get_seed_column_quoted_csv(model, column_names) %}\n {%- set quote_seed_column = model['config'].get('quote_columns', None) -%}\n {% set quoted = [] %}\n {% for col in column_names -%}\n {%- do quoted.append(adapter.quote_seed_column(col, quote_seed_column)) -%}\n {%- endfor %}\n\n {%- set dest_cols_csv = quoted | join(', ') -%}\n {{ return(dest_cols_csv) }}\n{% endmacro %}", + "meta": {}, + "name": "get_seed_column_quoted_csv", + "original_file_path": "macros/materializations/seeds/helpers.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/seeds/helpers.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_seed_column_quoted_csv" + }, + "macro.dbt.get_select_subquery": { + "arguments": [], + "created_at": 1687942822.955651, + "depends_on": { + "macros": [ + "macro.dbt.default__get_select_subquery" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro get_select_subquery(sql) %}\n {{ return(adapter.dispatch('get_select_subquery', 'dbt')(sql)) }}\n{% endmacro %}", + "meta": {}, + "name": "get_select_subquery", + "original_file_path": "macros/materializations/models/table/create_table_as.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/table/create_table_as.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_select_subquery" + }, + "macro.dbt.get_show_grant_sql": { + "arguments": [], + "created_at": 1687942823.0228322, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres__get_show_grant_sql" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro get_show_grant_sql(relation) %}\n {{ return(adapter.dispatch(\"get_show_grant_sql\", \"dbt\")(relation)) }}\n{% endmacro %}", + "meta": {}, + "name": "get_show_grant_sql", + "original_file_path": "macros/adapters/apply_grants.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/apply_grants.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_show_grant_sql" + }, + "macro.dbt.get_table_columns_and_constraints": { + "arguments": [], + "created_at": 1687942822.945817, + "depends_on": { + "macros": [ + "macro.dbt.default__get_table_columns_and_constraints" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{%- macro get_table_columns_and_constraints() -%}\n {{ adapter.dispatch('get_table_columns_and_constraints', 'dbt')() }}\n{%- endmacro -%}\n\n", + "meta": {}, + "name": "get_table_columns_and_constraints", + "original_file_path": "macros/materializations/models/table/columns_spec_ddl.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/table/columns_spec_ddl.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_table_columns_and_constraints" + }, + "macro.dbt.get_test_sql": { + "arguments": [], + "created_at": 1687942822.904258, + "depends_on": { + "macros": [ + "macro.dbt.default__get_test_sql" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro get_test_sql(main_sql, fail_calc, warn_if, error_if, limit) -%}\n {{ adapter.dispatch('get_test_sql', 'dbt')(main_sql, fail_calc, warn_if, error_if, limit) }}\n{%- endmacro %}", + "meta": {}, + "name": "get_test_sql", + "original_file_path": "macros/materializations/tests/helpers.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/tests/helpers.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_test_sql" + }, + "macro.dbt.get_true_sql": { + "arguments": [], + "created_at": 1687942822.890275, + "depends_on": { + "macros": [ + "macro.dbt.default__get_true_sql" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro get_true_sql() %}\n {{ adapter.dispatch('get_true_sql', 'dbt')() }}\n{% endmacro %}", + "meta": {}, + "name": "get_true_sql", + "original_file_path": "macros/materializations/snapshots/helpers.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/snapshots/helpers.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_true_sql" + }, + "macro.dbt.get_where_subquery": { + "arguments": [], + "created_at": 1687942822.905386, + "depends_on": { + "macros": [ + "macro.dbt.default__get_where_subquery" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro get_where_subquery(relation) -%}\n {% do return(adapter.dispatch('get_where_subquery', 'dbt')(relation)) %}\n{%- endmacro %}", + "meta": {}, + "name": "get_where_subquery", + "original_file_path": "macros/materializations/tests/where_subquery.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/tests/where_subquery.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.get_where_subquery" + }, + "macro.dbt.handle_existing_table": { + "arguments": [], + "created_at": 1687942822.9593441, + "depends_on": { + "macros": [ + "macro.dbt.default__handle_existing_table" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro handle_existing_table(full_refresh, old_relation) %}\n {{ adapter.dispatch('handle_existing_table', 'dbt')(full_refresh, old_relation) }}\n{% endmacro %}", + "meta": {}, + "name": "handle_existing_table", + "original_file_path": "macros/materializations/models/view/helpers.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/view/helpers.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.handle_existing_table" + }, + "macro.dbt.hash": { + "arguments": [], + "created_at": 1687942822.9966311, + "depends_on": { + "macros": [ + "macro.dbt.default__hash" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro hash(field) -%}\n {{ return(adapter.dispatch('hash', 'dbt') (field)) }}\n{%- endmacro %}", + "meta": {}, + "name": "hash", + "original_file_path": "macros/utils/hash.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/hash.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.hash" + }, + "macro.dbt.in_transaction": { + "arguments": [], + "created_at": 1687942822.871236, + "depends_on": { + "macros": [ + "macro.dbt.make_hook_config" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro in_transaction(sql) %}\n {{ make_hook_config(sql, inside_transaction=True) }}\n{% endmacro %}", + "meta": {}, + "name": "in_transaction", + "original_file_path": "macros/materializations/hooks.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/hooks.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.in_transaction" + }, + "macro.dbt.incremental_validate_on_schema_change": { + "arguments": [], + "created_at": 1687942822.9404979, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro incremental_validate_on_schema_change(on_schema_change, default='ignore') %}\n\n {% if on_schema_change not in ['sync_all_columns', 'append_new_columns', 'fail', 'ignore'] %}\n\n {% set log_message = 'Invalid value for on_schema_change (%s) specified. Setting default value of %s.' % (on_schema_change, default) %}\n {% do log(log_message) %}\n\n {{ return(default) }}\n\n {% else %}\n\n {{ return(on_schema_change) }}\n\n {% endif %}\n\n{% endmacro %}", + "meta": {}, + "name": "incremental_validate_on_schema_change", + "original_file_path": "macros/materializations/models/incremental/on_schema_change.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/incremental/on_schema_change.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.incremental_validate_on_schema_change" + }, + "macro.dbt.information_schema_name": { + "arguments": [], + "created_at": 1687942823.032001, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres__information_schema_name" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro information_schema_name(database) %}\n {{ return(adapter.dispatch('information_schema_name', 'dbt')(database)) }}\n{% endmacro %}", + "meta": {}, + "name": "information_schema_name", + "original_file_path": "macros/adapters/metadata.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/metadata.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.information_schema_name" + }, + "macro.dbt.intersect": { + "arguments": [], + "created_at": 1687942822.992756, + "depends_on": { + "macros": [ + "macro.dbt.default__intersect" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro intersect() %}\n {{ return(adapter.dispatch('intersect', 'dbt')()) }}\n{% endmacro %}", + "meta": {}, + "name": "intersect", + "original_file_path": "macros/utils/intersect.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/intersect.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.intersect" + }, + "macro.dbt.is_incremental": { + "arguments": [], + "created_at": 1687942822.924459, + "depends_on": { + "macros": [ + "macro.dbt.should_full_refresh" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro is_incremental() %}\n {#-- do not run introspective queries in parsing #}\n {% if not execute %}\n {{ return(False) }}\n {% else %}\n {% set relation = adapter.get_relation(this.database, this.schema, this.table) %}\n {{ return(relation is not none\n and relation.type == 'table'\n and model.config.materialized == 'incremental'\n and not should_full_refresh()) }}\n {% endif %}\n{% endmacro %}", + "meta": {}, + "name": "is_incremental", + "original_file_path": "macros/materializations/models/incremental/is_incremental.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/incremental/is_incremental.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.is_incremental" + }, + "macro.dbt.last_day": { + "arguments": [], + "created_at": 1687942823.0035672, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres__last_day" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro last_day(date, datepart) %}\n {{ return(adapter.dispatch('last_day', 'dbt') (date, datepart)) }}\n{% endmacro %}", + "meta": {}, + "name": "last_day", + "original_file_path": "macros/utils/last_day.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/last_day.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.last_day" + }, + "macro.dbt.length": { + "arguments": [], + "created_at": 1687942822.9916742, + "depends_on": { + "macros": [ + "macro.dbt.default__length" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro length(expression) -%}\n {{ return(adapter.dispatch('length', 'dbt') (expression)) }}\n{% endmacro %}", + "meta": {}, + "name": "length", + "original_file_path": "macros/utils/length.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/length.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.length" + }, + "macro.dbt.list_relations_without_caching": { + "arguments": [], + "created_at": 1687942823.03344, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres__list_relations_without_caching" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro list_relations_without_caching(schema_relation) %}\n {{ return(adapter.dispatch('list_relations_without_caching', 'dbt')(schema_relation)) }}\n{% endmacro %}", + "meta": {}, + "name": "list_relations_without_caching", + "original_file_path": "macros/adapters/metadata.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/metadata.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.list_relations_without_caching" + }, + "macro.dbt.list_schemas": { + "arguments": [], + "created_at": 1687942823.032376, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres__list_schemas" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro list_schemas(database) -%}\n {{ return(adapter.dispatch('list_schemas', 'dbt')(database)) }}\n{% endmacro %}", + "meta": {}, + "name": "list_schemas", + "original_file_path": "macros/adapters/metadata.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/metadata.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.list_schemas" + }, + "macro.dbt.listagg": { + "arguments": [], + "created_at": 1687942822.9947362, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres__listagg" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro listagg(measure, delimiter_text=\"','\", order_by_clause=none, limit_num=none) -%}\n {{ return(adapter.dispatch('listagg', 'dbt') (measure, delimiter_text, order_by_clause, limit_num)) }}\n{%- endmacro %}", + "meta": {}, + "name": "listagg", + "original_file_path": "macros/utils/listagg.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/listagg.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.listagg" + }, + "macro.dbt.load_cached_relation": { + "arguments": [], + "created_at": 1687942823.018537, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro load_cached_relation(relation) %}\n {% do return(adapter.get_relation(\n database=relation.database,\n schema=relation.schema,\n identifier=relation.identifier\n )) -%}\n{% endmacro %}", + "meta": {}, + "name": "load_cached_relation", + "original_file_path": "macros/adapters/relation.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/relation.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.load_cached_relation" + }, + "macro.dbt.load_csv_rows": { + "arguments": [], + "created_at": 1687942822.976654, + "depends_on": { + "macros": [ + "macro.dbt.default__load_csv_rows" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro load_csv_rows(model, agate_table) -%}\n {{ adapter.dispatch('load_csv_rows', 'dbt')(model, agate_table) }}\n{%- endmacro %}", + "meta": {}, + "name": "load_csv_rows", + "original_file_path": "macros/materializations/seeds/helpers.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/seeds/helpers.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.load_csv_rows" + }, + "macro.dbt.load_relation": { + "arguments": [], + "created_at": 1687942823.018698, + "depends_on": { + "macros": [ + "macro.dbt.load_cached_relation" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro load_relation(relation) %}\n {{ return(load_cached_relation(relation)) }}\n{% endmacro %}", + "meta": {}, + "name": "load_relation", + "original_file_path": "macros/adapters/relation.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/relation.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.load_relation" + }, + "macro.dbt.make_backup_relation": { + "arguments": [], + "created_at": 1687942823.015619, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres__make_backup_relation" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro make_backup_relation(base_relation, backup_relation_type, suffix='__dbt_backup') %}\n {{ return(adapter.dispatch('make_backup_relation', 'dbt')(base_relation, backup_relation_type, suffix)) }}\n{% endmacro %}", + "meta": {}, + "name": "make_backup_relation", + "original_file_path": "macros/adapters/relation.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/relation.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.make_backup_relation" + }, + "macro.dbt.make_hook_config": { + "arguments": [], + "created_at": 1687942822.8709161, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro make_hook_config(sql, inside_transaction) %}\n {{ tojson({\"sql\": sql, \"transaction\": inside_transaction}) }}\n{% endmacro %}", + "meta": {}, + "name": "make_hook_config", + "original_file_path": "macros/materializations/hooks.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/hooks.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.make_hook_config" + }, + "macro.dbt.make_intermediate_relation": { + "arguments": [], + "created_at": 1687942823.014596, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres__make_intermediate_relation" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro make_intermediate_relation(base_relation, suffix='__dbt_tmp') %}\n {{ return(adapter.dispatch('make_intermediate_relation', 'dbt')(base_relation, suffix)) }}\n{% endmacro %}", + "meta": {}, + "name": "make_intermediate_relation", + "original_file_path": "macros/adapters/relation.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/relation.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.make_intermediate_relation" + }, + "macro.dbt.make_temp_relation": { + "arguments": [], + "created_at": 1687942823.0150259, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres__make_temp_relation" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro make_temp_relation(base_relation, suffix='__dbt_tmp') %}\n {{ return(adapter.dispatch('make_temp_relation', 'dbt')(base_relation, suffix)) }}\n{% endmacro %}", + "meta": {}, + "name": "make_temp_relation", + "original_file_path": "macros/adapters/relation.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/relation.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.make_temp_relation" + }, + "macro.dbt.materialization_incremental_default": { + "arguments": [], + "created_at": 1687942822.9339032, + "depends_on": { + "macros": [ + "macro.dbt.load_cached_relation", + "macro.dbt.make_temp_relation", + "macro.dbt.make_intermediate_relation", + "macro.dbt.make_backup_relation", + "macro.dbt.should_full_refresh", + "macro.dbt.incremental_validate_on_schema_change", + "macro.dbt.drop_relation_if_exists", + "macro.dbt.run_hooks", + "macro.dbt.get_create_table_as_sql", + "macro.dbt.run_query", + "macro.dbt.process_schema_changes", + "macro.dbt.statement", + "macro.dbt.should_revoke", + "macro.dbt.apply_grants", + "macro.dbt.persist_docs", + "macro.dbt.create_indexes" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% materialization incremental, default -%}\n\n -- relations\n {%- set existing_relation = load_cached_relation(this) -%}\n {%- set target_relation = this.incorporate(type='table') -%}\n {%- set temp_relation = make_temp_relation(target_relation)-%}\n {%- set intermediate_relation = make_intermediate_relation(target_relation)-%}\n {%- set backup_relation_type = 'table' if existing_relation is none else existing_relation.type -%}\n {%- set backup_relation = make_backup_relation(target_relation, backup_relation_type) -%}\n\n -- configs\n {%- set unique_key = config.get('unique_key') -%}\n {%- set full_refresh_mode = (should_full_refresh() or existing_relation.is_view) -%}\n {%- set on_schema_change = incremental_validate_on_schema_change(config.get('on_schema_change'), default='ignore') -%}\n\n -- the temp_ and backup_ relations should not already exist in the database; get_relation\n -- will return None in that case. Otherwise, we get a relation that we can drop\n -- later, before we try to use this name for the current operation. This has to happen before\n -- BEGIN, in a separate transaction\n {%- set preexisting_intermediate_relation = load_cached_relation(intermediate_relation)-%}\n {%- set preexisting_backup_relation = load_cached_relation(backup_relation) -%}\n -- grab current tables grants config for comparison later on\n {% set grant_config = config.get('grants') %}\n {{ drop_relation_if_exists(preexisting_intermediate_relation) }}\n {{ drop_relation_if_exists(preexisting_backup_relation) }}\n\n {{ run_hooks(pre_hooks, inside_transaction=False) }}\n\n -- `BEGIN` happens here:\n {{ run_hooks(pre_hooks, inside_transaction=True) }}\n\n {% set to_drop = [] %}\n\n {% if existing_relation is none %}\n {% set build_sql = get_create_table_as_sql(False, target_relation, sql) %}\n {% elif full_refresh_mode %}\n {% set build_sql = get_create_table_as_sql(False, intermediate_relation, sql) %}\n {% set need_swap = true %}\n {% else %}\n {% do run_query(get_create_table_as_sql(True, temp_relation, sql)) %}\n {% do adapter.expand_target_column_types(\n from_relation=temp_relation,\n to_relation=target_relation) %}\n {#-- Process schema changes. Returns dict of changes if successful. Use source columns for upserting/merging --#}\n {% set dest_columns = process_schema_changes(on_schema_change, temp_relation, existing_relation) %}\n {% if not dest_columns %}\n {% set dest_columns = adapter.get_columns_in_relation(existing_relation) %}\n {% endif %}\n\n {#-- Get the incremental_strategy, the macro to use for the strategy, and build the sql --#}\n {% set incremental_strategy = config.get('incremental_strategy') or 'default' %}\n {% set incremental_predicates = config.get('predicates', none) or config.get('incremental_predicates', none) %}\n {% set strategy_sql_macro_func = adapter.get_incremental_strategy_macro(context, incremental_strategy) %}\n {% set strategy_arg_dict = ({'target_relation': target_relation, 'temp_relation': temp_relation, 'unique_key': unique_key, 'dest_columns': dest_columns, 'incremental_predicates': incremental_predicates }) %}\n {% set build_sql = strategy_sql_macro_func(strategy_arg_dict) %}\n\n {% endif %}\n\n {% call statement(\"main\") %}\n {{ build_sql }}\n {% endcall %}\n\n {% if need_swap %}\n {% do adapter.rename_relation(target_relation, backup_relation) %}\n {% do adapter.rename_relation(intermediate_relation, target_relation) %}\n {% do to_drop.append(backup_relation) %}\n {% endif %}\n\n {% set should_revoke = should_revoke(existing_relation, full_refresh_mode) %}\n {% do apply_grants(target_relation, grant_config, should_revoke=should_revoke) %}\n\n {% do persist_docs(target_relation, model) %}\n\n {% if existing_relation is none or existing_relation.is_view or should_full_refresh() %}\n {% do create_indexes(target_relation) %}\n {% endif %}\n\n {{ run_hooks(post_hooks, inside_transaction=True) }}\n\n -- `COMMIT` happens here\n {% do adapter.commit() %}\n\n {% for rel in to_drop %}\n {% do adapter.drop_relation(rel) %}\n {% endfor %}\n\n {{ run_hooks(post_hooks, inside_transaction=False) }}\n\n {{ return({'relations': [target_relation]}) }}\n\n{%- endmaterialization %}", + "meta": {}, + "name": "materialization_incremental_default", + "original_file_path": "macros/materializations/models/incremental/incremental.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/incremental/incremental.sql", + "resource_type": "macro", + "supported_languages": [ + "sql" + ], + "unique_id": "macro.dbt.materialization_incremental_default" + }, + "macro.dbt.materialization_seed_default": { + "arguments": [], + "created_at": 1687942822.966659, + "depends_on": { + "macros": [ + "macro.dbt.should_full_refresh", + "macro.dbt.run_hooks", + "macro.dbt.reset_csv_table", + "macro.dbt.create_csv_table", + "macro.dbt.load_csv_rows", + "macro.dbt.noop_statement", + "macro.dbt.get_csv_sql", + "macro.dbt.should_revoke", + "macro.dbt.apply_grants", + "macro.dbt.persist_docs", + "macro.dbt.create_indexes" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% materialization seed, default %}\n\n {%- set identifier = model['alias'] -%}\n {%- set full_refresh_mode = (should_full_refresh()) -%}\n\n {%- set old_relation = adapter.get_relation(database=database, schema=schema, identifier=identifier) -%}\n\n {%- set exists_as_table = (old_relation is not none and old_relation.is_table) -%}\n {%- set exists_as_view = (old_relation is not none and old_relation.is_view) -%}\n\n {%- set grant_config = config.get('grants') -%}\n {%- set agate_table = load_agate_table() -%}\n -- grab current tables grants config for comparison later on\n\n {%- do store_result('agate_table', response='OK', agate_table=agate_table) -%}\n\n {{ run_hooks(pre_hooks, inside_transaction=False) }}\n\n -- `BEGIN` happens here:\n {{ run_hooks(pre_hooks, inside_transaction=True) }}\n\n -- build model\n {% set create_table_sql = \"\" %}\n {% if exists_as_view %}\n {{ exceptions.raise_compiler_error(\"Cannot seed to '{}', it is a view\".format(old_relation)) }}\n {% elif exists_as_table %}\n {% set create_table_sql = reset_csv_table(model, full_refresh_mode, old_relation, agate_table) %}\n {% else %}\n {% set create_table_sql = create_csv_table(model, agate_table) %}\n {% endif %}\n\n {% set code = 'CREATE' if full_refresh_mode else 'INSERT' %}\n {% set rows_affected = (agate_table.rows | length) %}\n {% set sql = load_csv_rows(model, agate_table) %}\n\n {% call noop_statement('main', code ~ ' ' ~ rows_affected, code, rows_affected) %}\n {{ get_csv_sql(create_table_sql, sql) }};\n {% endcall %}\n\n {% set target_relation = this.incorporate(type='table') %}\n\n {% set should_revoke = should_revoke(old_relation, full_refresh_mode) %}\n {% do apply_grants(target_relation, grant_config, should_revoke=should_revoke) %}\n\n {% do persist_docs(target_relation, model) %}\n\n {% if full_refresh_mode or not exists_as_table %}\n {% do create_indexes(target_relation) %}\n {% endif %}\n\n {{ run_hooks(post_hooks, inside_transaction=True) }}\n\n -- `COMMIT` happens here\n {{ adapter.commit() }}\n\n {{ run_hooks(post_hooks, inside_transaction=False) }}\n\n {{ return({'relations': [target_relation]}) }}\n\n{% endmaterialization %}", + "meta": {}, + "name": "materialization_seed_default", + "original_file_path": "macros/materializations/seeds/seed.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/seeds/seed.sql", + "resource_type": "macro", + "supported_languages": [ + "sql" + ], + "unique_id": "macro.dbt.materialization_seed_default" + }, + "macro.dbt.materialization_snapshot_default": { + "arguments": [], + "created_at": 1687942822.901087, + "depends_on": { + "macros": [ + "macro.dbt.get_or_create_relation", + "macro.dbt.run_hooks", + "macro.dbt.strategy_dispatch", + "macro.dbt.build_snapshot_table", + "macro.dbt.create_table_as", + "macro.dbt.build_snapshot_staging_table", + "macro.dbt.create_columns", + "macro.dbt.snapshot_merge_sql", + "macro.dbt.statement", + "macro.dbt.should_revoke", + "macro.dbt.apply_grants", + "macro.dbt.persist_docs", + "macro.dbt.create_indexes", + "macro.dbt.post_snapshot" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% materialization snapshot, default %}\n {%- set config = model['config'] -%}\n\n {%- set target_table = model.get('alias', model.get('name')) -%}\n\n {%- set strategy_name = config.get('strategy') -%}\n {%- set unique_key = config.get('unique_key') %}\n -- grab current tables grants config for comparison later on\n {%- set grant_config = config.get('grants') -%}\n\n {% set target_relation_exists, target_relation = get_or_create_relation(\n database=model.database,\n schema=model.schema,\n identifier=target_table,\n type='table') -%}\n\n {%- if not target_relation.is_table -%}\n {% do exceptions.relation_wrong_type(target_relation, 'table') %}\n {%- endif -%}\n\n\n {{ run_hooks(pre_hooks, inside_transaction=False) }}\n\n {{ run_hooks(pre_hooks, inside_transaction=True) }}\n\n {% set strategy_macro = strategy_dispatch(strategy_name) %}\n {% set strategy = strategy_macro(model, \"snapshotted_data\", \"source_data\", config, target_relation_exists) %}\n\n {% if not target_relation_exists %}\n\n {% set build_sql = build_snapshot_table(strategy, model['compiled_code']) %}\n {% set final_sql = create_table_as(False, target_relation, build_sql) %}\n\n {% else %}\n\n {{ adapter.valid_snapshot_target(target_relation) }}\n\n {% set staging_table = build_snapshot_staging_table(strategy, sql, target_relation) %}\n\n -- this may no-op if the database does not require column expansion\n {% do adapter.expand_target_column_types(from_relation=staging_table,\n to_relation=target_relation) %}\n\n {% set missing_columns = adapter.get_missing_columns(staging_table, target_relation)\n | rejectattr('name', 'equalto', 'dbt_change_type')\n | rejectattr('name', 'equalto', 'DBT_CHANGE_TYPE')\n | rejectattr('name', 'equalto', 'dbt_unique_key')\n | rejectattr('name', 'equalto', 'DBT_UNIQUE_KEY')\n | list %}\n\n {% do create_columns(target_relation, missing_columns) %}\n\n {% set source_columns = adapter.get_columns_in_relation(staging_table)\n | rejectattr('name', 'equalto', 'dbt_change_type')\n | rejectattr('name', 'equalto', 'DBT_CHANGE_TYPE')\n | rejectattr('name', 'equalto', 'dbt_unique_key')\n | rejectattr('name', 'equalto', 'DBT_UNIQUE_KEY')\n | list %}\n\n {% set quoted_source_columns = [] %}\n {% for column in source_columns %}\n {% do quoted_source_columns.append(adapter.quote(column.name)) %}\n {% endfor %}\n\n {% set final_sql = snapshot_merge_sql(\n target = target_relation,\n source = staging_table,\n insert_cols = quoted_source_columns\n )\n %}\n\n {% endif %}\n\n {% call statement('main') %}\n {{ final_sql }}\n {% endcall %}\n\n {% set should_revoke = should_revoke(target_relation_exists, full_refresh_mode=False) %}\n {% do apply_grants(target_relation, grant_config, should_revoke=should_revoke) %}\n\n {% do persist_docs(target_relation, model) %}\n\n {% if not target_relation_exists %}\n {% do create_indexes(target_relation) %}\n {% endif %}\n\n {{ run_hooks(post_hooks, inside_transaction=True) }}\n\n {{ adapter.commit() }}\n\n {% if staging_table is defined %}\n {% do post_snapshot(staging_table) %}\n {% endif %}\n\n {{ run_hooks(post_hooks, inside_transaction=False) }}\n\n {{ return({'relations': [target_relation]}) }}\n\n{% endmaterialization %}", + "meta": {}, + "name": "materialization_snapshot_default", + "original_file_path": "macros/materializations/snapshots/snapshot.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/snapshots/snapshot.sql", + "resource_type": "macro", + "supported_languages": [ + "sql" + ], + "unique_id": "macro.dbt.materialization_snapshot_default" + }, + "macro.dbt.materialization_table_default": { + "arguments": [], + "created_at": 1687942822.952439, + "depends_on": { + "macros": [ + "macro.dbt.load_cached_relation", + "macro.dbt.make_intermediate_relation", + "macro.dbt.make_backup_relation", + "macro.dbt.drop_relation_if_exists", + "macro.dbt.run_hooks", + "macro.dbt.statement", + "macro.dbt.get_create_table_as_sql", + "macro.dbt.create_indexes", + "macro.dbt.should_revoke", + "macro.dbt.apply_grants", + "macro.dbt.persist_docs" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% materialization table, default %}\n\n {%- set existing_relation = load_cached_relation(this) -%}\n {%- set target_relation = this.incorporate(type='table') %}\n {%- set intermediate_relation = make_intermediate_relation(target_relation) -%}\n -- the intermediate_relation should not already exist in the database; get_relation\n -- will return None in that case. Otherwise, we get a relation that we can drop\n -- later, before we try to use this name for the current operation\n {%- set preexisting_intermediate_relation = load_cached_relation(intermediate_relation) -%}\n /*\n See ../view/view.sql for more information about this relation.\n */\n {%- set backup_relation_type = 'table' if existing_relation is none else existing_relation.type -%}\n {%- set backup_relation = make_backup_relation(target_relation, backup_relation_type) -%}\n -- as above, the backup_relation should not already exist\n {%- set preexisting_backup_relation = load_cached_relation(backup_relation) -%}\n -- grab current tables grants config for comparison later on\n {% set grant_config = config.get('grants') %}\n\n -- drop the temp relations if they exist already in the database\n {{ drop_relation_if_exists(preexisting_intermediate_relation) }}\n {{ drop_relation_if_exists(preexisting_backup_relation) }}\n\n {{ run_hooks(pre_hooks, inside_transaction=False) }}\n\n -- `BEGIN` happens here:\n {{ run_hooks(pre_hooks, inside_transaction=True) }}\n\n -- build model\n {% call statement('main') -%}\n {{ get_create_table_as_sql(False, intermediate_relation, sql) }}\n {%- endcall %}\n\n -- cleanup\n {% if existing_relation is not none %}\n {{ adapter.rename_relation(existing_relation, backup_relation) }}\n {% endif %}\n\n {{ adapter.rename_relation(intermediate_relation, target_relation) }}\n\n {% do create_indexes(target_relation) %}\n\n {{ run_hooks(post_hooks, inside_transaction=True) }}\n\n {% set should_revoke = should_revoke(existing_relation, full_refresh_mode=True) %}\n {% do apply_grants(target_relation, grant_config, should_revoke=should_revoke) %}\n\n {% do persist_docs(target_relation, model) %}\n\n -- `COMMIT` happens here\n {{ adapter.commit() }}\n\n -- finally, drop the existing/backup relation after the commit\n {{ drop_relation_if_exists(backup_relation) }}\n\n {{ run_hooks(post_hooks, inside_transaction=False) }}\n\n {{ return({'relations': [target_relation]}) }}\n{% endmaterialization %}", + "meta": {}, + "name": "materialization_table_default", + "original_file_path": "macros/materializations/models/table/table.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/table/table.sql", + "resource_type": "macro", + "supported_languages": [ + "sql" + ], + "unique_id": "macro.dbt.materialization_table_default" + }, + "macro.dbt.materialization_test_default": { + "arguments": [], + "created_at": 1687942822.903701, + "depends_on": { + "macros": [ + "macro.dbt.should_store_failures", + "macro.dbt.statement", + "macro.dbt.create_table_as", + "macro.dbt.get_test_sql" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{%- materialization test, default -%}\n\n {% set relations = [] %}\n\n {% if should_store_failures() %}\n\n {% set identifier = model['alias'] %}\n {% set old_relation = adapter.get_relation(database=database, schema=schema, identifier=identifier) %}\n {% set target_relation = api.Relation.create(\n identifier=identifier, schema=schema, database=database, type='table') -%} %}\n\n {% if old_relation %}\n {% do adapter.drop_relation(old_relation) %}\n {% endif %}\n\n {% call statement(auto_begin=True) %}\n {{ create_table_as(False, target_relation, sql) }}\n {% endcall %}\n\n {% do relations.append(target_relation) %}\n\n {% set main_sql %}\n select *\n from {{ target_relation }}\n {% endset %}\n\n {{ adapter.commit() }}\n\n {% else %}\n\n {% set main_sql = sql %}\n\n {% endif %}\n\n {% set limit = config.get('limit') %}\n {% set fail_calc = config.get('fail_calc') %}\n {% set warn_if = config.get('warn_if') %}\n {% set error_if = config.get('error_if') %}\n\n {% call statement('main', fetch_result=True) -%}\n\n {{ get_test_sql(main_sql, fail_calc, warn_if, error_if, limit)}}\n\n {%- endcall %}\n\n {{ return({'relations': relations}) }}\n\n{%- endmaterialization -%}", + "meta": {}, + "name": "materialization_test_default", + "original_file_path": "macros/materializations/tests/test.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/tests/test.sql", + "resource_type": "macro", + "supported_languages": [ + "sql" + ], + "unique_id": "macro.dbt.materialization_test_default" + }, + "macro.dbt.materialization_view_default": { + "arguments": [], + "created_at": 1687942822.9589398, + "depends_on": { + "macros": [ + "macro.dbt.load_cached_relation", + "macro.dbt.make_intermediate_relation", + "macro.dbt.make_backup_relation", + "macro.dbt.run_hooks", + "macro.dbt.drop_relation_if_exists", + "macro.dbt.statement", + "macro.dbt.get_create_view_as_sql", + "macro.dbt.should_revoke", + "macro.dbt.apply_grants", + "macro.dbt.persist_docs" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{%- materialization view, default -%}\n\n {%- set existing_relation = load_cached_relation(this) -%}\n {%- set target_relation = this.incorporate(type='view') -%}\n {%- set intermediate_relation = make_intermediate_relation(target_relation) -%}\n\n -- the intermediate_relation should not already exist in the database; get_relation\n -- will return None in that case. Otherwise, we get a relation that we can drop\n -- later, before we try to use this name for the current operation\n {%- set preexisting_intermediate_relation = load_cached_relation(intermediate_relation) -%}\n /*\n This relation (probably) doesn't exist yet. If it does exist, it's a leftover from\n a previous run, and we're going to try to drop it immediately. At the end of this\n materialization, we're going to rename the \"existing_relation\" to this identifier,\n and then we're going to drop it. In order to make sure we run the correct one of:\n - drop view ...\n - drop table ...\n\n We need to set the type of this relation to be the type of the existing_relation, if it exists,\n or else \"view\" as a sane default if it does not. Note that if the existing_relation does not\n exist, then there is nothing to move out of the way and subsequentally drop. In that case,\n this relation will be effectively unused.\n */\n {%- set backup_relation_type = 'view' if existing_relation is none else existing_relation.type -%}\n {%- set backup_relation = make_backup_relation(target_relation, backup_relation_type) -%}\n -- as above, the backup_relation should not already exist\n {%- set preexisting_backup_relation = load_cached_relation(backup_relation) -%}\n -- grab current tables grants config for comparison later on\n {% set grant_config = config.get('grants') %}\n\n {{ run_hooks(pre_hooks, inside_transaction=False) }}\n\n -- drop the temp relations if they exist already in the database\n {{ drop_relation_if_exists(preexisting_intermediate_relation) }}\n {{ drop_relation_if_exists(preexisting_backup_relation) }}\n\n -- `BEGIN` happens here:\n {{ run_hooks(pre_hooks, inside_transaction=True) }}\n\n -- build model\n {% call statement('main') -%}\n {{ get_create_view_as_sql(intermediate_relation, sql) }}\n {%- endcall %}\n\n -- cleanup\n -- move the existing view out of the way\n {% if existing_relation is not none %}\n {{ adapter.rename_relation(existing_relation, backup_relation) }}\n {% endif %}\n {{ adapter.rename_relation(intermediate_relation, target_relation) }}\n\n {% set should_revoke = should_revoke(existing_relation, full_refresh_mode=True) %}\n {% do apply_grants(target_relation, grant_config, should_revoke=should_revoke) %}\n\n {% do persist_docs(target_relation, model) %}\n\n {{ run_hooks(post_hooks, inside_transaction=True) }}\n\n {{ adapter.commit() }}\n\n {{ drop_relation_if_exists(backup_relation) }}\n\n {{ run_hooks(post_hooks, inside_transaction=False) }}\n\n {{ return({'relations': [target_relation]}) }}\n\n{%- endmaterialization -%}", + "meta": {}, + "name": "materialization_view_default", + "original_file_path": "macros/materializations/models/view/view.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/view/view.sql", + "resource_type": "macro", + "supported_languages": [ + "sql" + ], + "unique_id": "macro.dbt.materialization_view_default" + }, + "macro.dbt.noop_statement": { + "arguments": [], + "created_at": 1687942822.984918, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro noop_statement(name=None, message=None, code=None, rows_affected=None, res=None) -%}\n {%- set sql = caller() -%}\n\n {%- if name == 'main' -%}\n {{ log('Writing runtime SQL for node \"{}\"'.format(model['unique_id'])) }}\n {{ write(sql) }}\n {%- endif -%}\n\n {%- if name is not none -%}\n {{ store_raw_result(name, message=message, code=code, rows_affected=rows_affected, agate_table=res) }}\n {%- endif -%}\n\n{%- endmacro %}", + "meta": {}, + "name": "noop_statement", + "original_file_path": "macros/etc/statement.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/etc/statement.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.noop_statement" + }, + "macro.dbt.partition_range": { + "arguments": [], + "created_at": 1687942822.989682, + "depends_on": { + "macros": [ + "macro.dbt.dates_in_range" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro partition_range(raw_partition_date, date_fmt='%Y%m%d') %}\n {% set partition_range = (raw_partition_date | string).split(\",\") %}\n\n {% if (partition_range | length) == 1 %}\n {% set start_date = partition_range[0] %}\n {% set end_date = none %}\n {% elif (partition_range | length) == 2 %}\n {% set start_date = partition_range[0] %}\n {% set end_date = partition_range[1] %}\n {% else %}\n {{ exceptions.raise_compiler_error(\"Invalid partition time. Expected format: {Start Date}[,{End Date}]. Got: \" ~ raw_partition_date) }}\n {% endif %}\n\n {{ return(dates_in_range(start_date, end_date, in_fmt=date_fmt)) }}\n{% endmacro %}", + "meta": {}, + "name": "partition_range", + "original_file_path": "macros/etc/datetime.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/etc/datetime.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.partition_range" + }, + "macro.dbt.persist_docs": { + "arguments": [], + "created_at": 1687942823.029068, + "depends_on": { + "macros": [ + "macro.dbt.default__persist_docs" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro persist_docs(relation, model, for_relation=true, for_columns=true) -%}\n {{ return(adapter.dispatch('persist_docs', 'dbt')(relation, model, for_relation, for_columns)) }}\n{% endmacro %}", + "meta": {}, + "name": "persist_docs", + "original_file_path": "macros/adapters/persist_docs.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/persist_docs.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.persist_docs" + }, + "macro.dbt.position": { + "arguments": [], + "created_at": 1687942822.998125, + "depends_on": { + "macros": [ + "macro.dbt.default__position" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro position(substring_text, string_text) -%}\n {{ return(adapter.dispatch('position', 'dbt') (substring_text, string_text)) }}\n{% endmacro %}", + "meta": {}, + "name": "position", + "original_file_path": "macros/utils/position.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/position.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.position" + }, + "macro.dbt.post_snapshot": { + "arguments": [], + "created_at": 1687942822.889555, + "depends_on": { + "macros": [ + "macro.dbt.default__post_snapshot" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro post_snapshot(staging_relation) %}\n {{ adapter.dispatch('post_snapshot', 'dbt')(staging_relation) }}\n{% endmacro %}", + "meta": {}, + "name": "post_snapshot", + "original_file_path": "macros/materializations/snapshots/helpers.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/snapshots/helpers.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.post_snapshot" + }, + "macro.dbt.process_schema_changes": { + "arguments": [], + "created_at": 1687942822.9446778, + "depends_on": { + "macros": [ + "macro.dbt.check_for_schema_changes", + "macro.dbt.sync_column_schemas" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro process_schema_changes(on_schema_change, source_relation, target_relation) %}\n\n {% if on_schema_change == 'ignore' %}\n\n {{ return({}) }}\n\n {% else %}\n\n {% set schema_changes_dict = check_for_schema_changes(source_relation, target_relation) %}\n\n {% if schema_changes_dict['schema_changed'] %}\n\n {% if on_schema_change == 'fail' %}\n\n {% set fail_msg %}\n The source and target schemas on this incremental model are out of sync!\n They can be reconciled in several ways:\n - set the `on_schema_change` config to either append_new_columns or sync_all_columns, depending on your situation.\n - Re-run the incremental model with `full_refresh: True` to update the target schema.\n - update the schema manually and re-run the process.\n\n Additional troubleshooting context:\n Source columns not in target: {{ schema_changes_dict['source_not_in_target'] }}\n Target columns not in source: {{ schema_changes_dict['target_not_in_source'] }}\n New column types: {{ schema_changes_dict['new_target_types'] }}\n {% endset %}\n\n {% do exceptions.raise_compiler_error(fail_msg) %}\n\n {# -- unless we ignore, run the sync operation per the config #}\n {% else %}\n\n {% do sync_column_schemas(on_schema_change, target_relation, schema_changes_dict) %}\n\n {% endif %}\n\n {% endif %}\n\n {{ return(schema_changes_dict['source_columns']) }}\n\n {% endif %}\n\n{% endmacro %}", + "meta": {}, + "name": "process_schema_changes", + "original_file_path": "macros/materializations/models/incremental/on_schema_change.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/incremental/on_schema_change.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.process_schema_changes" + }, + "macro.dbt.py_current_timestring": { + "arguments": [], + "created_at": 1687942822.989971, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro py_current_timestring() %}\n {% set dt = modules.datetime.datetime.now() %}\n {% do return(dt.strftime(\"%Y%m%d%H%M%S%f\")) %}\n{% endmacro %}", + "meta": {}, + "name": "py_current_timestring", + "original_file_path": "macros/etc/datetime.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/etc/datetime.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.py_current_timestring" + }, + "macro.dbt.py_script_comment": { + "arguments": [], + "created_at": 1687942823.046462, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{%macro py_script_comment()%}\n{%endmacro%}", + "meta": {}, + "name": "py_script_comment", + "original_file_path": "macros/python_model/python.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/python_model/python.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.py_script_comment" + }, + "macro.dbt.py_script_postfix": { + "arguments": [], + "created_at": 1687942823.04638, + "depends_on": { + "macros": [ + "macro.dbt.build_ref_function", + "macro.dbt.build_source_function", + "macro.dbt.build_config_dict", + "macro.dbt.resolve_model_name", + "macro.dbt.is_incremental", + "macro.dbt.py_script_comment" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro py_script_postfix(model) %}\n# This part is user provided model code\n# you will need to copy the next section to run the code\n# COMMAND ----------\n# this part is dbt logic for get ref work, do not modify\n\n{{ build_ref_function(model ) }}\n{{ build_source_function(model ) }}\n{{ build_config_dict(model) }}\n\nclass config:\n def __init__(self, *args, **kwargs):\n pass\n\n @staticmethod\n def get(key, default=None):\n return config_dict.get(key, default)\n\nclass this:\n \"\"\"dbt.this() or dbt.this.identifier\"\"\"\n database = \"{{ this.database }}\"\n schema = \"{{ this.schema }}\"\n identifier = \"{{ this.identifier }}\"\n {% set this_relation_name = resolve_model_name(this) %}\n def __repr__(self):\n return '{{ this_relation_name }}'\n\n\nclass dbtObj:\n def __init__(self, load_df_function) -> None:\n self.source = lambda *args: source(*args, dbt_load_df_function=load_df_function)\n self.ref = lambda *args, **kwargs: ref(*args, **kwargs, dbt_load_df_function=load_df_function)\n self.config = config\n self.this = this()\n self.is_incremental = {{ is_incremental() }}\n\n# COMMAND ----------\n{{py_script_comment()}}\n{% endmacro %}", + "meta": {}, + "name": "py_script_postfix", + "original_file_path": "macros/python_model/python.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/python_model/python.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.py_script_postfix" + }, + "macro.dbt.rename_relation": { + "arguments": [], + "created_at": 1687942823.017083, + "depends_on": { + "macros": [ + "macro.dbt.default__rename_relation" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro rename_relation(from_relation, to_relation) -%}\n {{ return(adapter.dispatch('rename_relation', 'dbt')(from_relation, to_relation)) }}\n{% endmacro %}", + "meta": {}, + "name": "rename_relation", + "original_file_path": "macros/adapters/relation.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/relation.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.rename_relation" + }, + "macro.dbt.replace": { + "arguments": [], + "created_at": 1687942822.990763, + "depends_on": { + "macros": [ + "macro.dbt.default__replace" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro replace(field, old_chars, new_chars) -%}\n {{ return(adapter.dispatch('replace', 'dbt') (field, old_chars, new_chars)) }}\n{% endmacro %}", + "meta": {}, + "name": "replace", + "original_file_path": "macros/utils/replace.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/replace.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.replace" + }, + "macro.dbt.reset_csv_table": { + "arguments": [], + "created_at": 1687942822.974001, + "depends_on": { + "macros": [ + "macro.dbt.default__reset_csv_table" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro reset_csv_table(model, full_refresh, old_relation, agate_table) -%}\n {{ adapter.dispatch('reset_csv_table', 'dbt')(model, full_refresh, old_relation, agate_table) }}\n{%- endmacro %}", + "meta": {}, + "name": "reset_csv_table", + "original_file_path": "macros/materializations/seeds/helpers.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/seeds/helpers.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.reset_csv_table" + }, + "macro.dbt.resolve_model_name": { + "arguments": [], + "created_at": 1687942823.043636, + "depends_on": { + "macros": [ + "macro.dbt.default__resolve_model_name" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro resolve_model_name(input_model_name) %}\n {{ return(adapter.dispatch('resolve_model_name', 'dbt')(input_model_name)) }}\n{% endmacro %}", + "meta": {}, + "name": "resolve_model_name", + "original_file_path": "macros/python_model/python.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/python_model/python.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.resolve_model_name" + }, + "macro.dbt.right": { + "arguments": [], + "created_at": 1687942822.9938078, + "depends_on": { + "macros": [ + "macro.dbt.default__right" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro right(string_text, length_expression) -%}\n {{ return(adapter.dispatch('right', 'dbt') (string_text, length_expression)) }}\n{% endmacro %}", + "meta": {}, + "name": "right", + "original_file_path": "macros/utils/right.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/right.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.right" + }, + "macro.dbt.run_hooks": { + "arguments": [], + "created_at": 1687942822.870699, + "depends_on": { + "macros": [ + "macro.dbt.statement" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro run_hooks(hooks, inside_transaction=True) %}\n {% for hook in hooks | selectattr('transaction', 'equalto', inside_transaction) %}\n {% if not inside_transaction and loop.first %}\n {% call statement(auto_begin=inside_transaction) %}\n commit;\n {% endcall %}\n {% endif %}\n {% set rendered = render(hook.get('sql')) | trim %}\n {% if (rendered | length) > 0 %}\n {% call statement(auto_begin=inside_transaction) %}\n {{ rendered }}\n {% endcall %}\n {% endif %}\n {% endfor %}\n{% endmacro %}", + "meta": {}, + "name": "run_hooks", + "original_file_path": "macros/materializations/hooks.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/hooks.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.run_hooks" + }, + "macro.dbt.run_query": { + "arguments": [], + "created_at": 1687942822.985245, + "depends_on": { + "macros": [ + "macro.dbt.statement" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro run_query(sql) %}\n {% call statement(\"run_query_statement\", fetch_result=true, auto_begin=false) %}\n {{ sql }}\n {% endcall %}\n\n {% do return(load_result(\"run_query_statement\").table) %}\n{% endmacro %}", + "meta": {}, + "name": "run_query", + "original_file_path": "macros/etc/statement.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/etc/statement.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.run_query" + }, + "macro.dbt.safe_cast": { + "arguments": [], + "created_at": 1687942822.996167, + "depends_on": { + "macros": [ + "macro.dbt.default__safe_cast" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro safe_cast(field, type) %}\n {{ return(adapter.dispatch('safe_cast', 'dbt') (field, type)) }}\n{% endmacro %}", + "meta": {}, + "name": "safe_cast", + "original_file_path": "macros/utils/safe_cast.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/safe_cast.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.safe_cast" + }, + "macro.dbt.set_sql_header": { + "arguments": [], + "created_at": 1687942822.871808, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro set_sql_header(config) -%}\n {{ config.set('sql_header', caller()) }}\n{%- endmacro %}", + "meta": {}, + "name": "set_sql_header", + "original_file_path": "macros/materializations/configs.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/configs.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.set_sql_header" + }, + "macro.dbt.should_full_refresh": { + "arguments": [], + "created_at": 1687942822.8721411, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro should_full_refresh() %}\n {% set config_full_refresh = config.get('full_refresh') %}\n {% if config_full_refresh is none %}\n {% set config_full_refresh = flags.FULL_REFRESH %}\n {% endif %}\n {% do return(config_full_refresh) %}\n{% endmacro %}", + "meta": {}, + "name": "should_full_refresh", + "original_file_path": "macros/materializations/configs.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/configs.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.should_full_refresh" + }, + "macro.dbt.should_revoke": { + "arguments": [], + "created_at": 1687942823.022611, + "depends_on": { + "macros": [ + "macro.dbt.copy_grants" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro should_revoke(existing_relation, full_refresh_mode=True) %}\n\n {% if not existing_relation %}\n {#-- The table doesn't already exist, so no grants to copy over --#}\n {{ return(False) }}\n {% elif full_refresh_mode %}\n {#-- The object is being REPLACED -- whether grants are copied over depends on the value of user config --#}\n {{ return(copy_grants()) }}\n {% else %}\n {#-- The table is being merged/upserted/inserted -- grants will be carried over --#}\n {{ return(True) }}\n {% endif %}\n\n{% endmacro %}", + "meta": {}, + "name": "should_revoke", + "original_file_path": "macros/adapters/apply_grants.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/apply_grants.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.should_revoke" + }, + "macro.dbt.should_store_failures": { + "arguments": [], + "created_at": 1687942822.872481, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro should_store_failures() %}\n {% set config_store_failures = config.get('store_failures') %}\n {% if config_store_failures is none %}\n {% set config_store_failures = flags.STORE_FAILURES %}\n {% endif %}\n {% do return(config_store_failures) %}\n{% endmacro %}", + "meta": {}, + "name": "should_store_failures", + "original_file_path": "macros/materializations/configs.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/configs.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.should_store_failures" + }, + "macro.dbt.snapshot_check_all_get_existing_columns": { + "arguments": [], + "created_at": 1687942822.881519, + "depends_on": { + "macros": [ + "macro.dbt.get_columns_in_query" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro snapshot_check_all_get_existing_columns(node, target_exists, check_cols_config) -%}\n {%- if not target_exists -%}\n {#-- no table yet -> return whatever the query does --#}\n {{ return((false, query_columns)) }}\n {%- endif -%}\n\n {#-- handle any schema changes --#}\n {%- set target_relation = adapter.get_relation(database=node.database, schema=node.schema, identifier=node.alias) -%}\n\n {% if check_cols_config == 'all' %}\n {%- set query_columns = get_columns_in_query(node['compiled_code']) -%}\n\n {% elif check_cols_config is iterable and (check_cols_config | length) > 0 %}\n {#-- query for proper casing/quoting, to support comparison below --#}\n {%- set select_check_cols_from_target -%}\n {#-- N.B. The whitespace below is necessary to avoid edge case issue with comments --#}\n {#-- See: https://github.com/dbt-labs/dbt-core/issues/6781 --#}\n select {{ check_cols_config | join(', ') }} from (\n {{ node['compiled_code'] }}\n ) subq\n {%- endset -%}\n {% set query_columns = get_columns_in_query(select_check_cols_from_target) %}\n\n {% else %}\n {% do exceptions.raise_compiler_error(\"Invalid value for 'check_cols': \" ~ check_cols_config) %}\n {% endif %}\n\n {%- set existing_cols = adapter.get_columns_in_relation(target_relation) | map(attribute = 'name') | list -%}\n {%- set ns = namespace() -%} {#-- handle for-loop scoping with a namespace --#}\n {%- set ns.column_added = false -%}\n\n {%- set intersection = [] -%}\n {%- for col in query_columns -%}\n {%- if col in existing_cols -%}\n {%- do intersection.append(adapter.quote(col)) -%}\n {%- else -%}\n {% set ns.column_added = true %}\n {%- endif -%}\n {%- endfor -%}\n {{ return((ns.column_added, intersection)) }}\n{%- endmacro %}", + "meta": {}, + "name": "snapshot_check_all_get_existing_columns", + "original_file_path": "macros/materializations/snapshots/strategies.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/snapshots/strategies.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.snapshot_check_all_get_existing_columns" + }, + "macro.dbt.snapshot_check_strategy": { + "arguments": [], + "created_at": 1687942822.8830268, + "depends_on": { + "macros": [ + "macro.dbt.snapshot_get_time", + "macro.dbt.snapshot_check_all_get_existing_columns", + "macro.dbt.get_true_sql", + "macro.dbt.snapshot_hash_arguments" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro snapshot_check_strategy(node, snapshotted_rel, current_rel, config, target_exists) %}\n {% set check_cols_config = config['check_cols'] %}\n {% set primary_key = config['unique_key'] %}\n {% set invalidate_hard_deletes = config.get('invalidate_hard_deletes', false) %}\n {% set updated_at = config.get('updated_at', snapshot_get_time()) %}\n\n {% set column_added = false %}\n\n {% set column_added, check_cols = snapshot_check_all_get_existing_columns(node, target_exists, check_cols_config) %}\n\n {%- set row_changed_expr -%}\n (\n {%- if column_added -%}\n {{ get_true_sql() }}\n {%- else -%}\n {%- for col in check_cols -%}\n {{ snapshotted_rel }}.{{ col }} != {{ current_rel }}.{{ col }}\n or\n (\n (({{ snapshotted_rel }}.{{ col }} is null) and not ({{ current_rel }}.{{ col }} is null))\n or\n ((not {{ snapshotted_rel }}.{{ col }} is null) and ({{ current_rel }}.{{ col }} is null))\n )\n {%- if not loop.last %} or {% endif -%}\n {%- endfor -%}\n {%- endif -%}\n )\n {%- endset %}\n\n {% set scd_id_expr = snapshot_hash_arguments([primary_key, updated_at]) %}\n\n {% do return({\n \"unique_key\": primary_key,\n \"updated_at\": updated_at,\n \"row_changed\": row_changed_expr,\n \"scd_id\": scd_id_expr,\n \"invalidate_hard_deletes\": invalidate_hard_deletes\n }) %}\n{% endmacro %}", + "meta": {}, + "name": "snapshot_check_strategy", + "original_file_path": "macros/materializations/snapshots/strategies.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/snapshots/strategies.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.snapshot_check_strategy" + }, + "macro.dbt.snapshot_get_time": { + "arguments": [], + "created_at": 1687942823.008611, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres__snapshot_get_time" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "\n\n{%- macro snapshot_get_time() -%}\n {{ adapter.dispatch('snapshot_get_time', 'dbt')() }}\n{%- endmacro -%}\n\n", + "meta": {}, + "name": "snapshot_get_time", + "original_file_path": "macros/adapters/timestamps.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/timestamps.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.snapshot_get_time" + }, + "macro.dbt.snapshot_hash_arguments": { + "arguments": [], + "created_at": 1687942822.877908, + "depends_on": { + "macros": [ + "macro.dbt.default__snapshot_hash_arguments" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro snapshot_hash_arguments(args) -%}\n {{ adapter.dispatch('snapshot_hash_arguments', 'dbt')(args) }}\n{%- endmacro %}", + "meta": {}, + "name": "snapshot_hash_arguments", + "original_file_path": "macros/materializations/snapshots/strategies.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/snapshots/strategies.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.snapshot_hash_arguments" + }, + "macro.dbt.snapshot_merge_sql": { + "arguments": [], + "created_at": 1687942822.872962, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres__snapshot_merge_sql" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro snapshot_merge_sql(target, source, insert_cols) -%}\n {{ adapter.dispatch('snapshot_merge_sql', 'dbt')(target, source, insert_cols) }}\n{%- endmacro %}", + "meta": {}, + "name": "snapshot_merge_sql", + "original_file_path": "macros/materializations/snapshots/snapshot_merge.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/snapshots/snapshot_merge.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.snapshot_merge_sql" + }, + "macro.dbt.snapshot_staging_table": { + "arguments": [], + "created_at": 1687942822.890684, + "depends_on": { + "macros": [ + "macro.dbt.default__snapshot_staging_table" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro snapshot_staging_table(strategy, source_sql, target_relation) -%}\n {{ adapter.dispatch('snapshot_staging_table', 'dbt')(strategy, source_sql, target_relation) }}\n{% endmacro %}", + "meta": {}, + "name": "snapshot_staging_table", + "original_file_path": "macros/materializations/snapshots/helpers.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/snapshots/helpers.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.snapshot_staging_table" + }, + "macro.dbt.snapshot_string_as_time": { + "arguments": [], + "created_at": 1687942822.8794858, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres__snapshot_string_as_time" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro snapshot_string_as_time(timestamp) -%}\n {{ adapter.dispatch('snapshot_string_as_time', 'dbt')(timestamp) }}\n{%- endmacro %}", + "meta": {}, + "name": "snapshot_string_as_time", + "original_file_path": "macros/materializations/snapshots/strategies.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/snapshots/strategies.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.snapshot_string_as_time" + }, + "macro.dbt.snapshot_timestamp_strategy": { + "arguments": [], + "created_at": 1687942822.879221, + "depends_on": { + "macros": [ + "macro.dbt.snapshot_hash_arguments" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro snapshot_timestamp_strategy(node, snapshotted_rel, current_rel, config, target_exists) %}\n {% set primary_key = config['unique_key'] %}\n {% set updated_at = config['updated_at'] %}\n {% set invalidate_hard_deletes = config.get('invalidate_hard_deletes', false) %}\n\n {#/*\n The snapshot relation might not have an {{ updated_at }} value if the\n snapshot strategy is changed from `check` to `timestamp`. We\n should use a dbt-created column for the comparison in the snapshot\n table instead of assuming that the user-supplied {{ updated_at }}\n will be present in the historical data.\n\n See https://github.com/dbt-labs/dbt-core/issues/2350\n */ #}\n {% set row_changed_expr -%}\n ({{ snapshotted_rel }}.dbt_valid_from < {{ current_rel }}.{{ updated_at }})\n {%- endset %}\n\n {% set scd_id_expr = snapshot_hash_arguments([primary_key, updated_at]) %}\n\n {% do return({\n \"unique_key\": primary_key,\n \"updated_at\": updated_at,\n \"row_changed\": row_changed_expr,\n \"scd_id\": scd_id_expr,\n \"invalidate_hard_deletes\": invalidate_hard_deletes\n }) %}\n{% endmacro %}", + "meta": {}, + "name": "snapshot_timestamp_strategy", + "original_file_path": "macros/materializations/snapshots/strategies.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/snapshots/strategies.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.snapshot_timestamp_strategy" + }, + "macro.dbt.split_part": { + "arguments": [], + "created_at": 1687942823.004601, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres__split_part" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro split_part(string_text, delimiter_text, part_number) %}\n {{ return(adapter.dispatch('split_part', 'dbt') (string_text, delimiter_text, part_number)) }}\n{% endmacro %}", + "meta": {}, + "name": "split_part", + "original_file_path": "macros/utils/split_part.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/split_part.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.split_part" + }, + "macro.dbt.sql_convert_columns_in_relation": { + "arguments": [], + "created_at": 1687942823.0370119, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro sql_convert_columns_in_relation(table) -%}\n {% set columns = [] %}\n {% for row in table %}\n {% do columns.append(api.Column(*row)) %}\n {% endfor %}\n {{ return(columns) }}\n{% endmacro %}", + "meta": {}, + "name": "sql_convert_columns_in_relation", + "original_file_path": "macros/adapters/columns.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/columns.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.sql_convert_columns_in_relation" + }, + "macro.dbt.statement": { + "arguments": [], + "created_at": 1687942822.9842792, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "\n{%- macro statement(name=None, fetch_result=False, auto_begin=True, language='sql') -%}\n {%- if execute: -%}\n {%- set compiled_code = caller() -%}\n\n {%- if name == 'main' -%}\n {{ log('Writing runtime {} for node \"{}\"'.format(language, model['unique_id'])) }}\n {{ write(compiled_code) }}\n {%- endif -%}\n {%- if language == 'sql'-%}\n {%- set res, table = adapter.execute(compiled_code, auto_begin=auto_begin, fetch=fetch_result) -%}\n {%- elif language == 'python' -%}\n {%- set res = submit_python_job(model, compiled_code) -%}\n {#-- TODO: What should table be for python models? --#}\n {%- set table = None -%}\n {%- else -%}\n {% do exceptions.raise_compiler_error(\"statement macro didn't get supported language\") %}\n {%- endif -%}\n\n {%- if name is not none -%}\n {{ store_result(name, response=res, agate_table=table) }}\n {%- endif -%}\n\n {%- endif -%}\n{%- endmacro %}", + "meta": {}, + "name": "statement", + "original_file_path": "macros/etc/statement.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/etc/statement.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.statement" + }, + "macro.dbt.strategy_dispatch": { + "arguments": [], + "created_at": 1687942822.877471, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro strategy_dispatch(name) -%}\n{% set original_name = name %}\n {% if '.' in name %}\n {% set package_name, name = name.split(\".\", 1) %}\n {% else %}\n {% set package_name = none %}\n {% endif %}\n\n {% if package_name is none %}\n {% set package_context = context %}\n {% elif package_name in context %}\n {% set package_context = context[package_name] %}\n {% else %}\n {% set error_msg %}\n Could not find package '{{package_name}}', called with '{{original_name}}'\n {% endset %}\n {{ exceptions.raise_compiler_error(error_msg | trim) }}\n {% endif %}\n\n {%- set search_name = 'snapshot_' ~ name ~ '_strategy' -%}\n\n {% if search_name not in package_context %}\n {% set error_msg %}\n The specified strategy macro '{{name}}' was not found in package '{{ package_name }}'\n {% endset %}\n {{ exceptions.raise_compiler_error(error_msg | trim) }}\n {% endif %}\n {{ return(package_context[search_name]) }}\n{%- endmacro %}", + "meta": {}, + "name": "strategy_dispatch", + "original_file_path": "macros/materializations/snapshots/strategies.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/snapshots/strategies.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.strategy_dispatch" + }, + "macro.dbt.string_literal": { + "arguments": [], + "created_at": 1687942822.998741, + "depends_on": { + "macros": [ + "macro.dbt.default__string_literal" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{%- macro string_literal(value) -%}\n {{ return(adapter.dispatch('string_literal', 'dbt') (value)) }}\n{%- endmacro -%}\n\n", + "meta": {}, + "name": "string_literal", + "original_file_path": "macros/utils/literal.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/literal.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.string_literal" + }, + "macro.dbt.support_multiple_grantees_per_dcl_statement": { + "arguments": [], + "created_at": 1687942823.022024, + "depends_on": { + "macros": [ + "macro.dbt.default__support_multiple_grantees_per_dcl_statement" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro support_multiple_grantees_per_dcl_statement() %}\n {{ return(adapter.dispatch('support_multiple_grantees_per_dcl_statement', 'dbt')()) }}\n{% endmacro %}", + "meta": {}, + "name": "support_multiple_grantees_per_dcl_statement", + "original_file_path": "macros/adapters/apply_grants.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/apply_grants.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.support_multiple_grantees_per_dcl_statement" + }, + "macro.dbt.sync_column_schemas": { + "arguments": [], + "created_at": 1687942822.943712, + "depends_on": { + "macros": [ + "macro.dbt.alter_relation_add_remove_columns", + "macro.dbt.alter_column_type" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro sync_column_schemas(on_schema_change, target_relation, schema_changes_dict) %}\n\n {%- set add_to_target_arr = schema_changes_dict['source_not_in_target'] -%}\n\n {%- if on_schema_change == 'append_new_columns'-%}\n {%- if add_to_target_arr | length > 0 -%}\n {%- do alter_relation_add_remove_columns(target_relation, add_to_target_arr, none) -%}\n {%- endif -%}\n\n {% elif on_schema_change == 'sync_all_columns' %}\n {%- set remove_from_target_arr = schema_changes_dict['target_not_in_source'] -%}\n {%- set new_target_types = schema_changes_dict['new_target_types'] -%}\n\n {% if add_to_target_arr | length > 0 or remove_from_target_arr | length > 0 %}\n {%- do alter_relation_add_remove_columns(target_relation, add_to_target_arr, remove_from_target_arr) -%}\n {% endif %}\n\n {% if new_target_types != [] %}\n {% for ntt in new_target_types %}\n {% set column_name = ntt['column_name'] %}\n {% set new_type = ntt['new_type'] %}\n {% do alter_column_type(target_relation, column_name, new_type) %}\n {% endfor %}\n {% endif %}\n\n {% endif %}\n\n {% set schema_change_message %}\n In {{ target_relation }}:\n Schema change approach: {{ on_schema_change }}\n Columns added: {{ add_to_target_arr }}\n Columns removed: {{ remove_from_target_arr }}\n Data types changed: {{ new_target_types }}\n {% endset %}\n\n {% do log(schema_change_message) %}\n\n{% endmacro %}", + "meta": {}, + "name": "sync_column_schemas", + "original_file_path": "macros/materializations/models/incremental/on_schema_change.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/incremental/on_schema_change.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.sync_column_schemas" + }, + "macro.dbt.table_columns_and_constraints": { + "arguments": [], + "created_at": 1687942822.9465652, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro table_columns_and_constraints() %}\n {# loop through user_provided_columns to create DDL with data types and constraints #}\n {%- set raw_column_constraints = adapter.render_raw_columns_constraints(raw_columns=model['columns']) -%}\n {%- set raw_model_constraints = adapter.render_raw_model_constraints(raw_constraints=model['constraints']) -%}\n (\n {% for c in raw_column_constraints -%}\n {{ c }}{{ \",\" if not loop.last or raw_model_constraints }}\n {% endfor %}\n {% for c in raw_model_constraints -%}\n {{ c }}{{ \",\" if not loop.last }}\n {% endfor -%}\n )\n{% endmacro %}", + "meta": {}, + "name": "table_columns_and_constraints", + "original_file_path": "macros/materializations/models/table/columns_spec_ddl.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/materializations/models/table/columns_spec_ddl.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.table_columns_and_constraints" + }, + "macro.dbt.test_accepted_values": { + "arguments": [], + "created_at": 1687942823.047759, + "depends_on": { + "macros": [ + "macro.dbt.default__test_accepted_values" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% test accepted_values(model, column_name, values, quote=True) %}\n {% set macro = adapter.dispatch('test_accepted_values', 'dbt') %}\n {{ macro(model, column_name, values, quote) }}\n{% endtest %}", + "meta": {}, + "name": "test_accepted_values", + "original_file_path": "tests/generic/builtin.sql", + "package_name": "dbt", + "patch_path": null, + "path": "tests/generic/builtin.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.test_accepted_values" + }, + "macro.dbt.test_not_null": { + "arguments": [], + "created_at": 1687942823.047362, + "depends_on": { + "macros": [ + "macro.dbt.default__test_not_null" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% test not_null(model, column_name) %}\n {% set macro = adapter.dispatch('test_not_null', 'dbt') %}\n {{ macro(model, column_name) }}\n{% endtest %}", + "meta": {}, + "name": "test_not_null", + "original_file_path": "tests/generic/builtin.sql", + "package_name": "dbt", + "patch_path": null, + "path": "tests/generic/builtin.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.test_not_null" + }, + "macro.dbt.test_relationships": { + "arguments": [], + "created_at": 1687942823.048068, + "depends_on": { + "macros": [ + "macro.dbt.default__test_relationships" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% test relationships(model, column_name, to, field) %}\n {% set macro = adapter.dispatch('test_relationships', 'dbt') %}\n {{ macro(model, column_name, to, field) }}\n{% endtest %}", + "meta": {}, + "name": "test_relationships", + "original_file_path": "tests/generic/builtin.sql", + "package_name": "dbt", + "patch_path": null, + "path": "tests/generic/builtin.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.test_relationships" + }, + "macro.dbt.test_unique": { + "arguments": [], + "created_at": 1687942823.047106, + "depends_on": { + "macros": [ + "macro.dbt.default__test_unique" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% test unique(model, column_name) %}\n {% set macro = adapter.dispatch('test_unique', 'dbt') %}\n {{ macro(model, column_name) }}\n{% endtest %}", + "meta": {}, + "name": "test_unique", + "original_file_path": "tests/generic/builtin.sql", + "package_name": "dbt", + "patch_path": null, + "path": "tests/generic/builtin.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.test_unique" + }, + "macro.dbt.truncate_relation": { + "arguments": [], + "created_at": 1687942823.0166788, + "depends_on": { + "macros": [ + "macro.dbt.default__truncate_relation" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro truncate_relation(relation) -%}\n {{ return(adapter.dispatch('truncate_relation', 'dbt')(relation)) }}\n{% endmacro %}", + "meta": {}, + "name": "truncate_relation", + "original_file_path": "macros/adapters/relation.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/adapters/relation.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.truncate_relation" + }, + "macro.dbt.type_bigint": { + "arguments": [], + "created_at": 1687942823.001445, + "depends_on": { + "macros": [ + "macro.dbt.default__type_bigint" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "\n\n{%- macro type_bigint() -%}\n {{ return(adapter.dispatch('type_bigint', 'dbt')()) }}\n{%- endmacro -%}\n\n", + "meta": {}, + "name": "type_bigint", + "original_file_path": "macros/utils/data_types.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/data_types.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.type_bigint" + }, + "macro.dbt.type_boolean": { + "arguments": [], + "created_at": 1687942823.002202, + "depends_on": { + "macros": [ + "macro.dbt.default__type_boolean" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "\n\n{%- macro type_boolean() -%}\n {{ return(adapter.dispatch('type_boolean', 'dbt')()) }}\n{%- endmacro -%}\n\n", + "meta": {}, + "name": "type_boolean", + "original_file_path": "macros/utils/data_types.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/data_types.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.type_boolean" + }, + "macro.dbt.type_float": { + "arguments": [], + "created_at": 1687942823.000724, + "depends_on": { + "macros": [ + "macro.dbt.default__type_float" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "\n\n{%- macro type_float() -%}\n {{ return(adapter.dispatch('type_float', 'dbt')()) }}\n{%- endmacro -%}\n\n", + "meta": {}, + "name": "type_float", + "original_file_path": "macros/utils/data_types.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/data_types.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.type_float" + }, + "macro.dbt.type_int": { + "arguments": [], + "created_at": 1687942823.001866, + "depends_on": { + "macros": [ + "macro.dbt.default__type_int" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "\n\n{%- macro type_int() -%}\n {{ return(adapter.dispatch('type_int', 'dbt')()) }}\n{%- endmacro -%}\n\n", + "meta": {}, + "name": "type_int", + "original_file_path": "macros/utils/data_types.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/data_types.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.type_int" + }, + "macro.dbt.type_numeric": { + "arguments": [], + "created_at": 1687942823.0010638, + "depends_on": { + "macros": [ + "macro.dbt.default__type_numeric" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "\n\n{%- macro type_numeric() -%}\n {{ return(adapter.dispatch('type_numeric', 'dbt')()) }}\n{%- endmacro -%}\n\n", + "meta": {}, + "name": "type_numeric", + "original_file_path": "macros/utils/data_types.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/data_types.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.type_numeric" + }, + "macro.dbt.type_string": { + "arguments": [], + "created_at": 1687942823.00002, + "depends_on": { + "macros": [ + "macro.dbt.default__type_string" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "\n\n{%- macro type_string() -%}\n {{ return(adapter.dispatch('type_string', 'dbt')()) }}\n{%- endmacro -%}\n\n", + "meta": {}, + "name": "type_string", + "original_file_path": "macros/utils/data_types.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/data_types.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.type_string" + }, + "macro.dbt.type_timestamp": { + "arguments": [], + "created_at": 1687942823.000371, + "depends_on": { + "macros": [ + "macro.dbt.default__type_timestamp" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "\n\n{%- macro type_timestamp() -%}\n {{ return(adapter.dispatch('type_timestamp', 'dbt')()) }}\n{%- endmacro -%}\n\n", + "meta": {}, + "name": "type_timestamp", + "original_file_path": "macros/utils/data_types.sql", + "package_name": "dbt", + "patch_path": null, + "path": "macros/utils/data_types.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt.type_timestamp" + }, + "macro.dbt_postgres.postgres__alter_column_comment": { + "arguments": [], + "created_at": 1687942822.8619561, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres_escape_comment" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro postgres__alter_column_comment(relation, column_dict) %}\n {% set existing_columns = adapter.get_columns_in_relation(relation) | map(attribute=\"name\") | list %}\n {% for column_name in column_dict if (column_name in existing_columns) %}\n {% set comment = column_dict[column_name]['description'] %}\n {% set escaped_comment = postgres_escape_comment(comment) %}\n comment on column {{ relation }}.{{ adapter.quote(column_name) if column_dict[column_name]['quote'] else column_name }} is {{ escaped_comment }};\n {% endfor %}\n{% endmacro %}", + "meta": {}, + "name": "postgres__alter_column_comment", + "original_file_path": "macros/adapters.sql", + "package_name": "dbt_postgres", + "patch_path": null, + "path": "macros/adapters.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt_postgres.postgres__alter_column_comment" + }, + "macro.dbt_postgres.postgres__alter_relation_comment": { + "arguments": [], + "created_at": 1687942822.861244, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres_escape_comment" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro postgres__alter_relation_comment(relation, comment) %}\n {% set escaped_comment = postgres_escape_comment(comment) %}\n comment on {{ relation.type }} {{ relation }} is {{ escaped_comment }};\n{% endmacro %}", + "meta": {}, + "name": "postgres__alter_relation_comment", + "original_file_path": "macros/adapters.sql", + "package_name": "dbt_postgres", + "patch_path": null, + "path": "macros/adapters.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt_postgres.postgres__alter_relation_comment" + }, + "macro.dbt_postgres.postgres__any_value": { + "arguments": [], + "created_at": 1687942822.868335, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro postgres__any_value(expression) -%}\n\n min({{ expression }})\n\n{%- endmacro %}", + "meta": {}, + "name": "postgres__any_value", + "original_file_path": "macros/utils/any_value.sql", + "package_name": "dbt_postgres", + "patch_path": null, + "path": "macros/utils/any_value.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt_postgres.postgres__any_value" + }, + "macro.dbt_postgres.postgres__check_schema_exists": { + "arguments": [], + "created_at": 1687942822.857978, + "depends_on": { + "macros": [ + "macro.dbt.statement" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro postgres__check_schema_exists(information_schema, schema) -%}\n {% if information_schema.database -%}\n {{ adapter.verify_database(information_schema.database) }}\n {%- endif -%}\n {% call statement('check_schema_exists', fetch_result=True, auto_begin=False) %}\n select count(*) from pg_namespace where nspname = '{{ schema }}'\n {% endcall %}\n {{ return(load_result('check_schema_exists').table) }}\n{% endmacro %}", + "meta": {}, + "name": "postgres__check_schema_exists", + "original_file_path": "macros/adapters.sql", + "package_name": "dbt_postgres", + "patch_path": null, + "path": "macros/adapters.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt_postgres.postgres__check_schema_exists" + }, + "macro.dbt_postgres.postgres__copy_grants": { + "arguments": [], + "created_at": 1687942822.862329, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro postgres__copy_grants() %}\n {{ return(False) }}\n{% endmacro %}", + "meta": {}, + "name": "postgres__copy_grants", + "original_file_path": "macros/adapters.sql", + "package_name": "dbt_postgres", + "patch_path": null, + "path": "macros/adapters.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt_postgres.postgres__copy_grants" + }, + "macro.dbt_postgres.postgres__create_schema": { + "arguments": [], + "created_at": 1687942822.8555672, + "depends_on": { + "macros": [ + "macro.dbt.statement" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro postgres__create_schema(relation) -%}\n {% if relation.database -%}\n {{ adapter.verify_database(relation.database) }}\n {%- endif -%}\n {%- call statement('create_schema') -%}\n create schema if not exists {{ relation.without_identifier().include(database=False) }}\n {%- endcall -%}\n{% endmacro %}", + "meta": {}, + "name": "postgres__create_schema", + "original_file_path": "macros/adapters.sql", + "package_name": "dbt_postgres", + "patch_path": null, + "path": "macros/adapters.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt_postgres.postgres__create_schema" + }, + "macro.dbt_postgres.postgres__create_table_as": { + "arguments": [], + "created_at": 1687942822.854578, + "depends_on": { + "macros": [ + "macro.dbt.get_assert_columns_equivalent", + "macro.dbt.get_table_columns_and_constraints", + "macro.dbt.default__get_column_names", + "macro.dbt.get_select_subquery" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro postgres__create_table_as(temporary, relation, sql) -%}\n {%- set unlogged = config.get('unlogged', default=false) -%}\n {%- set sql_header = config.get('sql_header', none) -%}\n\n {{ sql_header if sql_header is not none }}\n\n create {% if temporary -%}\n temporary\n {%- elif unlogged -%}\n unlogged\n {%- endif %} table {{ relation }}\n {% set contract_config = config.get('contract') %}\n {% if contract_config.enforced %}\n {{ get_assert_columns_equivalent(sql) }}\n {{ get_table_columns_and_constraints() }} ;\n insert into {{ relation }} (\n {{ adapter.dispatch('get_column_names', 'dbt')() }}\n )\n {%- set sql = get_select_subquery(sql) %}\n {% else %}\n as\n {% endif %}\n (\n {{ sql }}\n );\n{%- endmacro %}", + "meta": {}, + "name": "postgres__create_table_as", + "original_file_path": "macros/adapters.sql", + "package_name": "dbt_postgres", + "patch_path": null, + "path": "macros/adapters.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt_postgres.postgres__create_table_as" + }, + "macro.dbt_postgres.postgres__current_timestamp": { + "arguments": [], + "created_at": 1687942822.8427362, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro postgres__current_timestamp() -%}\n now()\n{%- endmacro %}", + "meta": {}, + "name": "postgres__current_timestamp", + "original_file_path": "macros/timestamps.sql", + "package_name": "dbt_postgres", + "patch_path": null, + "path": "macros/timestamps.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt_postgres.postgres__current_timestamp" + }, + "macro.dbt_postgres.postgres__current_timestamp_backcompat": { + "arguments": [], + "created_at": 1687942822.843909, + "depends_on": { + "macros": [ + "macro.dbt.type_timestamp" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro postgres__current_timestamp_backcompat() %}\n current_timestamp::{{ type_timestamp() }}\n{% endmacro %}", + "meta": {}, + "name": "postgres__current_timestamp_backcompat", + "original_file_path": "macros/timestamps.sql", + "package_name": "dbt_postgres", + "patch_path": null, + "path": "macros/timestamps.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt_postgres.postgres__current_timestamp_backcompat" + }, + "macro.dbt_postgres.postgres__current_timestamp_in_utc_backcompat": { + "arguments": [], + "created_at": 1687942822.8440351, + "depends_on": { + "macros": [ + "macro.dbt.type_timestamp" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro postgres__current_timestamp_in_utc_backcompat() %}\n (current_timestamp at time zone 'utc')::{{ type_timestamp() }}\n{% endmacro %}", + "meta": {}, + "name": "postgres__current_timestamp_in_utc_backcompat", + "original_file_path": "macros/timestamps.sql", + "package_name": "dbt_postgres", + "patch_path": null, + "path": "macros/timestamps.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt_postgres.postgres__current_timestamp_in_utc_backcompat" + }, + "macro.dbt_postgres.postgres__dateadd": { + "arguments": [], + "created_at": 1687942822.863829, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro postgres__dateadd(datepart, interval, from_date_or_timestamp) %}\n\n {{ from_date_or_timestamp }} + ((interval '1 {{ datepart }}') * ({{ interval }}))\n\n{% endmacro %}", + "meta": {}, + "name": "postgres__dateadd", + "original_file_path": "macros/utils/dateadd.sql", + "package_name": "dbt_postgres", + "patch_path": null, + "path": "macros/utils/dateadd.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt_postgres.postgres__dateadd" + }, + "macro.dbt_postgres.postgres__datediff": { + "arguments": [], + "created_at": 1687942822.868163, + "depends_on": { + "macros": [ + "macro.dbt.datediff" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro postgres__datediff(first_date, second_date, datepart) -%}\n\n {% if datepart == 'year' %}\n (date_part('year', ({{second_date}})::date) - date_part('year', ({{first_date}})::date))\n {% elif datepart == 'quarter' %}\n ({{ datediff(first_date, second_date, 'year') }} * 4 + date_part('quarter', ({{second_date}})::date) - date_part('quarter', ({{first_date}})::date))\n {% elif datepart == 'month' %}\n ({{ datediff(first_date, second_date, 'year') }} * 12 + date_part('month', ({{second_date}})::date) - date_part('month', ({{first_date}})::date))\n {% elif datepart == 'day' %}\n (({{second_date}})::date - ({{first_date}})::date)\n {% elif datepart == 'week' %}\n ({{ datediff(first_date, second_date, 'day') }} / 7 + case\n when date_part('dow', ({{first_date}})::timestamp) <= date_part('dow', ({{second_date}})::timestamp) then\n case when {{first_date}} <= {{second_date}} then 0 else -1 end\n else\n case when {{first_date}} <= {{second_date}} then 1 else 0 end\n end)\n {% elif datepart == 'hour' %}\n ({{ datediff(first_date, second_date, 'day') }} * 24 + date_part('hour', ({{second_date}})::timestamp) - date_part('hour', ({{first_date}})::timestamp))\n {% elif datepart == 'minute' %}\n ({{ datediff(first_date, second_date, 'hour') }} * 60 + date_part('minute', ({{second_date}})::timestamp) - date_part('minute', ({{first_date}})::timestamp))\n {% elif datepart == 'second' %}\n ({{ datediff(first_date, second_date, 'minute') }} * 60 + floor(date_part('second', ({{second_date}})::timestamp)) - floor(date_part('second', ({{first_date}})::timestamp)))\n {% elif datepart == 'millisecond' %}\n ({{ datediff(first_date, second_date, 'minute') }} * 60000 + floor(date_part('millisecond', ({{second_date}})::timestamp)) - floor(date_part('millisecond', ({{first_date}})::timestamp)))\n {% elif datepart == 'microsecond' %}\n ({{ datediff(first_date, second_date, 'minute') }} * 60000000 + floor(date_part('microsecond', ({{second_date}})::timestamp)) - floor(date_part('microsecond', ({{first_date}})::timestamp)))\n {% else %}\n {{ exceptions.raise_compiler_error(\"Unsupported datepart for macro datediff in postgres: {!r}\".format(datepart)) }}\n {% endif %}\n\n{%- endmacro %}", + "meta": {}, + "name": "postgres__datediff", + "original_file_path": "macros/utils/datediff.sql", + "package_name": "dbt_postgres", + "patch_path": null, + "path": "macros/utils/datediff.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt_postgres.postgres__datediff" + }, + "macro.dbt_postgres.postgres__drop_schema": { + "arguments": [], + "created_at": 1687942822.855936, + "depends_on": { + "macros": [ + "macro.dbt.statement" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro postgres__drop_schema(relation) -%}\n {% if relation.database -%}\n {{ adapter.verify_database(relation.database) }}\n {%- endif -%}\n {%- call statement('drop_schema') -%}\n drop schema if exists {{ relation.without_identifier().include(database=False) }} cascade\n {%- endcall -%}\n{% endmacro %}", + "meta": {}, + "name": "postgres__drop_schema", + "original_file_path": "macros/adapters.sql", + "package_name": "dbt_postgres", + "patch_path": null, + "path": "macros/adapters.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt_postgres.postgres__drop_schema" + }, + "macro.dbt_postgres.postgres__get_catalog": { + "arguments": [], + "created_at": 1687942822.845242, + "depends_on": { + "macros": [ + "macro.dbt.statement" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro postgres__get_catalog(information_schema, schemas) -%}\n\n {%- call statement('catalog', fetch_result=True) -%}\n {#\n If the user has multiple databases set and the first one is wrong, this will fail.\n But we won't fail in the case where there are multiple quoting-difference-only dbs, which is better.\n #}\n {% set database = information_schema.database %}\n {{ adapter.verify_database(database) }}\n\n select\n '{{ database }}' as table_database,\n sch.nspname as table_schema,\n tbl.relname as table_name,\n case tbl.relkind\n when 'v' then 'VIEW'\n else 'BASE TABLE'\n end as table_type,\n tbl_desc.description as table_comment,\n col.attname as column_name,\n col.attnum as column_index,\n pg_catalog.format_type(col.atttypid, col.atttypmod) as column_type,\n col_desc.description as column_comment,\n pg_get_userbyid(tbl.relowner) as table_owner\n\n from pg_catalog.pg_namespace sch\n join pg_catalog.pg_class tbl on tbl.relnamespace = sch.oid\n join pg_catalog.pg_attribute col on col.attrelid = tbl.oid\n left outer join pg_catalog.pg_description tbl_desc on (tbl_desc.objoid = tbl.oid and tbl_desc.objsubid = 0)\n left outer join pg_catalog.pg_description col_desc on (col_desc.objoid = tbl.oid and col_desc.objsubid = col.attnum)\n\n where (\n {%- for schema in schemas -%}\n upper(sch.nspname) = upper('{{ schema }}'){%- if not loop.last %} or {% endif -%}\n {%- endfor -%}\n )\n and not pg_is_other_temp_schema(sch.oid) -- not a temporary schema belonging to another session\n and tbl.relpersistence in ('p', 'u') -- [p]ermanent table or [u]nlogged table. Exclude [t]emporary tables\n and tbl.relkind in ('r', 'v', 'f', 'p') -- o[r]dinary table, [v]iew, [f]oreign table, [p]artitioned table. Other values are [i]ndex, [S]equence, [c]omposite type, [t]OAST table, [m]aterialized view\n and col.attnum > 0 -- negative numbers are used for system columns such as oid\n and not col.attisdropped -- column as not been dropped\n\n order by\n sch.nspname,\n tbl.relname,\n col.attnum\n\n {%- endcall -%}\n\n {{ return(load_result('catalog').table) }}\n\n{%- endmacro %}", + "meta": {}, + "name": "postgres__get_catalog", + "original_file_path": "macros/catalog.sql", + "package_name": "dbt_postgres", + "patch_path": null, + "path": "macros/catalog.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt_postgres.postgres__get_catalog" + }, + "macro.dbt_postgres.postgres__get_columns_in_relation": { + "arguments": [], + "created_at": 1687942822.8564942, + "depends_on": { + "macros": [ + "macro.dbt.statement", + "macro.dbt.sql_convert_columns_in_relation" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro postgres__get_columns_in_relation(relation) -%}\n {% call statement('get_columns_in_relation', fetch_result=True) %}\n select\n column_name,\n data_type,\n character_maximum_length,\n numeric_precision,\n numeric_scale\n\n from {{ relation.information_schema('columns') }}\n where table_name = '{{ relation.identifier }}'\n {% if relation.schema %}\n and table_schema = '{{ relation.schema }}'\n {% endif %}\n order by ordinal_position\n\n {% endcall %}\n {% set table = load_result('get_columns_in_relation').table %}\n {{ return(sql_convert_columns_in_relation(table)) }}\n{% endmacro %}", + "meta": {}, + "name": "postgres__get_columns_in_relation", + "original_file_path": "macros/adapters.sql", + "package_name": "dbt_postgres", + "patch_path": null, + "path": "macros/adapters.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt_postgres.postgres__get_columns_in_relation" + }, + "macro.dbt_postgres.postgres__get_create_index_sql": { + "arguments": [], + "created_at": 1687942822.855187, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro postgres__get_create_index_sql(relation, index_dict) -%}\n {%- set index_config = adapter.parse_index(index_dict) -%}\n {%- set comma_separated_columns = \", \".join(index_config.columns) -%}\n {%- set index_name = index_config.render(relation) -%}\n\n create {% if index_config.unique -%}\n unique\n {%- endif %} index if not exists\n \"{{ index_name }}\"\n on {{ relation }} {% if index_config.type -%}\n using {{ index_config.type }}\n {%- endif %}\n ({{ comma_separated_columns }});\n{%- endmacro %}", + "meta": {}, + "name": "postgres__get_create_index_sql", + "original_file_path": "macros/adapters.sql", + "package_name": "dbt_postgres", + "patch_path": null, + "path": "macros/adapters.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt_postgres.postgres__get_create_index_sql" + }, + "macro.dbt_postgres.postgres__get_incremental_default_sql": { + "arguments": [], + "created_at": 1687942822.8627782, + "depends_on": { + "macros": [ + "macro.dbt.get_incremental_delete_insert_sql", + "macro.dbt.get_incremental_append_sql" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro postgres__get_incremental_default_sql(arg_dict) %}\n\n {% if arg_dict[\"unique_key\"] %}\n {% do return(get_incremental_delete_insert_sql(arg_dict)) %}\n {% else %}\n {% do return(get_incremental_append_sql(arg_dict)) %}\n {% endif %}\n\n{% endmacro %}", + "meta": {}, + "name": "postgres__get_incremental_default_sql", + "original_file_path": "macros/materializations/incremental_strategies.sql", + "package_name": "dbt_postgres", + "patch_path": null, + "path": "macros/materializations/incremental_strategies.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt_postgres.postgres__get_incremental_default_sql" + }, + "macro.dbt_postgres.postgres__get_show_grant_sql": { + "arguments": [], + "created_at": 1687942822.8621938, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "\n\n{%- macro postgres__get_show_grant_sql(relation) -%}\n select grantee, privilege_type\n from {{ relation.information_schema('role_table_grants') }}\n where grantor = current_role\n and grantee != current_role\n and table_schema = '{{ relation.schema }}'\n and table_name = '{{ relation.identifier }}'\n{%- endmacro -%}\n\n", + "meta": {}, + "name": "postgres__get_show_grant_sql", + "original_file_path": "macros/adapters.sql", + "package_name": "dbt_postgres", + "patch_path": null, + "path": "macros/adapters.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt_postgres.postgres__get_show_grant_sql" + }, + "macro.dbt_postgres.postgres__information_schema_name": { + "arguments": [], + "created_at": 1687942822.8571231, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro postgres__information_schema_name(database) -%}\n {% if database_name -%}\n {{ adapter.verify_database(database_name) }}\n {%- endif -%}\n information_schema\n{%- endmacro %}", + "meta": {}, + "name": "postgres__information_schema_name", + "original_file_path": "macros/adapters.sql", + "package_name": "dbt_postgres", + "patch_path": null, + "path": "macros/adapters.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt_postgres.postgres__information_schema_name" + }, + "macro.dbt_postgres.postgres__last_day": { + "arguments": [], + "created_at": 1687942822.868885, + "depends_on": { + "macros": [ + "macro.dbt.dateadd", + "macro.dbt.date_trunc", + "macro.dbt.default_last_day" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro postgres__last_day(date, datepart) -%}\n\n {%- if datepart == 'quarter' -%}\n -- postgres dateadd does not support quarter interval.\n cast(\n {{dbt.dateadd('day', '-1',\n dbt.dateadd('month', '3', dbt.date_trunc(datepart, date))\n )}}\n as date)\n {%- else -%}\n {{dbt.default_last_day(date, datepart)}}\n {%- endif -%}\n\n{%- endmacro %}", + "meta": {}, + "name": "postgres__last_day", + "original_file_path": "macros/utils/last_day.sql", + "package_name": "dbt_postgres", + "patch_path": null, + "path": "macros/utils/last_day.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt_postgres.postgres__last_day" + }, + "macro.dbt_postgres.postgres__list_relations_without_caching": { + "arguments": [], + "created_at": 1687942822.856927, + "depends_on": { + "macros": [ + "macro.dbt.statement" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro postgres__list_relations_without_caching(schema_relation) %}\n {% call statement('list_relations_without_caching', fetch_result=True) -%}\n select\n '{{ schema_relation.database }}' as database,\n tablename as name,\n schemaname as schema,\n 'table' as type\n from pg_tables\n where schemaname ilike '{{ schema_relation.schema }}'\n union all\n select\n '{{ schema_relation.database }}' as database,\n viewname as name,\n schemaname as schema,\n 'view' as type\n from pg_views\n where schemaname ilike '{{ schema_relation.schema }}'\n {% endcall %}\n {{ return(load_result('list_relations_without_caching').table) }}\n{% endmacro %}", + "meta": {}, + "name": "postgres__list_relations_without_caching", + "original_file_path": "macros/adapters.sql", + "package_name": "dbt_postgres", + "patch_path": null, + "path": "macros/adapters.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt_postgres.postgres__list_relations_without_caching" + }, + "macro.dbt_postgres.postgres__list_schemas": { + "arguments": [], + "created_at": 1687942822.857524, + "depends_on": { + "macros": [ + "macro.dbt.statement" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro postgres__list_schemas(database) %}\n {% if database -%}\n {{ adapter.verify_database(database) }}\n {%- endif -%}\n {% call statement('list_schemas', fetch_result=True, auto_begin=False) %}\n select distinct nspname from pg_namespace\n {% endcall %}\n {{ return(load_result('list_schemas').table) }}\n{% endmacro %}", + "meta": {}, + "name": "postgres__list_schemas", + "original_file_path": "macros/adapters.sql", + "package_name": "dbt_postgres", + "patch_path": null, + "path": "macros/adapters.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt_postgres.postgres__list_schemas" + }, + "macro.dbt_postgres.postgres__listagg": { + "arguments": [], + "created_at": 1687942822.864536, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro postgres__listagg(measure, delimiter_text, order_by_clause, limit_num) -%}\n\n {% if limit_num -%}\n array_to_string(\n (array_agg(\n {{ measure }}\n {% if order_by_clause -%}\n {{ order_by_clause }}\n {%- endif %}\n ))[1:{{ limit_num }}],\n {{ delimiter_text }}\n )\n {%- else %}\n string_agg(\n {{ measure }},\n {{ delimiter_text }}\n {% if order_by_clause -%}\n {{ order_by_clause }}\n {%- endif %}\n )\n {%- endif %}\n\n{%- endmacro %}", + "meta": {}, + "name": "postgres__listagg", + "original_file_path": "macros/utils/listagg.sql", + "package_name": "dbt_postgres", + "patch_path": null, + "path": "macros/utils/listagg.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt_postgres.postgres__listagg" + }, + "macro.dbt_postgres.postgres__make_backup_relation": { + "arguments": [], + "created_at": 1687942822.860425, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres__make_relation_with_suffix" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro postgres__make_backup_relation(base_relation, backup_relation_type, suffix) %}\n {% set backup_relation = postgres__make_relation_with_suffix(base_relation, suffix, dstring=False) %}\n {{ return(backup_relation.incorporate(type=backup_relation_type)) }}\n{% endmacro %}", + "meta": {}, + "name": "postgres__make_backup_relation", + "original_file_path": "macros/adapters.sql", + "package_name": "dbt_postgres", + "patch_path": null, + "path": "macros/adapters.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt_postgres.postgres__make_backup_relation" + }, + "macro.dbt_postgres.postgres__make_intermediate_relation": { + "arguments": [], + "created_at": 1687942822.859671, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres__make_relation_with_suffix" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro postgres__make_intermediate_relation(base_relation, suffix) %}\n {{ return(postgres__make_relation_with_suffix(base_relation, suffix, dstring=False)) }}\n{% endmacro %}", + "meta": {}, + "name": "postgres__make_intermediate_relation", + "original_file_path": "macros/adapters.sql", + "package_name": "dbt_postgres", + "patch_path": null, + "path": "macros/adapters.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt_postgres.postgres__make_intermediate_relation" + }, + "macro.dbt_postgres.postgres__make_relation_with_suffix": { + "arguments": [], + "created_at": 1687942822.8594298, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro postgres__make_relation_with_suffix(base_relation, suffix, dstring) %}\n {% if dstring %}\n {% set dt = modules.datetime.datetime.now() %}\n {% set dtstring = dt.strftime(\"%H%M%S%f\") %}\n {% set suffix = suffix ~ dtstring %}\n {% endif %}\n {% set suffix_length = suffix|length %}\n {% set relation_max_name_length = base_relation.relation_max_name_length() %}\n {% if suffix_length > relation_max_name_length %}\n {% do exceptions.raise_compiler_error('Relation suffix is too long (' ~ suffix_length ~ ' characters). Maximum length is ' ~ relation_max_name_length ~ ' characters.') %}\n {% endif %}\n {% set identifier = base_relation.identifier[:relation_max_name_length - suffix_length] ~ suffix %}\n\n {{ return(base_relation.incorporate(path={\"identifier\": identifier })) }}\n\n {% endmacro %}", + "meta": {}, + "name": "postgres__make_relation_with_suffix", + "original_file_path": "macros/adapters.sql", + "package_name": "dbt_postgres", + "patch_path": null, + "path": "macros/adapters.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt_postgres.postgres__make_relation_with_suffix" + }, + "macro.dbt_postgres.postgres__make_temp_relation": { + "arguments": [], + "created_at": 1687942822.86004, + "depends_on": { + "macros": [ + "macro.dbt_postgres.postgres__make_relation_with_suffix" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro postgres__make_temp_relation(base_relation, suffix) %}\n {% set temp_relation = postgres__make_relation_with_suffix(base_relation, suffix, dstring=True) %}\n {{ return(temp_relation.incorporate(path={\"schema\": none,\n \"database\": none})) }}\n{% endmacro %}", + "meta": {}, + "name": "postgres__make_temp_relation", + "original_file_path": "macros/adapters.sql", + "package_name": "dbt_postgres", + "patch_path": null, + "path": "macros/adapters.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt_postgres.postgres__make_temp_relation" + }, + "macro.dbt_postgres.postgres__snapshot_get_time": { + "arguments": [], + "created_at": 1687942822.84378, + "depends_on": { + "macros": [ + "macro.dbt.current_timestamp" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro postgres__snapshot_get_time() -%}\n {{ current_timestamp() }}::timestamp without time zone\n{%- endmacro %}", + "meta": {}, + "name": "postgres__snapshot_get_time", + "original_file_path": "macros/timestamps.sql", + "package_name": "dbt_postgres", + "patch_path": null, + "path": "macros/timestamps.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt_postgres.postgres__snapshot_get_time" + }, + "macro.dbt_postgres.postgres__snapshot_merge_sql": { + "arguments": [], + "created_at": 1687942822.8635678, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro postgres__snapshot_merge_sql(target, source, insert_cols) -%}\n {%- set insert_cols_csv = insert_cols | join(', ') -%}\n\n update {{ target }}\n set dbt_valid_to = DBT_INTERNAL_SOURCE.dbt_valid_to\n from {{ source }} as DBT_INTERNAL_SOURCE\n where DBT_INTERNAL_SOURCE.dbt_scd_id::text = {{ target }}.dbt_scd_id::text\n and DBT_INTERNAL_SOURCE.dbt_change_type::text in ('update'::text, 'delete'::text)\n and {{ target }}.dbt_valid_to is null;\n\n insert into {{ target }} ({{ insert_cols_csv }})\n select {% for column in insert_cols -%}\n DBT_INTERNAL_SOURCE.{{ column }} {%- if not loop.last %}, {%- endif %}\n {%- endfor %}\n from {{ source }} as DBT_INTERNAL_SOURCE\n where DBT_INTERNAL_SOURCE.dbt_change_type::text = 'insert'::text;\n{% endmacro %}", + "meta": {}, + "name": "postgres__snapshot_merge_sql", + "original_file_path": "macros/materializations/snapshot_merge.sql", + "package_name": "dbt_postgres", + "patch_path": null, + "path": "macros/materializations/snapshot_merge.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt_postgres.postgres__snapshot_merge_sql" + }, + "macro.dbt_postgres.postgres__snapshot_string_as_time": { + "arguments": [], + "created_at": 1687942822.8435972, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro postgres__snapshot_string_as_time(timestamp) -%}\n {%- set result = \"'\" ~ timestamp ~ \"'::timestamp without time zone\" -%}\n {{ return(result) }}\n{%- endmacro %}", + "meta": {}, + "name": "postgres__snapshot_string_as_time", + "original_file_path": "macros/timestamps.sql", + "package_name": "dbt_postgres", + "patch_path": null, + "path": "macros/timestamps.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt_postgres.postgres__snapshot_string_as_time" + }, + "macro.dbt_postgres.postgres__split_part": { + "arguments": [], + "created_at": 1687942822.86934, + "depends_on": { + "macros": [ + "macro.dbt.default__split_part", + "macro.dbt._split_part_negative" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro postgres__split_part(string_text, delimiter_text, part_number) %}\n\n {% if part_number >= 0 %}\n {{ dbt.default__split_part(string_text, delimiter_text, part_number) }}\n {% else %}\n {{ dbt._split_part_negative(string_text, delimiter_text, part_number) }}\n {% endif %}\n\n{% endmacro %}", + "meta": {}, + "name": "postgres__split_part", + "original_file_path": "macros/utils/split_part.sql", + "package_name": "dbt_postgres", + "patch_path": null, + "path": "macros/utils/split_part.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt_postgres.postgres__split_part" + }, + "macro.dbt_postgres.postgres_escape_comment": { + "arguments": [], + "created_at": 1687942822.860971, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro postgres_escape_comment(comment) -%}\n {% if comment is not string %}\n {% do exceptions.raise_compiler_error('cannot escape a non-string: ' ~ comment) %}\n {% endif %}\n {%- set magic = '$dbt_comment_literal_block$' -%}\n {%- if magic in comment -%}\n {%- do exceptions.raise_compiler_error('The string ' ~ magic ~ ' is not allowed in comments.') -%}\n {%- endif -%}\n {{ magic }}{{ comment }}{{ magic }}\n{%- endmacro %}", + "meta": {}, + "name": "postgres_escape_comment", + "original_file_path": "macros/adapters.sql", + "package_name": "dbt_postgres", + "patch_path": null, + "path": "macros/adapters.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt_postgres.postgres_escape_comment" + }, + "macro.dbt_postgres.postgres_get_relations": { + "arguments": [], + "created_at": 1687942822.845938, + "depends_on": { + "macros": [ + "macro.dbt.statement" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{% macro postgres_get_relations () -%}\n\n {#\n -- in pg_depend, objid is the dependent, refobjid is the referenced object\n -- > a pg_depend entry indicates that the referenced object cannot be\n -- > dropped without also dropping the dependent object.\n #}\n\n {%- call statement('relations', fetch_result=True) -%}\n with relation as (\n select\n pg_rewrite.ev_class as class,\n pg_rewrite.oid as id\n from pg_rewrite\n ),\n class as (\n select\n oid as id,\n relname as name,\n relnamespace as schema,\n relkind as kind\n from pg_class\n ),\n dependency as (\n select distinct\n pg_depend.objid as id,\n pg_depend.refobjid as ref\n from pg_depend\n ),\n schema as (\n select\n pg_namespace.oid as id,\n pg_namespace.nspname as name\n from pg_namespace\n where nspname != 'information_schema' and nspname not like 'pg\\_%'\n ),\n referenced as (\n select\n relation.id AS id,\n referenced_class.name ,\n referenced_class.schema ,\n referenced_class.kind\n from relation\n join class as referenced_class on relation.class=referenced_class.id\n where referenced_class.kind in ('r', 'v')\n ),\n relationships as (\n select\n referenced.name as referenced_name,\n referenced.schema as referenced_schema_id,\n dependent_class.name as dependent_name,\n dependent_class.schema as dependent_schema_id,\n referenced.kind as kind\n from referenced\n join dependency on referenced.id=dependency.id\n join class as dependent_class on dependency.ref=dependent_class.id\n where\n (referenced.name != dependent_class.name or\n referenced.schema != dependent_class.schema)\n )\n\n select\n referenced_schema.name as referenced_schema,\n relationships.referenced_name as referenced_name,\n dependent_schema.name as dependent_schema,\n relationships.dependent_name as dependent_name\n from relationships\n join schema as dependent_schema on relationships.dependent_schema_id=dependent_schema.id\n join schema as referenced_schema on relationships.referenced_schema_id=referenced_schema.id\n group by referenced_schema, referenced_name, dependent_schema, dependent_name\n order by referenced_schema, referenced_name, dependent_schema, dependent_name;\n\n {%- endcall -%}\n\n {{ return(load_result('relations').table) }}\n{% endmacro %}", + "meta": {}, + "name": "postgres_get_relations", + "original_file_path": "macros/relations.sql", + "package_name": "dbt_postgres", + "patch_path": null, + "path": "macros/relations.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.dbt_postgres.postgres_get_relations" + }, + "macro.jaffle_shop.drop_table": { + "arguments": [], + "created_at": 1687942822.842345, + "depends_on": { + "macros": [ + "macro.dbt.run_query" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "macro_sql": "{%- macro drop_table(table_name) -%}\n {%- set drop_query -%}\n DROP TABLE IF EXISTS {{ target.schema }}.{{ table_name }} CASCADE\n {%- endset -%}\n {% do run_query(drop_query) %}\n{%- endmacro -%}", + "meta": {}, + "name": "drop_table", + "original_file_path": "macros/drop_table.sql", + "package_name": "jaffle_shop", + "patch_path": null, + "path": "macros/drop_table.sql", + "resource_type": "macro", + "supported_languages": null, + "unique_id": "macro.jaffle_shop.drop_table" + } + }, + "metadata": { + "adapter_type": "postgres", + "dbt_schema_version": "https://schemas.getdbt.com/dbt/manifest/v9.json", + "dbt_version": "1.5.2", + "env": {}, + "generated_at": "2023-06-28T09:00:22.812217Z", + "invocation_id": "67cf034b-da45-43ca-973c-71bafad9f92c", + "project_id": "06e5b98c2db46f8a72cc4f66410e9b3b", + "send_anonymous_usage_stats": true, + "user_id": "842c63fe-29b4-41e3-b5ef-9e065d34bcc0" + }, + "metrics": {}, + "nodes": { + "model.jaffle_shop.customers": { + "access": "protected", + "alias": "customers", + "build_path": null, + "checksum": { + "checksum": "60bd72e33da43fff3a7e7609135c17cd4468bd22afec0735dd36018bfb5af30a", + "name": "sha256" + }, + "columns": { + "customer_id": { + "constraints": [], + "data_type": null, + "description": "This is a unique identifier for a customer", + "meta": {}, + "name": "customer_id", + "quote": null, + "tags": [] + }, + "first_name": { + "constraints": [], + "data_type": null, + "description": "Customer's first name. PII.", + "meta": {}, + "name": "first_name", + "quote": null, + "tags": [] + }, + "first_order": { + "constraints": [], + "data_type": null, + "description": "Date (UTC) of a customer's first order", + "meta": {}, + "name": "first_order", + "quote": null, + "tags": [] + }, + "last_name": { + "constraints": [], + "data_type": null, + "description": "Customer's last name. PII.", + "meta": {}, + "name": "last_name", + "quote": null, + "tags": [] + }, + "most_recent_order": { + "constraints": [], + "data_type": null, + "description": "Date (UTC) of a customer's most recent order", + "meta": {}, + "name": "most_recent_order", + "quote": null, + "tags": [] + }, + "number_of_orders": { + "constraints": [], + "data_type": null, + "description": "Count of the number of orders a customer has placed", + "meta": {}, + "name": "number_of_orders", + "quote": null, + "tags": [] + }, + "total_order_amount": { + "constraints": [], + "data_type": null, + "description": "Total value (AUD) of a customer's orders", + "meta": {}, + "name": "total_order_amount", + "quote": null, + "tags": [] + } + }, + "compiled_path": null, + "config": { + "alias": null, + "column_types": {}, + "contract": { + "enforced": false + }, + "database": null, + "docs": { + "node_color": null, + "show": true + }, + "enabled": true, + "full_refresh": null, + "grants": {}, + "group": null, + "incremental_strategy": null, + "materialized": "table", + "meta": {}, + "on_schema_change": "ignore", + "packages": [], + "persist_docs": {}, + "post-hook": [], + "pre-hook": [], + "quoting": {}, + "schema": null, + "tags": [], + "unique_key": null + }, + "constraints": [], + "contract": { + "checksum": null, + "enforced": false + }, + "created_at": 1687942823.256299, + "database": "postgres", + "deferred": false, + "depends_on": { + "macros": [], + "nodes": [ + "model.jaffle_shop.stg_customers", + "model.jaffle_shop.stg_orders", + "model.jaffle_shop.stg_payments" + ] + }, + "description": "This table has basic information about a customer, as well as some derived facts based on a customer's orders", + "docs": { + "node_color": null, + "show": true + }, + "fqn": [ + "jaffle_shop", + "customers" + ], + "group": null, + "language": "sql", + "latest_version": null, + "meta": {}, + "metrics": [], + "name": "customers", + "original_file_path": "models/customers.sql", + "package_name": "jaffle_shop", + "patch_path": "jaffle_shop://models/schema.yml", + "path": "customers.sql", + "raw_code": "with customers as (\n\n select * from {{ ref('stg_customers') }}\n\n),\n\norders as (\n\n select * from {{ ref('stg_orders') }}\n\n),\n\npayments as (\n\n select * from {{ ref('stg_payments') }}\n\n),\n\ncustomer_orders as (\n\n select\n customer_id,\n\n min(order_date) as first_order,\n max(order_date) as most_recent_order,\n count(order_id) as number_of_orders\n from orders\n\n group by customer_id\n\n),\n\ncustomer_payments as (\n\n select\n orders.customer_id,\n sum(amount) as total_amount\n\n from payments\n\n left join orders on\n payments.order_id = orders.order_id\n\n group by orders.customer_id\n\n),\n\nfinal as (\n\n select\n customers.customer_id,\n customers.first_name,\n customers.last_name,\n customer_orders.first_order,\n customer_orders.most_recent_order,\n customer_orders.number_of_orders,\n customer_payments.total_amount as customer_lifetime_value\n\n from customers\n\n left join customer_orders\n on customers.customer_id = customer_orders.customer_id\n\n left join customer_payments\n on customers.customer_id = customer_payments.customer_id\n\n)\n\nselect * from final", + "refs": [ + { + "name": "stg_customers", + "package": null, + "version": null + }, + { + "name": "stg_orders", + "package": null, + "version": null + }, + { + "name": "stg_payments", + "package": null, + "version": null + } + ], + "relation_name": "\"postgres\".\"public\".\"customers\"", + "resource_type": "model", + "schema": "public", + "sources": [], + "tags": [], + "unique_id": "model.jaffle_shop.customers", + "unrendered_config": { + "materialized": "table" + }, + "version": null + }, + "model.jaffle_shop.orders": { + "access": "protected", + "alias": "orders", + "build_path": null, + "checksum": { + "checksum": "27f8c79aad1cfd8411ab9c3d2ce8da1d787f7f05c58bbee1d247510dc426be0f", + "name": "sha256" + }, + "columns": { + "amount": { + "constraints": [], + "data_type": null, + "description": "Total amount (AUD) of the order", + "meta": {}, + "name": "amount", + "quote": null, + "tags": [] + }, + "bank_transfer_amount": { + "constraints": [], + "data_type": null, + "description": "Amount of the order (AUD) paid for by bank transfer", + "meta": {}, + "name": "bank_transfer_amount", + "quote": null, + "tags": [] + }, + "coupon_amount": { + "constraints": [], + "data_type": null, + "description": "Amount of the order (AUD) paid for by coupon", + "meta": {}, + "name": "coupon_amount", + "quote": null, + "tags": [] + }, + "credit_card_amount": { + "constraints": [], + "data_type": null, + "description": "Amount of the order (AUD) paid for by credit card", + "meta": {}, + "name": "credit_card_amount", + "quote": null, + "tags": [] + }, + "customer_id": { + "constraints": [], + "data_type": null, + "description": "Foreign key to the customers table", + "meta": {}, + "name": "customer_id", + "quote": null, + "tags": [] + }, + "gift_card_amount": { + "constraints": [], + "data_type": null, + "description": "Amount of the order (AUD) paid for by gift card", + "meta": {}, + "name": "gift_card_amount", + "quote": null, + "tags": [] + }, + "order_date": { + "constraints": [], + "data_type": null, + "description": "Date (UTC) that the order was placed", + "meta": {}, + "name": "order_date", + "quote": null, + "tags": [] + }, + "order_id": { + "constraints": [], + "data_type": null, + "description": "This is a unique identifier for an order", + "meta": {}, + "name": "order_id", + "quote": null, + "tags": [] + }, + "status": { + "constraints": [], + "data_type": null, + "description": "Orders can be one of the following statuses:\n\n| status | description |\n|----------------|------------------------------------------------------------------------------------------------------------------------|\n| placed | The order has been placed but has not yet left the warehouse |\n| shipped | The order has ben shipped to the customer and is currently in transit |\n| completed | The order has been received by the customer |\n| return_pending | The customer has indicated that they would like to return the order, but it has not yet been received at the warehouse |\n| returned | The order has been returned by the customer and received at the warehouse |", + "meta": {}, + "name": "status", + "quote": null, + "tags": [] + } + }, + "compiled_path": null, + "config": { + "alias": null, + "column_types": {}, + "contract": { + "enforced": false + }, + "database": null, + "docs": { + "node_color": null, + "show": true + }, + "enabled": true, + "full_refresh": null, + "grants": {}, + "group": null, + "incremental_strategy": null, + "materialized": "table", + "meta": {}, + "on_schema_change": "ignore", + "packages": [], + "persist_docs": {}, + "post-hook": [], + "pre-hook": [], + "quoting": {}, + "schema": null, + "tags": [], + "unique_key": null + }, + "constraints": [], + "contract": { + "checksum": null, + "enforced": false + }, + "created_at": 1687942823.258613, + "database": "postgres", + "deferred": false, + "depends_on": { + "macros": [], + "nodes": [ + "model.jaffle_shop.stg_orders", + "model.jaffle_shop.stg_payments" + ] + }, + "description": "This table has basic information about orders, as well as some derived facts based on payments", + "docs": { + "node_color": null, + "show": true + }, + "fqn": [ + "jaffle_shop", + "orders" + ], + "group": null, + "language": "sql", + "latest_version": null, + "meta": {}, + "metrics": [], + "name": "orders", + "original_file_path": "models/orders.sql", + "package_name": "jaffle_shop", + "patch_path": "jaffle_shop://models/schema.yml", + "path": "orders.sql", + "raw_code": "{% set payment_methods = ['credit_card', 'coupon', 'bank_transfer', 'gift_card'] %}\n\nwith orders as (\n\n select * from {{ ref('stg_orders') }}\n\n),\n\npayments as (\n\n select * from {{ ref('stg_payments') }}\n\n),\n\norder_payments as (\n\n select\n order_id,\n\n {% for payment_method in payment_methods -%}\n sum(case when payment_method = '{{ payment_method }}' then amount else 0 end) as {{ payment_method }}_amount,\n {% endfor -%}\n\n sum(amount) as total_amount\n\n from payments\n\n group by order_id\n\n),\n\nfinal as (\n\n select\n orders.order_id,\n orders.customer_id,\n orders.order_date,\n orders.status,\n\n {% for payment_method in payment_methods -%}\n\n order_payments.{{ payment_method }}_amount,\n\n {% endfor -%}\n\n order_payments.total_amount as amount\n\n from orders\n\n\n left join order_payments\n on orders.order_id = order_payments.order_id\n\n)\n\nselect * from final", + "refs": [ + { + "name": "stg_orders", + "package": null, + "version": null + }, + { + "name": "stg_payments", + "package": null, + "version": null + } + ], + "relation_name": "\"postgres\".\"public\".\"orders\"", + "resource_type": "model", + "schema": "public", + "sources": [], + "tags": [], + "unique_id": "model.jaffle_shop.orders", + "unrendered_config": { + "materialized": "table" + }, + "version": null + }, + "model.jaffle_shop.stg_customers": { + "access": "protected", + "alias": "stg_customers", + "build_path": null, + "checksum": { + "checksum": "80e3223cd54387e11fa16cd0f4cbe15f8ff74dcd9900b93856b9e39416178c9d", + "name": "sha256" + }, + "columns": { + "customer_id": { + "constraints": [], + "data_type": null, + "description": "", + "meta": {}, + "name": "customer_id", + "quote": null, + "tags": [] + } + }, + "compiled_path": null, + "config": { + "alias": null, + "column_types": {}, + "contract": { + "enforced": false + }, + "database": null, + "docs": { + "node_color": null, + "show": true + }, + "enabled": true, + "full_refresh": null, + "grants": {}, + "group": null, + "incremental_strategy": null, + "materialized": "view", + "meta": {}, + "on_schema_change": "ignore", + "packages": [], + "persist_docs": {}, + "post-hook": [], + "pre-hook": [], + "quoting": {}, + "schema": null, + "tags": [], + "unique_key": null + }, + "constraints": [], + "contract": { + "checksum": null, + "enforced": false + }, + "created_at": 1687942823.2960231, + "database": "postgres", + "deferred": false, + "depends_on": { + "macros": [], + "nodes": [ + "seed.jaffle_shop.raw_customers" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "fqn": [ + "jaffle_shop", + "staging", + "stg_customers" + ], + "group": null, + "language": "sql", + "latest_version": null, + "meta": {}, + "metrics": [], + "name": "stg_customers", + "original_file_path": "models/staging/stg_customers.sql", + "package_name": "jaffle_shop", + "patch_path": "jaffle_shop://models/staging/schema.yml", + "path": "staging/stg_customers.sql", + "raw_code": "with source as (\n\n {#-\n Normally we would select from the table here, but we are using seeds to load\n our data in this project\n #}\n select * from {{ ref('raw_customers') }}\n\n),\n\nrenamed as (\n\n select\n id as customer_id,\n first_name,\n last_name\n\n from source\n\n)\n\nselect * from renamed", + "refs": [ + { + "name": "raw_customers", + "package": null, + "version": null + } + ], + "relation_name": "\"postgres\".\"public\".\"stg_customers\"", + "resource_type": "model", + "schema": "public", + "sources": [], + "tags": [], + "unique_id": "model.jaffle_shop.stg_customers", + "unrendered_config": { + "materialized": "view" + }, + "version": null + }, + "model.jaffle_shop.stg_orders": { + "access": "protected", + "alias": "stg_orders", + "build_path": null, + "checksum": { + "checksum": "f4f881cb09d2c4162200fc331d7401df6d1abd4fed492554a7db70dede347108", + "name": "sha256" + }, + "columns": { + "order_id": { + "constraints": [], + "data_type": null, + "description": "", + "meta": {}, + "name": "order_id", + "quote": null, + "tags": [] + }, + "status": { + "constraints": [], + "data_type": null, + "description": "", + "meta": {}, + "name": "status", + "quote": null, + "tags": [] + } + }, + "compiled_path": null, + "config": { + "alias": null, + "column_types": {}, + "contract": { + "enforced": false + }, + "database": null, + "docs": { + "node_color": null, + "show": true + }, + "enabled": true, + "full_refresh": null, + "grants": {}, + "group": null, + "incremental_strategy": null, + "materialized": "view", + "meta": {}, + "on_schema_change": "ignore", + "packages": [], + "persist_docs": {}, + "post-hook": [], + "pre-hook": [], + "quoting": {}, + "schema": null, + "tags": [], + "unique_key": null + }, + "constraints": [], + "contract": { + "checksum": null, + "enforced": false + }, + "created_at": 1687942823.296747, + "database": "postgres", + "deferred": false, + "depends_on": { + "macros": [], + "nodes": [ + "seed.jaffle_shop.raw_orders" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "fqn": [ + "jaffle_shop", + "staging", + "stg_orders" + ], + "group": null, + "language": "sql", + "latest_version": null, + "meta": {}, + "metrics": [], + "name": "stg_orders", + "original_file_path": "models/staging/stg_orders.sql", + "package_name": "jaffle_shop", + "patch_path": "jaffle_shop://models/staging/schema.yml", + "path": "staging/stg_orders.sql", + "raw_code": "with source as (\n\n {#-\n Normally we would select from the table here, but we are using seeds to load\n our data in this project\n #}\n select * from {{ ref('raw_orders') }}\n\n),\n\nrenamed as (\n\n select\n id as order_id,\n user_id as customer_id,\n order_date,\n status\n\n from source\n\n)\n\nselect * from renamed", + "refs": [ + { + "name": "raw_orders", + "package": null, + "version": null + } + ], + "relation_name": "\"postgres\".\"public\".\"stg_orders\"", + "resource_type": "model", + "schema": "public", + "sources": [], + "tags": [], + "unique_id": "model.jaffle_shop.stg_orders", + "unrendered_config": { + "materialized": "view" + }, + "version": null + }, + "model.jaffle_shop.stg_payments": { + "access": "protected", + "alias": "stg_payments", + "build_path": null, + "checksum": { + "checksum": "30f346f66ef7bca4c8865a471086303720c3daab58870c805b6f45e92d19fd65", + "name": "sha256" + }, + "columns": { + "payment_id": { + "constraints": [], + "data_type": null, + "description": "", + "meta": {}, + "name": "payment_id", + "quote": null, + "tags": [] + }, + "payment_method": { + "constraints": [], + "data_type": null, + "description": "", + "meta": {}, + "name": "payment_method", + "quote": null, + "tags": [] + } + }, + "compiled_path": null, + "config": { + "alias": null, + "column_types": {}, + "contract": { + "enforced": false + }, + "database": null, + "docs": { + "node_color": null, + "show": true + }, + "enabled": true, + "full_refresh": null, + "grants": {}, + "group": null, + "incremental_strategy": null, + "materialized": "view", + "meta": {}, + "on_schema_change": "ignore", + "packages": [], + "persist_docs": {}, + "post-hook": [], + "pre-hook": [], + "quoting": {}, + "schema": null, + "tags": [], + "unique_key": null + }, + "constraints": [], + "contract": { + "checksum": null, + "enforced": false + }, + "created_at": 1687942823.297438, + "database": "postgres", + "deferred": false, + "depends_on": { + "macros": [], + "nodes": [ + "seed.jaffle_shop.raw_payments" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "fqn": [ + "jaffle_shop", + "staging", + "stg_payments" + ], + "group": null, + "language": "sql", + "latest_version": null, + "meta": {}, + "metrics": [], + "name": "stg_payments", + "original_file_path": "models/staging/stg_payments.sql", + "package_name": "jaffle_shop", + "patch_path": "jaffle_shop://models/staging/schema.yml", + "path": "staging/stg_payments.sql", + "raw_code": "with source as (\n\n {#-\n Normally we would select from the table here, but we are using seeds to load\n our data in this project\n #}\n select * from {{ ref('raw_payments') }}\n\n),\n\nrenamed as (\n\n select\n id as payment_id,\n order_id,\n payment_method,\n\n -- `amount` is currently stored in cents, so we convert it to dollars\n amount / 100 as amount\n\n from source\n\n)\n\nselect * from renamed", + "refs": [ + { + "name": "raw_payments", + "package": null, + "version": null + } + ], + "relation_name": "\"postgres\".\"public\".\"stg_payments\"", + "resource_type": "model", + "schema": "public", + "sources": [], + "tags": [], + "unique_id": "model.jaffle_shop.stg_payments", + "unrendered_config": { + "materialized": "view" + }, + "version": null + }, + "seed.jaffle_shop.raw_customers": { + "alias": "raw_customers", + "build_path": null, + "checksum": { + "checksum": "357d173dda65a741ad97d6683502286cc2655bb396ab5f4dfad12b8c39bd2a63", + "name": "sha256" + }, + "columns": {}, + "config": { + "alias": null, + "column_types": {}, + "contract": { + "enforced": false + }, + "database": null, + "docs": { + "node_color": null, + "show": true + }, + "enabled": true, + "full_refresh": null, + "grants": {}, + "group": null, + "incremental_strategy": null, + "materialized": "seed", + "meta": {}, + "on_schema_change": "ignore", + "packages": [], + "persist_docs": {}, + "post-hook": [], + "pre-hook": [], + "quote_columns": null, + "quoting": {}, + "schema": null, + "tags": [], + "unique_key": null + }, + "created_at": 1687942823.236107, + "database": "postgres", + "deferred": false, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "fqn": [ + "jaffle_shop", + "raw_customers" + ], + "group": null, + "meta": {}, + "name": "raw_customers", + "original_file_path": "seeds/raw_customers.csv", + "package_name": "jaffle_shop", + "patch_path": null, + "path": "raw_customers.csv", + "raw_code": "", + "relation_name": "\"postgres\".\"public\".\"raw_customers\"", + "resource_type": "seed", + "root_path": "/Users/tati/Code/astronomer-cosmos/dev/dags/dbt/jaffle_shop", + "schema": "public", + "tags": [], + "unique_id": "seed.jaffle_shop.raw_customers", + "unrendered_config": {} + }, + "seed.jaffle_shop.raw_orders": { + "alias": "raw_orders", + "build_path": null, + "checksum": { + "checksum": "6228dde8e17b9621f35c13e272ec67d3ff55b55499433f47d303adf2be72c17f", + "name": "sha256" + }, + "columns": {}, + "config": { + "alias": null, + "column_types": {}, + "contract": { + "enforced": false + }, + "database": null, + "docs": { + "node_color": null, + "show": true + }, + "enabled": true, + "full_refresh": null, + "grants": {}, + "group": null, + "incremental_strategy": null, + "materialized": "seed", + "meta": {}, + "on_schema_change": "ignore", + "packages": [], + "persist_docs": {}, + "post-hook": [], + "pre-hook": [], + "quote_columns": null, + "quoting": {}, + "schema": null, + "tags": [], + "unique_key": null + }, + "created_at": 1687942823.237559, + "database": "postgres", + "deferred": false, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "fqn": [ + "jaffle_shop", + "raw_orders" + ], + "group": null, + "meta": {}, + "name": "raw_orders", + "original_file_path": "seeds/raw_orders.csv", + "package_name": "jaffle_shop", + "patch_path": null, + "path": "raw_orders.csv", + "raw_code": "", + "relation_name": "\"postgres\".\"public\".\"raw_orders\"", + "resource_type": "seed", + "root_path": "/Users/tati/Code/astronomer-cosmos/dev/dags/dbt/jaffle_shop", + "schema": "public", + "tags": [], + "unique_id": "seed.jaffle_shop.raw_orders", + "unrendered_config": {} + }, + "seed.jaffle_shop.raw_payments": { + "alias": "raw_payments", + "build_path": null, + "checksum": { + "checksum": "6de0626a8db9c1750eefd1b2e17fac4c2a4b9f778eb50532d8b377b90de395e6", + "name": "sha256" + }, + "columns": {}, + "config": { + "alias": null, + "column_types": {}, + "contract": { + "enforced": false + }, + "database": null, + "docs": { + "node_color": null, + "show": true + }, + "enabled": true, + "full_refresh": null, + "grants": {}, + "group": null, + "incremental_strategy": null, + "materialized": "seed", + "meta": {}, + "on_schema_change": "ignore", + "packages": [], + "persist_docs": {}, + "post-hook": [], + "pre-hook": [], + "quote_columns": null, + "quoting": {}, + "schema": null, + "tags": [], + "unique_key": null + }, + "created_at": 1687942823.2389288, + "database": "postgres", + "deferred": false, + "depends_on": { + "macros": [] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "fqn": [ + "jaffle_shop", + "raw_payments" + ], + "group": null, + "meta": {}, + "name": "raw_payments", + "original_file_path": "seeds/raw_payments.csv", + "package_name": "jaffle_shop", + "patch_path": null, + "path": "raw_payments.csv", + "raw_code": "", + "relation_name": "\"postgres\".\"public\".\"raw_payments\"", + "resource_type": "seed", + "root_path": "/Users/tati/Code/astronomer-cosmos/dev/dags/dbt/jaffle_shop", + "schema": "public", + "tags": [], + "unique_id": "seed.jaffle_shop.raw_payments", + "unrendered_config": {} + }, + "test.jaffle_shop.accepted_values_orders_status__placed__shipped__completed__return_pending__returned.be6b5b5ec3": { + "alias": "accepted_values_orders_1ce6ab157c285f7cd2ac656013faf758", + "attached_node": "model.jaffle_shop.orders", + "build_path": null, + "checksum": { + "checksum": "", + "name": "none" + }, + "column_name": "status", + "columns": {}, + "compiled_path": null, + "config": { + "alias": "accepted_values_orders_1ce6ab157c285f7cd2ac656013faf758", + "database": null, + "enabled": true, + "error_if": "!= 0", + "fail_calc": "count(*)", + "group": null, + "limit": null, + "materialized": "test", + "meta": {}, + "schema": "dbt_test__audit", + "severity": "ERROR", + "store_failures": null, + "tags": [], + "warn_if": "!= 0", + "where": null + }, + "contract": { + "checksum": null, + "enforced": false + }, + "created_at": 1687942823.283273, + "database": "postgres", + "deferred": false, + "depends_on": { + "macros": [ + "macro.dbt.test_accepted_values", + "macro.dbt.get_where_subquery" + ], + "nodes": [ + "model.jaffle_shop.orders" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "file_key_name": "models.orders", + "fqn": [ + "jaffle_shop", + "accepted_values_orders_status__placed__shipped__completed__return_pending__returned" + ], + "group": null, + "language": "sql", + "meta": {}, + "metrics": [], + "name": "accepted_values_orders_status__placed__shipped__completed__return_pending__returned", + "original_file_path": "models/schema.yml", + "package_name": "jaffle_shop", + "patch_path": null, + "path": "accepted_values_orders_1ce6ab157c285f7cd2ac656013faf758.sql", + "raw_code": "{{ test_accepted_values(**_dbt_generic_test_kwargs) }}{{ config(alias=\"accepted_values_orders_1ce6ab157c285f7cd2ac656013faf758\") }}", + "refs": [ + { + "name": "orders", + "package": null, + "version": null + } + ], + "relation_name": null, + "resource_type": "test", + "schema": "public_dbt_test__audit", + "sources": [], + "tags": [], + "test_metadata": { + "kwargs": { + "column_name": "status", + "model": "{{ get_where_subquery(ref('orders')) }}", + "values": [ + "placed", + "shipped", + "completed", + "return_pending", + "returned" + ] + }, + "name": "accepted_values", + "namespace": null + }, + "unique_id": "test.jaffle_shop.accepted_values_orders_status__placed__shipped__completed__return_pending__returned.be6b5b5ec3", + "unrendered_config": { + "alias": "accepted_values_orders_1ce6ab157c285f7cd2ac656013faf758" + } + }, + "test.jaffle_shop.accepted_values_stg_orders_status__placed__shipped__completed__return_pending__returned.080fb20aad": { + "alias": "accepted_values_stg_orders_4f514bf94b77b7ea437830eec4421c58", + "attached_node": "model.jaffle_shop.stg_orders", + "build_path": null, + "checksum": { + "checksum": "", + "name": "none" + }, + "column_name": "status", + "columns": {}, + "compiled_path": null, + "config": { + "alias": "accepted_values_stg_orders_4f514bf94b77b7ea437830eec4421c58", + "database": null, + "enabled": true, + "error_if": "!= 0", + "fail_calc": "count(*)", + "group": null, + "limit": null, + "materialized": "test", + "meta": {}, + "schema": "dbt_test__audit", + "severity": "ERROR", + "store_failures": null, + "tags": [], + "warn_if": "!= 0", + "where": null + }, + "contract": { + "checksum": null, + "enforced": false + }, + "created_at": 1687942823.3024309, + "database": "postgres", + "deferred": false, + "depends_on": { + "macros": [ + "macro.dbt.test_accepted_values", + "macro.dbt.get_where_subquery" + ], + "nodes": [ + "model.jaffle_shop.stg_orders" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "file_key_name": "models.stg_orders", + "fqn": [ + "jaffle_shop", + "staging", + "accepted_values_stg_orders_status__placed__shipped__completed__return_pending__returned" + ], + "group": null, + "language": "sql", + "meta": {}, + "metrics": [], + "name": "accepted_values_stg_orders_status__placed__shipped__completed__return_pending__returned", + "original_file_path": "models/staging/schema.yml", + "package_name": "jaffle_shop", + "patch_path": null, + "path": "accepted_values_stg_orders_4f514bf94b77b7ea437830eec4421c58.sql", + "raw_code": "{{ test_accepted_values(**_dbt_generic_test_kwargs) }}{{ config(alias=\"accepted_values_stg_orders_4f514bf94b77b7ea437830eec4421c58\") }}", + "refs": [ + { + "name": "stg_orders", + "package": null, + "version": null + } + ], + "relation_name": null, + "resource_type": "test", + "schema": "public_dbt_test__audit", + "sources": [], + "tags": [], + "test_metadata": { + "kwargs": { + "column_name": "status", + "model": "{{ get_where_subquery(ref('stg_orders')) }}", + "values": [ + "placed", + "shipped", + "completed", + "return_pending", + "returned" + ] + }, + "name": "accepted_values", + "namespace": null + }, + "unique_id": "test.jaffle_shop.accepted_values_stg_orders_status__placed__shipped__completed__return_pending__returned.080fb20aad", + "unrendered_config": { + "alias": "accepted_values_stg_orders_4f514bf94b77b7ea437830eec4421c58" + } + }, + "test.jaffle_shop.accepted_values_stg_payments_payment_method__credit_card__coupon__bank_transfer__gift_card.3c3820f278": { + "alias": "accepted_values_stg_payments_c7909fb19b1f0177c2bf99c7912f06ef", + "attached_node": "model.jaffle_shop.stg_payments", + "build_path": null, + "checksum": { + "checksum": "", + "name": "none" + }, + "column_name": "payment_method", + "columns": {}, + "compiled_path": null, + "config": { + "alias": "accepted_values_stg_payments_c7909fb19b1f0177c2bf99c7912f06ef", + "database": null, + "enabled": true, + "error_if": "!= 0", + "fail_calc": "count(*)", + "group": null, + "limit": null, + "materialized": "test", + "meta": {}, + "schema": "dbt_test__audit", + "severity": "ERROR", + "store_failures": null, + "tags": [], + "warn_if": "!= 0", + "where": null + }, + "contract": { + "checksum": null, + "enforced": false + }, + "created_at": 1687942823.308291, + "database": "postgres", + "deferred": false, + "depends_on": { + "macros": [ + "macro.dbt.test_accepted_values", + "macro.dbt.get_where_subquery" + ], + "nodes": [ + "model.jaffle_shop.stg_payments" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "file_key_name": "models.stg_payments", + "fqn": [ + "jaffle_shop", + "staging", + "accepted_values_stg_payments_payment_method__credit_card__coupon__bank_transfer__gift_card" + ], + "group": null, + "language": "sql", + "meta": {}, + "metrics": [], + "name": "accepted_values_stg_payments_payment_method__credit_card__coupon__bank_transfer__gift_card", + "original_file_path": "models/staging/schema.yml", + "package_name": "jaffle_shop", + "patch_path": null, + "path": "accepted_values_stg_payments_c7909fb19b1f0177c2bf99c7912f06ef.sql", + "raw_code": "{{ test_accepted_values(**_dbt_generic_test_kwargs) }}{{ config(alias=\"accepted_values_stg_payments_c7909fb19b1f0177c2bf99c7912f06ef\") }}", + "refs": [ + { + "name": "stg_payments", + "package": null, + "version": null + } + ], + "relation_name": null, + "resource_type": "test", + "schema": "public_dbt_test__audit", + "sources": [], + "tags": [], + "test_metadata": { + "kwargs": { + "column_name": "payment_method", + "model": "{{ get_where_subquery(ref('stg_payments')) }}", + "values": [ + "credit_card", + "coupon", + "bank_transfer", + "gift_card" + ] + }, + "name": "accepted_values", + "namespace": null + }, + "unique_id": "test.jaffle_shop.accepted_values_stg_payments_payment_method__credit_card__coupon__bank_transfer__gift_card.3c3820f278", + "unrendered_config": { + "alias": "accepted_values_stg_payments_c7909fb19b1f0177c2bf99c7912f06ef" + } + }, + "test.jaffle_shop.not_null_customers_customer_id.5c9bf9911d": { + "alias": "not_null_customers_customer_id", + "attached_node": "model.jaffle_shop.customers", + "build_path": null, + "checksum": { + "checksum": "", + "name": "none" + }, + "column_name": "customer_id", + "columns": {}, + "compiled_path": null, + "config": { + "alias": null, + "database": null, + "enabled": true, + "error_if": "!= 0", + "fail_calc": "count(*)", + "group": null, + "limit": null, + "materialized": "test", + "meta": {}, + "schema": "dbt_test__audit", + "severity": "ERROR", + "store_failures": null, + "tags": [], + "warn_if": "!= 0", + "where": null + }, + "contract": { + "checksum": null, + "enforced": false + }, + "created_at": 1687942823.272043, + "database": "postgres", + "deferred": false, + "depends_on": { + "macros": [ + "macro.dbt.test_not_null" + ], + "nodes": [ + "model.jaffle_shop.customers" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "file_key_name": "models.customers", + "fqn": [ + "jaffle_shop", + "not_null_customers_customer_id" + ], + "group": null, + "language": "sql", + "meta": {}, + "metrics": [], + "name": "not_null_customers_customer_id", + "original_file_path": "models/schema.yml", + "package_name": "jaffle_shop", + "patch_path": null, + "path": "not_null_customers_customer_id.sql", + "raw_code": "{{ test_not_null(**_dbt_generic_test_kwargs) }}", + "refs": [ + { + "name": "customers", + "package": null, + "version": null + } + ], + "relation_name": null, + "resource_type": "test", + "schema": "public_dbt_test__audit", + "sources": [], + "tags": [], + "test_metadata": { + "kwargs": { + "column_name": "customer_id", + "model": "{{ get_where_subquery(ref('customers')) }}" + }, + "name": "not_null", + "namespace": null + }, + "unique_id": "test.jaffle_shop.not_null_customers_customer_id.5c9bf9911d", + "unrendered_config": {} + }, + "test.jaffle_shop.not_null_orders_amount.106140f9fd": { + "alias": "not_null_orders_amount", + "attached_node": "model.jaffle_shop.orders", + "build_path": null, + "checksum": { + "checksum": "", + "name": "none" + }, + "column_name": "amount", + "columns": {}, + "compiled_path": null, + "config": { + "alias": null, + "database": null, + "enabled": true, + "error_if": "!= 0", + "fail_calc": "count(*)", + "group": null, + "limit": null, + "materialized": "test", + "meta": {}, + "schema": "dbt_test__audit", + "severity": "ERROR", + "store_failures": null, + "tags": [], + "warn_if": "!= 0", + "where": null + }, + "contract": { + "checksum": null, + "enforced": false + }, + "created_at": 1687942823.28982, + "database": "postgres", + "deferred": false, + "depends_on": { + "macros": [ + "macro.dbt.test_not_null" + ], + "nodes": [ + "model.jaffle_shop.orders" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "file_key_name": "models.orders", + "fqn": [ + "jaffle_shop", + "not_null_orders_amount" + ], + "group": null, + "language": "sql", + "meta": {}, + "metrics": [], + "name": "not_null_orders_amount", + "original_file_path": "models/schema.yml", + "package_name": "jaffle_shop", + "patch_path": null, + "path": "not_null_orders_amount.sql", + "raw_code": "{{ test_not_null(**_dbt_generic_test_kwargs) }}", + "refs": [ + { + "name": "orders", + "package": null, + "version": null + } + ], + "relation_name": null, + "resource_type": "test", + "schema": "public_dbt_test__audit", + "sources": [], + "tags": [], + "test_metadata": { + "kwargs": { + "column_name": "amount", + "model": "{{ get_where_subquery(ref('orders')) }}" + }, + "name": "not_null", + "namespace": null + }, + "unique_id": "test.jaffle_shop.not_null_orders_amount.106140f9fd", + "unrendered_config": {} + }, + "test.jaffle_shop.not_null_orders_bank_transfer_amount.7743500c49": { + "alias": "not_null_orders_bank_transfer_amount", + "attached_node": "model.jaffle_shop.orders", + "build_path": null, + "checksum": { + "checksum": "", + "name": "none" + }, + "column_name": "bank_transfer_amount", + "columns": {}, + "compiled_path": null, + "config": { + "alias": null, + "database": null, + "enabled": true, + "error_if": "!= 0", + "fail_calc": "count(*)", + "group": null, + "limit": null, + "materialized": "test", + "meta": {}, + "schema": "dbt_test__audit", + "severity": "ERROR", + "store_failures": null, + "tags": [], + "warn_if": "!= 0", + "where": null + }, + "contract": { + "checksum": null, + "enforced": false + }, + "created_at": 1687942823.2937589, + "database": "postgres", + "deferred": false, + "depends_on": { + "macros": [ + "macro.dbt.test_not_null" + ], + "nodes": [ + "model.jaffle_shop.orders" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "file_key_name": "models.orders", + "fqn": [ + "jaffle_shop", + "not_null_orders_bank_transfer_amount" + ], + "group": null, + "language": "sql", + "meta": {}, + "metrics": [], + "name": "not_null_orders_bank_transfer_amount", + "original_file_path": "models/schema.yml", + "package_name": "jaffle_shop", + "patch_path": null, + "path": "not_null_orders_bank_transfer_amount.sql", + "raw_code": "{{ test_not_null(**_dbt_generic_test_kwargs) }}", + "refs": [ + { + "name": "orders", + "package": null, + "version": null + } + ], + "relation_name": null, + "resource_type": "test", + "schema": "public_dbt_test__audit", + "sources": [], + "tags": [], + "test_metadata": { + "kwargs": { + "column_name": "bank_transfer_amount", + "model": "{{ get_where_subquery(ref('orders')) }}" + }, + "name": "not_null", + "namespace": null + }, + "unique_id": "test.jaffle_shop.not_null_orders_bank_transfer_amount.7743500c49", + "unrendered_config": {} + }, + "test.jaffle_shop.not_null_orders_coupon_amount.ab90c90625": { + "alias": "not_null_orders_coupon_amount", + "attached_node": "model.jaffle_shop.orders", + "build_path": null, + "checksum": { + "checksum": "", + "name": "none" + }, + "column_name": "coupon_amount", + "columns": {}, + "compiled_path": null, + "config": { + "alias": null, + "database": null, + "enabled": true, + "error_if": "!= 0", + "fail_calc": "count(*)", + "group": null, + "limit": null, + "materialized": "test", + "meta": {}, + "schema": "dbt_test__audit", + "severity": "ERROR", + "store_failures": null, + "tags": [], + "warn_if": "!= 0", + "where": null + }, + "contract": { + "checksum": null, + "enforced": false + }, + "created_at": 1687942823.292507, + "database": "postgres", + "deferred": false, + "depends_on": { + "macros": [ + "macro.dbt.test_not_null" + ], + "nodes": [ + "model.jaffle_shop.orders" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "file_key_name": "models.orders", + "fqn": [ + "jaffle_shop", + "not_null_orders_coupon_amount" + ], + "group": null, + "language": "sql", + "meta": {}, + "metrics": [], + "name": "not_null_orders_coupon_amount", + "original_file_path": "models/schema.yml", + "package_name": "jaffle_shop", + "patch_path": null, + "path": "not_null_orders_coupon_amount.sql", + "raw_code": "{{ test_not_null(**_dbt_generic_test_kwargs) }}", + "refs": [ + { + "name": "orders", + "package": null, + "version": null + } + ], + "relation_name": null, + "resource_type": "test", + "schema": "public_dbt_test__audit", + "sources": [], + "tags": [], + "test_metadata": { + "kwargs": { + "column_name": "coupon_amount", + "model": "{{ get_where_subquery(ref('orders')) }}" + }, + "name": "not_null", + "namespace": null + }, + "unique_id": "test.jaffle_shop.not_null_orders_coupon_amount.ab90c90625", + "unrendered_config": {} + }, + "test.jaffle_shop.not_null_orders_credit_card_amount.d3ca593b59": { + "alias": "not_null_orders_credit_card_amount", + "attached_node": "model.jaffle_shop.orders", + "build_path": null, + "checksum": { + "checksum": "", + "name": "none" + }, + "column_name": "credit_card_amount", + "columns": {}, + "compiled_path": null, + "config": { + "alias": null, + "database": null, + "enabled": true, + "error_if": "!= 0", + "fail_calc": "count(*)", + "group": null, + "limit": null, + "materialized": "test", + "meta": {}, + "schema": "dbt_test__audit", + "severity": "ERROR", + "store_failures": null, + "tags": [], + "warn_if": "!= 0", + "where": null + }, + "contract": { + "checksum": null, + "enforced": false + }, + "created_at": 1687942823.291426, + "database": "postgres", + "deferred": false, + "depends_on": { + "macros": [ + "macro.dbt.test_not_null" + ], + "nodes": [ + "model.jaffle_shop.orders" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "file_key_name": "models.orders", + "fqn": [ + "jaffle_shop", + "not_null_orders_credit_card_amount" + ], + "group": null, + "language": "sql", + "meta": {}, + "metrics": [], + "name": "not_null_orders_credit_card_amount", + "original_file_path": "models/schema.yml", + "package_name": "jaffle_shop", + "patch_path": null, + "path": "not_null_orders_credit_card_amount.sql", + "raw_code": "{{ test_not_null(**_dbt_generic_test_kwargs) }}", + "refs": [ + { + "name": "orders", + "package": null, + "version": null + } + ], + "relation_name": null, + "resource_type": "test", + "schema": "public_dbt_test__audit", + "sources": [], + "tags": [], + "test_metadata": { + "kwargs": { + "column_name": "credit_card_amount", + "model": "{{ get_where_subquery(ref('orders')) }}" + }, + "name": "not_null", + "namespace": null + }, + "unique_id": "test.jaffle_shop.not_null_orders_credit_card_amount.d3ca593b59", + "unrendered_config": {} + }, + "test.jaffle_shop.not_null_orders_customer_id.c5f02694af": { + "alias": "not_null_orders_customer_id", + "attached_node": "model.jaffle_shop.orders", + "build_path": null, + "checksum": { + "checksum": "", + "name": "none" + }, + "column_name": "customer_id", + "columns": {}, + "compiled_path": null, + "config": { + "alias": null, + "database": null, + "enabled": true, + "error_if": "!= 0", + "fail_calc": "count(*)", + "group": null, + "limit": null, + "materialized": "test", + "meta": {}, + "schema": "dbt_test__audit", + "severity": "ERROR", + "store_failures": null, + "tags": [], + "warn_if": "!= 0", + "where": null + }, + "contract": { + "checksum": null, + "enforced": false + }, + "created_at": 1687942823.275403, + "database": "postgres", + "deferred": false, + "depends_on": { + "macros": [ + "macro.dbt.test_not_null" + ], + "nodes": [ + "model.jaffle_shop.orders" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "file_key_name": "models.orders", + "fqn": [ + "jaffle_shop", + "not_null_orders_customer_id" + ], + "group": null, + "language": "sql", + "meta": {}, + "metrics": [], + "name": "not_null_orders_customer_id", + "original_file_path": "models/schema.yml", + "package_name": "jaffle_shop", + "patch_path": null, + "path": "not_null_orders_customer_id.sql", + "raw_code": "{{ test_not_null(**_dbt_generic_test_kwargs) }}", + "refs": [ + { + "name": "orders", + "package": null, + "version": null + } + ], + "relation_name": null, + "resource_type": "test", + "schema": "public_dbt_test__audit", + "sources": [], + "tags": [], + "test_metadata": { + "kwargs": { + "column_name": "customer_id", + "model": "{{ get_where_subquery(ref('orders')) }}" + }, + "name": "not_null", + "namespace": null + }, + "unique_id": "test.jaffle_shop.not_null_orders_customer_id.c5f02694af", + "unrendered_config": {} + }, + "test.jaffle_shop.not_null_orders_gift_card_amount.413a0d2d7a": { + "alias": "not_null_orders_gift_card_amount", + "attached_node": "model.jaffle_shop.orders", + "build_path": null, + "checksum": { + "checksum": "", + "name": "none" + }, + "column_name": "gift_card_amount", + "columns": {}, + "compiled_path": null, + "config": { + "alias": null, + "database": null, + "enabled": true, + "error_if": "!= 0", + "fail_calc": "count(*)", + "group": null, + "limit": null, + "materialized": "test", + "meta": {}, + "schema": "dbt_test__audit", + "severity": "ERROR", + "store_failures": null, + "tags": [], + "warn_if": "!= 0", + "where": null + }, + "contract": { + "checksum": null, + "enforced": false + }, + "created_at": 1687942823.294739, + "database": "postgres", + "deferred": false, + "depends_on": { + "macros": [ + "macro.dbt.test_not_null" + ], + "nodes": [ + "model.jaffle_shop.orders" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "file_key_name": "models.orders", + "fqn": [ + "jaffle_shop", + "not_null_orders_gift_card_amount" + ], + "group": null, + "language": "sql", + "meta": {}, + "metrics": [], + "name": "not_null_orders_gift_card_amount", + "original_file_path": "models/schema.yml", + "package_name": "jaffle_shop", + "patch_path": null, + "path": "not_null_orders_gift_card_amount.sql", + "raw_code": "{{ test_not_null(**_dbt_generic_test_kwargs) }}", + "refs": [ + { + "name": "orders", + "package": null, + "version": null + } + ], + "relation_name": null, + "resource_type": "test", + "schema": "public_dbt_test__audit", + "sources": [], + "tags": [], + "test_metadata": { + "kwargs": { + "column_name": "gift_card_amount", + "model": "{{ get_where_subquery(ref('orders')) }}" + }, + "name": "not_null", + "namespace": null + }, + "unique_id": "test.jaffle_shop.not_null_orders_gift_card_amount.413a0d2d7a", + "unrendered_config": {} + }, + "test.jaffle_shop.not_null_orders_order_id.cf6c17daed": { + "alias": "not_null_orders_order_id", + "attached_node": "model.jaffle_shop.orders", + "build_path": null, + "checksum": { + "checksum": "", + "name": "none" + }, + "column_name": "order_id", + "columns": {}, + "compiled_path": null, + "config": { + "alias": null, + "database": null, + "enabled": true, + "error_if": "!= 0", + "fail_calc": "count(*)", + "group": null, + "limit": null, + "materialized": "test", + "meta": {}, + "schema": "dbt_test__audit", + "severity": "ERROR", + "store_failures": null, + "tags": [], + "warn_if": "!= 0", + "where": null + }, + "contract": { + "checksum": null, + "enforced": false + }, + "created_at": 1687942823.274264, + "database": "postgres", + "deferred": false, + "depends_on": { + "macros": [ + "macro.dbt.test_not_null" + ], + "nodes": [ + "model.jaffle_shop.orders" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "file_key_name": "models.orders", + "fqn": [ + "jaffle_shop", + "not_null_orders_order_id" + ], + "group": null, + "language": "sql", + "meta": {}, + "metrics": [], + "name": "not_null_orders_order_id", + "original_file_path": "models/schema.yml", + "package_name": "jaffle_shop", + "patch_path": null, + "path": "not_null_orders_order_id.sql", + "raw_code": "{{ test_not_null(**_dbt_generic_test_kwargs) }}", + "refs": [ + { + "name": "orders", + "package": null, + "version": null + } + ], + "relation_name": null, + "resource_type": "test", + "schema": "public_dbt_test__audit", + "sources": [], + "tags": [], + "test_metadata": { + "kwargs": { + "column_name": "order_id", + "model": "{{ get_where_subquery(ref('orders')) }}" + }, + "name": "not_null", + "namespace": null + }, + "unique_id": "test.jaffle_shop.not_null_orders_order_id.cf6c17daed", + "unrendered_config": {} + }, + "test.jaffle_shop.not_null_stg_customers_customer_id.e2cfb1f9aa": { + "alias": "not_null_stg_customers_customer_id", + "attached_node": "model.jaffle_shop.stg_customers", + "build_path": null, + "checksum": { + "checksum": "", + "name": "none" + }, + "column_name": "customer_id", + "columns": {}, + "compiled_path": null, + "config": { + "alias": null, + "database": null, + "enabled": true, + "error_if": "!= 0", + "fail_calc": "count(*)", + "group": null, + "limit": null, + "materialized": "test", + "meta": {}, + "schema": "dbt_test__audit", + "severity": "ERROR", + "store_failures": null, + "tags": [], + "warn_if": "!= 0", + "where": null + }, + "contract": { + "checksum": null, + "enforced": false + }, + "created_at": 1687942823.29916, + "database": "postgres", + "deferred": false, + "depends_on": { + "macros": [ + "macro.dbt.test_not_null" + ], + "nodes": [ + "model.jaffle_shop.stg_customers" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "file_key_name": "models.stg_customers", + "fqn": [ + "jaffle_shop", + "staging", + "not_null_stg_customers_customer_id" + ], + "group": null, + "language": "sql", + "meta": {}, + "metrics": [], + "name": "not_null_stg_customers_customer_id", + "original_file_path": "models/staging/schema.yml", + "package_name": "jaffle_shop", + "patch_path": null, + "path": "not_null_stg_customers_customer_id.sql", + "raw_code": "{{ test_not_null(**_dbt_generic_test_kwargs) }}", + "refs": [ + { + "name": "stg_customers", + "package": null, + "version": null + } + ], + "relation_name": null, + "resource_type": "test", + "schema": "public_dbt_test__audit", + "sources": [], + "tags": [], + "test_metadata": { + "kwargs": { + "column_name": "customer_id", + "model": "{{ get_where_subquery(ref('stg_customers')) }}" + }, + "name": "not_null", + "namespace": null + }, + "unique_id": "test.jaffle_shop.not_null_stg_customers_customer_id.e2cfb1f9aa", + "unrendered_config": {} + }, + "test.jaffle_shop.not_null_stg_orders_order_id.81cfe2fe64": { + "alias": "not_null_stg_orders_order_id", + "attached_node": "model.jaffle_shop.stg_orders", + "build_path": null, + "checksum": { + "checksum": "", + "name": "none" + }, + "column_name": "order_id", + "columns": {}, + "compiled_path": null, + "config": { + "alias": null, + "database": null, + "enabled": true, + "error_if": "!= 0", + "fail_calc": "count(*)", + "group": null, + "limit": null, + "materialized": "test", + "meta": {}, + "schema": "dbt_test__audit", + "severity": "ERROR", + "store_failures": null, + "tags": [], + "warn_if": "!= 0", + "where": null + }, + "contract": { + "checksum": null, + "enforced": false + }, + "created_at": 1687942823.301339, + "database": "postgres", + "deferred": false, + "depends_on": { + "macros": [ + "macro.dbt.test_not_null" + ], + "nodes": [ + "model.jaffle_shop.stg_orders" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "file_key_name": "models.stg_orders", + "fqn": [ + "jaffle_shop", + "staging", + "not_null_stg_orders_order_id" + ], + "group": null, + "language": "sql", + "meta": {}, + "metrics": [], + "name": "not_null_stg_orders_order_id", + "original_file_path": "models/staging/schema.yml", + "package_name": "jaffle_shop", + "patch_path": null, + "path": "not_null_stg_orders_order_id.sql", + "raw_code": "{{ test_not_null(**_dbt_generic_test_kwargs) }}", + "refs": [ + { + "name": "stg_orders", + "package": null, + "version": null + } + ], + "relation_name": null, + "resource_type": "test", + "schema": "public_dbt_test__audit", + "sources": [], + "tags": [], + "test_metadata": { + "kwargs": { + "column_name": "order_id", + "model": "{{ get_where_subquery(ref('stg_orders')) }}" + }, + "name": "not_null", + "namespace": null + }, + "unique_id": "test.jaffle_shop.not_null_stg_orders_order_id.81cfe2fe64", + "unrendered_config": {} + }, + "test.jaffle_shop.not_null_stg_payments_payment_id.c19cc50075": { + "alias": "not_null_stg_payments_payment_id", + "attached_node": "model.jaffle_shop.stg_payments", + "build_path": null, + "checksum": { + "checksum": "", + "name": "none" + }, + "column_name": "payment_id", + "columns": {}, + "compiled_path": null, + "config": { + "alias": null, + "database": null, + "enabled": true, + "error_if": "!= 0", + "fail_calc": "count(*)", + "group": null, + "limit": null, + "materialized": "test", + "meta": {}, + "schema": "dbt_test__audit", + "severity": "ERROR", + "store_failures": null, + "tags": [], + "warn_if": "!= 0", + "where": null + }, + "contract": { + "checksum": null, + "enforced": false + }, + "created_at": 1687942823.3071492, + "database": "postgres", + "deferred": false, + "depends_on": { + "macros": [ + "macro.dbt.test_not_null" + ], + "nodes": [ + "model.jaffle_shop.stg_payments" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "file_key_name": "models.stg_payments", + "fqn": [ + "jaffle_shop", + "staging", + "not_null_stg_payments_payment_id" + ], + "group": null, + "language": "sql", + "meta": {}, + "metrics": [], + "name": "not_null_stg_payments_payment_id", + "original_file_path": "models/staging/schema.yml", + "package_name": "jaffle_shop", + "patch_path": null, + "path": "not_null_stg_payments_payment_id.sql", + "raw_code": "{{ test_not_null(**_dbt_generic_test_kwargs) }}", + "refs": [ + { + "name": "stg_payments", + "package": null, + "version": null + } + ], + "relation_name": null, + "resource_type": "test", + "schema": "public_dbt_test__audit", + "sources": [], + "tags": [], + "test_metadata": { + "kwargs": { + "column_name": "payment_id", + "model": "{{ get_where_subquery(ref('stg_payments')) }}" + }, + "name": "not_null", + "namespace": null + }, + "unique_id": "test.jaffle_shop.not_null_stg_payments_payment_id.c19cc50075", + "unrendered_config": {} + }, + "test.jaffle_shop.relationships_orders_customer_id__customer_id__ref_customers_.c6ec7f58f2": { + "alias": "relationships_orders_customer_id__customer_id__ref_customers_", + "attached_node": "model.jaffle_shop.orders", + "build_path": null, + "checksum": { + "checksum": "", + "name": "none" + }, + "column_name": "customer_id", + "columns": {}, + "compiled_path": null, + "config": { + "alias": null, + "database": null, + "enabled": true, + "error_if": "!= 0", + "fail_calc": "count(*)", + "group": null, + "limit": null, + "materialized": "test", + "meta": {}, + "schema": "dbt_test__audit", + "severity": "ERROR", + "store_failures": null, + "tags": [], + "warn_if": "!= 0", + "where": null + }, + "contract": { + "checksum": null, + "enforced": false + }, + "created_at": 1687942823.276394, + "database": "postgres", + "deferred": false, + "depends_on": { + "macros": [ + "macro.dbt.test_relationships", + "macro.dbt.get_where_subquery" + ], + "nodes": [ + "model.jaffle_shop.customers", + "model.jaffle_shop.orders" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "file_key_name": "models.orders", + "fqn": [ + "jaffle_shop", + "relationships_orders_customer_id__customer_id__ref_customers_" + ], + "group": null, + "language": "sql", + "meta": {}, + "metrics": [], + "name": "relationships_orders_customer_id__customer_id__ref_customers_", + "original_file_path": "models/schema.yml", + "package_name": "jaffle_shop", + "patch_path": null, + "path": "relationships_orders_customer_id__customer_id__ref_customers_.sql", + "raw_code": "{{ test_relationships(**_dbt_generic_test_kwargs) }}", + "refs": [ + { + "name": "customers", + "package": null, + "version": null + }, + { + "name": "orders", + "package": null, + "version": null + } + ], + "relation_name": null, + "resource_type": "test", + "schema": "public_dbt_test__audit", + "sources": [], + "tags": [], + "test_metadata": { + "kwargs": { + "column_name": "customer_id", + "field": "customer_id", + "model": "{{ get_where_subquery(ref('orders')) }}", + "to": "ref('customers')" + }, + "name": "relationships", + "namespace": null + }, + "unique_id": "test.jaffle_shop.relationships_orders_customer_id__customer_id__ref_customers_.c6ec7f58f2", + "unrendered_config": {} + }, + "test.jaffle_shop.unique_customers_customer_id.c5af1ff4b1": { + "alias": "unique_customers_customer_id", + "attached_node": "model.jaffle_shop.customers", + "build_path": null, + "checksum": { + "checksum": "", + "name": "none" + }, + "column_name": "customer_id", + "columns": {}, + "compiled_path": null, + "config": { + "alias": null, + "database": null, + "enabled": true, + "error_if": "!= 0", + "fail_calc": "count(*)", + "group": null, + "limit": null, + "materialized": "test", + "meta": {}, + "schema": "dbt_test__audit", + "severity": "ERROR", + "store_failures": null, + "tags": [], + "warn_if": "!= 0", + "where": null + }, + "contract": { + "checksum": null, + "enforced": false + }, + "created_at": 1687942823.270796, + "database": "postgres", + "deferred": false, + "depends_on": { + "macros": [ + "macro.dbt.test_unique" + ], + "nodes": [ + "model.jaffle_shop.customers" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "file_key_name": "models.customers", + "fqn": [ + "jaffle_shop", + "unique_customers_customer_id" + ], + "group": null, + "language": "sql", + "meta": {}, + "metrics": [], + "name": "unique_customers_customer_id", + "original_file_path": "models/schema.yml", + "package_name": "jaffle_shop", + "patch_path": null, + "path": "unique_customers_customer_id.sql", + "raw_code": "{{ test_unique(**_dbt_generic_test_kwargs) }}", + "refs": [ + { + "name": "customers", + "package": null, + "version": null + } + ], + "relation_name": null, + "resource_type": "test", + "schema": "public_dbt_test__audit", + "sources": [], + "tags": [], + "test_metadata": { + "kwargs": { + "column_name": "customer_id", + "model": "{{ get_where_subquery(ref('customers')) }}" + }, + "name": "unique", + "namespace": null + }, + "unique_id": "test.jaffle_shop.unique_customers_customer_id.c5af1ff4b1", + "unrendered_config": {} + }, + "test.jaffle_shop.unique_orders_order_id.fed79b3a6e": { + "alias": "unique_orders_order_id", + "attached_node": "model.jaffle_shop.orders", + "build_path": null, + "checksum": { + "checksum": "", + "name": "none" + }, + "column_name": "order_id", + "columns": {}, + "compiled_path": null, + "config": { + "alias": null, + "database": null, + "enabled": true, + "error_if": "!= 0", + "fail_calc": "count(*)", + "group": null, + "limit": null, + "materialized": "test", + "meta": {}, + "schema": "dbt_test__audit", + "severity": "ERROR", + "store_failures": null, + "tags": [], + "warn_if": "!= 0", + "where": null + }, + "contract": { + "checksum": null, + "enforced": false + }, + "created_at": 1687942823.273182, + "database": "postgres", + "deferred": false, + "depends_on": { + "macros": [ + "macro.dbt.test_unique" + ], + "nodes": [ + "model.jaffle_shop.orders" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "file_key_name": "models.orders", + "fqn": [ + "jaffle_shop", + "unique_orders_order_id" + ], + "group": null, + "language": "sql", + "meta": {}, + "metrics": [], + "name": "unique_orders_order_id", + "original_file_path": "models/schema.yml", + "package_name": "jaffle_shop", + "patch_path": null, + "path": "unique_orders_order_id.sql", + "raw_code": "{{ test_unique(**_dbt_generic_test_kwargs) }}", + "refs": [ + { + "name": "orders", + "package": null, + "version": null + } + ], + "relation_name": null, + "resource_type": "test", + "schema": "public_dbt_test__audit", + "sources": [], + "tags": [], + "test_metadata": { + "kwargs": { + "column_name": "order_id", + "model": "{{ get_where_subquery(ref('orders')) }}" + }, + "name": "unique", + "namespace": null + }, + "unique_id": "test.jaffle_shop.unique_orders_order_id.fed79b3a6e", + "unrendered_config": {} + }, + "test.jaffle_shop.unique_stg_customers_customer_id.c7614daada": { + "alias": "unique_stg_customers_customer_id", + "attached_node": "model.jaffle_shop.stg_customers", + "build_path": null, + "checksum": { + "checksum": "", + "name": "none" + }, + "column_name": "customer_id", + "columns": {}, + "compiled_path": null, + "config": { + "alias": null, + "database": null, + "enabled": true, + "error_if": "!= 0", + "fail_calc": "count(*)", + "group": null, + "limit": null, + "materialized": "test", + "meta": {}, + "schema": "dbt_test__audit", + "severity": "ERROR", + "store_failures": null, + "tags": [], + "warn_if": "!= 0", + "where": null + }, + "contract": { + "checksum": null, + "enforced": false + }, + "created_at": 1687942823.298065, + "database": "postgres", + "deferred": false, + "depends_on": { + "macros": [ + "macro.dbt.test_unique" + ], + "nodes": [ + "model.jaffle_shop.stg_customers" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "file_key_name": "models.stg_customers", + "fqn": [ + "jaffle_shop", + "staging", + "unique_stg_customers_customer_id" + ], + "group": null, + "language": "sql", + "meta": {}, + "metrics": [], + "name": "unique_stg_customers_customer_id", + "original_file_path": "models/staging/schema.yml", + "package_name": "jaffle_shop", + "patch_path": null, + "path": "unique_stg_customers_customer_id.sql", + "raw_code": "{{ test_unique(**_dbt_generic_test_kwargs) }}", + "refs": [ + { + "name": "stg_customers", + "package": null, + "version": null + } + ], + "relation_name": null, + "resource_type": "test", + "schema": "public_dbt_test__audit", + "sources": [], + "tags": [], + "test_metadata": { + "kwargs": { + "column_name": "customer_id", + "model": "{{ get_where_subquery(ref('stg_customers')) }}" + }, + "name": "unique", + "namespace": null + }, + "unique_id": "test.jaffle_shop.unique_stg_customers_customer_id.c7614daada", + "unrendered_config": {} + }, + "test.jaffle_shop.unique_stg_orders_order_id.e3b841c71a": { + "alias": "unique_stg_orders_order_id", + "attached_node": "model.jaffle_shop.stg_orders", + "build_path": null, + "checksum": { + "checksum": "", + "name": "none" + }, + "column_name": "order_id", + "columns": {}, + "compiled_path": null, + "config": { + "alias": null, + "database": null, + "enabled": true, + "error_if": "!= 0", + "fail_calc": "count(*)", + "group": null, + "limit": null, + "materialized": "test", + "meta": {}, + "schema": "dbt_test__audit", + "severity": "ERROR", + "store_failures": null, + "tags": [], + "warn_if": "!= 0", + "where": null + }, + "contract": { + "checksum": null, + "enforced": false + }, + "created_at": 1687942823.3003101, + "database": "postgres", + "deferred": false, + "depends_on": { + "macros": [ + "macro.dbt.test_unique" + ], + "nodes": [ + "model.jaffle_shop.stg_orders" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "file_key_name": "models.stg_orders", + "fqn": [ + "jaffle_shop", + "staging", + "unique_stg_orders_order_id" + ], + "group": null, + "language": "sql", + "meta": {}, + "metrics": [], + "name": "unique_stg_orders_order_id", + "original_file_path": "models/staging/schema.yml", + "package_name": "jaffle_shop", + "patch_path": null, + "path": "unique_stg_orders_order_id.sql", + "raw_code": "{{ test_unique(**_dbt_generic_test_kwargs) }}", + "refs": [ + { + "name": "stg_orders", + "package": null, + "version": null + } + ], + "relation_name": null, + "resource_type": "test", + "schema": "public_dbt_test__audit", + "sources": [], + "tags": [], + "test_metadata": { + "kwargs": { + "column_name": "order_id", + "model": "{{ get_where_subquery(ref('stg_orders')) }}" + }, + "name": "unique", + "namespace": null + }, + "unique_id": "test.jaffle_shop.unique_stg_orders_order_id.e3b841c71a", + "unrendered_config": {} + }, + "test.jaffle_shop.unique_stg_payments_payment_id.3744510712": { + "alias": "unique_stg_payments_payment_id", + "attached_node": "model.jaffle_shop.stg_payments", + "build_path": null, + "checksum": { + "checksum": "", + "name": "none" + }, + "column_name": "payment_id", + "columns": {}, + "compiled_path": null, + "config": { + "alias": null, + "database": null, + "enabled": true, + "error_if": "!= 0", + "fail_calc": "count(*)", + "group": null, + "limit": null, + "materialized": "test", + "meta": {}, + "schema": "dbt_test__audit", + "severity": "ERROR", + "store_failures": null, + "tags": [], + "warn_if": "!= 0", + "where": null + }, + "contract": { + "checksum": null, + "enforced": false + }, + "created_at": 1687942823.3060858, + "database": "postgres", + "deferred": false, + "depends_on": { + "macros": [ + "macro.dbt.test_unique" + ], + "nodes": [ + "model.jaffle_shop.stg_payments" + ] + }, + "description": "", + "docs": { + "node_color": null, + "show": true + }, + "file_key_name": "models.stg_payments", + "fqn": [ + "jaffle_shop", + "staging", + "unique_stg_payments_payment_id" + ], + "group": null, + "language": "sql", + "meta": {}, + "metrics": [], + "name": "unique_stg_payments_payment_id", + "original_file_path": "models/staging/schema.yml", + "package_name": "jaffle_shop", + "patch_path": null, + "path": "unique_stg_payments_payment_id.sql", + "raw_code": "{{ test_unique(**_dbt_generic_test_kwargs) }}", + "refs": [ + { + "name": "stg_payments", + "package": null, + "version": null + } + ], + "relation_name": null, + "resource_type": "test", + "schema": "public_dbt_test__audit", + "sources": [], + "tags": [], + "test_metadata": { + "kwargs": { + "column_name": "payment_id", + "model": "{{ get_where_subquery(ref('stg_payments')) }}" + }, + "name": "unique", + "namespace": null + }, + "unique_id": "test.jaffle_shop.unique_stg_payments_payment_id.3744510712", + "unrendered_config": {} + } + }, + "parent_map": { + "model.jaffle_shop.customers": [ + "model.jaffle_shop.stg_customers", + "model.jaffle_shop.stg_orders", + "model.jaffle_shop.stg_payments" + ], + "model.jaffle_shop.orders": [ + "model.jaffle_shop.stg_orders", + "model.jaffle_shop.stg_payments" + ], + "model.jaffle_shop.stg_customers": [ + "seed.jaffle_shop.raw_customers" + ], + "model.jaffle_shop.stg_orders": [ + "seed.jaffle_shop.raw_orders" + ], + "model.jaffle_shop.stg_payments": [ + "seed.jaffle_shop.raw_payments" + ], + "seed.jaffle_shop.raw_customers": [], + "seed.jaffle_shop.raw_orders": [], + "seed.jaffle_shop.raw_payments": [], + "test.jaffle_shop.accepted_values_orders_status__placed__shipped__completed__return_pending__returned.be6b5b5ec3": [ + "model.jaffle_shop.orders" + ], + "test.jaffle_shop.accepted_values_stg_orders_status__placed__shipped__completed__return_pending__returned.080fb20aad": [ + "model.jaffle_shop.stg_orders" + ], + "test.jaffle_shop.accepted_values_stg_payments_payment_method__credit_card__coupon__bank_transfer__gift_card.3c3820f278": [ + "model.jaffle_shop.stg_payments" + ], + "test.jaffle_shop.not_null_customers_customer_id.5c9bf9911d": [ + "model.jaffle_shop.customers" + ], + "test.jaffle_shop.not_null_orders_amount.106140f9fd": [ + "model.jaffle_shop.orders" + ], + "test.jaffle_shop.not_null_orders_bank_transfer_amount.7743500c49": [ + "model.jaffle_shop.orders" + ], + "test.jaffle_shop.not_null_orders_coupon_amount.ab90c90625": [ + "model.jaffle_shop.orders" + ], + "test.jaffle_shop.not_null_orders_credit_card_amount.d3ca593b59": [ + "model.jaffle_shop.orders" + ], + "test.jaffle_shop.not_null_orders_customer_id.c5f02694af": [ + "model.jaffle_shop.orders" + ], + "test.jaffle_shop.not_null_orders_gift_card_amount.413a0d2d7a": [ + "model.jaffle_shop.orders" + ], + "test.jaffle_shop.not_null_orders_order_id.cf6c17daed": [ + "model.jaffle_shop.orders" + ], + "test.jaffle_shop.not_null_stg_customers_customer_id.e2cfb1f9aa": [ + "model.jaffle_shop.stg_customers" + ], + "test.jaffle_shop.not_null_stg_orders_order_id.81cfe2fe64": [ + "model.jaffle_shop.stg_orders" + ], + "test.jaffle_shop.not_null_stg_payments_payment_id.c19cc50075": [ + "model.jaffle_shop.stg_payments" + ], + "test.jaffle_shop.relationships_orders_customer_id__customer_id__ref_customers_.c6ec7f58f2": [ + "model.jaffle_shop.customers", + "model.jaffle_shop.orders" + ], + "test.jaffle_shop.unique_customers_customer_id.c5af1ff4b1": [ + "model.jaffle_shop.customers" + ], + "test.jaffle_shop.unique_orders_order_id.fed79b3a6e": [ + "model.jaffle_shop.orders" + ], + "test.jaffle_shop.unique_stg_customers_customer_id.c7614daada": [ + "model.jaffle_shop.stg_customers" + ], + "test.jaffle_shop.unique_stg_orders_order_id.e3b841c71a": [ + "model.jaffle_shop.stg_orders" + ], + "test.jaffle_shop.unique_stg_payments_payment_id.3744510712": [ + "model.jaffle_shop.stg_payments" + ] + }, + "selectors": {}, + "sources": {} +} diff --git a/tests/test_airflow.py b/tests/test_airflow.py deleted file mode 100644 index b47a6e826..000000000 --- a/tests/test_airflow.py +++ /dev/null @@ -1,258 +0,0 @@ -"""Tests for the Airflow integration.""" -from __future__ import annotations - -from airflow.models.baseoperator import BaseOperator -from airflow.models.taskmixin import DAGNode - -try: - from typing import TypedDict -except ImportError: - from typing_extensions import TypedDict - -import pendulum -import pytest -from airflow.models import DAG - -from cosmos.core.airflow import CosmosDag, CosmosTaskGroup -from cosmos.core.graph.entities import CosmosEntity, Group, Task - - -def get_tasks(dag_node: DAGNode) -> list[BaseOperator]: - """ - Get a list of all tasks in a DAGNode. - """ - tasks = [] - if isinstance(dag_node, BaseOperator): - tasks.append(dag_node) - elif isinstance(dag_node, DAG): - tasks.extend(dag_node.tasks) - elif isinstance(dag_node, CosmosTaskGroup): - for task in dag_node.children.values(): - tasks.extend(get_tasks(task)) - else: - raise TypeError(f"Unknown type: {type(dag_node)}") - - return tasks - - -def simple_group() -> Group: - """ - A Group containing: - - a single Task - """ - group = Group(id="simple_group") - group.add_entity(Task(id="task_1")) - - return group - - -def nested_group() -> Group: - """ - A Group containing: - - a single Task - - a Group with two Tasks - """ - group = Group(id="nested_group") - group.add_entity(Task(id="task_1")) - - sub_group = Group(id="group_2") - sub_group.add_entity(Task(id="task_2")) - sub_group.add_entity(Task(id="task_3")) - - group.add_entity(sub_group) - - return group - - -def nested_group_no_add_entity() -> Group: - """ - A Group containing: - - a single Task - - a Group with two Tasks - - This is the same as nested_group, but uses the `entities` attribute - instead of the `add_entity` method. - """ - task_1 = Task(id="task_1") - task_2 = Task(id="task_2") - task_3 = Task(id="task_3") - - sub_group = Group( - id="group_2", - entities=[task_2, task_3], - ) - - group = Group( - id="nested_group_no_add_entity", - entities=[task_1, sub_group], - ) - - return group - - -def nested_group_with_upstream() -> Group: - """ - A Group containing: - - a single Task - - a Group with two Tasks - """ - group = Group(id="nested_group_with_upstream") - task_1 = Task(id="task_1") - group.add_entity(task_1) - - sub_group = Group(id="group_2") - task_2 = Task(id="task_2") - sub_group.add_entity(task_2) - task_3 = Task(id="task_3") - sub_group.add_entity(task_3) - - sub_group.add_upstream(task_1) - - group.add_entity(sub_group) - - return group - - -def double_nested_groups() -> Group: - """ - A Group containing: - - a single Task - - a Group with two Tasks and a nested Group - """ - group = Group(id="double_nested_groups") - task_1 = Task(id="task_1") - group.add_entity(task_1) - - sub_group = Group(id="group_2") - task_2 = Task(id="task_2") - sub_group.add_entity(task_2) - - sub_sub_group = Group(id="group_3") - task_3 = Task(id="task_3") - sub_sub_group.add_entity(task_3) - task_4 = Task(id="task_4") - sub_sub_group.add_entity(task_4) - - sub_group.add_entity(sub_sub_group) - - group.add_entity(sub_group) - - return group - - -class AirflowTestTask(TypedDict): - """A test task for Airflow.""" - - operator_class: str - arguments: dict[str, str] - upstream_entity_ids: list[str] - - -@pytest.mark.parametrize( - "group,num_tasks", - [ - pytest.param(simple_group(), 1, id="simple_group"), - pytest.param(nested_group(), 3, id="nested_group"), - pytest.param( - nested_group_no_add_entity(), - 3, - id="nested_group_no_add_entity", - ), - pytest.param( - nested_group_with_upstream(), - 3, - id="nested_group_with_upstream", - ), - pytest.param(double_nested_groups(), 4, id="double_nested_groups"), - ], -) -def test_cosmos_dag_and_task_group( - group: Group, - num_tasks: int, -) -> None: - """ - Tests that the CosmosDag properly renders a Group as an Airflow DAG. - """ - expected: dict[str, AirflowTestTask] = {} - - def flatten_entities( - entities: list[CosmosEntity], - upstream_from_parent: list[str] | None = None, - ) -> None: - """ - Flatten a list of CosmosEntities into a list of AirflowTestTasks. - """ - upstreams = upstream_from_parent or [] - for entity in entities: - if isinstance(entity, Task): - expected[entity.id] = { - "operator_class": entity.operator_class, - "arguments": entity.arguments, - "upstream_entity_ids": entity.upstream_entity_ids + upstreams, - } - elif isinstance(entity, Group): - flatten_entities(entity.entities, upstreams + entity.upstream_entity_ids) - - flatten_entities(group.entities) - - ############################ - # CosmosDag - ############################ - dag = CosmosDag( - start_date=pendulum.datetime(2021, 1, 1), - cosmos_group=group, - ) - - # basic dag checks - assert dag.dag_id == group.id - assert dag.task_count == num_tasks - - for task in dag.tasks: - task_id = task.task_id.split(".")[-1] - cosmos_task = expected[task_id] - - assert task_id in expected - - class_name = str(type(task)) - assert class_name == f"" - - airflow_upstream_task_ids = list(task.upstream_task_ids) - assert airflow_upstream_task_ids == cosmos_task["upstream_entity_ids"] - - ############################ - # CosmosTaskGroup - ############################ - with DAG(dag_id="test", start_date=pendulum.datetime(2021, 1, 1)) as dag: - task_group = CosmosTaskGroup( - cosmos_group=group, - ) - - tasks = get_tasks(task_group) - - # basic task group checks - assert task_group.group_id == group.id - assert len(tasks) == num_tasks - - for tg_task in tasks: - task_id = tg_task.task_id.split(".")[-1] - cosmos_task = expected[task_id] - - assert task_id in expected - - class_name = str(type(tg_task)) - assert class_name == f"" - - airflow_upstream_task_ids = [id.split(".")[-1] for id in list(tg_task.upstream_task_ids)] - assert airflow_upstream_task_ids == cosmos_task["upstream_entity_ids"] - - -def test_invalid_operator() -> None: - """Tests that an invalid operator raises an error.""" - with pytest.raises(ValueError): - group = Group(id="group_1") - group.add_entity(Task(id="task_1", operator_class="InvalidOperator")) - - CosmosDag( - start_date=pendulum.datetime(2021, 1, 1), - cosmos_group=group, - ) diff --git a/tests/test_converter.py b/tests/test_converter.py new file mode 100644 index 000000000..3df683734 --- /dev/null +++ b/tests/test_converter.py @@ -0,0 +1,17 @@ +import pytest +from airflow.exceptions import AirflowException + +from cosmos.converter import validate_arguments + + +@pytest.mark.parametrize("argument_key", ["tags", "paths"]) +def test_validate_arguments_tags(argument_key): + selector_name = argument_key[:-1] + select = [f"{selector_name}:a,{selector_name}:b"] + exclude = [f"{selector_name}:b,{selector_name}:c"] + profile_args = {} + task_args = {} + with pytest.raises(AirflowException) as err: + validate_arguments(select, exclude, profile_args, task_args) + expected = f"Can't specify the same {selector_name} in `select` and `exclude`: {{'b'}}" + assert err.value.args[0] == expected diff --git a/tests/test_render.py b/tests/test_render.py deleted file mode 100644 index e92dccd52..000000000 --- a/tests/test_render.py +++ /dev/null @@ -1,136 +0,0 @@ -from pathlib import Path - -import pytest -from airflow.exceptions import AirflowException - -from cosmos.core.graph.entities import Group -from cosmos.render import calculate_operator_class, render_project - -DBT_PROJECT_PATH = Path(__name__).parent.parent.parent.parent.parent / "dev/dags/dbt/" - - -def test_calculate_operator_class(): - class_module_import_path = calculate_operator_class(execution_mode="kubernetes", dbt_class="Seed") - assert class_module_import_path == "cosmos.operators.kubernetes.SeedKubernetesOperator" - - -def test_render_project_default(): - computed = render_project(dbt_project_name="jaffle_shop", dbt_root_path=DBT_PROJECT_PATH) - assert isinstance(computed, Group) - assert computed.id == "jaffle_shop" - assert len(computed.entities) == 8 - entities_ids = [entity.id for entity in computed.entities] - expected_ids = [ - "customers", - "orders", - "raw_customers_seed", - "raw_orders_seed", - "raw_payments_seed", - "stg_customers", - "stg_orders", - "stg_payments", - ] - assert sorted(entities_ids) == expected_ids - - -def test_render_project_test_behavior_none(): - computed = render_project( - dbt_project_name="jaffle_shop", - dbt_root_path=DBT_PROJECT_PATH, - test_behavior="none", - ) - assert isinstance(computed, Group) - assert computed.id == "jaffle_shop" - assert len(computed.entities) == 8 - entities_ids = [entity.id for entity in computed.entities] - expected_ids = [ - "customers_run", - "orders_run", - "raw_customers_seed", - "raw_orders_seed", - "raw_payments_seed", - "stg_customers_run", - "stg_orders_run", - "stg_payments_run", - ] - assert sorted(entities_ids) == expected_ids - - -def test_render_project_test_behavior_after_all(): - computed = render_project( - dbt_project_name="jaffle_shop", - dbt_root_path=DBT_PROJECT_PATH, - test_behavior="after_all", - ) - assert isinstance(computed, Group) - assert computed.id == "jaffle_shop" - assert len(computed.entities) == 9 - entities_ids = [entity.id for entity in computed.entities] - expected_ids = [ - "customers_run", - "jaffle_shop_test", - "orders_run", - "raw_customers_seed", - "raw_orders_seed", - "raw_payments_seed", - "stg_customers_run", - "stg_orders_run", - "stg_payments_run", - ] - assert sorted(entities_ids) == expected_ids - - -@pytest.mark.parametrize( - "selectors,error_message", - [ - ( - {"tags": ["tag_1"]}, - "Can't specify the same tag in `select` and `include`: {'tag_1'}", - ), - ( - {"paths": ["/tmp"]}, - "Can't specify the same path in `select` and `include`: {'/tmp'}", - ), - ], - ids=["tags", "paths"], -) -def test_render_project_select_and_exclude_conflict(selectors, error_message): - with pytest.raises(AirflowException) as exc_info: - render_project( - dbt_project_name="jaffle_shop", - dbt_root_path=DBT_PROJECT_PATH, - select=selectors, - exclude=selectors, - ) - computed = exc_info.value.args[0] - assert computed == error_message - - -def test_render_project_select_models_by_including_path(): - computed = render_project( - dbt_project_name="jaffle_shop", - dbt_root_path=DBT_PROJECT_PATH, - select={"paths": ["models/staging/"]}, - ) - assert len(computed.entities) == 3 - entities_ids = [entity.id for entity in computed.entities] - expected_ids = ["stg_customers", "stg_orders", "stg_payments"] - assert sorted(entities_ids) == expected_ids - - -def test_render_project_select_models_by_excluding_path(): - computed = render_project( - dbt_project_name="jaffle_shop", - dbt_root_path=DBT_PROJECT_PATH, - exclude={"paths": ["models/staging/"]}, - ) - assert len(computed.entities) == 5 - entities_ids = [entity.id for entity in computed.entities] - expected_ids = [ - "customers", - "orders", - "raw_customers_seed", - "raw_orders_seed", - "raw_payments_seed", - ] - assert sorted(entities_ids) == expected_ids