Skip to content

Commit

Permalink
[config] clean-up memoization
Browse files Browse the repository at this point in the history
  • Loading branch information
alangenfeld committed Feb 3, 2025
1 parent 9682bde commit 40d93ab
Show file tree
Hide file tree
Showing 8 changed files with 23 additions and 40 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -204,7 +204,7 @@ def resolve_automationCondition(
# we only store one of automation_condition or automation_condition_snapshot
automation_condition
if isinstance(automation_condition, AutomationConditionSnapshot)
else automation_condition.get_snapshot()
else automation_condition.snapshot
)
return None

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -840,7 +840,7 @@ def resolve_automationCondition(
# we only store one of automation_condition or automation_condition_snapshot
automation_condition
if isinstance(automation_condition, AutomationConditionSnapshot)
else automation_condition.get_snapshot()
else automation_condition.snapshot
)
return None

Expand Down
45 changes: 14 additions & 31 deletions python_modules/dagster/dagster/_config/config_type.py
Original file line number Diff line number Diff line change
@@ -1,12 +1,14 @@
import typing
from collections.abc import Iterator, Sequence
from enum import Enum as PythonEnum
from functools import cached_property
from typing import TYPE_CHECKING, Optional, cast

import dagster._check as check
from dagster._annotations import public
from dagster._builtins import BuiltinEnum
from dagster._config import UserConfigSchema
from dagster._record import record
from dagster._serdes import whitelist_for_serdes

if TYPE_CHECKING:
Expand Down Expand Up @@ -56,33 +58,15 @@ def is_selector(kind: "ConfigTypeKind") -> bool:
return kind == ConfigTypeKind.SELECTOR


@record
class ConfigType:
"""The class backing DagsterTypes as they are used processing configuration data."""

def __init__(
self,
key: str,
kind: ConfigTypeKind,
given_name: Optional[str] = None,
description: Optional[str] = None,
type_params: Optional[Sequence["ConfigType"]] = None,
):
self.key: str = check.str_param(key, "key")
self.kind: ConfigTypeKind = check.inst_param(kind, "kind", ConfigTypeKind)
self.given_name: Optional[str] = check.opt_str_param(given_name, "given_name")
self._description: Optional[str] = check.opt_str_param(description, "description")
self.type_params: Optional[Sequence[ConfigType]] = (
check.sequence_param(type_params, "type_params", of_type=ConfigType)
if type_params
else None
)

# memoized snap representation
self._snap: Optional[ConfigTypeSnap] = None

@property
def description(self) -> Optional[str]:
return self._description
key: str
kind: ConfigTypeKind
given_name: Optional[str] = None
description: Optional[str] = None
type_params: Optional[Sequence["ConfigType"]] = None

@staticmethod
def from_builtin_enum(builtin_enum: typing.Any) -> "ConfigType":
Expand All @@ -98,22 +82,21 @@ def post_process(self, value):
"""
return value

def get_snapshot(self) -> "ConfigTypeSnap":
@cached_property
def snapshot(self) -> "ConfigTypeSnap":
from dagster._config.snap import snap_from_config_type

if self._snap is None:
self._snap = snap_from_config_type(self)

return self._snap
return snap_from_config_type(self)

def type_iterator(self) -> Iterator["ConfigType"]:
yield self

def get_schema_snapshot(self) -> "ConfigSchemaSnapshot":
@cached_property
def schema_snapshot(self) -> "ConfigSchemaSnapshot":
from dagster._config.snap import ConfigSchemaSnapshot

return ConfigSchemaSnapshot(
all_config_snaps_by_key={ct.key: ct.get_snapshot() for ct in self.type_iterator()}
all_config_snaps_by_key={ct.key: ct.snapshot for ct in self.type_iterator()}
)


Expand Down
4 changes: 2 additions & 2 deletions python_modules/dagster/dagster/_config/traversal_context.py
Original file line number Diff line number Diff line change
Expand Up @@ -135,8 +135,8 @@ def from_config_type(
traversal_type: TraversalType,
) -> "TraversalContext":
return TraversalContext(
config_schema_snapshot=config_type.get_schema_snapshot(),
config_type_snap=config_type.get_snapshot(),
config_schema_snapshot=config_type.schema_snapshot,
config_type_snap=config_type.snapshot,
config_type=config_type,
stack=stack,
traversal_type=traversal_type,
Expand Down
2 changes: 1 addition & 1 deletion python_modules/dagster/dagster/_config/type_printer.py
Original file line number Diff line number Diff line change
Expand Up @@ -7,7 +7,7 @@

def _print_type_from_config_type(config_type, print_fn=print, with_lines=True):
check.inst_param(config_type, "config_type", ConfigType)
return _print_type(config_type.get_schema_snapshot(), config_type.key, print_fn, with_lines)
return _print_type(config_type.schema_snapshot, config_type.key, print_fn, with_lines)


def _print_type(config_schema_snapshot, config_type_key, print_fn, with_lines):
Expand Down
2 changes: 1 addition & 1 deletion python_modules/dagster/dagster/_config/validate.py
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ def validate_config(config_schema: object, config_value: T) -> EvaluateValueResu
config_type = check.inst(resolve_to_config_type(config_schema), ConfigType)

return validate_config_from_snap(
config_schema_snapshot=config_type.get_schema_snapshot(),
config_schema_snapshot=config_type.schema_snapshot,
config_type_key=config_type.key,
config_value=config_value,
)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1289,7 +1289,7 @@ def from_def(
resource_snapshot=build_resource_def_snap(name, resource_def),
configured_values=configured_values,
config_field_snaps=unconfigured_config_type_snap.fields or [],
config_schema_snap=config_type.get_schema_snapshot(),
config_schema_snap=config_type.schema_snapshot,
nested_resources=nested_resources,
parent_resources=parent_resources,
is_top_level=True,
Expand Down Expand Up @@ -1867,7 +1867,7 @@ def resolve_automation_condition_args(
return automation_condition, None
else:
# for non-serializable conditions, only include the snapshot
return None, automation_condition.get_snapshot()
return None, automation_condition.snapshot


def _extract_fast(serialized_job_data: str):
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,7 @@

def assert_inner_types(parent_type, *dagster_types):
config_type = resolve_to_config_type(parent_type)
config_schema_snapshot = config_type.get_schema_snapshot() # pyright: ignore[reportAttributeAccessIssue]
config_schema_snapshot = config_type.schema_snapshot # pyright: ignore[reportAttributeAccessIssue]

all_type_keys = get_recursive_type_keys(
snap_from_config_type(config_type), # pyright: ignore[reportArgumentType]
Expand Down

0 comments on commit 40d93ab

Please sign in to comment.