Skip to content

Commit

Permalink
WIP
Browse files Browse the repository at this point in the history
  • Loading branch information
courtneyholcomb committed Jul 22, 2024
1 parent 33161a3 commit 06df37e
Show file tree
Hide file tree
Showing 51 changed files with 196 additions and 28 deletions.
1 change: 1 addition & 0 deletions core/dbt/artifacts/resources/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -86,6 +86,7 @@
SourceDefinition,
)
from dbt.artifacts.resources.v1.sql_operation import SqlOperation
from dbt.artifacts.resources.v1.time_spines import TimeSpine
from dbt.artifacts.resources.v1.unit_test_definition import (
UnitTestConfig,
UnitTestDefinition,
Expand Down
35 changes: 35 additions & 0 deletions core/dbt/artifacts/resources/v1/time_spines.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,35 @@
from dbt.artifacts.resources.base import GraphResource
from dataclasses import dataclass, field
from typing import List
from dbt_common.dataclass_schema import dbtClassMixin
from dbt.artifacts.resources.v1.semantic_model import NodeRelation
from dbt_semantic_interfaces.type_enums.time_granularity import TimeGranularity
from dbt.artifacts.resources.v1.components import DependsOn, RefArgs


# ====================================
# TimeSpine objects
# TimeSpine protocols: https://github.com/dbt-labs/dbt-semantic-interfaces/blob/main/dbt_semantic_interfaces/protocols/time_spine.py
# ====================================


@dataclass
class TimeSpinePrimaryColumn(dbtClassMixin):
"""The column in the time spine that maps to a standard granularityf."""

name: str
time_granularity: TimeGranularity


@dataclass
class TimeSpine(GraphResource):
"""Describes a table that contains dates at a specific time grain.
One column must map to a standard granularity (one of the TimeGranularity enum members). Others might represent
custom granularity columns. Custom granularity columns are not yet implemented.
"""

node_relation: NodeRelation
primary_column: TimeSpinePrimaryColumn
depends_on: DependsOn = field(default_factory=DependsOn)
refs: List[RefArgs] = field(default_factory=list)
4 changes: 4 additions & 0 deletions core/dbt/artifacts/schemas/manifest/v12/manifest.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
Snapshot,
SourceDefinition,
SqlOperation,
TimeSpine,
UnitTestDefinition,
)
from dbt.artifacts.schemas.base import (
Expand Down Expand Up @@ -148,6 +149,9 @@ class WritableManifest(ArtifactMixin):
semantic_models: Mapping[UniqueID, SemanticModel] = field(
metadata=dict(description=("The semantic models defined in the dbt project"))
)
time_spines: Mapping[UniqueID, TimeSpine] = field(
metadata=dict(description=("The time spine models defined in the dbt project."))
)
metadata: ManifestMetadata = field(
metadata=dict(
description="Metadata about the manifest",
Expand Down
2 changes: 2 additions & 0 deletions core/dbt/artifacts/schemas/upgrades/upgrade_manifest.py
Original file line number Diff line number Diff line change
Expand Up @@ -169,6 +169,8 @@ def upgrade_manifest_json(manifest: dict, manifest_schema_version: int) -> dict:
doc_content["resource_type"] = "doc"
if "semantic_models" not in manifest:
manifest["semantic_models"] = {}
if "time_spines" not in manifest:
manifest["time_spines"] = {}
if "saved_queries" not in manifest:
manifest["saved_queries"] = {}
return manifest
1 change: 1 addition & 0 deletions core/dbt/config/runtime.py
Original file line number Diff line number Diff line change
Expand Up @@ -339,6 +339,7 @@ def get_resource_config_paths(self) -> Dict[str, PathSet]:
"unit_tests": self._get_config_paths(self.unit_tests),
"metrics": self._get_config_paths(self.metrics),
"semantic_models": self._get_config_paths(self.semantic_models),
"time_spines": self._get_config_paths(self.time_spines),
"saved_queries": self._get_config_paths(self.saved_queries),
"exposures": self._get_config_paths(self.exposures),
}
Expand Down
2 changes: 1 addition & 1 deletion core/dbt/constants.py
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,6 @@
PACKAGE_LOCK_FILE_NAME = "package-lock.yml"
MANIFEST_FILE_NAME = "manifest.json"
SEMANTIC_MANIFEST_FILE_NAME = "semantic_manifest.json"
TIME_SPINE_MODEL_NAME = "metricflow_time_spine"
LEGACY_TIME_SPINE_MODEL_NAME = "metricflow_time_spine"
PARTIAL_PARSE_FILE_NAME = "partial_parse.msgpack"
PACKAGE_LOCK_HASH_KEY = "sha1_hash"
2 changes: 2 additions & 0 deletions core/dbt/context/context_config.py
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,8 @@ def get_config_dict(self, resource_type: NodeType) -> Dict[str, Any]:
model_configs = unrendered.get("metrics")
elif resource_type == NodeType.SemanticModel:
model_configs = unrendered.get("semantic_models")
# here & this file!

elif resource_type == NodeType.SavedQuery:
model_configs = unrendered.get("saved_queries")
elif resource_type == NodeType.Exposure:
Expand Down
1 change: 1 addition & 0 deletions core/dbt/contracts/files.py
Original file line number Diff line number Diff line change
Expand Up @@ -203,6 +203,7 @@ class SchemaSourceFile(BaseSourceFile):
# node patches contain models, seeds, snapshots, analyses
ndp: List[str] = field(default_factory=list)
semantic_models: List[str] = field(default_factory=list)
# here & this file!
unit_tests: List[str] = field(default_factory=list)
saved_queries: List[str] = field(default_factory=list)
# any macro patches in this file by macro unique_id.
Expand Down
4 changes: 4 additions & 0 deletions core/dbt/contracts/graph/manifest.py
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,7 @@
SeedNode,
SemanticModel,
SourceDefinition,
TimeSpine,
UnitTestDefinition,
UnitTestFileFixture,
UnpatchedSourceDefinition,
Expand Down Expand Up @@ -823,6 +824,8 @@ class Manifest(MacroMethods, dbtClassMixin):
disabled: MutableMapping[str, List[GraphMemberNode]] = field(default_factory=dict)
env_vars: MutableMapping[str, str] = field(default_factory=dict)
semantic_models: MutableMapping[str, SemanticModel] = field(default_factory=dict)
time_spines: MutableMapping[str, TimeSpine] = field(default_factory=dict)

unit_tests: MutableMapping[str, UnitTestDefinition] = field(default_factory=dict)
saved_queries: MutableMapping[str, SavedQuery] = field(default_factory=dict)
fixtures: MutableMapping[str, UnitTestFileFixture] = field(default_factory=dict)
Expand Down Expand Up @@ -1154,6 +1157,7 @@ def writable_manifest(self) -> "WritableManifest":
parent_map=self.parent_map,
group_map=self.group_map,
semantic_models=self._map_nodes_to_map_resources(self.semantic_models),
time_spines=self._map_nodes_to_map_resources(self.time_spines),
unit_tests=self._map_nodes_to_map_resources(self.unit_tests),
saved_queries=self._map_nodes_to_map_resources(self.saved_queries),
)
Expand Down
38 changes: 36 additions & 2 deletions core/dbt/contracts/graph/nodes.py
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,7 @@
from dbt.artifacts.resources import Snapshot as SnapshotResource
from dbt.artifacts.resources import SourceDefinition as SourceDefinitionResource
from dbt.artifacts.resources import SqlOperation as SqlOperationResource
from dbt.artifacts.resources import TimeSpine as TimeSpineResource
from dbt.artifacts.resources import UnitTestDefinition as UnitTestDefinitionResource
from dbt.contracts.graph.model_config import UnitTestNodeConfig
from dbt.contracts.graph.node_args import ModelNodeArgs
Expand Down Expand Up @@ -1419,7 +1420,7 @@ def same_contents(self, old: Optional["Metric"]) -> bool:
# existing when it didn't before is a change!
# metadata/tags changes are not "changes"
if old is None:
return True
return True # is this a bug?

return (
self.same_filter(old)
Expand Down Expand Up @@ -1501,7 +1502,7 @@ def same_contents(self, old: Optional["SemanticModel"]) -> bool:
# existing when it didn't before is a change!
# metadata/tags changes are not "changes"
if old is None:
return True
return True # is this a bug?

return (
self.same_model(old)
Expand All @@ -1517,6 +1518,39 @@ def same_contents(self, old: Optional["SemanticModel"]) -> bool:
)


# ====================================
# TimeSpine node
# ====================================


@dataclass
class TimeSpine(GraphNode, TimeSpineResource):
@property
def depends_on_nodes(self):
return self.depends_on.nodes

@property
def depends_on_macros(self):
return self.depends_on.macros

@classmethod
def resource_class(cls) -> Type[TimeSpineResource]:
return TimeSpineResource

def same_model(self, old: "TimeSpine") -> bool:
return self.model == old.model

def same_primary_column(self, old: "TimeSpine") -> bool:
return self.primary_column == old.primary_column

def same_contents(self, old: Optional["TimeSpine"]) -> bool:
# existing when it didn't before is a change!
if old is None:
return False # is this right?

return self.same_model(old) and self.same_primary_column(old)


# ====================================
# SavedQuery
# ====================================
Expand Down
64 changes: 45 additions & 19 deletions core/dbt/contracts/graph/semantic_manifest.py
Original file line number Diff line number Diff line change
@@ -1,4 +1,8 @@
from dbt.constants import TIME_SPINE_MODEL_NAME
from typing import Optional, Set

from dbt.constants import LEGACY_TIME_SPINE_MODEL_NAME
from dbt.contracts.graph.manifest import Manifest
from dbt.contracts.graph.nodes import ManifestNode, TimeSpine
from dbt.events.types import SemanticValidationFailure
from dbt.exceptions import ParsingError
from dbt_common.clients.system import write_file
Expand All @@ -23,7 +27,7 @@


class SemanticManifest:
def __init__(self, manifest) -> None:
def __init__(self, manifest: Manifest) -> None:
self.manifest = manifest

def validate(self) -> bool:
Expand Down Expand Up @@ -59,8 +63,10 @@ def write_json_to_file(self, file_path: str):
write_file(file_path, json)

def _get_pydantic_semantic_manifest(self) -> PydanticSemanticManifest:
time_spines = self.manifest.time_spines.values()

project_config = PydanticProjectConfiguration(
time_spine_table_configurations=[],
time_spine_table_configurations=[], time_spines=time_spines
)
pydantic_semantic_manifest = PydanticSemanticManifest(
metrics=[], semantic_models=[], project_configuration=project_config
Expand All @@ -79,24 +85,44 @@ def _get_pydantic_semantic_manifest(self) -> PydanticSemanticManifest:
PydanticSavedQuery.parse_obj(saved_query.to_dict())
)

# Look for time-spine table model and create time spine table configuration
if self.manifest.semantic_models:
# Get model for time_spine_table
model = self.manifest.ref_lookup.find(TIME_SPINE_MODEL_NAME, None, None, self.manifest)
if not model:
# Validate that there is a time spine configured for the semantic manifest.
time_spine_models_not_found: Set[str] = set()
daily_time_spine: Optional[TimeSpine] = None
for time_spine in time_spines:
alias = time_spine.node_relation.alias
if time_spine.primary_column.time_granularity == TimeGranularity.DAY:
daily_time_spine = time_spine
time_spine_model = self.manifest.ref_lookup.find(alias, None, None, self.manifest)
if not time_spine_model:
time_spine_models_not_found.add(alias)
if time_spine_models_not_found:
raise ParsingError(
"The semantic layer requires a 'metricflow_time_spine' model in the project, but none was found. "
"Guidance on creating this model can be found on our docs site ("
"https://docs.getdbt.com/docs/build/metricflow-time-spine) "
f"Error parsing time spines. Referenced models do not exist: {time_spine_models_not_found}"
)
# Create time_spine_table_config, set it in project_config, and add to semantic manifest
time_spine_table_config = PydanticTimeSpineTableConfiguration(
location=model.relation_name,
column_name="date_day",
grain=TimeGranularity.DAY,
)
pydantic_semantic_manifest.project_configuration.time_spine_table_configurations = [
time_spine_table_config
]

# If no daily time spine has beem configured, look for legacy time spine model. This logic is included to
# avoid breaking projects that have not migrated to the new time spine config yet.
legacy_time_spine_model: Optional[ManifestNode] = None
if not daily_time_spine:
legacy_time_spine_model = self.manifest.ref_lookup.find(
LEGACY_TIME_SPINE_MODEL_NAME, None, None, self.manifest
)
# If no time spines have been configured AND legacy time spine model does not exist, error.
if not legacy_time_spine_model:
raise ParsingError(
"The semantic layer requires a time spine model in the project, but none was found. "
"Guidance on creating this model can be found on our docs site ("
"https://docs.getdbt.com/docs/build/metricflow-time-spine) " # TODO: update docs link!
)
# Create time_spine_table_config, set it in project_config, and add to semantic manifest
time_spine_table_config = PydanticTimeSpineTableConfiguration(
location=legacy_time_spine_model.relation_name,
column_name="date_day",
grain=TimeGranularity.DAY,
)
pydantic_semantic_manifest.project_configuration.time_spine_table_configurations = [
time_spine_table_config
]

return pydantic_semantic_manifest
1 change: 1 addition & 0 deletions core/dbt/contracts/project.py
Original file line number Diff line number Diff line change
Expand Up @@ -247,6 +247,7 @@ class Project(dbtClassMixin):
unit_tests: Dict[str, Any] = field(default_factory=dict)
metrics: Dict[str, Any] = field(default_factory=dict)
semantic_models: Dict[str, Any] = field(default_factory=dict)
# here & this file!
saved_queries: Dict[str, Any] = field(default_factory=dict)
exposures: Dict[str, Any] = field(default_factory=dict)
vars: Optional[Dict[str, Any]] = field(
Expand Down
2 changes: 2 additions & 0 deletions core/dbt/graph/selector.py
Original file line number Diff line number Diff line change
Expand Up @@ -168,6 +168,8 @@ def _is_graph_member(self, unique_id: UniqueId) -> bool:
return metric.config.enabled
elif unique_id in self.manifest.semantic_models:
semantic_model = self.manifest.semantic_models[unique_id]
# here & this file? what is this enabled/disabled stuff?

return semantic_model.config.enabled
elif unique_id in self.manifest.unit_tests:
return True
Expand Down
1 change: 1 addition & 0 deletions core/dbt/graph/selector_methods.py
Original file line number Diff line number Diff line change
Expand Up @@ -175,6 +175,7 @@ def parsed_and_unit_nodes(self, included_nodes: Set[UniqueId]):
def semantic_model_nodes(
self, included_nodes: Set[UniqueId]
) -> Iterator[Tuple[UniqueId, SemanticModel]]:
# here & this file?

for key, semantic_model in self.manifest.semantic_models.items():
unique_id = UniqueId(key)
Expand Down
1 change: 1 addition & 0 deletions core/dbt/parser/manifest.py
Original file line number Diff line number Diff line change
Expand Up @@ -1119,6 +1119,7 @@ def process_refs(self, current_project: str, dependencies: Optional[Mapping[str,
continue
_process_refs(self.manifest, current_project, semantic_model, dependencies)
self.update_semantic_model(semantic_model)
# here & this file!

# Takes references in 'metrics' array of nodes and exposures, finds the target
# node, and updates 'depends_on.nodes' with the unique id
Expand Down
2 changes: 2 additions & 0 deletions core/dbt/parser/partial.py
Original file line number Diff line number Diff line change
Expand Up @@ -443,6 +443,8 @@ def schedule_nodes_for_parsing(self, unique_ids):
semantic_model.name,
self.delete_schema_semantic_model,
)
# here & this file!

elif unique_id in self.saved_manifest.saved_queries:
saved_query = self.saved_manifest.saved_queries[unique_id]
self._schedule_for_parsing(
Expand Down
1 change: 1 addition & 0 deletions core/dbt/parser/schema_yaml_readers.py
Original file line number Diff line number Diff line change
Expand Up @@ -523,6 +523,7 @@ def parse(self):
self.parse_group(unparsed)


# here & this file!
class SemanticModelParser(YamlReader):
def __init__(self, schema_parser: SchemaParser, yaml: YamlBlock) -> None:
super().__init__(schema_parser, yaml, "semantic_models")
Expand Down
1 change: 1 addition & 0 deletions core/dbt/parser/schemas.py
Original file line number Diff line number Diff line change
Expand Up @@ -76,6 +76,7 @@
"exposures",
"metrics",
"semantic_models",
# here & this file!
"saved_queries",
)

Expand Down
4 changes: 3 additions & 1 deletion core/dbt/task/docs/index.html

Large diffs are not rendered by default.

2 changes: 2 additions & 0 deletions core/dbt/task/list.py
Original file line number Diff line number Diff line change
Expand Up @@ -84,6 +84,8 @@ def _iterate_selected_nodes(self):
yield self.manifest.metrics[unique_id]
elif unique_id in self.manifest.semantic_models:
yield self.manifest.semantic_models[unique_id]
# here & this file!

elif unique_id in self.manifest.unit_tests:
yield self.manifest.unit_tests[unique_id]
elif unique_id in self.manifest.saved_queries:
Expand Down
2 changes: 1 addition & 1 deletion core/setup.py
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,7 @@
# Accept patches but avoid automatically updating past a set minor version range.
"dbt-extractor>=0.5.0,<=0.6",
"minimal-snowplow-tracker>=0.0.2,<0.1",
"dbt-semantic-interfaces>=0.6.8,<0.7",
"dbt-semantic-interfaces>=0.6.9,<0.7",
# Minor versions for these are expected to be backwards-compatible
"dbt-common>=1.3.0,<2.0",
"dbt-adapters>=1.1.1,<2.0",
Expand Down
2 changes: 2 additions & 0 deletions schemas/dbt/manifest/v12.json
Original file line number Diff line number Diff line change
Expand Up @@ -20139,6 +20139,8 @@
"type": "string"
}
},
# here & this file!

"semantic_models": {
"type": "object",
"description": "The semantic models defined in the dbt project",
Expand Down
1 change: 1 addition & 0 deletions tests/functional/access/test_access.py
Original file line number Diff line number Diff line change
Expand Up @@ -140,6 +140,7 @@
union all
select 1 as id, 'Callum' as first_name, 'McCann' as last_name, 'emerald' as favorite_color, true as loves_dbt, 0 as tenure, current_timestamp as created_at
"""
# here & this file!

people_semantic_model_yml = """
semantic_models:
Expand Down
4 changes: 3 additions & 1 deletion tests/functional/artifacts/data/state/v12/manifest.json

Large diffs are not rendered by default.

1 change: 1 addition & 0 deletions tests/functional/artifacts/expected_manifest.py
Original file line number Diff line number Diff line change
Expand Up @@ -885,6 +885,7 @@ def expected_seeded_manifest(project, model_database=None, quote_model=False):
"doc.test.macro_arg_info": ANY,
},
"disabled": {},
# here & this file!
"semantic_models": {},
"unit_tests": {},
"saved_queries": {},
Expand Down
1 change: 1 addition & 0 deletions tests/functional/artifacts/test_artifacts.py
Original file line number Diff line number Diff line change
Expand Up @@ -474,6 +474,7 @@ def verify_manifest(project, expected_manifest, start_time, manifest_schema_path
"disabled",
"exposures",
"selectors",
# here & this file!
"semantic_models",
"unit_tests",
"saved_queries",
Expand Down
Loading

0 comments on commit 06df37e

Please sign in to comment.