diff --git a/python_modules/dagster/dagster/_check/__init__.py b/python_modules/dagster/dagster/_check/__init__.py index da4d16892b737..b39308971e9d6 100644 --- a/python_modules/dagster/dagster/_check/__init__.py +++ b/python_modules/dagster/dagster/_check/__init__.py @@ -22,6 +22,8 @@ overload, ) +from typing_extensions import Literal + TypeOrTupleOfTypes = Union[type, Tuple[type, ...]] Numeric = Union[int, float] T = TypeVar("T") @@ -768,12 +770,8 @@ def list_param( of_type: Optional[TypeOrTupleOfTypes] = None, additional_message: Optional[str] = None, ) -> List[Any]: - from dagster._utils import frozenlist - - if not isinstance(obj, (frozenlist, list)): - raise _param_type_mismatch_exception( - obj, (frozenlist, list), param_name, additional_message - ) + if not isinstance(obj, list): + raise _param_type_mismatch_exception(obj, list, param_name, additional_message) if not of_type: return obj @@ -793,12 +791,8 @@ def opt_list_param( If the of_type argument is provided, also ensures that list items conform to the type specified by of_type. """ - from dagster._utils import frozenlist - - if obj is not None and not isinstance(obj, (frozenlist, list)): - raise _param_type_mismatch_exception( - obj, (frozenlist, list), param_name, additional_message - ) + if obj is not None and not isinstance(obj, list): + raise _param_type_mismatch_exception(obj, list, param_name, additional_message) if not obj: return [] @@ -840,12 +834,8 @@ def opt_nullable_list_param( If the of_type argument is provided, also ensures that list items conform to the type specified by of_type. """ - from dagster._utils import frozenlist - - if obj is not None and not isinstance(obj, (frozenlist, list)): - raise _param_type_mismatch_exception( - obj, (frozenlist, list), param_name, additional_message - ) + if obj is not None and not isinstance(obj, list): + raise _param_type_mismatch_exception(obj, list, param_name, additional_message) if not obj: return None if obj is None else [] @@ -1180,39 +1170,88 @@ def opt_path_param( # ######################## +@overload +def sequence_param( + obj: Sequence[T], + param_name: str, + of_type: Optional[TypeOrTupleOfTypes] = ..., + additional_message: Optional[str] = ..., + as_tuple: Literal[True] = ..., +) -> Tuple[T, ...]: + ... + + +@overload def sequence_param( obj: Sequence[T], param_name: str, of_type: Optional[TypeOrTupleOfTypes] = None, additional_message: Optional[str] = None, + as_tuple: bool = ..., ) -> Sequence[T]: + ... + + +def sequence_param( + obj: Sequence[T], + param_name: str, + of_type: Optional[TypeOrTupleOfTypes] = None, + additional_message: Optional[str] = None, + as_tuple: bool = False, +) -> Union[Sequence[T], Tuple[T, ...]]: if not isinstance(obj, collections.abc.Sequence): raise _param_type_mismatch_exception( obj, (collections.abc.Sequence,), param_name, additional_message ) + norm_obj = tuple(obj) if as_tuple else obj if not of_type: - return obj + return norm_obj + + return _check_iterable_items(norm_obj, of_type, "sequence") - return _check_iterable_items(obj, of_type, "sequence") +@overload +def opt_sequence_param( + obj: Optional[Sequence[T]], + param_name: str, + of_type: Optional[TypeOrTupleOfTypes] = ..., + additional_message: Optional[str] = ..., + as_tuple: Literal[True] = ..., +) -> Tuple[T, ...]: + ... + +@overload def opt_sequence_param( obj: Optional[Sequence[T]], param_name: str, of_type: Optional[TypeOrTupleOfTypes] = None, additional_message: Optional[str] = None, + as_tuple: bool = ..., ) -> Sequence[T]: + ... + + +def opt_sequence_param( + obj: Optional[Sequence[T]], + param_name: str, + of_type: Optional[TypeOrTupleOfTypes] = None, + additional_message: Optional[str] = None, + as_tuple: bool = False, +) -> Union[Sequence[T], Tuple[T, ...]]: if obj is None: - return [] + return tuple() if as_tuple else [] elif not isinstance(obj, collections.abc.Sequence): raise _param_type_mismatch_exception( obj, (collections.abc.Sequence,), param_name, additional_message ) - elif of_type is not None: - return _check_iterable_items(obj, of_type, "sequence") + + norm_obj = tuple(obj) if as_tuple else obj + if of_type is not None: + return _check_iterable_items(norm_obj, of_type, "sequence") else: - return obj + return norm_obj @overload @@ -1221,16 +1260,29 @@ def opt_nullable_sequence_param( param_name: str, of_type: Optional[TypeOrTupleOfTypes] = ..., additional_message: Optional[str] = ..., + as_tuple: bool = ..., ) -> None: ... +@overload +def opt_nullable_sequence_param( + obj: Sequence[T], + param_name: str, + of_type: Optional[TypeOrTupleOfTypes] = ..., + additional_message: Optional[str] = ..., + as_tuple: Literal[True] = ..., +) -> Tuple[T, ...]: + ... + + @overload def opt_nullable_sequence_param( obj: Sequence[T], param_name: str, of_type: Optional[TypeOrTupleOfTypes] = None, additional_message: Optional[str] = ..., + as_tuple: bool = ..., ) -> Sequence[T]: ... @@ -1240,11 +1292,12 @@ def opt_nullable_sequence_param( param_name: str, of_type: Optional[TypeOrTupleOfTypes] = None, additional_message: Optional[str] = None, + as_tuple: bool = False, ) -> Optional[Sequence[T]]: if obj is None: return None else: - return opt_sequence_param(obj, param_name, of_type, additional_message) + return opt_sequence_param(obj, param_name, of_type, additional_message, as_tuple) # ######################## diff --git a/python_modules/dagster/dagster/_config/post_process.py b/python_modules/dagster/dagster/_config/post_process.py index e8d21ee5dc7e8..eac5ca6465626 100644 --- a/python_modules/dagster/dagster/_config/post_process.py +++ b/python_modules/dagster/dagster/_config/post_process.py @@ -2,7 +2,7 @@ from typing import Any, Dict, List, Mapping, Optional, cast import dagster._check as check -from dagster._utils import ensure_single_item, frozendict, frozenlist +from dagster._utils import ensure_single_item, frozendict from dagster._utils.error import serializable_error_info_from_exc_info from .config_type import ConfigType, ConfigTypeKind @@ -210,7 +210,7 @@ def _recurse_in_to_array(context: TraversalContext, config_value: Any) -> Evalua if errors: return EvaluateValueResult.for_errors(errors) - return EvaluateValueResult.for_value(frozenlist([result.value for result in results])) + return EvaluateValueResult.for_value([result.value for result in results]) def _recurse_in_to_map(context: TraversalContext, config_value: Any) -> EvaluateValueResult[Any]: diff --git a/python_modules/dagster/dagster/_core/code_pointer.py b/python_modules/dagster/dagster/_core/code_pointer.py index e42c5e66ddffc..faffd38619826 100644 --- a/python_modules/dagster/dagster/_core/code_pointer.py +++ b/python_modules/dagster/dagster/_core/code_pointer.py @@ -4,13 +4,13 @@ import sys from abc import ABC, abstractmethod from types import ModuleType -from typing import Callable, List, NamedTuple, Optional, Sequence, cast +from typing import Callable, List, NamedTuple, Optional, Sequence, Tuple, cast import dagster._check as check from dagster._core.errors import DagsterImportError, DagsterInvariantViolationError from dagster._serdes import whitelist_for_serdes from dagster._seven import get_import_error_message, import_module_from_path -from dagster._utils import alter_sys_path, frozenlist +from dagster._utils import alter_sys_path class CodePointer(ABC): @@ -269,8 +269,8 @@ class CustomPointer( "_CustomPointer", [ ("reconstructor_pointer", ModuleCodePointer), - ("reconstructable_args", Sequence[object]), - ("reconstructable_kwargs", Sequence[Sequence]), + ("reconstructable_args", Tuple[object, ...]), + ("reconstructable_kwargs", Tuple[Tuple[str, object], ...]), ], ), CodePointer, @@ -298,9 +298,9 @@ def __new__( # These are frozenlists, rather than lists, so that they can be hashed and the pointer # stored in the lru_cache on the repository and pipeline get_definition methods - reconstructable_args = frozenlist(reconstructable_args) - reconstructable_kwargs = frozenlist( - [frozenlist(reconstructable_kwarg) for reconstructable_kwarg in reconstructable_kwargs] + reconstructable_args = tuple(reconstructable_args) + reconstructable_kwargs = tuple( + tuple(reconstructable_kwarg) for reconstructable_kwarg in reconstructable_kwargs ) return super(CustomPointer, cls).__new__( diff --git a/python_modules/dagster/dagster/_core/definitions/cacheable_assets.py b/python_modules/dagster/dagster/_core/definitions/cacheable_assets.py index 68688853d3e0c..3eecba6e70f06 100644 --- a/python_modules/dagster/dagster/_core/definitions/cacheable_assets.py +++ b/python_modules/dagster/dagster/_core/definitions/cacheable_assets.py @@ -2,7 +2,7 @@ import inspect import json from abc import ABC, abstractmethod -from typing import AbstractSet, Any, List, Mapping, NamedTuple, Optional, Sequence +from typing import AbstractSet, Any, List, Mapping, NamedTuple, Optional, Sequence, Tuple import dagster._check as check import dagster._seven as seven @@ -14,7 +14,7 @@ from dagster._core.definitions.resource_definition import ResourceDefinition from dagster._core.definitions.resource_requirement import ResourceAddable from dagster._serdes import whitelist_for_serdes -from dagster._utils import frozendict, frozenlist, make_readonly_value +from dagster._utils import frozendict, make_readonly_value @whitelist_for_serdes @@ -27,7 +27,7 @@ class AssetsDefinitionCacheableData( ("internal_asset_deps", Optional[Mapping[str, AbstractSet[AssetKey]]]), ("group_name", Optional[str]), ("metadata_by_output_name", Optional[Mapping[str, MetadataUserInput]]), - ("key_prefix", Optional[CoercibleToAssetKeyPrefix]), + ("key_prefix", Optional[Tuple[str, ...]]), ("can_subset", bool), ("extra_metadata", Optional[Mapping[Any, Any]]), ("freshness_policies_by_output_name", Optional[Mapping[str, FreshnessPolicy]]), @@ -95,7 +95,7 @@ def __new__( metadata_by_output_name=make_readonly_value(metadata_by_output_name) if metadata_by_output_name else None, - key_prefix=frozenlist(key_prefix) if key_prefix else None, + key_prefix=tuple(key_prefix) if key_prefix else None, can_subset=check.opt_bool_param(can_subset, "can_subset", default=False), extra_metadata=make_readonly_value(extra_metadata) if extra_metadata else None, freshness_policies_by_output_name=frozendict(freshness_policies_by_output_name) diff --git a/python_modules/dagster/dagster/_core/definitions/metadata/table.py b/python_modules/dagster/dagster/_core/definitions/metadata/table.py index 832e0964f9673..cf2f60b78b8e9 100644 --- a/python_modules/dagster/dagster/_core/definitions/metadata/table.py +++ b/python_modules/dagster/dagster/_core/definitions/metadata/table.py @@ -5,7 +5,6 @@ from dagster._serdes.serdes import ( whitelist_for_serdes, ) -from dagster._utils import frozenlist # ######################## # ##### TABLE RECORD @@ -109,7 +108,7 @@ def __new__( ): return super(TableSchema, cls).__new__( cls, - columns=frozenlist(check.sequence_param(columns, "columns", of_type=TableColumn)), + columns=check.sequence_param(columns, "columns", of_type=TableColumn), constraints=check.opt_inst_param( constraints, "constraints", TableConstraints, default=_DEFAULT_TABLE_CONSTRAINTS ), @@ -157,7 +156,7 @@ def __new__( ): return super(TableConstraints, cls).__new__( cls, - other=frozenlist(check.sequence_param(other, "other", of_type=str)), + other=check.sequence_param(other, "other", of_type=str), ) @@ -255,7 +254,7 @@ def __new__( cls, nullable=check.bool_param(nullable, "nullable"), unique=check.bool_param(unique, "unique"), - other=frozenlist(check.opt_sequence_param(other, "other")), + other=check.opt_sequence_param(other, "other"), ) diff --git a/python_modules/dagster/dagster/_core/definitions/node_definition.py b/python_modules/dagster/dagster/_core/definitions/node_definition.py index 3b118571a2597..fdba775dde948 100644 --- a/python_modules/dagster/dagster/_core/definitions/node_definition.py +++ b/python_modules/dagster/dagster/_core/definitions/node_definition.py @@ -13,7 +13,7 @@ import dagster._check as check from dagster._core.definitions.configurable import NamedConfigurableDefinition from dagster._core.definitions.policy import RetryPolicy -from dagster._utils import frozendict, frozenlist +from dagster._utils import frozendict from .hook_definition import HookDefinition from .utils import check_valid_name, validate_tags @@ -54,10 +54,10 @@ def __init__( self._name = check_valid_name(name) self._description = check.opt_str_param(description, "description") self._tags = validate_tags(tags) - self._input_defs = frozenlist(input_defs) + self._input_defs = tuple(input_defs) self._input_dict = frozendict({input_def.name: input_def for input_def in input_defs}) check.invariant(len(self._input_defs) == len(self._input_dict), "Duplicate input def names") - self._output_defs = frozenlist(output_defs) + self._output_defs = tuple(output_defs) self._output_dict = frozendict({output_def.name: output_def for output_def in output_defs}) check.invariant( len(self._output_defs) == len(self._output_dict), "Duplicate output def names" diff --git a/python_modules/dagster/dagster/_core/definitions/partition.py b/python_modules/dagster/dagster/_core/definitions/partition.py index 79eac14703796..8654107eec2d5 100644 --- a/python_modules/dagster/dagster/_core/definitions/partition.py +++ b/python_modules/dagster/dagster/_core/definitions/partition.py @@ -38,7 +38,6 @@ from dagster._core.storage.tags import PARTITION_NAME_TAG from dagster._serdes import whitelist_for_serdes from dagster._seven.compat.pendulum import PendulumDateTime, to_timezone -from dagster._utils import frozenlist from dagster._utils.backcompat import deprecation_warning, experimental_arg_warning from dagster._utils.merger import merge_dicts from dagster._utils.schedules import schedule_execution_time_iterator @@ -949,9 +948,7 @@ def _execution_fn(context): return selected_partitions = ( - selector_result - if isinstance(selector_result, (frozenlist, list)) - else [selector_result] + selector_result if isinstance(selector_result, list) else [selector_result] ) check.is_list(selected_partitions, of_type=Partition) diff --git a/python_modules/dagster/dagster/_core/definitions/reconstruct.py b/python_modules/dagster/dagster/_core/definitions/reconstruct.py index b13cda3a20256..3b3a6b0cc344c 100644 --- a/python_modules/dagster/dagster/_core/definitions/reconstruct.py +++ b/python_modules/dagster/dagster/_core/definitions/reconstruct.py @@ -40,7 +40,7 @@ ) from dagster._core.selector import parse_solid_selection from dagster._serdes import pack_value, unpack_value, whitelist_for_serdes -from dagster._utils import frozenlist, make_readonly_value +from dagster._utils import make_readonly_value from .events import AssetKey from .pipeline_base import IPipeline @@ -71,7 +71,7 @@ class ReconstructableRepository( ("pointer", CodePointer), ("container_image", Optional[str]), ("executable_path", Optional[str]), - ("entry_point", Sequence[str]), + ("entry_point", Tuple[str, ...]), ("container_context", Optional[Mapping[str, Any]]), ("repository_load_data", Optional["RepositoryLoadData"]), ], @@ -94,7 +94,7 @@ def __new__( container_image=check.opt_str_param(container_image, "container_image"), executable_path=check.opt_str_param(executable_path, "executable_path"), entry_point=( - frozenlist(check.sequence_param(entry_point, "entry_point", of_type=str)) + tuple(check.sequence_param(entry_point, "entry_point", of_type=str)) if entry_point is not None else DEFAULT_DAGSTER_ENTRY_POINT ), diff --git a/python_modules/dagster/dagster/_core/origin.py b/python_modules/dagster/dagster/_core/origin.py index d13b41310d22c..34e0dcd040b47 100644 --- a/python_modules/dagster/dagster/_core/origin.py +++ b/python_modules/dagster/dagster/_core/origin.py @@ -1,15 +1,16 @@ -from typing import Any, List, Mapping, NamedTuple, Optional, Sequence +from typing import Any, Mapping, NamedTuple, Optional, Sequence, Tuple + +from typing_extensions import Final import dagster._check as check from dagster._core.code_pointer import CodePointer from dagster._serdes import create_snapshot_id, whitelist_for_serdes -from dagster._utils import frozenlist -DEFAULT_DAGSTER_ENTRY_POINT = frozenlist(["dagster"]) +DEFAULT_DAGSTER_ENTRY_POINT: Final = ("dagster",) -def get_python_environment_entry_point(executable_path: str) -> List[str]: - return frozenlist([executable_path, "-m", "dagster"]) +def get_python_environment_entry_point(executable_path: str) -> Tuple[str, ...]: + return (executable_path, "-m", "dagster") @whitelist_for_serdes @@ -20,7 +21,7 @@ class RepositoryPythonOrigin( ("executable_path", str), ("code_pointer", CodePointer), ("container_image", Optional[str]), - ("entry_point", Optional[Sequence[str]]), + ("entry_point", Optional[Tuple[str, ...]]), ("container_context", Optional[Mapping[str, Any]]), ], ), @@ -52,7 +53,7 @@ def __new__( check.inst_param(code_pointer, "code_pointer", CodePointer), check.opt_str_param(container_image, "container_image"), ( - frozenlist(check.sequence_param(entry_point, "entry_point", of_type=str)) + check.sequence_param(entry_point, "entry_point", of_type=str, as_tuple=True) if entry_point is not None else None ), diff --git a/python_modules/dagster/dagster/_grpc/server.py b/python_modules/dagster/dagster/_grpc/server.py index f45a5b67f3de0..44889bbdbce71 100644 --- a/python_modules/dagster/dagster/_grpc/server.py +++ b/python_modules/dagster/dagster/_grpc/server.py @@ -39,7 +39,6 @@ from dagster._serdes.ipc import IPCErrorMessage, ipc_write_stream, open_ipc_subprocess from dagster._utils import ( find_free_port, - frozenlist, get_run_crash_explanation, safe_tempfile_path_unmanaged, ) @@ -237,7 +236,7 @@ def __init__( self._serializable_load_error = None self._entry_point = ( - frozenlist(check.sequence_param(entry_point, "entry_point", of_type=str)) + list(check.sequence_param(entry_point, "entry_point", of_type=str)) if entry_point is not None else DEFAULT_DAGSTER_ENTRY_POINT ) @@ -1092,24 +1091,24 @@ def open_server_process( executable_path = loadable_target_origin.executable_path if loadable_target_origin else None - subprocess_args = ( - get_python_environment_entry_point(executable_path or sys.executable) - + ["api", "grpc"] - + ["--lazy-load-user-code"] - + (["--port", str(port)] if port else []) - + (["--socket", socket] if socket else []) - + (["-n", str(max_workers)] if max_workers else []) - + (["--heartbeat"] if heartbeat else []) - + (["--heartbeat-timeout", str(heartbeat_timeout)] if heartbeat_timeout else []) - + (["--fixed-server-id", fixed_server_id] if fixed_server_id else []) - + (["--override-system-timezone", mocked_system_timezone] if mocked_system_timezone else []) - + (["--log-level", log_level]) + subprocess_args = [ + *get_python_environment_entry_point(executable_path or sys.executable), + *["api", "grpc"], + *["--lazy-load-user-code"], + *(["--port", str(port)] if port else []), + *(["--socket", socket] if socket else []), + *(["-n", str(max_workers)] if max_workers else []), + *(["--heartbeat"] if heartbeat else []), + *(["--heartbeat-timeout", str(heartbeat_timeout)] if heartbeat_timeout else []), + *(["--fixed-server-id", fixed_server_id] if fixed_server_id else []), + *(["--override-system-timezone", mocked_system_timezone] if mocked_system_timezone else []), + *(["--log-level", log_level]), # only use the Python environment if it has been explicitly set in the workspace - + (["--use-python-environment-entry-point"] if executable_path else []) - + (["--inject-env-vars-from-instance"]) - + (["--instance-ref", serialize_value(instance_ref)]) - + (["--location-name", location_name] if location_name else []) - ) + *(["--use-python-environment-entry-point"] if executable_path else []), + *(["--inject-env-vars-from-instance"]), + *(["--instance-ref", serialize_value(instance_ref)]), + *(["--location-name", location_name] if location_name else []), + ] if loadable_target_origin: subprocess_args += loadable_target_origin.get_cli_args() diff --git a/python_modules/dagster/dagster/_grpc/types.py b/python_modules/dagster/dagster/_grpc/types.py index eae7b043934f0..5ddb26f22f5e9 100644 --- a/python_modules/dagster/dagster/_grpc/types.py +++ b/python_modules/dagster/dagster/_grpc/types.py @@ -1,6 +1,6 @@ import base64 import zlib -from typing import Any, FrozenSet, Mapping, NamedTuple, Optional, Sequence +from typing import Any, FrozenSet, Mapping, NamedTuple, Optional, Sequence, Tuple import dagster._check as check from dagster._core.code_pointer import CodePointer @@ -15,7 +15,6 @@ from dagster._core.instance.ref import InstanceRef from dagster._core.origin import PipelinePythonOrigin, get_python_environment_entry_point from dagster._serdes import serialize_value, whitelist_for_serdes -from dagster._utils import frozenlist from dagster._utils.error import SerializableErrorInfo @@ -70,7 +69,7 @@ def __new__( ) -def _get_entry_point(origin: PipelinePythonOrigin): +def _get_entry_point(origin: PipelinePythonOrigin) -> Tuple[str, ...]: return ( origin.repository_origin.entry_point if origin.repository_origin.entry_point @@ -116,7 +115,8 @@ def __new__( ) def get_command_args(self) -> Sequence[str]: - return _get_entry_point(self.pipeline_origin) + [ + return [ + *_get_entry_point(self.pipeline_origin), "api", "execute_run", serialize_value(self), @@ -161,7 +161,8 @@ def __new__( ) def get_command_args(self) -> Sequence[str]: - return _get_entry_point(self.pipeline_origin) + [ + return [ + *_get_entry_point(self.pipeline_origin), "api", "resume_run", serialize_value(self), @@ -248,15 +249,16 @@ def get_command_args(self, skip_serialized_namedtuple: bool = False) -> Sequence """Get the command args to run this step. If skip_serialized_namedtuple is True, then get_command_env should be used to pass the args to Click using an env var. """ - return ( - _get_entry_point(self.pipeline_origin) - + ["api", "execute_step"] - + ( + return [ + *_get_entry_point(self.pipeline_origin), + "api", + "execute_step", + *( ["--compressed-input-json", self._get_compressed_args()] if not skip_serialized_namedtuple else [] - ) - ) + ), + ] def get_command_env(self) -> Sequence[Mapping[str, str]]: """Get the env vars for overriding the Click args of this step. Used in conjuction with @@ -317,7 +319,7 @@ def __new__( value_type=CodePointer, ), entry_point=( - frozenlist(check.sequence_param(entry_point, "entry_point", of_type=str)) + list(check.sequence_param(entry_point, "entry_point", of_type=str)) if entry_point is not None else None ), diff --git a/python_modules/dagster/dagster/_utils/__init__.py b/python_modules/dagster/dagster/_utils/__init__.py index 6bab21dc9a91f..ff3fe949feaea 100644 --- a/python_modules/dagster/dagster/_utils/__init__.py +++ b/python_modules/dagster/dagster/_utils/__init__.py @@ -243,44 +243,13 @@ def __hash__(self): return hash(tuple(sorted(self.items()))) -class frozenlist(list): - def __readonly__(self, *args, **kwargs): - raise RuntimeError("Cannot modify ReadOnlyList") - - # https://docs.python.org/3/library/pickle.html#object.__reduce__ - # - # Like frozendict, implement __reduce__ and __setstate__ to handle pickling. - # Otherwise, __setstate__ will be called to restore the frozenlist, causing - # a RuntimeError because frozenlist is not mutable. - - def __reduce__(self): - return (frozenlist, (), list(self)) - - def __setstate__(self, state): - self.__init__(state) - - __setitem__ = __readonly__ - __delitem__ = __readonly__ - append = __readonly__ - clear = __readonly__ - extend = __readonly__ - insert = __readonly__ - pop = __readonly__ - remove = __readonly__ - reverse = __readonly__ - sort = __readonly__ # type: ignore[assignment] - - def __hash__(self): - return hash(tuple(self)) - - @overload -def make_readonly_value(value: List[T]) -> Sequence[T]: +def make_readonly_value(value: Dict[T, U]) -> Mapping[T, U]: ... @overload -def make_readonly_value(value: Dict[T, U]) -> Mapping[T, U]: +def make_readonly_value(value: List[T]) -> Tuple[T, ...]: ... @@ -289,9 +258,9 @@ def make_readonly_value(value: T) -> T: ... -def make_readonly_value(value: Any) -> Any: +def make_readonly_value(value: object) -> object: if isinstance(value, list): - return frozenlist(list(map(make_readonly_value, value))) + return tuple(map(make_readonly_value, value)) elif isinstance(value, dict): return frozendict({key: make_readonly_value(value) for key, value in value.items()}) elif isinstance(value, set): diff --git a/python_modules/dagster/dagster_tests/core_tests/test_external_execution_plan.py b/python_modules/dagster/dagster_tests/core_tests/test_external_execution_plan.py index 82aaf6461f941..003ed61cae65c 100644 --- a/python_modules/dagster/dagster_tests/core_tests/test_external_execution_plan.py +++ b/python_modules/dagster/dagster_tests/core_tests/test_external_execution_plan.py @@ -343,7 +343,7 @@ def compute_cacheable_data(self): def build_definitions(self, data): assert len(data) == 1 - assert data == [self._cacheable_data] + assert data == (self._cacheable_data,) # used for tracking how many times this function gets called over an execution instance = DagsterInstance.get() diff --git a/python_modules/dagster/dagster_tests/general_tests/check_tests/test_check.py b/python_modules/dagster/dagster_tests/general_tests/check_tests/test_check.py index e9490895c596c..e389bed608d67 100644 --- a/python_modules/dagster/dagster_tests/general_tests/check_tests/test_check.py +++ b/python_modules/dagster/dagster_tests/general_tests/check_tests/test_check.py @@ -12,7 +12,7 @@ NotImplementedCheckError, ParameterCheckError, ) -from dagster._utils import frozendict, frozenlist +from dagster._utils import frozendict @contextmanager @@ -776,7 +776,6 @@ class Baaz: def test_list_param(): assert check.list_param([], "list_param") == [] - assert check.list_param(frozenlist(), "list_param") == [] assert check.list_param(["foo"], "list_param", of_type=str) == ["foo"] @@ -812,7 +811,6 @@ def test_opt_list_param(): assert check.opt_list_param(None, "list_param") == [] assert check.opt_list_param(None, "list_param", of_type=str) == [] assert check.opt_list_param([], "list_param") == [] - assert check.opt_list_param(frozenlist(), "list_param") == [] obj_list = [1] assert check.list_param(obj_list, "list_param") == obj_list assert check.opt_list_param(["foo"], "list_param", of_type=str) == ["foo"] @@ -852,7 +850,6 @@ class Bar: def test_opt_nullable_list_param(): assert check.opt_nullable_list_param(None, "list_param") is None assert check.opt_nullable_list_param([], "list_param") == [] - assert check.opt_nullable_list_param(frozenlist(), "list_param") == [] obj_list = [1] assert check.opt_nullable_list_param(obj_list, "list_param") == obj_list diff --git a/python_modules/dagster/dagster_tests/general_tests/utils_tests/test_frozenlist.py b/python_modules/dagster/dagster_tests/general_tests/utils_tests/test_frozenlist.py deleted file mode 100644 index b3119e4d669f8..0000000000000 --- a/python_modules/dagster/dagster_tests/general_tests/utils_tests/test_frozenlist.py +++ /dev/null @@ -1,23 +0,0 @@ -import pickle - -import pytest -from dagster._utils import frozenlist - - -def test_pickle_frozenlist(): - orig_list = frozenlist([1, "a", {}]) - data = pickle.dumps(orig_list) - loaded_list = pickle.loads(data) - - assert orig_list == loaded_list - - -def test_hash_frozen_list(): - assert hash(frozenlist([])) - assert hash(frozenlist(["foo", "bar"])) - - with pytest.raises(TypeError, match="unhashable type"): - hash(frozenlist([[]])) - - with pytest.raises(TypeError, match="unhashable type"): - hash(frozenlist([{}]))