Compare commits

...

15 Commits

Author SHA1 Message Date
Dave Connors
776f0a36f2 source partial parsing 2024-07-19 11:36:07 -05:00
Dave Connors
efe23ac9b7 add methods for partial parsing 2024-07-19 10:54:24 -05:00
Dave Connors
e0780d17a2 update artifacts readme 2024-07-19 10:45:11 -05:00
Dave Connors
7af4674d4f schemas update and artifacts testing 2024-07-19 10:40:34 -05:00
Dave Connors
c53fcaad56 Merge branch 'main' into dc/expoure-and-source-groups 2024-07-19 10:27:44 -05:00
Dave Connors
00a3f7acef run checks on exposure source dependencies too 2024-07-10 11:29:13 -05:00
Dave Connors
5ce8ba866d unit test for parsed sources and exposures 2024-07-10 11:19:45 -05:00
Dave Connors
6143d91840 validate source access settings 2024-07-10 09:35:37 -05:00
Dave Connors
b99aa4c2a2 a wee syntax error 2024-07-10 09:01:33 -05:00
Dave Connors
af73db6282 update test_access with the basic cases 2024-07-10 08:29:45 -05:00
Dave Connors
c1b84d3ca1 update manifest parsing to check source function validity 2024-07-10 08:29:08 -05:00
Dave Connors
bb9ac066c6 update unparsed to parsed functions for source and exposure 2024-07-10 08:25:57 -05:00
Dave Connors
f535169c56 add checks to validity of source jinja calls based on access levels 2024-07-10 08:25:21 -05:00
Dave Connors
5fb997d5ce remove explicit group overrides for source and exposure 2024-07-10 08:24:38 -05:00
Dave Connors
594b918659 add new fields to source definition, exposure definition, and unparsed yml classes 2024-07-10 08:24:07 -05:00
16 changed files with 630 additions and 968 deletions

View File

@@ -46,9 +46,12 @@ These types of minor, non-breaking changes are tested by [tests/unit/artifacts/t
#### Updating [schemas.getdbt.com](https://schemas.getdbt.com)
Non-breaking changes to artifact schemas require an update to the corresponding jsonschemas published to [schemas.getdbt.com](https://schemas.getdbt.com), which are defined in https://github.com/dbt-labs/schemas.getdbt.com. To do so:
1. Create a PR in https://github.com/dbt-labs/schemas.getdbt.com which reflects the schema changes to the artifact. The schema can be updated in-place for non-breaking changes. Example PR: https://github.com/dbt-labs/schemas.getdbt.com/pull/39
2. Merge the https://github.com/dbt-labs/schemas.getdbt.com PR
3. Observe the `Artifact Schema Check` CI check pass on the `dbt-core` PR that updates the artifact schemas, and merge the `dbt-core` PR!
1. Run `scripts/collect-artifact-schema.py --path schemas --artifact [manifest | catalog | run-results | sources]` to generate the jsonschema inclusive of your changes.
2. Copy the generated schema into the corresponding artifact file in the schemas.getdbt.com repository. (e.g. `cp schemas/dbt/manifest/v12.json ../../schemas.getdbt.com/dbt/manifest/`)
3. Create a PR in https://github.com/dbt-labs/schemas.getdbt.com which reflects the schema changes to the artifact. The schema can be updated in-place for non-breaking changes. Example PR: https://github.com/dbt-labs/schemas.getdbt.com/pull/39
4. Merge the https://github.com/dbt-labs/schemas.getdbt.com PR
5. Observe the `Artifact Schema Check` CI check pass on the `dbt-core` PR that updates the artifact schemas, and merge the `dbt-core` PR!
Note: Although `jsonschema` validation using the schemas in [schemas.getdbt.com](https://schemas.getdbt.com) is not encouraged or formally supported, `jsonschema` validation should still continue to work once the schemas are updated because they are forward-compatible and can therefore be used to validate previous minor versions of the schema.

View File

@@ -3,7 +3,7 @@ from dataclasses import dataclass, field
from typing import Any, Dict, List, Literal, Optional
from dbt.artifacts.resources.base import GraphResource
from dbt.artifacts.resources.types import NodeType
from dbt.artifacts.resources.types import AccessType, NodeType
from dbt.artifacts.resources.v1.components import DependsOn, RefArgs
from dbt.artifacts.resources.v1.owner import Owner
from dbt_common.contracts.config.base import BaseConfig
@@ -47,3 +47,5 @@ class Exposure(GraphResource):
sources: List[List[str]] = field(default_factory=list)
metrics: List[List[str]] = field(default_factory=list)
created_at: float = field(default_factory=lambda: time.time())
access: AccessType = AccessType.Public
group: Optional[str] = None

View File

@@ -3,7 +3,7 @@ from dataclasses import dataclass, field
from typing import Any, Dict, List, Literal, Optional, Union
from dbt.artifacts.resources.base import GraphResource
from dbt.artifacts.resources.types import NodeType
from dbt.artifacts.resources.types import AccessType, NodeType
from dbt.artifacts.resources.v1.components import (
ColumnInfo,
FreshnessThreshold,
@@ -70,3 +70,5 @@ class SourceDefinition(ParsedSourceMandatory):
unrendered_config: Dict[str, Any] = field(default_factory=dict)
relation_name: Optional[str] = None
created_at: float = field(default_factory=lambda: time.time())
access: AccessType = AccessType.Protected
group: Optional[str] = None

View File

@@ -643,6 +643,8 @@ MaybeParsedSource = Optional[
MaybeNonSource = Optional[Union[ManifestNode, Disabled[ManifestNode]]]
MaybeSource = Optional[Union[SourceDefinition, Disabled[SourceDefinition]]]
T = TypeVar("T", bound=GraphMemberNode)
@@ -1467,6 +1469,36 @@ class Manifest(MacroMethods, dbtClassMixin):
node.package_name != target_model.package_name and restrict_package_access
)
def is_invalid_private_source(self, node: GraphMemberNode, target_source: MaybeSource) -> bool:
if not isinstance(target_source, SourceDefinition):
return False
is_private_source = target_source.access == AccessType.Private
return is_private_source and (
not hasattr(node, "group") or not node.group or node.group != target_source.group
)
def is_invalid_protected_source(
self, node: GraphMemberNode, target_source: MaybeSource, dependencies: Optional[Mapping]
) -> bool:
dependencies = dependencies or {}
if not isinstance(target_source, SourceDefinition):
return False
is_protected_source = (
target_source.access == AccessType.Protected
# don't raise this reference error for ad hoc 'preview' queries
and node.resource_type != NodeType.SqlOperation
and node.resource_type != NodeType.RPCCall # TODO: rm
)
target_dependency = dependencies.get(target_source.package_name)
restrict_package_access = target_dependency.restrict_access if target_dependency else False
return is_protected_source and (
node.package_name != target_source.package_name and restrict_package_access
)
# Called in GraphRunnableTask.before_run, RunTask.before_run, CloneTask.before_run
def merge_from_artifact(self, other: "Manifest") -> None:
"""Update this manifest by adding the 'defer_relation' attribute to all nodes

View File

@@ -1232,6 +1232,9 @@ class SourceDefinition(
old.unrendered_config,
)
def same_source_representation(self, old: "SourceDefinition") -> bool:
return self.access == old.access and self.group == old.group
def same_contents(self, old: Optional["SourceDefinition"]) -> bool:
# existing when it didn't before is a change!
if old is None:
@@ -1252,6 +1255,7 @@ class SourceDefinition(
and self.same_quoting(old)
and self.same_freshness(old)
and self.same_external(old)
and self.same_source_representation(old)
and True
)
@@ -1297,10 +1301,6 @@ class SourceDefinition(
def search_name(self):
return f"{self.source_name}.{self.name}"
@property
def group(self):
return None
# ====================================
# Exposure node
@@ -1342,6 +1342,9 @@ class Exposure(GraphNode, ExposureResource):
def same_url(self, old: "Exposure") -> bool:
return self.url == old.url
def same_group(self, old: "Exposure") -> bool:
return self.group == old.group
def same_config(self, old: "Exposure") -> bool:
return self.config.same_contents(
self.unrendered_config,
@@ -1364,13 +1367,10 @@ class Exposure(GraphNode, ExposureResource):
and self.same_label(old)
and self.same_depends_on(old)
and self.same_config(old)
and self.same_group(old)
and True
)
@property
def group(self):
return None
# ====================================
# Metric node

View File

@@ -302,6 +302,8 @@ class UnparsedSourceDefinition(dbtClassMixin):
tables: List[UnparsedSourceTableDefinition] = field(default_factory=list)
tags: List[str] = field(default_factory=list)
config: Dict[str, Any] = field(default_factory=dict)
group: Optional[str] = None
access: Optional[str] = None
@classmethod
def validate(cls, data):
@@ -462,6 +464,7 @@ class UnparsedExposure(dbtClassMixin):
url: Optional[str] = None
depends_on: List[str] = field(default_factory=list)
config: Dict[str, Any] = field(default_factory=dict)
group: Optional[str] = None
@classmethod
def validate(cls, data):

View File

@@ -458,7 +458,7 @@ class ManifestLoader:
# These check the created_at time on the nodes to
# determine whether they need processing.
start_process = time.perf_counter()
self.process_sources(self.root_project.project_name)
self.process_sources(self.root_project.project_name, self.root_project.dependencies)
self.process_refs(self.root_project.project_name, self.root_project.dependencies)
self.process_unit_tests(self.root_project.project_name)
self.process_docs(self.root_project)
@@ -1246,18 +1246,18 @@ class ManifestLoader:
# Loops through all nodes and exposures, for each element in
# 'sources' array finds the source node and updates the
# 'depends_on.nodes' array with the unique id
def process_sources(self, current_project: str):
def process_sources(self, current_project: str, dependencies: Optional[Mapping[str, Project]]):
for node in self.manifest.nodes.values():
if node.resource_type == NodeType.Source:
continue
assert not isinstance(node, SourceDefinition)
if node.created_at < self.started_at:
continue
_process_sources_for_node(self.manifest, current_project, node)
_process_sources_for_node(self.manifest, current_project, node, dependencies)
for exposure in self.manifest.exposures.values():
if exposure.created_at < self.started_at:
continue
_process_sources_for_exposure(self.manifest, current_project, exposure)
_process_sources_for_exposure(self.manifest, current_project, exposure, dependencies)
# Loops through all nodes, for each element in
# 'unit_test' array finds the node and updates the
@@ -1793,7 +1793,12 @@ def remove_dependent_project_references(manifest, external_node_unique_id):
node.created_at = time.time()
def _process_sources_for_exposure(manifest: Manifest, current_project: str, exposure: Exposure):
def _process_sources_for_exposure(
manifest: Manifest,
current_project: str,
exposure: Exposure,
dependencies: Optional[Mapping[str, Project]],
):
target_source: Optional[Union[Disabled, SourceDefinition]] = None
for source_name, table_name in exposure.sources:
target_source = manifest.resolve_source(
@@ -1811,6 +1816,21 @@ def _process_sources_for_exposure(manifest: Manifest, current_project: str, expo
disabled=(isinstance(target_source, Disabled)),
)
continue
if manifest.is_invalid_private_source(exposure, target_source):
raise dbt.exceptions.DbtReferenceError(
unique_id=exposure.unique_id,
ref_unique_id=target_source.unique_id,
access=AccessType.Private,
scope=dbt_common.utils.cast_to_str(target_source.group),
)
elif manifest.is_invalid_protected_source(exposure, target_source, dependencies):
raise dbt.exceptions.DbtReferenceError(
unique_id=exposure.unique_id,
ref_unique_id=target_source.unique_id,
access=AccessType.Protected,
scope=exposure.package_name,
)
target_source_id = target_source.unique_id
exposure.depends_on.add_node(target_source_id)
@@ -1837,7 +1857,12 @@ def _process_sources_for_metric(manifest: Manifest, current_project: str, metric
metric.depends_on.add_node(target_source_id)
def _process_sources_for_node(manifest: Manifest, current_project: str, node: ManifestNode):
def _process_sources_for_node(
manifest: Manifest,
current_project: str,
node: ManifestNode,
dependencies: Optional[Mapping[str, Project]],
):
if isinstance(node, SeedNode):
return
@@ -1851,6 +1876,31 @@ def _process_sources_for_node(manifest: Manifest, current_project: str, node: Ma
)
if target_source is None or isinstance(target_source, Disabled):
node.config.enabled = False
invalid_target_fail_unless_test(
node=node,
target_name=f"{source_name}.{table_name}",
target_kind="source",
disabled=(isinstance(target_source, Disabled)),
)
continue
if manifest.is_invalid_private_source(node, target_source):
raise dbt.exceptions.DbtReferenceError(
unique_id=node.unique_id,
ref_unique_id=target_source.unique_id,
access=AccessType.Private,
scope=dbt_common.utils.cast_to_str(target_source.group),
)
elif manifest.is_invalid_protected_source(node, target_source, dependencies):
raise dbt.exceptions.DbtReferenceError(
unique_id=node.unique_id,
ref_unique_id=target_source.unique_id,
access=AccessType.Protected,
scope=node.package_name,
)
elif target_source is None or isinstance(target_source, Disabled):
# this follows the same pattern as refs
node.config.enabled = False
invalid_target_fail_unless_test(
@@ -1879,7 +1929,7 @@ def process_macro(config: RuntimeConfig, manifest: Manifest, macro: Macro) -> No
# This is called in task.rpc.sql_commands when a "dynamic" node is
# created in the manifest, in 'add_refs'
def process_node(config: RuntimeConfig, manifest: Manifest, node: ManifestNode):
_process_sources_for_node(manifest, config.project_name, node)
_process_sources_for_node(manifest, config.project_name, node, config.dependencies)
_process_refs(manifest, config.project_name, node, config.dependencies)
ctx = generate_runtime_docs_context(config, node, manifest, config.project_name)
_process_docs_for_node(ctx, node)

View File

@@ -123,6 +123,7 @@ class ExposureParser(YamlReader):
unique_id=unique_id,
fqn=fqn,
name=unparsed.name,
group=unparsed.group,
type=unparsed.type,
url=unparsed.url,
meta=unparsed.meta,

View File

@@ -6,6 +6,7 @@ from typing import Any, Dict, Iterable, List, Optional, Set
from dbt.adapters.capability import Capability
from dbt.adapters.factory import get_adapter
from dbt.artifacts.resources import FreshnessThreshold, SourceConfig, Time
from dbt.artifacts.resources.types import AccessType
from dbt.config import RuntimeConfig
from dbt.context.context_config import (
BaseContextConfigGenerator,
@@ -26,6 +27,7 @@ from dbt.contracts.graph.unparsed import (
UnparsedSourceTableDefinition,
)
from dbt.events.types import FreshnessConfigProblem, UnusedTables
from dbt.exceptions import InvalidAccessTypeError
from dbt.node_types import NodeType
from dbt.parser.common import ParserRef
from dbt.parser.schema_generic_tests import SchemaGenericTestParser
@@ -127,6 +129,19 @@ class SourcePatcher:
unique_id = target.unique_id
description = table.description or ""
source_description = source.description or ""
group = source.group or None
access = AccessType.Protected
# make sure sources are not public and are a valid access type
if source.access:
if (
not AccessType.is_valid(source.access)
or AccessType(source.access) == AccessType.Public
):
raise InvalidAccessTypeError(
unique_id=unique_id,
field_value=source.access,
)
access = AccessType(source.access)
# We need to be able to tell the difference between explicitly setting the loaded_at_field to None/null
# and when it's simply not set. This allows a user to override the source level loaded_at_field so that
@@ -190,6 +205,8 @@ class SourcePatcher:
tags=tags,
config=config,
unrendered_config=unrendered_config,
group=group,
access=access,
)
if (

File diff suppressed because it is too large Load Diff

View File

@@ -12,6 +12,51 @@ another_model_sql = "select 1234 as notfun"
yet_another_model_sql = "select 999 as weird"
stg_model_sql = """
-- depends_on: {{ source('my_source', 'my_table') }}
select 999 as weird
"""
stg_model_yml = """
models:
- name: stg_model
description: "my model"
group: analytics
"""
invalid_access_source_yml = """
sources:
- name: my_source
access: invalid
tables:
- name: my_table
"""
non_grouped_source_yml = """
sources:
- name: my_source
tables:
- name: my_table
"""
protected_analytics_source_yml = """
sources:
- name: my_source
group: analytics
tables:
- name: my_table
"""
private_marts_source_yml = """
sources:
- name: my_source
group: marts
access: private
tables:
- name: my_table
"""
schema_yml = """
models:
- name: my_model
@@ -314,6 +359,34 @@ class TestAccess:
# verify it works again
manifest = run_dbt(["parse"])
assert len(manifest.nodes) == 3
# add source with invalid access
write_file(invalid_access_source_yml, project.project_root, "models", "my_source.yml")
with pytest.raises(InvalidAccessTypeError):
run_dbt(["parse"])
# add source without group, new model in group analytics that depends on it
write_file(non_grouped_source_yml, project.project_root, "models", "my_source.yml")
write_file(stg_model_yml, project.project_root, "models", "stg_model.yml")
write_file(stg_model_sql, project.project_root, "models", "stg_model.sql")
manifest = run_dbt(["parse"])
assert len(manifest.nodes) == 4
assert len(manifest.sources) == 1
# add source to group analytics, implicitly set it to protected
write_file(protected_analytics_source_yml, project.project_root, "models", "my_source.yml")
manifest = run_dbt(["parse"])
assert len(manifest.nodes) == 4
assert len(manifest.sources) == 1
# add source to group mart, explicitly set it to private
write_file(private_marts_source_yml, project.project_root, "models", "my_source.yml")
with pytest.raises(DbtReferenceError):
run_dbt(["parse"])
# revert source to no group, protected access
write_file(non_grouped_source_yml, project.project_root, "models", "my_source.yml")
# Write out exposure refing private my_model
write_file(simple_exposure_yml, project.project_root, "models", "simple_exposure.yml")
# Fails with reference error
@@ -329,14 +402,14 @@ class TestAccess:
write_file(metricflow_time_spine_sql, "models", "metricflow_time_spine.sql")
# Should succeed
manifest = run_dbt(["parse"])
assert len(manifest.nodes) == 5
assert len(manifest.nodes) == 6
metric_id = "metric.test.number_of_people"
assert manifest.metrics[metric_id].group == "analytics"
# Use access and group in config
write_file(v5_schema_yml, project.project_root, "models", "schema.yml")
manifest = run_dbt(["parse"])
assert len(manifest.nodes) == 5
assert len(manifest.nodes) == 6
assert manifest.nodes["model.test.my_model"].access == AccessType.Private
assert manifest.nodes["model.test.my_model"].group == "analytics"
assert manifest.nodes["model.test.ref_my_model"].access == AccessType.Protected

View File

@@ -770,6 +770,8 @@ def expected_seeded_manifest(project, model_database=None, quote_model=False):
"unique_id": "source.test.my_source.my_table",
"fqn": ["test", "my_source", "my_table"],
"unrendered_config": {},
"group": None,
"access": "protected",
},
},
"exposures": {
@@ -804,6 +806,8 @@ def expected_seeded_manifest(project, model_database=None, quote_model=False):
"unique_id": "exposure.test.notebook_exposure",
"url": "http://example.com/notebook/1",
"unrendered_config": {},
"group": None,
"access": "public",
},
"exposure.test.simple_exposure": {
"created_at": ANY,
@@ -836,6 +840,8 @@ def expected_seeded_manifest(project, model_database=None, quote_model=False):
"meta": {},
"tags": [],
"unrendered_config": {},
"group": None,
"access": "public",
},
},
"metrics": {},
@@ -1258,6 +1264,8 @@ def expected_references_manifest(project):
"unique_id": "source.test.my_source.my_table",
"fqn": ["test", "my_source", "my_table"],
"unrendered_config": {},
"group": None,
"access": "protected",
},
},
"exposures": {
@@ -1289,6 +1297,8 @@ def expected_references_manifest(project):
"unique_id": "exposure.test.notebook_exposure",
"url": "http://example.com/notebook/1",
"unrendered_config": {},
"group": None,
"access": "public",
},
},
"metrics": {},
@@ -1859,6 +1869,8 @@ def expected_versions_manifest(project):
"unique_id": "exposure.test.notebook_exposure",
"url": None,
"unrendered_config": {},
"group": None,
"access": "public",
},
},
"metrics": {},

View File

@@ -164,6 +164,62 @@ seeds:
"""
schema_sources_add_group_yml = """
groups:
- name: dave_land
owner:
name: dave
sources:
- name: seed_sources
schema: "{{ target.schema }}"
group: dave_land
tables:
- name: raw_customers
columns:
- name: id
data_tests:
- not_null:
severity: "{{ 'error' if target.name == 'prod' else 'warn' }}"
- unique
- every_value_is_blue
- name: first_name
- name: last_name
- name: email
"""
schema_sources_add_access_yml = """
groups:
- name: dave_land
owner:
name: dave
sources:
- name: seed_sources
schema: "{{ target.schema }}"
group: dave_land
access: private
tables:
- name: raw_customers
columns:
- name: id
data_tests:
- not_null:
severity: "{{ 'error' if target.name == 'prod' else 'warn' }}"
- unique
- every_value_is_blue
- name: first_name
- name: last_name
- name: email
"""
env_var_schema_yml = """
models:

View File

@@ -9,7 +9,7 @@ import yaml
import dbt.flags as flags
from dbt.contracts.files import ParseFileType
from dbt.contracts.results import TestStatus
from dbt.exceptions import CompilationError
from dbt.exceptions import CompilationError, DbtReferenceError
from dbt.plugins.manifest import ModelNodeArgs, PluginNodes
from dbt.tests.fixtures.project import write_project_files
from dbt.tests.util import (
@@ -72,6 +72,8 @@ from tests.functional.partial_parsing.fixtures import (
schema_sources3_yml,
schema_sources4_yml,
schema_sources5_yml,
schema_sources_add_access_yml,
schema_sources_add_group_yml,
snapshot2_sql,
snapshot_sql,
sources_tests1_sql,
@@ -428,6 +430,18 @@ class TestSources:
write_file(sources_tests2_sql, project.project_root, "macros", "tests.sql")
results = run_dbt(["--partial-parse", "run"])
# Add group and add source to group
write_file(schema_sources_add_group_yml, project.project_root, "models", "sources.yml")
results = run_dbt(["--partial-parse", "run"])
manifest = get_manifest(project.project_root)
assert len(manifest.sources) == 1
assert manifest.sources["source.test.seed_sources.raw_customers"].group == "dave_land"
# Add group and add source to group
write_file(schema_sources_add_access_yml, project.project_root, "models", "sources.yml")
with pytest.raises(DbtReferenceError):
run_dbt(["--partial-parse", "run"])
class TestPartialParsingDependency:
@pytest.fixture(scope="class")

View File

@@ -34,4 +34,6 @@ def basic_parsed_source_definition_object():
unique_id="test.source.my_source.my_source_table",
tags=[],
config=SourceConfig(),
group="",
access=AccessType.Protected, # noqa
)

View File

@@ -1817,6 +1817,8 @@ def minimum_parsed_source_definition_dict():
"identifier": "my_source_table",
"resource_type": str(NodeType.Source),
"unique_id": "test.source.my_source.my_source_table",
"group": "",
"access": AccessType.Protected.value,
}
@@ -1847,6 +1849,8 @@ def basic_parsed_source_definition_dict():
"enabled": True,
},
"unrendered_config": {},
"group": "",
"access": AccessType.Protected.value,
}
@@ -1879,6 +1883,8 @@ def complex_parsed_source_definition_dict():
"freshness": {"warn_after": {"period": "hour", "count": 1}, "error_after": {}},
"loaded_at_field": "loaded_at",
"unrendered_config": {},
"group": "",
"access": AccessType.Protected.value,
}
@@ -1905,6 +1911,8 @@ def complex_parsed_source_definition_object():
config=SourceConfig(),
freshness=FreshnessThreshold(warn_after=Time(period=TimePeriod.hour, count=1)),
loaded_at_field="loaded_at",
group="",
access=AccessType.Protected,
)
@@ -2018,6 +2026,8 @@ def minimal_parsed_exposure_dict():
"description": "",
"created_at": 1.0,
"resource_type": "exposure",
"group": "",
"access": AccessType.Public.value,
}
@@ -2050,6 +2060,8 @@ def basic_parsed_exposure_dict():
"enabled": True,
},
"unrendered_config": {},
"access": AccessType.Public.value,
"group": "",
}
@@ -2070,6 +2082,8 @@ def basic_parsed_exposure_object():
tags=[],
config=ExposureConfig(),
unrendered_config={},
group="",
access=AccessType.Public,
)
@@ -2105,6 +2119,8 @@ def complex_parsed_exposure_dict():
"enabled": True,
},
"unrendered_config": {},
"group": "",
"access": AccessType.Public.value,
}
@@ -2128,6 +2144,8 @@ def complex_parsed_exposure_object():
original_file_path="models/something.yml",
config=ExposureConfig(),
unrendered_config={},
group="",
access=AccessType.Public,
)