From 1b2ac55361f152740ca2924a407c602c649579bc Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Tue, 31 Oct 2023 17:30:32 -0400 Subject: [PATCH 01/44] how crazy is it to not have bound op execution context --- .../_core/definitions/op_invocation.py | 32 +- .../_core/execution/context/invocation.py | 370 ++++++------------ .../dagster/dagster/_core/pipes/context.py | 10 +- 3 files changed, 137 insertions(+), 275 deletions(-) diff --git a/python_modules/dagster/dagster/_core/definitions/op_invocation.py b/python_modules/dagster/dagster/_core/definitions/op_invocation.py index 984dd8003626b..7b38150ad329d 100644 --- a/python_modules/dagster/dagster/_core/definitions/op_invocation.py +++ b/python_modules/dagster/dagster/_core/definitions/op_invocation.py @@ -32,7 +32,7 @@ from .result import MaterializeResult if TYPE_CHECKING: - from ..execution.context.invocation import BoundOpExecutionContext + from ..execution.context.invocation import DirectInvocationOpExecutionContext from .assets import AssetsDefinition from .composition import PendingNodeInvocation from .decorators.op_decorator import DecoratedOpFunction @@ -109,7 +109,7 @@ def direct_invocation_result( ) -> Any: from dagster._config.pythonic_config import Config from dagster._core.execution.context.invocation import ( - UnboundOpExecutionContext, + DirectInvocationOpExecutionContext, build_op_context, ) @@ -149,12 +149,12 @@ def direct_invocation_result( " no context was provided when invoking." ) if len(args) > 0: - if args[0] is not None and not isinstance(args[0], UnboundOpExecutionContext): + if args[0] is not None and not isinstance(args[0], DirectInvocationOpExecutionContext): raise DagsterInvalidInvocationError( f"Decorated function '{compute_fn.name}' has context argument, " "but no context was provided when invoking." ) - context = cast(UnboundOpExecutionContext, args[0]) + context = cast(DirectInvocationOpExecutionContext, args[0]) # update args to omit context args = args[1:] else: # context argument is provided under kwargs @@ -165,14 +165,14 @@ def direct_invocation_result( f"'{context_param_name}', but no value for '{context_param_name}' was " f"found when invoking. Provided kwargs: {kwargs}" ) - context = cast(UnboundOpExecutionContext, kwargs[context_param_name]) + context = cast(DirectInvocationOpExecutionContext, kwargs[context_param_name]) # update kwargs to remove context kwargs = { kwarg: val for kwarg, val in kwargs.items() if not kwarg == context_param_name } # allow passing context, even if the function doesn't have an arg for it - elif len(args) > 0 and isinstance(args[0], UnboundOpExecutionContext): - context = cast(UnboundOpExecutionContext, args[0]) + elif len(args) > 0 and isinstance(args[0], DirectInvocationOpExecutionContext): + context = cast(DirectInvocationOpExecutionContext, args[0]) args = args[1:] resource_arg_mapping = {arg.name: arg.name for arg in compute_fn.get_resource_args()} @@ -223,7 +223,7 @@ def direct_invocation_result( def _resolve_inputs( - op_def: "OpDefinition", args, kwargs, context: "BoundOpExecutionContext" + op_def: "OpDefinition", args, kwargs, context: "DirectInvocationOpExecutionContext" ) -> Mapping[str, Any]: from dagster._core.execution.plan.execute_step import do_type_check @@ -326,7 +326,9 @@ def _resolve_inputs( return input_dict -def _key_for_result(result: MaterializeResult, context: "BoundOpExecutionContext") -> AssetKey: +def _key_for_result( + result: MaterializeResult, context: "DirectInvocationOpExecutionContext" +) -> AssetKey: if result.asset_key: return result.asset_key @@ -341,7 +343,7 @@ def _key_for_result(result: MaterializeResult, context: "BoundOpExecutionContext def _output_name_for_result_obj( event: MaterializeResult, - context: "BoundOpExecutionContext", + context: "DirectInvocationOpExecutionContext", ): asset_key = _key_for_result(event, context) return context.assets_def.get_output_name_for_asset_key(asset_key) @@ -350,7 +352,7 @@ def _output_name_for_result_obj( def _handle_gen_event( event: T, op_def: "OpDefinition", - context: "BoundOpExecutionContext", + context: "DirectInvocationOpExecutionContext", output_defs: Mapping[str, OutputDefinition], outputs_seen: Set[str], ) -> T: @@ -384,7 +386,7 @@ def _handle_gen_event( def _type_check_output_wrapper( - op_def: "OpDefinition", result: Any, context: "BoundOpExecutionContext" + op_def: "OpDefinition", result: Any, context: "DirectInvocationOpExecutionContext" ) -> Any: """Type checks and returns the result of a op. @@ -458,7 +460,7 @@ def type_check_gen(gen): def _type_check_function_output( - op_def: "OpDefinition", result: T, context: "BoundOpExecutionContext" + op_def: "OpDefinition", result: T, context: "DirectInvocationOpExecutionContext" ) -> T: from ..execution.plan.compute_generator import validate_and_coerce_op_result_to_iterator @@ -476,14 +478,14 @@ def _type_check_function_output( def _type_check_output( output_def: "OutputDefinition", output: Union[Output, DynamicOutput], - context: "BoundOpExecutionContext", + context: "DirectInvocationOpExecutionContext", ) -> None: """Validates and performs core type check on a provided output. Args: output_def (OutputDefinition): The output definition to validate against. output (Any): The output to validate. - context (BoundOpExecutionContext): Context containing resources to be used for type + context (DirectInvocationOpExecutionContext): Context containing resources to be used for type check. """ from ..execution.plan.execute_step import do_type_check diff --git a/python_modules/dagster/dagster/_core/execution/context/invocation.py b/python_modules/dagster/dagster/_core/execution/context/invocation.py index c53d673e3200e..dfb6b8cd296b5 100644 --- a/python_modules/dagster/dagster/_core/execution/context/invocation.py +++ b/python_modules/dagster/dagster/_core/execution/context/invocation.py @@ -24,7 +24,6 @@ ExpectationResult, UserEvent, ) -from dagster._core.definitions.hook_definition import HookDefinition from dagster._core.definitions.job_definition import JobDefinition from dagster._core.definitions.multi_dimensional_partitions import MultiPartitionsDefinition from dagster._core.definitions.op_definition import OpDefinition @@ -61,12 +60,12 @@ def _property_msg(prop_name: str, method_name: str) -> str: return ( - f"The {prop_name} {method_name} is not set on the context when a solid is directly invoked." + f"The {prop_name} {method_name} is not set on the context when an op is directly invoked." ) -class UnboundOpExecutionContext(OpExecutionContext): - """The ``context`` object available as the first argument to a solid's compute function when +class DirectInvocationOpExecutionContext(OpExecutionContext): + """The ``context`` object available as the first argument to an op's compute function when being invoked directly. Can also be used as a context manager. """ @@ -119,6 +118,15 @@ def __init__( self._assets_def = check.opt_inst_param(assets_def, "assets_def", AssetsDefinition) + # Indicates whether the context has been bound to a particular invocation of an op + # @op + # def my_op(context): + # # context._bound is True + # ... + # ctx = build_op_context() # ctx._bound is False + # my_op(ctx) + self._bound = False + def __enter__(self): self._cm_scope_entered = True return self @@ -129,6 +137,10 @@ def __exit__(self, *exc): def __del__(self): self._exit_stack.close() + def _check_bound(self, fn_name: str, fn_type: str): + if not self._bound: + raise DagsterInvalidPropertyError(_property_msg(fn_name, fn_type)) + @property def op_config(self) -> Any: return self._op_config @@ -139,6 +151,7 @@ def resource_keys(self) -> AbstractSet[str]: @property def resources(self) -> Resources: + # TODO - figure out with context manager stuff if self._resources_contain_cm and not self._cm_scope_entered: raise DagsterInvariantViolationError( "At least one provided resource is a generator, but attempting to access " @@ -149,7 +162,7 @@ def resources(self) -> Resources: @property def dagster_run(self) -> DagsterRun: - raise DagsterInvalidPropertyError(_property_msg("pipeline_run", "property")) + raise DagsterInvalidPropertyError(_property_msg("dagster_run", "property")) @property def instance(self) -> DagsterInstance: @@ -183,7 +196,13 @@ def run_id(self) -> str: @property def run_config(self) -> dict: - raise DagsterInvalidPropertyError(_property_msg("run_config", "property")) + self._check_bound(fn_name="run_config", fn_type="property") + + run_config: Dict[str, object] = {} + if self._op_config: + run_config["ops"] = {self._op_def.name: {"config": self._op_config}} + run_config["resources"] = self._resources_config + return run_config @property def job_def(self) -> JobDefinition: @@ -203,7 +222,7 @@ def node_handle(self) -> NodeHandle: raise DagsterInvalidPropertyError(_property_msg("solid_handle", "property")) @property - def op(self) -> JobDefinition: + def op(self) -> Node: raise DagsterInvalidPropertyError(_property_msg("op", "property")) @property @@ -212,11 +231,23 @@ def solid(self) -> Node: @property def op_def(self) -> OpDefinition: - raise DagsterInvalidPropertyError(_property_msg("op_def", "property")) + self._check_bound(fn_name="op_def", fn_type="property") + return self._op_def + + @property + def has_assets_def(self) -> bool: + self._check_bound(fn_name="has_assets_def", fn_type="property") + return self._assets_def is not None @property def assets_def(self) -> AssetsDefinition: - raise DagsterInvalidPropertyError(_property_msg("assets_def", "property")) + self._check_bound(fn_name="assets_def", fn_type="property") + + if self._assets_def is None: + raise DagsterInvalidPropertyError( + f"Op {self.op_def.name} does not have an assets definition." + ) + return self._assets_def @property def has_partition_key(self) -> bool: @@ -246,13 +277,20 @@ def asset_partition_key_for_output(self, output_name: str = "result") -> str: return self.partition_key def has_tag(self, key: str) -> bool: - raise DagsterInvalidPropertyError(_property_msg("has_tag", "method")) + self._check_bound(fn_name="has_tag", fn_type="method") + return key in self._tags - def get_tag(self, key: str) -> str: - raise DagsterInvalidPropertyError(_property_msg("get_tag", "method")) + def get_tag(self, key: str) -> Optional[str]: + self._check_bound(fn_name="get_tag", fn_type="method") + return self._tags.get(key) + + @property + def alias(self) -> str: + self._check_bound(fn_name="alias", fn_type="property") + return self._alias def get_step_execution_context(self) -> StepExecutionContext: - raise DagsterInvalidPropertyError(_property_msg("get_step_execution_context", "methods")) + raise DagsterInvalidPropertyError(_property_msg("get_step_execution_context", "method")) def bind( self, @@ -261,9 +299,36 @@ def bind( assets_def: Optional[AssetsDefinition], config_from_args: Optional[Mapping[str, Any]], resources_from_args: Optional[Mapping[str, Any]], - ) -> "BoundOpExecutionContext": + ) -> "DirectInvocationOpExecutionContext": from dagster._core.definitions.resource_invocation import resolve_bound_config + self._op_def = op_def + + invocation_tags = ( + pending_invocation.tags + if isinstance(pending_invocation, PendingNodeInvocation) + else None + ) + self._tags = ( + merge_dicts(self._op_def.tags, invocation_tags) + if invocation_tags + else self._op_def.tags + ) + + self._hook_defs = ( + pending_invocation.hook_defs + if isinstance(pending_invocation, PendingNodeInvocation) + else None + ) + invocation_alias = ( + pending_invocation.given_alias + if isinstance(pending_invocation, PendingNodeInvocation) + else None + ) + self._alias = invocation_alias if invocation_alias else self._op_def.name + + self._assets_def = assets_def + if resources_from_args: if self._resource_defs: raise DagsterInvalidInvocationError( @@ -271,7 +336,7 @@ def bind( ) resource_defs = wrap_resources_for_execution(resources_from_args) # add new resources context to the stack to be cleared on exit - resources = self._exit_stack.enter_context( + self._resources = self._exit_stack.enter_context( build_resources(resource_defs, self.instance) ) elif assets_def and assets_def.resource_defs: @@ -286,49 +351,25 @@ def bind( {**self._resource_defs, **assets_def.resource_defs} ) # add new resources context to the stack to be cleared on exit - resources = self._exit_stack.enter_context( + self._resources = self._exit_stack.enter_context( build_resources(resource_defs, self.instance, self._resources_config) ) else: - resources = self.resources + self._resources = self.resources resource_defs = self._resource_defs _validate_resource_requirements(resource_defs, op_def) if self.op_config and config_from_args: raise DagsterInvalidInvocationError("Cannot provide config in both context and kwargs") - op_config = resolve_bound_config(config_from_args or self.op_config, op_def) - - return BoundOpExecutionContext( - op_def=op_def, - op_config=op_config, - resources=resources, - resources_config=self._resources_config, - instance=self.instance, - log_manager=self.log, - pdb=self.pdb, - tags=( - pending_invocation.tags - if isinstance(pending_invocation, PendingNodeInvocation) - else None - ), - hook_defs=( - pending_invocation.hook_defs - if isinstance(pending_invocation, PendingNodeInvocation) - else None - ), - alias=( - pending_invocation.given_alias - if isinstance(pending_invocation, PendingNodeInvocation) - else None - ), - user_events=self._user_events, - output_metadata=self._output_metadata, - mapping_key=self._mapping_key, - partition_key=self._partition_key, - partition_key_range=self._partition_key_range, - assets_def=assets_def, - ) + self._op_config = resolve_bound_config(config_from_args or self.op_config, op_def) + + self._seen_outputs = {} + self._requires_typed_event_stream = False + self._typed_event_stream_error_message = None + self._bound = True + + return self def get_events(self) -> Sequence[UserEvent]: """Retrieve the list of user-generated events that were logged via the context. @@ -375,186 +416,8 @@ def get_output_metadata( def get_mapping_key(self) -> Optional[str]: return self._mapping_key - -def _validate_resource_requirements( - resource_defs: Mapping[str, ResourceDefinition], op_def: OpDefinition -) -> None: - """Validate correctness of resources against required resource keys.""" - if cast(DecoratedOpFunction, op_def.compute_fn).has_context_arg(): - for requirement in op_def.get_resource_requirements(): - if not requirement.is_io_manager_requirement: - ensure_requirements_satisfied(resource_defs, [requirement]) - - -class BoundOpExecutionContext(OpExecutionContext): - """The op execution context that is passed to the compute function during invocation. - - This context is bound to a specific op definition, for which the resources and config have - been validated. - """ - - _op_def: OpDefinition - _op_config: Any - _resources: "Resources" - _resources_config: Mapping[str, Any] - _instance: DagsterInstance - _log_manager: DagsterLogManager - _pdb: Optional[ForkedPdb] - _tags: Mapping[str, str] - _hook_defs: Optional[AbstractSet[HookDefinition]] - _alias: str - _user_events: List[UserEvent] - _seen_outputs: Dict[str, Union[str, Set[str]]] - _output_metadata: Dict[str, Any] - _mapping_key: Optional[str] - _partition_key: Optional[str] - _partition_key_range: Optional[PartitionKeyRange] - _assets_def: Optional[AssetsDefinition] - - def __init__( - self, - op_def: OpDefinition, - op_config: Any, - resources: "Resources", - resources_config: Mapping[str, Any], - instance: DagsterInstance, - log_manager: DagsterLogManager, - pdb: Optional[ForkedPdb], - tags: Optional[Mapping[str, str]], - hook_defs: Optional[AbstractSet[HookDefinition]], - alias: Optional[str], - user_events: List[UserEvent], - output_metadata: Dict[str, Any], - mapping_key: Optional[str], - partition_key: Optional[str], - partition_key_range: Optional[PartitionKeyRange], - assets_def: Optional[AssetsDefinition], - ): - self._op_def = op_def - self._op_config = op_config - self._resources = resources - self._instance = instance - self._log = log_manager - self._pdb = pdb - self._tags = merge_dicts(self._op_def.tags, tags) if tags else self._op_def.tags - self._hook_defs = hook_defs - self._alias = alias if alias else self._op_def.name - self._resources_config = resources_config - self._user_events = user_events - self._seen_outputs = {} - self._output_metadata = output_metadata - self._mapping_key = mapping_key - self._partition_key = partition_key - self._partition_key_range = partition_key_range - self._assets_def = assets_def - self._requires_typed_event_stream = False - self._typed_event_stream_error_message = None - - @property - def op_config(self) -> Any: - return self._op_config - - @property - def resources(self) -> Resources: - return self._resources - - @property - def dagster_run(self) -> DagsterRun: - raise DagsterInvalidPropertyError(_property_msg("pipeline_run", "property")) - - @property - def instance(self) -> DagsterInstance: - return self._instance - - @property - def pdb(self) -> ForkedPdb: - """dagster.utils.forked_pdb.ForkedPdb: Gives access to pdb debugging from within the solid. - - Example: - .. code-block:: python - - @solid - def debug_solid(context): - context.pdb.set_trace() - - """ - if self._pdb is None: - self._pdb = ForkedPdb() - - return self._pdb - - @property - def step_launcher(self) -> Optional[StepLauncher]: - raise DagsterInvalidPropertyError(_property_msg("step_launcher", "property")) - - @property - def run_id(self) -> str: - """str: Hard-coded value to indicate that we are directly invoking solid.""" - return "EPHEMERAL" - - @property - def run_config(self) -> Mapping[str, object]: - run_config: Dict[str, object] = {} - if self._op_config: - run_config["ops"] = {self._op_def.name: {"config": self._op_config}} - run_config["resources"] = self._resources_config - return run_config - - @property - def job_def(self) -> JobDefinition: - raise DagsterInvalidPropertyError(_property_msg("job_def", "property")) - - @property - def job_name(self) -> str: - raise DagsterInvalidPropertyError(_property_msg("job_name", "property")) - - @property - def log(self) -> DagsterLogManager: - """DagsterLogManager: A console manager constructed for this context.""" - return self._log - - @property - def node_handle(self) -> NodeHandle: - raise DagsterInvalidPropertyError(_property_msg("node_handle", "property")) - - @property - def op(self) -> Node: - raise DagsterInvalidPropertyError(_property_msg("op", "property")) - - @property - def op_def(self) -> OpDefinition: - return self._op_def - - @property - def has_assets_def(self) -> bool: - return self._assets_def is not None - - @property - def assets_def(self) -> AssetsDefinition: - if self._assets_def is None: - raise DagsterInvalidPropertyError( - f"Op {self.op_def.name} does not have an assets definition." - ) - return self._assets_def - - @property - def has_partition_key(self) -> bool: - return self._partition_key is not None - - def has_tag(self, key: str) -> bool: - return key in self._tags - - def get_tag(self, key: str) -> Optional[str]: - return self._tags.get(key) - - @property - def alias(self) -> str: - return self._alias - - def get_step_execution_context(self) -> StepExecutionContext: - raise DagsterInvalidPropertyError(_property_msg("get_step_execution_context", "methods")) - def for_type(self, dagster_type: DagsterType) -> TypeCheckContext: + self._check_bound(fn_name="for_type", fn_type="method") resources = cast(NamedTuple, self.resources) return TypeCheckContext( self.run_id, @@ -563,16 +426,15 @@ def for_type(self, dagster_type: DagsterType) -> TypeCheckContext: dagster_type, ) - def get_mapping_key(self) -> Optional[str]: - return self._mapping_key - def describe_op(self) -> str: + self._check_bound(fn_name="describe_op", fn_type="method") if isinstance(self.op_def, OpDefinition): return f'op "{self.op_def.name}"' return f'solid "{self.op_def.name}"' def log_event(self, event: UserEvent) -> None: + self._check_bound(fn_name="log_event", fn_type="method") check.inst_param( event, "event", @@ -581,6 +443,7 @@ def log_event(self, event: UserEvent) -> None: self._user_events.append(event) def observe_output(self, output_name: str, mapping_key: Optional[str] = None) -> None: + self._check_bound(fn_name="observe_output", fn_type="method") if mapping_key: if output_name not in self._seen_outputs: self._seen_outputs[output_name] = set() @@ -589,36 +452,15 @@ def observe_output(self, output_name: str, mapping_key: Optional[str] = None) -> self._seen_outputs[output_name] = "seen" def has_seen_output(self, output_name: str, mapping_key: Optional[str] = None) -> bool: + self._check_bound(fn_name="has_seen_output", fn_type="method") if mapping_key: return ( output_name in self._seen_outputs and mapping_key in self._seen_outputs[output_name] ) return output_name in self._seen_outputs - @property - def partition_key(self) -> str: - if self._partition_key is not None: - return self._partition_key - check.failed("Tried to access partition_key for a non-partitioned asset") - - @property - def partition_key_range(self) -> PartitionKeyRange: - """The range of partition keys for the current run. - - If run is for a single partition key, return a `PartitionKeyRange` with the same start and - end. Raises an error if the current run is not a partitioned run. - """ - if self._partition_key_range: - return self._partition_key_range - elif self._partition_key: - return PartitionKeyRange(self._partition_key, self._partition_key) - else: - check.failed("Tried to access partition_key range for a non-partitioned run") - - def asset_partition_key_for_output(self, output_name: str = "result") -> str: - return self.partition_key - def asset_partitions_time_window_for_output(self, output_name: str = "result") -> TimeWindow: + self._check_bound(fn_name="asset_partitions_time_window_for_output", fn_type="method") partitions_def = self.assets_def.partitions_def if partitions_def is None: check.failed("Tried to access partition_key for a non-partitioned asset") @@ -667,6 +509,7 @@ def add_metadata_two_outputs(context) -> Tuple[str, int]: return ("dog", 5) """ + self._check_bound(fn_name="add_output_metadata", fn_type="method") metadata = check.mapping_param(metadata, "metadata", key_type=str) output_name = check.opt_str_param(output_name, "output_name") mapping_key = check.opt_str_param(mapping_key, "mapping_key") @@ -716,21 +559,34 @@ def add_metadata_two_outputs(context) -> Tuple[str, int]: else: self._output_metadata[output_name] = metadata - # In this mode no conversion is done on returned values and missing but expected outputs are not + # In bound mode no conversion is done on returned values and missing but expected outputs are not # allowed. @property def requires_typed_event_stream(self) -> bool: + self._check_bound(fn_name="requires_typed_event_stream", fn_type="property") return self._requires_typed_event_stream @property def typed_event_stream_error_message(self) -> Optional[str]: + self._check_bound(fn_name="typed_event_stream_error_message", fn_type="property") return self._typed_event_stream_error_message def set_requires_typed_event_stream(self, *, error_message: Optional[str]) -> None: + self._check_bound(fn_name="set_requires_typed_event_stream", fn_type="method") self._requires_typed_event_stream = True self._typed_event_stream_error_message = error_message +def _validate_resource_requirements( + resource_defs: Mapping[str, ResourceDefinition], op_def: OpDefinition +) -> None: + """Validate correctness of resources against required resource keys.""" + if cast(DecoratedOpFunction, op_def.compute_fn).has_context_arg(): + for requirement in op_def.get_resource_requirements(): + if not requirement.is_io_manager_requirement: + ensure_requirements_satisfied(resource_defs, [requirement]) + + def build_op_context( resources: Optional[Mapping[str, Any]] = None, op_config: Any = None, @@ -741,7 +597,7 @@ def build_op_context( partition_key_range: Optional[PartitionKeyRange] = None, mapping_key: Optional[str] = None, _assets_def: Optional[AssetsDefinition] = None, -) -> UnboundOpExecutionContext: +) -> DirectInvocationOpExecutionContext: """Builds op execution context from provided parameters. ``build_op_context`` can be used as either a function or context manager. If there is a @@ -779,7 +635,7 @@ def build_op_context( ) op_config = op_config if op_config else config - return UnboundOpExecutionContext( + return DirectInvocationOpExecutionContext( resources_dict=check.opt_mapping_param(resources, "resources", key_type=str), resources_config=check.opt_mapping_param( resources_config, "resources_config", key_type=str diff --git a/python_modules/dagster/dagster/_core/pipes/context.py b/python_modules/dagster/dagster/_core/pipes/context.py index 96004f036e027..60dd3678f3eb0 100644 --- a/python_modules/dagster/dagster/_core/pipes/context.py +++ b/python_modules/dagster/dagster/_core/pipes/context.py @@ -39,7 +39,7 @@ from dagster._core.errors import DagsterPipesExecutionError from dagster._core.events import EngineEventData from dagster._core.execution.context.compute import OpExecutionContext -from dagster._core.execution.context.invocation import BoundOpExecutionContext +from dagster._core.execution.context.invocation import DirectInvocationOpExecutionContext from dagster._utils.error import ( ExceptionInfo, SerializableErrorInfo, @@ -406,8 +406,12 @@ def build_external_execution_context_data( _convert_time_window(partition_time_window) if partition_time_window else None ), run_id=context.run_id, - job_name=None if isinstance(context, BoundOpExecutionContext) else context.job_name, - retry_number=0 if isinstance(context, BoundOpExecutionContext) else context.retry_number, + job_name=None + if isinstance(context, DirectInvocationOpExecutionContext) + else context.job_name, + retry_number=0 + if isinstance(context, DirectInvocationOpExecutionContext) + else context.retry_number, extras=extras or {}, ) From 579eb420306d7fd1081632b80bb8a20aaf44cbc2 Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Wed, 1 Nov 2023 09:29:43 -0400 Subject: [PATCH 02/44] fix test --- .../dagster_tests/asset_defs_tests/test_partitioned_assets.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python_modules/dagster/dagster_tests/asset_defs_tests/test_partitioned_assets.py b/python_modules/dagster/dagster_tests/asset_defs_tests/test_partitioned_assets.py index 4fa06e28b85e2..ac81b56b54a45 100644 --- a/python_modules/dagster/dagster_tests/asset_defs_tests/test_partitioned_assets.py +++ b/python_modules/dagster/dagster_tests/asset_defs_tests/test_partitioned_assets.py @@ -227,7 +227,7 @@ def partitioned_asset(context): @asset def non_partitioned_asset(context): with pytest.raises( - CheckError, match="Tried to access partition_key for a non-partitioned asset" + CheckError, match="Tried to access partition_key for a non-partitioned run" ): context.asset_partition_key_for_output() From c806e3c0881b04baf9a398ca5d2eef4d61da5323 Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Wed, 1 Nov 2023 15:59:44 -0400 Subject: [PATCH 03/44] wip test --- .../_core/execution/context/invocation.py | 8 +++++++- .../test_direct_invocation.py | 19 +++++++++++++++++++ 2 files changed, 26 insertions(+), 1 deletion(-) diff --git a/python_modules/dagster/dagster/_core/execution/context/invocation.py b/python_modules/dagster/dagster/_core/execution/context/invocation.py index dfb6b8cd296b5..1f0051a1835b9 100644 --- a/python_modules/dagster/dagster/_core/execution/context/invocation.py +++ b/python_modules/dagster/dagster/_core/execution/context/invocation.py @@ -118,6 +118,12 @@ def __init__( self._assets_def = check.opt_inst_param(assets_def, "assets_def", AssetsDefinition) + # These attributes will be set when the context is bound to an op invocation + self._op_def = None + self._alias = None + self._hook_defs = None + self._tags = {} + # Indicates whether the context has been bound to a particular invocation of an op # @op # def my_op(context): @@ -199,7 +205,7 @@ def run_config(self) -> dict: self._check_bound(fn_name="run_config", fn_type="property") run_config: Dict[str, object] = {} - if self._op_config: + if self._op_config and self._op_def: run_config["ops"] = {self._op_def.name: {"config": self._op_config}} run_config["resources"] = self._resources_config return run_config diff --git a/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py b/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py index 268677e9e51b9..dfd11b1002eaf 100644 --- a/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py +++ b/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py @@ -478,3 +478,22 @@ async def main(): result = asyncio.run(main()) assert result[0] == "one" assert result[1] == "two" + +def test_direct_invocation_resource_context_manager(): + from dagster import resource + + class YieldedResource: + def get_value(): + return 1 + + @resource + def yielding_resource(context): + yield YieldedResource() + + @asset(required_resource_keys={"yielded_resource"}) + def my_asset(context): + assert context.resources.yielded_resource.get_value() == 1 + + ctx = build_op_context(resources={"yielded_resource": yielding_resource}) + + my_asset(ctx) From da5b4e9ed037e8a65fda5ec9278c442180ef09e4 Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Wed, 1 Nov 2023 16:17:47 -0400 Subject: [PATCH 04/44] test for boundness --- .../pythonic_resources/test_direct_invocation.py | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py b/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py index dfd11b1002eaf..0250dff394655 100644 --- a/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py +++ b/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py @@ -483,7 +483,7 @@ def test_direct_invocation_resource_context_manager(): from dagster import resource class YieldedResource: - def get_value(): + def get_value(self): return 1 @resource @@ -494,6 +494,16 @@ def yielding_resource(context): def my_asset(context): assert context.resources.yielded_resource.get_value() == 1 - ctx = build_op_context(resources={"yielded_resource": yielding_resource}) + with build_op_context(resources={"yielded_resource": yielding_resource}) as ctx: + my_asset(ctx) + +def test_bound_state(): + @asset + def my_asset(context): + assert context._bound # noqa: SLF001 + + ctx = build_op_context() + assert not ctx._bound # noqa: SLF001 my_asset(ctx) + assert not ctx._bound # noqa: SLF001gi From 45d13defe1b7b4601a8f375c6fa35d61c78deb1b Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Wed, 1 Nov 2023 16:55:32 -0400 Subject: [PATCH 05/44] make a copy of the context instead --- .../_core/execution/context/invocation.py | 179 ++++++++++-------- .../test_direct_invocation.py | 2 +- 2 files changed, 101 insertions(+), 80 deletions(-) diff --git a/python_modules/dagster/dagster/_core/execution/context/invocation.py b/python_modules/dagster/dagster/_core/execution/context/invocation.py index 1f0051a1835b9..44fab07575c8d 100644 --- a/python_modules/dagster/dagster/_core/execution/context/invocation.py +++ b/python_modules/dagster/dagster/_core/execution/context/invocation.py @@ -123,6 +123,7 @@ def __init__( self._alias = None self._hook_defs = None self._tags = {} + self._seen_outputs = {} # Indicates whether the context has been bound to a particular invocation of an op # @op @@ -147,6 +148,105 @@ def _check_bound(self, fn_name: str, fn_type: str): if not self._bound: raise DagsterInvalidPropertyError(_property_msg(fn_name, fn_type)) + def bind( + self, + op_def: OpDefinition, + pending_invocation: Optional[PendingNodeInvocation[OpDefinition]], + assets_def: Optional[AssetsDefinition], + config_from_args: Optional[Mapping[str, Any]], + resources_from_args: Optional[Mapping[str, Any]], + ) -> "DirectInvocationOpExecutionContext": + from dagster._core.definitions.resource_invocation import resolve_bound_config + + if self._bound: + raise DagsterInvalidInvocationError( + "Cannot call bind() on a DirectInvocationOpExecutionContext that has already been bound." + ) + + # make a copy of the current context that will be bound to a particular op execution + bound_ctx = DirectInvocationOpExecutionContext( + op_config=self._op_config, + resources_dict=self._resource_defs, + instance=self._instance, + resources_config=self._resources_config, + partition_key=self._partition_key, + partition_key_range=self._partition_key_range, + mapping_key=self._mapping_key, + assets_def=self._assets_def, + ) + bound_ctx._user_events = self._user_events # noqa: SLF001 + bound_ctx._output_metadata = self._output_metadata # noqa: SLF001 + + # update the bound context with properties relevant to the execution of the op + bound_ctx._op_def = op_def # noqa: SLF001 + + invocation_tags = ( + pending_invocation.tags + if isinstance(pending_invocation, PendingNodeInvocation) + else None + ) + bound_ctx._tags = ( # noqa: SLF001 + merge_dicts(bound_ctx._op_def.tags, invocation_tags) # noqa: SLF001 + if invocation_tags + else bound_ctx._op_def.tags # noqa: SLF001 + ) + + bound_ctx._hook_defs = ( # noqa: SLF001 + pending_invocation.hook_defs + if isinstance(pending_invocation, PendingNodeInvocation) + else None + ) + invocation_alias = ( + pending_invocation.given_alias + if isinstance(pending_invocation, PendingNodeInvocation) + else None + ) + bound_ctx._alias = invocation_alias if invocation_alias else bound_ctx._op_def.name # noqa: SLF001 + + bound_ctx._assets_def = assets_def # noqa: SLF001 + + if resources_from_args: + if self._resource_defs: + raise DagsterInvalidInvocationError( + "Cannot provide resources in both context and kwargs" + ) + resource_defs = wrap_resources_for_execution(resources_from_args) + # add new resources context to the stack to be cleared on exit + bound_ctx._resources = self._exit_stack.enter_context( # noqa: SLF001 + build_resources(resource_defs, self.instance) + ) + elif assets_def and assets_def.resource_defs: + for key in sorted(list(assets_def.resource_defs.keys())): + if key in self._resource_defs: + raise DagsterInvalidInvocationError( + f"Error when invoking {assets_def!s} resource '{key}' " + "provided on both the definition and invocation context. Please " + "provide on only one or the other." + ) + resource_defs = wrap_resources_for_execution( + {**self._resource_defs, **assets_def.resource_defs} + ) + # add new resources context to the stack to be cleared on exit + bound_ctx._resources = self._exit_stack.enter_context( # noqa: SLF001 + build_resources(resource_defs, self.instance, self._resources_config) + ) + else: + bound_ctx._resources = self.resources # noqa: SLF001 + resource_defs = self._resource_defs + + _validate_resource_requirements(resource_defs, op_def) + + if self.op_config and config_from_args: + raise DagsterInvalidInvocationError("Cannot provide config in both context and kwargs") + bound_ctx._op_config = resolve_bound_config(config_from_args or self.op_config, op_def) # noqa: SLF001 + + bound_ctx._seen_outputs = {} # noqa: SLF001 + bound_ctx._requires_typed_event_stream = False # noqa: SLF001 + bound_ctx._typed_event_stream_error_message = None # noqa: SLF001 + bound_ctx._bound = True # noqa: SLF001 + + return bound_ctx + @property def op_config(self) -> Any: return self._op_config @@ -298,85 +398,6 @@ def alias(self) -> str: def get_step_execution_context(self) -> StepExecutionContext: raise DagsterInvalidPropertyError(_property_msg("get_step_execution_context", "method")) - def bind( - self, - op_def: OpDefinition, - pending_invocation: Optional[PendingNodeInvocation[OpDefinition]], - assets_def: Optional[AssetsDefinition], - config_from_args: Optional[Mapping[str, Any]], - resources_from_args: Optional[Mapping[str, Any]], - ) -> "DirectInvocationOpExecutionContext": - from dagster._core.definitions.resource_invocation import resolve_bound_config - - self._op_def = op_def - - invocation_tags = ( - pending_invocation.tags - if isinstance(pending_invocation, PendingNodeInvocation) - else None - ) - self._tags = ( - merge_dicts(self._op_def.tags, invocation_tags) - if invocation_tags - else self._op_def.tags - ) - - self._hook_defs = ( - pending_invocation.hook_defs - if isinstance(pending_invocation, PendingNodeInvocation) - else None - ) - invocation_alias = ( - pending_invocation.given_alias - if isinstance(pending_invocation, PendingNodeInvocation) - else None - ) - self._alias = invocation_alias if invocation_alias else self._op_def.name - - self._assets_def = assets_def - - if resources_from_args: - if self._resource_defs: - raise DagsterInvalidInvocationError( - "Cannot provide resources in both context and kwargs" - ) - resource_defs = wrap_resources_for_execution(resources_from_args) - # add new resources context to the stack to be cleared on exit - self._resources = self._exit_stack.enter_context( - build_resources(resource_defs, self.instance) - ) - elif assets_def and assets_def.resource_defs: - for key in sorted(list(assets_def.resource_defs.keys())): - if key in self._resource_defs: - raise DagsterInvalidInvocationError( - f"Error when invoking {assets_def!s} resource '{key}' " - "provided on both the definition and invocation context. Please " - "provide on only one or the other." - ) - resource_defs = wrap_resources_for_execution( - {**self._resource_defs, **assets_def.resource_defs} - ) - # add new resources context to the stack to be cleared on exit - self._resources = self._exit_stack.enter_context( - build_resources(resource_defs, self.instance, self._resources_config) - ) - else: - self._resources = self.resources - resource_defs = self._resource_defs - - _validate_resource_requirements(resource_defs, op_def) - - if self.op_config and config_from_args: - raise DagsterInvalidInvocationError("Cannot provide config in both context and kwargs") - self._op_config = resolve_bound_config(config_from_args or self.op_config, op_def) - - self._seen_outputs = {} - self._requires_typed_event_stream = False - self._typed_event_stream_error_message = None - self._bound = True - - return self - def get_events(self) -> Sequence[UserEvent]: """Retrieve the list of user-generated events that were logged via the context. diff --git a/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py b/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py index 0250dff394655..f0e5290bde618 100644 --- a/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py +++ b/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py @@ -506,4 +506,4 @@ def my_asset(context): ctx = build_op_context() assert not ctx._bound # noqa: SLF001 my_asset(ctx) - assert not ctx._bound # noqa: SLF001gi + assert not ctx._bound # noqa: SLF001 From bdb7cdf0d7854361a23faaa11bd2e4d623b38a92 Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Thu, 2 Nov 2023 15:07:16 -0400 Subject: [PATCH 06/44] with unbind --- .../_core/definitions/op_invocation.py | 6 +- .../_core/execution/context/invocation.py | 119 +++++++++++++++++- 2 files changed, 122 insertions(+), 3 deletions(-) diff --git a/python_modules/dagster/dagster/_core/definitions/op_invocation.py b/python_modules/dagster/dagster/_core/definitions/op_invocation.py index 7b38150ad329d..3ea9385253066 100644 --- a/python_modules/dagster/dagster/_core/definitions/op_invocation.py +++ b/python_modules/dagster/dagster/_core/definitions/op_invocation.py @@ -219,7 +219,11 @@ def direct_invocation_result( resource_args=resource_arg_mapping, ) - return _type_check_output_wrapper(op_def, result, bound_context) + res = _type_check_output_wrapper(op_def, result, bound_context) + + bound_context.unbind() + + return res def _resolve_inputs( diff --git a/python_modules/dagster/dagster/_core/execution/context/invocation.py b/python_modules/dagster/dagster/_core/execution/context/invocation.py index 44fab07575c8d..b20b7c2e800c8 100644 --- a/python_modules/dagster/dagster/_core/execution/context/invocation.py +++ b/python_modules/dagster/dagster/_core/execution/context/invocation.py @@ -84,6 +84,7 @@ def __init__( from dagster._core.execution.context_creation_job import initialize_console_manager self._op_config = op_config + self._init_op_config = op_config # so we can unbind the context back to it's original state self._mapping_key = mapping_key self._exit_stack = ExitStack() @@ -102,6 +103,9 @@ def __init__( resource_config=resources_config, ) ) + self._init_resources = ( + self._resources + ) # so we can unbind the context back to it's original state self._resources_contain_cm = isinstance(self._resources, IContainsGenerator) self._log = initialize_console_manager(None) @@ -155,6 +159,22 @@ def bind( assets_def: Optional[AssetsDefinition], config_from_args: Optional[Mapping[str, Any]], resources_from_args: Optional[Mapping[str, Any]], + ) -> "DirectInvocationOpExecutionContext": + return self.bind_self( + op_def=op_def, + pending_invocation=pending_invocation, + assets_def=assets_def, + config_from_args=config_from_args, + resources_from_args=resources_from_args, + ) + + def bind_copy( + self, + op_def: OpDefinition, + pending_invocation: Optional[PendingNodeInvocation[OpDefinition]], + assets_def: Optional[AssetsDefinition], + config_from_args: Optional[Mapping[str, Any]], + resources_from_args: Optional[Mapping[str, Any]], ) -> "DirectInvocationOpExecutionContext": from dagster._core.definitions.resource_invocation import resolve_bound_config @@ -240,13 +260,107 @@ def bind( raise DagsterInvalidInvocationError("Cannot provide config in both context and kwargs") bound_ctx._op_config = resolve_bound_config(config_from_args or self.op_config, op_def) # noqa: SLF001 - bound_ctx._seen_outputs = {} # noqa: SLF001 bound_ctx._requires_typed_event_stream = False # noqa: SLF001 bound_ctx._typed_event_stream_error_message = None # noqa: SLF001 bound_ctx._bound = True # noqa: SLF001 return bound_ctx + def bind_self( + self, + op_def: OpDefinition, + pending_invocation: Optional[PendingNodeInvocation[OpDefinition]], + assets_def: Optional[AssetsDefinition], + config_from_args: Optional[Mapping[str, Any]], + resources_from_args: Optional[Mapping[str, Any]], + ) -> "DirectInvocationOpExecutionContext": + from dagster._core.definitions.resource_invocation import resolve_bound_config + + if self._bound: + raise DagsterInvalidInvocationError( + "Cannot call bind() on a DirectInvocationOpExecutionContext that has already been bound." + ) + + # update the bound context with properties relevant to the execution of the op + self._op_def = op_def + + invocation_tags = ( + pending_invocation.tags + if isinstance(pending_invocation, PendingNodeInvocation) + else None + ) + self._tags = ( + merge_dicts(self._op_def.tags, invocation_tags) + if invocation_tags + else self._op_def.tags + ) + + self._hook_defs = ( + pending_invocation.hook_defs + if isinstance(pending_invocation, PendingNodeInvocation) + else None + ) + invocation_alias = ( + pending_invocation.given_alias + if isinstance(pending_invocation, PendingNodeInvocation) + else None + ) + self._alias = invocation_alias if invocation_alias else self._op_def.name + + self._assets_def = assets_def + + if resources_from_args: + if self._resource_defs: + raise DagsterInvalidInvocationError( + "Cannot provide resources in both context and kwargs" + ) + resource_defs = wrap_resources_for_execution(resources_from_args) + # add new resources context to the stack to be cleared on exit + self._resources = self._exit_stack.enter_context( + build_resources(resource_defs, self.instance) + ) + elif assets_def and assets_def.resource_defs: + for key in sorted(list(assets_def.resource_defs.keys())): + if key in self._resource_defs: + raise DagsterInvalidInvocationError( + f"Error when invoking {assets_def!s} resource '{key}' " + "provided on both the definition and invocation context. Please " + "provide on only one or the other." + ) + resource_defs = wrap_resources_for_execution( + {**self._resource_defs, **assets_def.resource_defs} + ) + # add new resources context to the stack to be cleared on exit + self._resources = self._exit_stack.enter_context( + build_resources(resource_defs, self.instance, self._resources_config) + ) + else: + self._resources = self.resources + resource_defs = self._resource_defs + + _validate_resource_requirements(resource_defs, op_def) + + if self.op_config and config_from_args: + raise DagsterInvalidInvocationError("Cannot provide config in both context and kwargs") + self._op_config = resolve_bound_config(config_from_args or self.op_config, op_def) + + self._requires_typed_event_stream = False + self._typed_event_stream_error_message = None + self._bound = True + + return self + + def unbind(self): + self._op_def = None + self._alias = None + self._hook_defs = None + self._tags = {} + self._seen_outputs = {} + self._resources = self._init_resources + self._op_config = self._init_op_config + + self._bound = False + @property def op_config(self) -> Any: return self._op_config @@ -257,7 +371,8 @@ def resource_keys(self) -> AbstractSet[str]: @property def resources(self) -> Resources: - # TODO - figure out with context manager stuff + if self._bound: + return self._resources if self._resources_contain_cm and not self._cm_scope_entered: raise DagsterInvariantViolationError( "At least one provided resource is a generator, but attempting to access " From 6c8cbe7d1de6e45fa19ba71ed0e44cadb00de0f9 Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Thu, 2 Nov 2023 15:40:09 -0400 Subject: [PATCH 07/44] remove unbind --- .../dagster/dagster/_core/definitions/op_invocation.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python_modules/dagster/dagster/_core/definitions/op_invocation.py b/python_modules/dagster/dagster/_core/definitions/op_invocation.py index 3ea9385253066..34b543320d34c 100644 --- a/python_modules/dagster/dagster/_core/definitions/op_invocation.py +++ b/python_modules/dagster/dagster/_core/definitions/op_invocation.py @@ -221,7 +221,7 @@ def direct_invocation_result( res = _type_check_output_wrapper(op_def, result, bound_context) - bound_context.unbind() + # bound_context.unbind() return res From edd30921bbe73cfeb6dc7acdb1df85d88d1ed08a Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Fri, 3 Nov 2023 11:30:25 -0400 Subject: [PATCH 08/44] return copy byt dont override enter exit del --- .../dagster/_core/execution/context/invocation.py | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/python_modules/dagster/dagster/_core/execution/context/invocation.py b/python_modules/dagster/dagster/_core/execution/context/invocation.py index b20b7c2e800c8..9404878fb7c68 100644 --- a/python_modules/dagster/dagster/_core/execution/context/invocation.py +++ b/python_modules/dagster/dagster/_core/execution/context/invocation.py @@ -139,14 +139,17 @@ def __init__( self._bound = False def __enter__(self): - self._cm_scope_entered = True + if not self._bound: + self._cm_scope_entered = True return self def __exit__(self, *exc): - self._exit_stack.close() + if not self._bound: + self._exit_stack.close() def __del__(self): - self._exit_stack.close() + if not self._bound: + self._exit_stack.close() def _check_bound(self, fn_name: str, fn_type: str): if not self._bound: @@ -160,7 +163,7 @@ def bind( config_from_args: Optional[Mapping[str, Any]], resources_from_args: Optional[Mapping[str, Any]], ) -> "DirectInvocationOpExecutionContext": - return self.bind_self( + return self.bind_copy( op_def=op_def, pending_invocation=pending_invocation, assets_def=assets_def, From 610f8db332e32e7803b39c72e208f9dc5c333cbf Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Fri, 3 Nov 2023 13:02:35 -0400 Subject: [PATCH 09/44] warn on double invocation --- .../_core/definitions/op_invocation.py | 6 +- .../_core/execution/context/invocation.py | 140 +++--------------- 2 files changed, 18 insertions(+), 128 deletions(-) diff --git a/python_modules/dagster/dagster/_core/definitions/op_invocation.py b/python_modules/dagster/dagster/_core/definitions/op_invocation.py index 34b543320d34c..7b38150ad329d 100644 --- a/python_modules/dagster/dagster/_core/definitions/op_invocation.py +++ b/python_modules/dagster/dagster/_core/definitions/op_invocation.py @@ -219,11 +219,7 @@ def direct_invocation_result( resource_args=resource_arg_mapping, ) - res = _type_check_output_wrapper(op_def, result, bound_context) - - # bound_context.unbind() - - return res + return _type_check_output_wrapper(op_def, result, bound_context) def _resolve_inputs( diff --git a/python_modules/dagster/dagster/_core/execution/context/invocation.py b/python_modules/dagster/dagster/_core/execution/context/invocation.py index 9404878fb7c68..283f15a44997f 100644 --- a/python_modules/dagster/dagster/_core/execution/context/invocation.py +++ b/python_modules/dagster/dagster/_core/execution/context/invocation.py @@ -1,3 +1,4 @@ +import warnings from contextlib import ExitStack from typing import ( AbstractSet, @@ -84,7 +85,6 @@ def __init__( from dagster._core.execution.context_creation_job import initialize_console_manager self._op_config = op_config - self._init_op_config = op_config # so we can unbind the context back to it's original state self._mapping_key = mapping_key self._exit_stack = ExitStack() @@ -129,6 +129,10 @@ def __init__( self._tags = {} self._seen_outputs = {} + # maintain init time versions of these values so we can unbind the context + self._init_op_config = op_config + self._init_resources = self._resources + # Indicates whether the context has been bound to a particular invocation of an op # @op # def my_op(context): @@ -139,17 +143,14 @@ def __init__( self._bound = False def __enter__(self): - if not self._bound: - self._cm_scope_entered = True + self._cm_scope_entered = True return self def __exit__(self, *exc): - if not self._bound: - self._exit_stack.close() + self._exit_stack.close() def __del__(self): - if not self._bound: - self._exit_stack.close() + self._exit_stack.close() def _check_bound(self, fn_name: str, fn_type: str): if not self._bound: @@ -162,127 +163,18 @@ def bind( assets_def: Optional[AssetsDefinition], config_from_args: Optional[Mapping[str, Any]], resources_from_args: Optional[Mapping[str, Any]], - ) -> "DirectInvocationOpExecutionContext": - return self.bind_copy( - op_def=op_def, - pending_invocation=pending_invocation, - assets_def=assets_def, - config_from_args=config_from_args, - resources_from_args=resources_from_args, - ) - - def bind_copy( - self, - op_def: OpDefinition, - pending_invocation: Optional[PendingNodeInvocation[OpDefinition]], - assets_def: Optional[AssetsDefinition], - config_from_args: Optional[Mapping[str, Any]], - resources_from_args: Optional[Mapping[str, Any]], - ) -> "DirectInvocationOpExecutionContext": - from dagster._core.definitions.resource_invocation import resolve_bound_config - - if self._bound: - raise DagsterInvalidInvocationError( - "Cannot call bind() on a DirectInvocationOpExecutionContext that has already been bound." - ) - - # make a copy of the current context that will be bound to a particular op execution - bound_ctx = DirectInvocationOpExecutionContext( - op_config=self._op_config, - resources_dict=self._resource_defs, - instance=self._instance, - resources_config=self._resources_config, - partition_key=self._partition_key, - partition_key_range=self._partition_key_range, - mapping_key=self._mapping_key, - assets_def=self._assets_def, - ) - bound_ctx._user_events = self._user_events # noqa: SLF001 - bound_ctx._output_metadata = self._output_metadata # noqa: SLF001 - - # update the bound context with properties relevant to the execution of the op - bound_ctx._op_def = op_def # noqa: SLF001 - - invocation_tags = ( - pending_invocation.tags - if isinstance(pending_invocation, PendingNodeInvocation) - else None - ) - bound_ctx._tags = ( # noqa: SLF001 - merge_dicts(bound_ctx._op_def.tags, invocation_tags) # noqa: SLF001 - if invocation_tags - else bound_ctx._op_def.tags # noqa: SLF001 - ) - - bound_ctx._hook_defs = ( # noqa: SLF001 - pending_invocation.hook_defs - if isinstance(pending_invocation, PendingNodeInvocation) - else None - ) - invocation_alias = ( - pending_invocation.given_alias - if isinstance(pending_invocation, PendingNodeInvocation) - else None - ) - bound_ctx._alias = invocation_alias if invocation_alias else bound_ctx._op_def.name # noqa: SLF001 - - bound_ctx._assets_def = assets_def # noqa: SLF001 - - if resources_from_args: - if self._resource_defs: - raise DagsterInvalidInvocationError( - "Cannot provide resources in both context and kwargs" - ) - resource_defs = wrap_resources_for_execution(resources_from_args) - # add new resources context to the stack to be cleared on exit - bound_ctx._resources = self._exit_stack.enter_context( # noqa: SLF001 - build_resources(resource_defs, self.instance) - ) - elif assets_def and assets_def.resource_defs: - for key in sorted(list(assets_def.resource_defs.keys())): - if key in self._resource_defs: - raise DagsterInvalidInvocationError( - f"Error when invoking {assets_def!s} resource '{key}' " - "provided on both the definition and invocation context. Please " - "provide on only one or the other." - ) - resource_defs = wrap_resources_for_execution( - {**self._resource_defs, **assets_def.resource_defs} - ) - # add new resources context to the stack to be cleared on exit - bound_ctx._resources = self._exit_stack.enter_context( # noqa: SLF001 - build_resources(resource_defs, self.instance, self._resources_config) - ) - else: - bound_ctx._resources = self.resources # noqa: SLF001 - resource_defs = self._resource_defs - - _validate_resource_requirements(resource_defs, op_def) - - if self.op_config and config_from_args: - raise DagsterInvalidInvocationError("Cannot provide config in both context and kwargs") - bound_ctx._op_config = resolve_bound_config(config_from_args or self.op_config, op_def) # noqa: SLF001 - - bound_ctx._requires_typed_event_stream = False # noqa: SLF001 - bound_ctx._typed_event_stream_error_message = None # noqa: SLF001 - bound_ctx._bound = True # noqa: SLF001 - - return bound_ctx - - def bind_self( - self, - op_def: OpDefinition, - pending_invocation: Optional[PendingNodeInvocation[OpDefinition]], - assets_def: Optional[AssetsDefinition], - config_from_args: Optional[Mapping[str, Any]], - resources_from_args: Optional[Mapping[str, Any]], ) -> "DirectInvocationOpExecutionContext": from dagster._core.definitions.resource_invocation import resolve_bound_config if self._bound: - raise DagsterInvalidInvocationError( - "Cannot call bind() on a DirectInvocationOpExecutionContext that has already been bound." + warnings.warn( + f"This context was already used to execute {self.alias}. The information about" + f" {self.alias} will be cleared, including user events and output metadata." + " If you would like to keep this information, you can create a new context" + " using build_op_context() to invoke other ops. You can also manually clear the" + f" information about {self.alias} using the unbind() method." ) + self.unbind() # update the bound context with properties relevant to the execution of the op self._op_def = op_def @@ -361,6 +253,8 @@ def unbind(self): self._seen_outputs = {} self._resources = self._init_resources self._op_config = self._init_op_config + self._user_events = [] + self._output_metadata = {} self._bound = False From 1b2adbfd00d337708e571f0b06aeab5ff78b3749 Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Fri, 3 Nov 2023 15:18:06 -0400 Subject: [PATCH 10/44] test the state and warnings --- .../test_direct_invocation.py | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py b/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py index f0e5290bde618..48ac2feffb054 100644 --- a/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py +++ b/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py @@ -1,4 +1,5 @@ import asyncio +import warnings import pytest from dagster import ( @@ -505,5 +506,29 @@ def my_asset(context): ctx = build_op_context() assert not ctx._bound # noqa: SLF001 + + my_asset(ctx) + assert ctx._bound # noqa: SLF001 + + ctx.unbind() + assert not ctx._bound # noqa: SLF001 my_asset(ctx) + assert ctx._bound # noqa: SLF001 + + +def test_bound_state_warning(): + warnings.resetwarnings() + warnings.filterwarnings("error") + + @asset + def my_asset(context): + assert context._bound # noqa: SLF001 + + ctx = build_op_context() assert not ctx._bound # noqa: SLF001 + + my_asset(ctx) + assert ctx._bound # noqa: SLF001 + + with pytest.raises(UserWarning, match="This context was already used to execute my_asset"): + my_asset(ctx) From cdd47334f07861b904f35b7749cb350412ea8034 Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Mon, 13 Nov 2023 12:20:10 -0500 Subject: [PATCH 11/44] lil ffixes --- .../dagster/_core/execution/context/invocation.py | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/python_modules/dagster/dagster/_core/execution/context/invocation.py b/python_modules/dagster/dagster/_core/execution/context/invocation.py index 283f15a44997f..5dd11e19bff59 100644 --- a/python_modules/dagster/dagster/_core/execution/context/invocation.py +++ b/python_modules/dagster/dagster/_core/execution/context/invocation.py @@ -103,9 +103,6 @@ def __init__( resource_config=resources_config, ) ) - self._init_resources = ( - self._resources - ) # so we can unbind the context back to it's original state self._resources_contain_cm = isinstance(self._resources, IContainsGenerator) self._log = initialize_console_manager(None) @@ -140,6 +137,7 @@ def __init__( # ... # ctx = build_op_context() # ctx._bound is False # my_op(ctx) + # ctx._bound is True, must call ctx.unbind() to unbind self._bound = False def __enter__(self): @@ -230,7 +228,6 @@ def bind( build_resources(resource_defs, self.instance, self._resources_config) ) else: - self._resources = self.resources resource_defs = self._resource_defs _validate_resource_requirements(resource_defs, op_def) @@ -337,7 +334,7 @@ def log(self) -> DagsterLogManager: @property def node_handle(self) -> NodeHandle: - raise DagsterInvalidPropertyError(_property_msg("solid_handle", "property")) + raise DagsterInvalidPropertyError(_property_msg("node_handle", "property")) @property def op(self) -> Node: @@ -350,7 +347,7 @@ def solid(self) -> Node: @property def op_def(self) -> OpDefinition: self._check_bound(fn_name="op_def", fn_type="property") - return self._op_def + return cast(OpDefinition, self._op_def) @property def has_assets_def(self) -> bool: @@ -405,7 +402,7 @@ def get_tag(self, key: str) -> Optional[str]: @property def alias(self) -> str: self._check_bound(fn_name="alias", fn_type="property") - return self._alias + return cast(str, self._alias) def get_step_execution_context(self) -> StepExecutionContext: raise DagsterInvalidPropertyError(_property_msg("get_step_execution_context", "method")) From 707d8afcb1eb485b85c06c587c2011cd58ff91a7 Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Mon, 13 Nov 2023 13:30:51 -0500 Subject: [PATCH 12/44] update test --- .../pythonic_resources/test_direct_invocation.py | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py b/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py index 48ac2feffb054..856d3a37ff3a6 100644 --- a/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py +++ b/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py @@ -9,7 +9,7 @@ asset, op, ) -from dagster._core.errors import DagsterInvalidInvocationError, DagsterInvariantViolationError +from dagster._core.errors import DagsterInvalidInvocationError from dagster._core.execution.context.invocation import build_asset_context, build_op_context @@ -446,11 +446,9 @@ def my_other_asset(context): my_asset(ctx) assert ctx.get_output_metadata("result") == {"foo": "bar"} - with pytest.raises( - DagsterInvariantViolationError, - match="attempted to log metadata for output 'result' more than once", - ): - my_other_asset(ctx) + # context in unbound when used in another invocation. This allows the metadata to be + # added in my_other_asset + my_other_asset(ctx) def test_async_assets_with_shared_context(): @@ -480,6 +478,7 @@ async def main(): assert result[0] == "one" assert result[1] == "two" + def test_direct_invocation_resource_context_manager(): from dagster import resource From 516a55630cbeb7e11799d591f0316c9350d822a4 Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Tue, 14 Nov 2023 13:02:26 -0500 Subject: [PATCH 13/44] fix most of failing tests --- .../dagster/dagster/_core/execution/context/invocation.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/python_modules/dagster/dagster/_core/execution/context/invocation.py b/python_modules/dagster/dagster/_core/execution/context/invocation.py index 5dd11e19bff59..670fcd6b63544 100644 --- a/python_modules/dagster/dagster/_core/execution/context/invocation.py +++ b/python_modules/dagster/dagster/_core/execution/context/invocation.py @@ -228,6 +228,9 @@ def bind( build_resources(resource_defs, self.instance, self._resources_config) ) else: + # this runs the check in resources() to ensure we are in a context manager if necessary + self._resources = self.resources + resource_defs = self._resource_defs _validate_resource_requirements(resource_defs, op_def) From 48cb1b98fc87c1839e22b5aaf508c4c5faa0a62f Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Tue, 28 Nov 2023 15:45:36 -0500 Subject: [PATCH 14/44] re-org to have sub objects --- .../_core/execution/context/invocation.py | 202 +++++++++++------- 1 file changed, 126 insertions(+), 76 deletions(-) diff --git a/python_modules/dagster/dagster/_core/execution/context/invocation.py b/python_modules/dagster/dagster/_core/execution/context/invocation.py index 670fcd6b63544..0d8379c98e3e7 100644 --- a/python_modules/dagster/dagster/_core/execution/context/invocation.py +++ b/python_modules/dagster/dagster/_core/execution/context/invocation.py @@ -25,6 +25,7 @@ ExpectationResult, UserEvent, ) +from dagster._core.definitions.hook_definition import HookDefinition from dagster._core.definitions.job_definition import JobDefinition from dagster._core.definitions.multi_dimensional_partitions import MultiPartitionsDefinition from dagster._core.definitions.op_definition import OpDefinition @@ -65,6 +66,62 @@ def _property_msg(prop_name: str, method_name: str) -> str: ) +class BoundProperties( + NamedTuple( + "_BoundProperties", + [ + ("op_def", OpDefinition), + ("tags", Mapping[Any, Any]), + ("hook_defs", Optional[AbstractSet[HookDefinition]]), + ("alias", str), + ("assets_def", Optional[AssetsDefinition]), + ("resources", Resources), + ("op_config", Dict[str, Any]), + ("requires_typed_event_stream", bool), + ("typed_event_stream_error_message", Optional[str]), + ], + ) +): + def __new__( + cls, + op_def: OpDefinition, + tags: Mapping[Any, Any], + hook_defs: Optional[AbstractSet[HookDefinition]], + alias: str, + assets_def: Optional[AssetsDefinition], + resources: Resources, + op_config: Dict[str, Any], + ): + return super(BoundProperties, cls).__new__( + cls, + op_def=op_def, + tags=tags, + hook_defs=hook_defs, + alias=alias, + assets_def=assets_def, + resources=resources, + op_config=op_config, + requires_typed_event_stream=False, + typed_event_stream_error_message=None, + ) + + +class InvocationProperties( + NamedTuple( + "_InvocationProperties", + [ + ("user_events", List[UserEvent]), + ("seen_outputs", Dict[str, Union[Set[str], str]]), + ("output_metadata", Dict[str, Any]), + ], + ) +): + def __new__(cls): + return super(InvocationProperties, cls).__new__( + cls, user_events=[], seen_outputs={}, output_metadata={} + ) + + class DirectInvocationOpExecutionContext(OpExecutionContext): """The ``context`` object available as the first argument to an op's compute function when being invoked directly. Can also be used as a context manager. @@ -114,31 +171,25 @@ def __init__( ) self._partition_key = partition_key self._partition_key_range = partition_key_range - self._user_events: List[UserEvent] = [] - self._output_metadata: Dict[str, Any] = {} - - self._assets_def = check.opt_inst_param(assets_def, "assets_def", AssetsDefinition) - - # These attributes will be set when the context is bound to an op invocation - self._op_def = None - self._alias = None - self._hook_defs = None - self._tags = {} - self._seen_outputs = {} + # self._user_events: List[UserEvent] = [] - # maintain init time versions of these values so we can unbind the context - self._init_op_config = op_config - self._init_resources = self._resources + self._assets_def = check.opt_inst_param( + assets_def, "assets_def", AssetsDefinition + ) # TODO - is this even used? - # Indicates whether the context has been bound to a particular invocation of an op + # Maintains the properties on the context that are bound to a particular invocation + # of an op # @op # def my_op(context): - # # context._bound is True + # # context._bound_properties.alias is "my_op" # ... - # ctx = build_op_context() # ctx._bound is False + # ctx = build_op_context() # ctx._bound_properties is None # my_op(ctx) - # ctx._bound is True, must call ctx.unbind() to unbind - self._bound = False + # ctx._bound_properties.alias is "my_op", must call ctx.unbind() to unbind + self._bound_properties = None + + # Maintians the properties on the context that are modified during invocation + self._invocation_properties = None def __enter__(self): self._cm_scope_entered = True @@ -151,7 +202,7 @@ def __del__(self): self._exit_stack.close() def _check_bound(self, fn_name: str, fn_type: str): - if not self._bound: + if self._bound_properties is None: raise DagsterInvalidPropertyError(_property_msg(fn_name, fn_type)) def bind( @@ -164,7 +215,7 @@ def bind( ) -> "DirectInvocationOpExecutionContext": from dagster._core.definitions.resource_invocation import resolve_bound_config - if self._bound: + if self._bound_properties is not None: warnings.warn( f"This context was already used to execute {self.alias}. The information about" f" {self.alias} will be cleared, including user events and output metadata." @@ -175,20 +226,15 @@ def bind( self.unbind() # update the bound context with properties relevant to the execution of the op - self._op_def = op_def invocation_tags = ( pending_invocation.tags if isinstance(pending_invocation, PendingNodeInvocation) else None ) - self._tags = ( - merge_dicts(self._op_def.tags, invocation_tags) - if invocation_tags - else self._op_def.tags - ) + tags = merge_dicts(op_def.tags, invocation_tags) if invocation_tags else op_def.tags - self._hook_defs = ( + hook_defs = ( pending_invocation.hook_defs if isinstance(pending_invocation, PendingNodeInvocation) else None @@ -198,9 +244,7 @@ def bind( if isinstance(pending_invocation, PendingNodeInvocation) else None ) - self._alias = invocation_alias if invocation_alias else self._op_def.name - - self._assets_def = assets_def + alias = invocation_alias if invocation_alias else op_def.name if resources_from_args: if self._resource_defs: @@ -209,7 +253,7 @@ def bind( ) resource_defs = wrap_resources_for_execution(resources_from_args) # add new resources context to the stack to be cleared on exit - self._resources = self._exit_stack.enter_context( + resources = self._exit_stack.enter_context( build_resources(resource_defs, self.instance) ) elif assets_def and assets_def.resource_defs: @@ -224,12 +268,12 @@ def bind( {**self._resource_defs, **assets_def.resource_defs} ) # add new resources context to the stack to be cleared on exit - self._resources = self._exit_stack.enter_context( + resources = self._exit_stack.enter_context( build_resources(resource_defs, self.instance, self._resources_config) ) else: # this runs the check in resources() to ensure we are in a context manager if necessary - self._resources = self.resources + resources = self.resources resource_defs = self._resource_defs @@ -237,26 +281,24 @@ def bind( if self.op_config and config_from_args: raise DagsterInvalidInvocationError("Cannot provide config in both context and kwargs") - self._op_config = resolve_bound_config(config_from_args or self.op_config, op_def) - - self._requires_typed_event_stream = False - self._typed_event_stream_error_message = None - self._bound = True + op_config = resolve_bound_config(config_from_args or self.op_config, op_def) + + self._bound_properties = BoundProperties( + op_def=op_def, + tags=tags, + hook_defs=hook_defs, + alias=alias, + assets_def=assets_def, + resources=resources, + op_config=op_config, + ) + self._invocation_properties = InvocationProperties() return self def unbind(self): - self._op_def = None - self._alias = None - self._hook_defs = None - self._tags = {} - self._seen_outputs = {} - self._resources = self._init_resources - self._op_config = self._init_op_config - self._user_events = [] - self._output_metadata = {} - - self._bound = False + self._bound_properties = None + self._invocation_properties = None @property def op_config(self) -> Any: @@ -268,8 +310,8 @@ def resource_keys(self) -> AbstractSet[str]: @property def resources(self) -> Resources: - if self._bound: - return self._resources + if self._bound_properties is not None: + return self._bound_properties.resources if self._resources_contain_cm and not self._cm_scope_entered: raise DagsterInvariantViolationError( "At least one provided resource is a generator, but attempting to access " @@ -317,8 +359,10 @@ def run_config(self) -> dict: self._check_bound(fn_name="run_config", fn_type="property") run_config: Dict[str, object] = {} - if self._op_config and self._op_def: - run_config["ops"] = {self._op_def.name: {"config": self._op_config}} + if self._op_config and self._bound_properties.op_def: + run_config["ops"] = { + self._bound_properties.op_def.name: {"config": self._bound_properties.op_config} + } run_config["resources"] = self._resources_config return run_config @@ -350,22 +394,22 @@ def solid(self) -> Node: @property def op_def(self) -> OpDefinition: self._check_bound(fn_name="op_def", fn_type="property") - return cast(OpDefinition, self._op_def) + return cast(OpDefinition, self._bound_properties.op_def) @property def has_assets_def(self) -> bool: self._check_bound(fn_name="has_assets_def", fn_type="property") - return self._assets_def is not None + return self._bound_properties.assets_def is not None @property def assets_def(self) -> AssetsDefinition: self._check_bound(fn_name="assets_def", fn_type="property") - if self._assets_def is None: + if self._bound_properties.assets_def is None: raise DagsterInvalidPropertyError( f"Op {self.op_def.name} does not have an assets definition." ) - return self._assets_def + return self._bound_properties.assets_def @property def has_partition_key(self) -> bool: @@ -396,16 +440,16 @@ def asset_partition_key_for_output(self, output_name: str = "result") -> str: def has_tag(self, key: str) -> bool: self._check_bound(fn_name="has_tag", fn_type="method") - return key in self._tags + return key in self._bound_properties.tags def get_tag(self, key: str) -> Optional[str]: self._check_bound(fn_name="get_tag", fn_type="method") - return self._tags.get(key) + return self._bound_properties.tags.get(key) @property def alias(self) -> str: self._check_bound(fn_name="alias", fn_type="property") - return cast(str, self._alias) + return cast(str, self._bound_properties.alias) def get_step_execution_context(self) -> StepExecutionContext: raise DagsterInvalidPropertyError(_property_msg("get_step_execution_context", "method")) @@ -431,7 +475,8 @@ def test_my_op(): expectation_results = [event for event in all_user_events if isinstance(event, ExpectationResult)] ... """ - return self._user_events + self._check_bound(fn_name="get_events", fn_type="method") + return self._invocation_properties.user_events def get_output_metadata( self, output_name: str, mapping_key: Optional[str] = None @@ -447,7 +492,8 @@ def get_output_metadata( Returns: Optional[Mapping[str, Any]]: The metadata values present for the output_name/mapping_key combination, if present. """ - metadata = self._output_metadata.get(output_name) + self._check_bound(fn_name="get_output_metadata", fn_type="method") + metadata = self._invocation_properties.output_metadata.get(output_name) if mapping_key and metadata: return metadata.get(mapping_key) return metadata @@ -479,24 +525,25 @@ def log_event(self, event: UserEvent) -> None: "event", (AssetMaterialization, AssetObservation, ExpectationResult), ) - self._user_events.append(event) + self._invocation_properties.user_events.append(event) def observe_output(self, output_name: str, mapping_key: Optional[str] = None) -> None: self._check_bound(fn_name="observe_output", fn_type="method") if mapping_key: - if output_name not in self._seen_outputs: - self._seen_outputs[output_name] = set() - cast(Set[str], self._seen_outputs[output_name]).add(mapping_key) + if output_name not in self._invocation_properties.seen_outputs: + self._invocation_properties.seen_outputs[output_name] = set() + cast(Set[str], self._invocation_properties.seen_outputs[output_name]).add(mapping_key) else: - self._seen_outputs[output_name] = "seen" + self._invocation_properties.seen_outputs[output_name] = "seen" def has_seen_output(self, output_name: str, mapping_key: Optional[str] = None) -> bool: self._check_bound(fn_name="has_seen_output", fn_type="method") if mapping_key: return ( - output_name in self._seen_outputs and mapping_key in self._seen_outputs[output_name] + output_name in self._invocation_properties.seen_outputs + and mapping_key in self._invocation_properties.seen_outputs[output_name] ) - return output_name in self._seen_outputs + return output_name in self._invocation_properties.seen_outputs def asset_partitions_time_window_for_output(self, output_name: str = "result") -> TimeWindow: self._check_bound(fn_name="asset_partitions_time_window_for_output", fn_type="method") @@ -584,19 +631,22 @@ def add_metadata_two_outputs(context) -> Tuple[str, int]: ) output_name = output_def.name - if output_name in self._output_metadata: - if not mapping_key or mapping_key in self._output_metadata[output_name]: + if output_name in self._invocation_properties.output_metadata: + if ( + not mapping_key + or mapping_key in self._invocation_properties.output_metadata[output_name] + ): raise DagsterInvariantViolationError( f"In {self.op_def.node_type_str} '{self.op_def.name}', attempted to log" f" metadata for output '{output_name}' more than once." ) if mapping_key: - if output_name not in self._output_metadata: - self._output_metadata[output_name] = {} - self._output_metadata[output_name][mapping_key] = metadata + if output_name not in self._invocation_properties.output_metadata: + self._invocation_properties.output_metadata[output_name] = {} + self._invocation_properties.output_metadata[output_name][mapping_key] = metadata else: - self._output_metadata[output_name] = metadata + self._invocation_properties.output_metadata[output_name] = metadata # In bound mode no conversion is done on returned values and missing but expected outputs are not # allowed. From df5e1d276bc1d0004e4cf4b45cf7c98ac6bccfd8 Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Tue, 28 Nov 2023 15:53:55 -0500 Subject: [PATCH 15/44] remove unused _assets_def --- .../_core/execution/context/invocation.py | 20 +++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/python_modules/dagster/dagster/_core/execution/context/invocation.py b/python_modules/dagster/dagster/_core/execution/context/invocation.py index 0d8379c98e3e7..f94c8678e749f 100644 --- a/python_modules/dagster/dagster/_core/execution/context/invocation.py +++ b/python_modules/dagster/dagster/_core/execution/context/invocation.py @@ -55,6 +55,7 @@ from dagster._core.types.dagster_type import DagsterType from dagster._utils.forked_pdb import ForkedPdb from dagster._utils.merger import merge_dicts +from dagster._utils.warnings import deprecation_warning from .compute import OpExecutionContext from .system import StepExecutionContext, TypeCheckContext @@ -136,7 +137,6 @@ def __init__( partition_key: Optional[str], partition_key_range: Optional[PartitionKeyRange], mapping_key: Optional[str], - assets_def: Optional[AssetsDefinition], ): from dagster._core.execution.api import ephemeral_instance_if_missing from dagster._core.execution.context_creation_job import initialize_console_manager @@ -171,11 +171,6 @@ def __init__( ) self._partition_key = partition_key self._partition_key_range = partition_key_range - # self._user_events: List[UserEvent] = [] - - self._assets_def = check.opt_inst_param( - assets_def, "assets_def", AssetsDefinition - ) # TODO - is this even used? # Maintains the properties on the context that are bound to a particular invocation # of an op @@ -188,7 +183,7 @@ def __init__( # ctx._bound_properties.alias is "my_op", must call ctx.unbind() to unbind self._bound_properties = None - # Maintians the properties on the context that are modified during invocation + # Maintains the properties on the context that are modified during invocation self._invocation_properties = None def __enter__(self): @@ -723,6 +718,16 @@ def build_op_context( "legacy version, ``config``. Please provide one or the other." ) + if _assets_def: + deprecation_warning( + subject="build_op_context", + additional_warn_text=( + "Parameter '_assets_def' was passed to build_op_context. This parameter was intended for internal use only, and has been deprecated " + ), + breaking_version="1.8.0", + stacklevel=1, + ) + op_config = op_config if op_config else config return DirectInvocationOpExecutionContext( resources_dict=check.opt_mapping_param(resources, "resources", key_type=str), @@ -736,7 +741,6 @@ def build_op_context( partition_key_range, "partition_key_range", PartitionKeyRange ), mapping_key=check.opt_str_param(mapping_key, "mapping_key"), - assets_def=check.opt_inst_param(_assets_def, "_assets_def", AssetsDefinition), ) From e1b8fcb43438f554d6d13de16379b29606f9eb71 Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Tue, 28 Nov 2023 16:24:10 -0500 Subject: [PATCH 16/44] fix prop access --- .../dagster/dagster/_core/execution/context/invocation.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/python_modules/dagster/dagster/_core/execution/context/invocation.py b/python_modules/dagster/dagster/_core/execution/context/invocation.py index f94c8678e749f..02037856c11bb 100644 --- a/python_modules/dagster/dagster/_core/execution/context/invocation.py +++ b/python_modules/dagster/dagster/_core/execution/context/invocation.py @@ -648,17 +648,17 @@ def add_metadata_two_outputs(context) -> Tuple[str, int]: @property def requires_typed_event_stream(self) -> bool: self._check_bound(fn_name="requires_typed_event_stream", fn_type="property") - return self._requires_typed_event_stream + return self._bound_properties.requires_typed_event_stream @property def typed_event_stream_error_message(self) -> Optional[str]: self._check_bound(fn_name="typed_event_stream_error_message", fn_type="property") - return self._typed_event_stream_error_message + return self._bound_properties.typed_event_stream_error_message def set_requires_typed_event_stream(self, *, error_message: Optional[str]) -> None: self._check_bound(fn_name="set_requires_typed_event_stream", fn_type="method") - self._requires_typed_event_stream = True - self._typed_event_stream_error_message = error_message + self._bound_properties.requires_typed_event_stream = True + self._bound_properties.typed_event_stream_error_message = error_message def _validate_resource_requirements( From ee4a6fef8646a649b2c35c3985ff23f5b51e3712 Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Tue, 28 Nov 2023 17:11:15 -0500 Subject: [PATCH 17/44] fix config --- .../dagster/dagster/_core/execution/context/invocation.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/python_modules/dagster/dagster/_core/execution/context/invocation.py b/python_modules/dagster/dagster/_core/execution/context/invocation.py index 02037856c11bb..7278a32926eee 100644 --- a/python_modules/dagster/dagster/_core/execution/context/invocation.py +++ b/python_modules/dagster/dagster/_core/execution/context/invocation.py @@ -274,9 +274,9 @@ def bind( _validate_resource_requirements(resource_defs, op_def) - if self.op_config and config_from_args: + if self._op_config and config_from_args: raise DagsterInvalidInvocationError("Cannot provide config in both context and kwargs") - op_config = resolve_bound_config(config_from_args or self.op_config, op_def) + op_config = resolve_bound_config(config_from_args or self._op_config, op_def) self._bound_properties = BoundProperties( op_def=op_def, @@ -297,7 +297,9 @@ def unbind(self): @property def op_config(self) -> Any: - return self._op_config + if self._bound_properties is None: + return self._op_config + return self._bound_properties.op_config @property def resource_keys(self) -> AbstractSet[str]: From e2827d5c39873b768049f532a72eeff32c3b8b65 Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Wed, 29 Nov 2023 10:46:40 -0500 Subject: [PATCH 18/44] update boundproperties to be a plain class so attrs are mutable --- .../_core/execution/context/invocation.py | 46 +++++++------------ .../test_direct_invocation.py | 16 +++---- 2 files changed, 24 insertions(+), 38 deletions(-) diff --git a/python_modules/dagster/dagster/_core/execution/context/invocation.py b/python_modules/dagster/dagster/_core/execution/context/invocation.py index 7278a32926eee..57568cf2779e1 100644 --- a/python_modules/dagster/dagster/_core/execution/context/invocation.py +++ b/python_modules/dagster/dagster/_core/execution/context/invocation.py @@ -67,24 +67,9 @@ def _property_msg(prop_name: str, method_name: str) -> str: ) -class BoundProperties( - NamedTuple( - "_BoundProperties", - [ - ("op_def", OpDefinition), - ("tags", Mapping[Any, Any]), - ("hook_defs", Optional[AbstractSet[HookDefinition]]), - ("alias", str), - ("assets_def", Optional[AssetsDefinition]), - ("resources", Resources), - ("op_config", Dict[str, Any]), - ("requires_typed_event_stream", bool), - ("typed_event_stream_error_message", Optional[str]), - ], - ) -): - def __new__( - cls, +class BoundProperties: + def __init__( + self, op_def: OpDefinition, tags: Mapping[Any, Any], hook_defs: Optional[AbstractSet[HookDefinition]], @@ -93,18 +78,19 @@ def __new__( resources: Resources, op_config: Dict[str, Any], ): - return super(BoundProperties, cls).__new__( - cls, - op_def=op_def, - tags=tags, - hook_defs=hook_defs, - alias=alias, - assets_def=assets_def, - resources=resources, - op_config=op_config, - requires_typed_event_stream=False, - typed_event_stream_error_message=None, - ) + """Maintains the properties of the context that are provided at bind time. + This class is not implemented as a NamedTuple because requires_typed_event_stream + and type_event_stream_error_message should be mutable. + """ + self.op_def = check.inst_param(op_def, "op_def", OpDefinition) + self.tags = check.dict_param(tags, "tags", Any, Any) + self.hook_defs = check.opt_set_param(hook_defs, "hook_defs", HookDefinition) + self.alias = check.str_param(alias, "alias") + self.assets_def = check.inst_param(assets_def, "assets_def", AssetsDefinition) + self.resources = check.inst_param(resources, "resources", Resources) + self.op_config = check.dict_param(op_config, "op_config", str, Any) + self.requires_typed_event_stream = False + self.typed_event_stream_error_message = None class InvocationProperties( diff --git a/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py b/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py index 856d3a37ff3a6..367e607fb1abf 100644 --- a/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py +++ b/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py @@ -501,18 +501,18 @@ def my_asset(context): def test_bound_state(): @asset def my_asset(context): - assert context._bound # noqa: SLF001 + assert context._bound_properties is not None # noqa: SLF001 ctx = build_op_context() - assert not ctx._bound # noqa: SLF001 + assert ctx._bound_properties is None # noqa: SLF001 my_asset(ctx) - assert ctx._bound # noqa: SLF001 + assert ctx._bound_properties is not None # noqa: SLF001 ctx.unbind() - assert not ctx._bound # noqa: SLF001 + assert ctx._bound_properties is None # noqa: SLF001 my_asset(ctx) - assert ctx._bound # noqa: SLF001 + assert ctx._bound_properties is not None # noqa: SLF001 def test_bound_state_warning(): @@ -521,13 +521,13 @@ def test_bound_state_warning(): @asset def my_asset(context): - assert context._bound # noqa: SLF001 + assert context._bound_properties is not None # noqa: SLF001 ctx = build_op_context() - assert not ctx._bound # noqa: SLF001 + assert ctx._bound_properties is None # noqa: SLF001 my_asset(ctx) - assert ctx._bound # noqa: SLF001 + assert ctx._bound_properties is not None # noqa: SLF001 with pytest.raises(UserWarning, match="This context was already used to execute my_asset"): my_asset(ctx) From 7fa3db6dd52ce1ee5f2119f95bae2de42d123e07 Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Wed, 29 Nov 2023 16:55:09 -0500 Subject: [PATCH 19/44] wip --- .../dagster/_core/execution/context/invocation.py | 9 ++++++--- .../pythonic_resources/test_direct_invocation.py | 4 ++-- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/python_modules/dagster/dagster/_core/execution/context/invocation.py b/python_modules/dagster/dagster/_core/execution/context/invocation.py index 57568cf2779e1..fe49c65d663a7 100644 --- a/python_modules/dagster/dagster/_core/execution/context/invocation.py +++ b/python_modules/dagster/dagster/_core/execution/context/invocation.py @@ -76,7 +76,7 @@ def __init__( alias: str, assets_def: Optional[AssetsDefinition], resources: Resources, - op_config: Dict[str, Any], + op_config: Optional[Dict[str, Any]], ): """Maintains the properties of the context that are provided at bind time. This class is not implemented as a NamedTuple because requires_typed_event_stream @@ -86,9 +86,9 @@ def __init__( self.tags = check.dict_param(tags, "tags", Any, Any) self.hook_defs = check.opt_set_param(hook_defs, "hook_defs", HookDefinition) self.alias = check.str_param(alias, "alias") - self.assets_def = check.inst_param(assets_def, "assets_def", AssetsDefinition) + self.assets_def = check.opt_inst_param(assets_def, "assets_def", AssetsDefinition) self.resources = check.inst_param(resources, "resources", Resources) - self.op_config = check.dict_param(op_config, "op_config", str, Any) + self.op_config = check.opt_dict_param(op_config, "op_config", str, Any) self.requires_typed_event_stream = False self.typed_event_stream_error_message = None @@ -197,6 +197,7 @@ def bind( from dagster._core.definitions.resource_invocation import resolve_bound_config if self._bound_properties is not None: + # if self._completed: warnings.warn( f"This context was already used to execute {self.alias}. The information about" f" {self.alias} will be cleared, including user events and output metadata." @@ -205,6 +206,8 @@ def bind( f" information about {self.alias} using the unbind() method." ) self.unbind() + # else: + # raise DagsterInvalidInvocationError(f"This context is currently being used to execute {self.alias}. The context cannot be used to execute another op until {self.alias} has finished executing.") # update the bound context with properties relevant to the execution of the op diff --git a/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py b/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py index 367e607fb1abf..81e4e196f1ab5 100644 --- a/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py +++ b/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py @@ -509,8 +509,8 @@ def my_asset(context): my_asset(ctx) assert ctx._bound_properties is not None # noqa: SLF001 - ctx.unbind() - assert ctx._bound_properties is None # noqa: SLF001 + # ctx.unbind() + # assert ctx._bound_properties is None my_asset(ctx) assert ctx._bound_properties is not None # noqa: SLF001 From 19ad2559c0753584992e323f331454386005fe37 Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Thu, 30 Nov 2023 11:58:30 -0500 Subject: [PATCH 20/44] add tests for different execution types --- .../_core/definitions/op_invocation.py | 3 + .../_core/execution/context/invocation.py | 31 ++++---- .../test_direct_invocation.py | 78 ++++++++++++++----- 3 files changed, 80 insertions(+), 32 deletions(-) diff --git a/python_modules/dagster/dagster/_core/definitions/op_invocation.py b/python_modules/dagster/dagster/_core/definitions/op_invocation.py index 7b38150ad329d..49e9f49ed6174 100644 --- a/python_modules/dagster/dagster/_core/definitions/op_invocation.py +++ b/python_modules/dagster/dagster/_core/definitions/op_invocation.py @@ -418,6 +418,7 @@ async def to_gen(async_gen): f"Invocation of {op_def.node_type_str} '{context.alias}' did not" f" return an output for non-optional output '{output_def.name}'" ) + context.unbind() return to_gen(result) @@ -452,6 +453,7 @@ def type_check_gen(gen): f'Invocation of {op_def.node_type_str} "{context.alias}" did not' f' return an output for non-optional output "{output_def.name}"' ) + context.unbind() return type_check_gen(result) @@ -472,6 +474,7 @@ def _type_check_function_output( # ensure result objects are contextually valid _output_name_for_result_obj(event, context) + context.unbind() return result diff --git a/python_modules/dagster/dagster/_core/execution/context/invocation.py b/python_modules/dagster/dagster/_core/execution/context/invocation.py index fe49c65d663a7..f5b4ed2bc8e9b 100644 --- a/python_modules/dagster/dagster/_core/execution/context/invocation.py +++ b/python_modules/dagster/dagster/_core/execution/context/invocation.py @@ -1,4 +1,3 @@ -import warnings from contextlib import ExitStack from typing import ( AbstractSet, @@ -103,6 +102,11 @@ class InvocationProperties( ], ) ): + """Maintains information about the invocation that is updated during execution time. This information + needs to be available to the user once invocation is complete, so that they can assert on events and + outputs. It needs to be cleared before the context is used for another invocation. + """ + def __new__(cls): return super(InvocationProperties, cls).__new__( cls, user_events=[], seen_outputs={}, output_metadata={} @@ -197,17 +201,20 @@ def bind( from dagster._core.definitions.resource_invocation import resolve_bound_config if self._bound_properties is not None: - # if self._completed: - warnings.warn( - f"This context was already used to execute {self.alias}. The information about" - f" {self.alias} will be cleared, including user events and output metadata." - " If you would like to keep this information, you can create a new context" - " using build_op_context() to invoke other ops. You can also manually clear the" - f" information about {self.alias} using the unbind() method." + # warnings.warn( + # f"This context was already used to execute {self.alias}. The information about" + # f" {self.alias} will be cleared, including user events and output metadata." + # " If you would like to keep this information, you can create a new context" + # " using build_op_context() to invoke other ops. You can also manually clear the" + # f" information about {self.alias} using the unbind() method." + # ) + # self.unbind() + raise DagsterInvalidInvocationError( + f"This context is currently being used to execute {self.alias}. The context cannot be used to execute another op until {self.alias} has finished executing." ) - self.unbind() - # else: - # raise DagsterInvalidInvocationError(f"This context is currently being used to execute {self.alias}. The context cannot be used to execute another op until {self.alias} has finished executing.") + + # reset invocation_properties + self._invocation_properties = InvocationProperties() # update the bound context with properties relevant to the execution of the op @@ -277,12 +284,10 @@ def bind( op_config=op_config, ) - self._invocation_properties = InvocationProperties() return self def unbind(self): self._bound_properties = None - self._invocation_properties = None @property def op_config(self) -> Any: diff --git a/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py b/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py index 81e4e196f1ab5..1eb9b5ca542c5 100644 --- a/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py +++ b/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py @@ -1,11 +1,12 @@ import asyncio -import warnings import pytest from dagster import ( AssetExecutionContext, ConfigurableResource, OpExecutionContext, + Out, + Output, asset, op, ) @@ -474,9 +475,11 @@ async def main(): async_asset_two(ctx), ) - result = asyncio.run(main()) - assert result[0] == "one" - assert result[1] == "two" + with pytest.raises( + DagsterInvalidInvocationError, + match=r"This context is currently being used to execute .* The context cannot be used to execute another op until .* has finished executing", + ): + asyncio.run(main()) def test_direct_invocation_resource_context_manager(): @@ -498,36 +501,73 @@ def my_asset(context): my_asset(ctx) -def test_bound_state(): +def test_context_bound_state_non_generator(): @asset def my_asset(context): assert context._bound_properties is not None # noqa: SLF001 ctx = build_op_context() assert ctx._bound_properties is None # noqa: SLF001 + assert ctx._invocation_properties is None # noqa: SLF001 my_asset(ctx) - assert ctx._bound_properties is not None # noqa: SLF001 + assert ctx._bound_properties is None # noqa: SLF001 + assert ctx._invocation_properties is not None # noqa: SLF001 - # ctx.unbind() - # assert ctx._bound_properties is None - my_asset(ctx) - assert ctx._bound_properties is not None # noqa: SLF001 +def test_context_bound_state_generator(): + @op(out={"first": Out(), "second": Out()}) + def generator(context): + assert context._bound_properties is not None # noqa: SLF001 + yield Output("one", output_name="first") + yield Output("two", output_name="second") + + ctx = build_op_context() + + result = list(generator(ctx)) + assert result[0].value == "one" + assert result[1].value == "two" + + assert ctx._bound_properties is None # noqa: SLF001 + assert ctx._invocation_properties is not None # noqa: SLF001 -def test_bound_state_warning(): - warnings.resetwarnings() - warnings.filterwarnings("error") +def test_context_bound_state_async(): @asset - def my_asset(context): + async def async_asset(context): + assert context._bound_properties is not None # noqa: SLF001 + assert context.asset_key.to_user_string() == "async_asset" + await asyncio.sleep(0.01) + return "one" + + ctx = build_asset_context() + + result = asyncio.run(async_asset(ctx)) + assert result == "one" + + assert ctx._bound_properties is None # noqa: SLF001 + assert ctx._invocation_properties is not None # noqa: SLF001 + + +def test_context_bound_state_async_generator(): + @op(out={"first": Out(), "second": Out()}) + async def async_generator(context): assert context._bound_properties is not None # noqa: SLF001 + yield Output("one", output_name="first") + await asyncio.sleep(0.01) + yield Output("two", output_name="second") ctx = build_op_context() - assert ctx._bound_properties is None # noqa: SLF001 - my_asset(ctx) - assert ctx._bound_properties is not None # noqa: SLF001 + async def get_results(): + res = [] + async for output in async_generator(ctx): + res.append(output) + return res - with pytest.raises(UserWarning, match="This context was already used to execute my_asset"): - my_asset(ctx) + result = asyncio.run(get_results()) + assert result[0].value == "one" + assert result[1].value == "two" + + assert ctx._bound_properties is None # noqa: SLF001 + assert ctx._invocation_properties is not None # noqa: SLF001 From aad8d50c4b1499cda07548a0514f2a38d66d2439 Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Thu, 30 Nov 2023 14:50:55 -0500 Subject: [PATCH 21/44] fix dictionary check --- .../dagster/dagster/_core/execution/context/invocation.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python_modules/dagster/dagster/_core/execution/context/invocation.py b/python_modules/dagster/dagster/_core/execution/context/invocation.py index f5b4ed2bc8e9b..d50351bfe60a8 100644 --- a/python_modules/dagster/dagster/_core/execution/context/invocation.py +++ b/python_modules/dagster/dagster/_core/execution/context/invocation.py @@ -82,12 +82,12 @@ def __init__( and type_event_stream_error_message should be mutable. """ self.op_def = check.inst_param(op_def, "op_def", OpDefinition) - self.tags = check.dict_param(tags, "tags", Any, Any) + self.tags = check.dict_param(tags, "tags") self.hook_defs = check.opt_set_param(hook_defs, "hook_defs", HookDefinition) self.alias = check.str_param(alias, "alias") self.assets_def = check.opt_inst_param(assets_def, "assets_def", AssetsDefinition) self.resources = check.inst_param(resources, "resources", Resources) - self.op_config = check.opt_dict_param(op_config, "op_config", str, Any) + self.op_config = check.opt_dict_param(op_config, "op_config") self.requires_typed_event_stream = False self.typed_event_stream_error_message = None From b23b9829cef279e54456b54e0ad01d4519c91c8f Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Thu, 30 Nov 2023 15:46:29 -0500 Subject: [PATCH 22/44] test fixes --- .../dagster/_core/execution/context/invocation.py | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/python_modules/dagster/dagster/_core/execution/context/invocation.py b/python_modules/dagster/dagster/_core/execution/context/invocation.py index d50351bfe60a8..6a34ce60a2d12 100644 --- a/python_modules/dagster/dagster/_core/execution/context/invocation.py +++ b/python_modules/dagster/dagster/_core/execution/context/invocation.py @@ -75,7 +75,7 @@ def __init__( alias: str, assets_def: Optional[AssetsDefinition], resources: Resources, - op_config: Optional[Dict[str, Any]], + op_config: Any, ): """Maintains the properties of the context that are provided at bind time. This class is not implemented as a NamedTuple because requires_typed_event_stream @@ -87,7 +87,7 @@ def __init__( self.alias = check.str_param(alias, "alias") self.assets_def = check.opt_inst_param(assets_def, "assets_def", AssetsDefinition) self.resources = check.inst_param(resources, "resources", Resources) - self.op_config = check.opt_dict_param(op_config, "op_config") + self.op_config = op_config self.requires_typed_event_stream = False self.typed_event_stream_error_message = None @@ -174,7 +174,7 @@ def __init__( self._bound_properties = None # Maintains the properties on the context that are modified during invocation - self._invocation_properties = None + self._invocation_properties = InvocationProperties() def __enter__(self): self._cm_scope_entered = True @@ -466,7 +466,6 @@ def test_my_op(): expectation_results = [event for event in all_user_events if isinstance(event, ExpectationResult)] ... """ - self._check_bound(fn_name="get_events", fn_type="method") return self._invocation_properties.user_events def get_output_metadata( @@ -483,7 +482,6 @@ def get_output_metadata( Returns: Optional[Mapping[str, Any]]: The metadata values present for the output_name/mapping_key combination, if present. """ - self._check_bound(fn_name="get_output_metadata", fn_type="method") metadata = self._invocation_properties.output_metadata.get(output_name) if mapping_key and metadata: return metadata.get(mapping_key) @@ -528,7 +526,6 @@ def observe_output(self, output_name: str, mapping_key: Optional[str] = None) -> self._invocation_properties.seen_outputs[output_name] = "seen" def has_seen_output(self, output_name: str, mapping_key: Optional[str] = None) -> bool: - self._check_bound(fn_name="has_seen_output", fn_type="method") if mapping_key: return ( output_name in self._invocation_properties.seen_outputs From 1a3aba4ca94bed0ee95b39bd351df256c413c5e3 Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Thu, 30 Nov 2023 16:12:08 -0500 Subject: [PATCH 23/44] test update --- .../pythonic_resources/test_direct_invocation.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py b/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py index 1eb9b5ca542c5..a8fcdbd98c563 100644 --- a/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py +++ b/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py @@ -477,7 +477,8 @@ async def main(): with pytest.raises( DagsterInvalidInvocationError, - match=r"This context is currently being used to execute .* The context cannot be used to execute another op until .* has finished executing", + match=r"This context is currently being used to execute .* The context" + r" cannot be used to execute another op until .* has finished executing", ): asyncio.run(main()) @@ -508,7 +509,6 @@ def my_asset(context): ctx = build_op_context() assert ctx._bound_properties is None # noqa: SLF001 - assert ctx._invocation_properties is None # noqa: SLF001 my_asset(ctx) assert ctx._bound_properties is None # noqa: SLF001 From a3b273df790094113521cde06296a8dc8515b71b Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Fri, 1 Dec 2023 11:05:02 -0500 Subject: [PATCH 24/44] test demo for unbinding on errors --- .../test_direct_invocation.py | 23 +++++++++++++++++++ 1 file changed, 23 insertions(+) diff --git a/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py b/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py index a8fcdbd98c563..973506f4b4777 100644 --- a/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py +++ b/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py @@ -10,6 +10,7 @@ asset, op, ) +from dagster._core.definitions.events import Failure from dagster._core.errors import DagsterInvalidInvocationError from dagster._core.execution.context.invocation import build_asset_context, build_op_context @@ -571,3 +572,25 @@ async def get_results(): assert ctx._bound_properties is None # noqa: SLF001 assert ctx._invocation_properties is not None # noqa: SLF001 + + +def test_bound_state_with_error(): + @asset + def throws_error(context): + assert context.alias == "throws_error" + raise Failure("something bad happened!") + + ctx = build_asset_context() + + with pytest.raises(Failure): + throws_error(ctx) + + # invocation pathway was interrupted, ctx is still in bound state + assert ctx._bound_properties is not None # noqa: SLF001 + + @asset + def no_error(context): + assert context.alias == "no_error" + + with pytest.raises(DagsterInvalidInvocationError): + no_error(ctx) From 08a9549e0744e0098655ab0c0eae40fcc605790b Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Fri, 1 Dec 2023 11:50:42 -0500 Subject: [PATCH 25/44] re-org tests --- .../test_direct_invocation.py | 148 ------------------ .../core_tests/test_op_invocation.py | 145 +++++++++++++++++ 2 files changed, 145 insertions(+), 148 deletions(-) diff --git a/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py b/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py index 973506f4b4777..3a259def54306 100644 --- a/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py +++ b/python_modules/dagster/dagster_tests/core_tests/resource_tests/pythonic_resources/test_direct_invocation.py @@ -1,16 +1,11 @@ -import asyncio - import pytest from dagster import ( AssetExecutionContext, ConfigurableResource, OpExecutionContext, - Out, - Output, asset, op, ) -from dagster._core.definitions.events import Failure from dagster._core.errors import DagsterInvalidInvocationError from dagster._core.execution.context.invocation import build_asset_context, build_op_context @@ -434,56 +429,6 @@ def an_asset( executed.clear() -def test_direct_invocation_output_metadata(): - @asset - def my_asset(context): - context.add_output_metadata({"foo": "bar"}) - - @asset - def my_other_asset(context): - context.add_output_metadata({"baz": "qux"}) - - ctx = build_asset_context() - - my_asset(ctx) - assert ctx.get_output_metadata("result") == {"foo": "bar"} - - # context in unbound when used in another invocation. This allows the metadata to be - # added in my_other_asset - my_other_asset(ctx) - - -def test_async_assets_with_shared_context(): - @asset - async def async_asset_one(context): - assert context.asset_key.to_user_string() == "async_asset_one" - await asyncio.sleep(0.01) - return "one" - - @asset - async def async_asset_two(context): - assert context.asset_key.to_user_string() == "async_asset_two" - await asyncio.sleep(0.01) - return "two" - - # test that we can run two ops/assets with the same context at the same time without - # overriding op/asset specific attributes - ctx = build_asset_context() - - async def main(): - return await asyncio.gather( - async_asset_one(ctx), - async_asset_two(ctx), - ) - - with pytest.raises( - DagsterInvalidInvocationError, - match=r"This context is currently being used to execute .* The context" - r" cannot be used to execute another op until .* has finished executing", - ): - asyncio.run(main()) - - def test_direct_invocation_resource_context_manager(): from dagster import resource @@ -501,96 +446,3 @@ def my_asset(context): with build_op_context(resources={"yielded_resource": yielding_resource}) as ctx: my_asset(ctx) - - -def test_context_bound_state_non_generator(): - @asset - def my_asset(context): - assert context._bound_properties is not None # noqa: SLF001 - - ctx = build_op_context() - assert ctx._bound_properties is None # noqa: SLF001 - - my_asset(ctx) - assert ctx._bound_properties is None # noqa: SLF001 - assert ctx._invocation_properties is not None # noqa: SLF001 - - -def test_context_bound_state_generator(): - @op(out={"first": Out(), "second": Out()}) - def generator(context): - assert context._bound_properties is not None # noqa: SLF001 - yield Output("one", output_name="first") - yield Output("two", output_name="second") - - ctx = build_op_context() - - result = list(generator(ctx)) - assert result[0].value == "one" - assert result[1].value == "two" - - assert ctx._bound_properties is None # noqa: SLF001 - assert ctx._invocation_properties is not None # noqa: SLF001 - - -def test_context_bound_state_async(): - @asset - async def async_asset(context): - assert context._bound_properties is not None # noqa: SLF001 - assert context.asset_key.to_user_string() == "async_asset" - await asyncio.sleep(0.01) - return "one" - - ctx = build_asset_context() - - result = asyncio.run(async_asset(ctx)) - assert result == "one" - - assert ctx._bound_properties is None # noqa: SLF001 - assert ctx._invocation_properties is not None # noqa: SLF001 - - -def test_context_bound_state_async_generator(): - @op(out={"first": Out(), "second": Out()}) - async def async_generator(context): - assert context._bound_properties is not None # noqa: SLF001 - yield Output("one", output_name="first") - await asyncio.sleep(0.01) - yield Output("two", output_name="second") - - ctx = build_op_context() - - async def get_results(): - res = [] - async for output in async_generator(ctx): - res.append(output) - return res - - result = asyncio.run(get_results()) - assert result[0].value == "one" - assert result[1].value == "two" - - assert ctx._bound_properties is None # noqa: SLF001 - assert ctx._invocation_properties is not None # noqa: SLF001 - - -def test_bound_state_with_error(): - @asset - def throws_error(context): - assert context.alias == "throws_error" - raise Failure("something bad happened!") - - ctx = build_asset_context() - - with pytest.raises(Failure): - throws_error(ctx) - - # invocation pathway was interrupted, ctx is still in bound state - assert ctx._bound_properties is not None # noqa: SLF001 - - @asset - def no_error(context): - assert context.alias == "no_error" - - with pytest.raises(DagsterInvalidInvocationError): - no_error(ctx) diff --git a/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py b/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py index d575e54ca5358..14b251904ff08 100644 --- a/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py +++ b/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py @@ -1333,3 +1333,148 @@ def foo(context: AssetExecutionContext): partition_key_range=PartitionKeyRange("2023-01-01", "2023-01-02"), ) assert foo(context) == {"2023-01-01": True, "2023-01-02": True} + + +def test_direct_invocation_output_metadata(): + @asset + def my_asset(context): + context.add_output_metadata({"foo": "bar"}) + + @asset + def my_other_asset(context): + context.add_output_metadata({"baz": "qux"}) + + ctx = build_asset_context() + + my_asset(ctx) + assert ctx.get_output_metadata("result") == {"foo": "bar"} + + # context in unbound when used in another invocation. This allows the metadata to be + # added in my_other_asset + my_other_asset(ctx) + + +def test_async_assets_with_shared_context(): + @asset + async def async_asset_one(context): + assert context.asset_key.to_user_string() == "async_asset_one" + await asyncio.sleep(0.01) + return "one" + + @asset + async def async_asset_two(context): + assert context.asset_key.to_user_string() == "async_asset_two" + await asyncio.sleep(0.01) + return "two" + + # test that we can run two ops/assets with the same context at the same time without + # overriding op/asset specific attributes + ctx = build_asset_context() + + async def main(): + return await asyncio.gather( + async_asset_one(ctx), + async_asset_two(ctx), + ) + + with pytest.raises( + DagsterInvalidInvocationError, + match=r"This context is currently being used to execute .* The context" + r" cannot be used to execute another op until .* has finished executing", + ): + asyncio.run(main()) + + +def test_context_bound_state_non_generator(): + @asset + def my_asset(context): + assert context._bound_properties is not None # noqa: SLF001 + + ctx = build_op_context() + assert ctx._bound_properties is None # noqa: SLF001 + + my_asset(ctx) + assert ctx._bound_properties is None # noqa: SLF001 + assert ctx._invocation_properties is not None # noqa: SLF001 + + +def test_context_bound_state_generator(): + @op(out={"first": Out(), "second": Out()}) + def generator(context): + assert context._bound_properties is not None # noqa: SLF001 + yield Output("one", output_name="first") + yield Output("two", output_name="second") + + ctx = build_op_context() + + result = list(generator(ctx)) + assert result[0].value == "one" + assert result[1].value == "two" + + assert ctx._bound_properties is None # noqa: SLF001 + assert ctx._invocation_properties is not None # noqa: SLF001 + + +def test_context_bound_state_async(): + @asset + async def async_asset(context): + assert context._bound_properties is not None # noqa: SLF001 + assert context.asset_key.to_user_string() == "async_asset" + await asyncio.sleep(0.01) + return "one" + + ctx = build_asset_context() + + result = asyncio.run(async_asset(ctx)) + assert result == "one" + + assert ctx._bound_properties is None # noqa: SLF001 + assert ctx._invocation_properties is not None # noqa: SLF001 + + +def test_context_bound_state_async_generator(): + @op(out={"first": Out(), "second": Out()}) + async def async_generator(context): + assert context._bound_properties is not None # noqa: SLF001 + yield Output("one", output_name="first") + await asyncio.sleep(0.01) + yield Output("two", output_name="second") + + ctx = build_op_context() + + async def get_results(): + res = [] + async for output in async_generator(ctx): + res.append(output) + return res + + result = asyncio.run(get_results()) + assert result[0].value == "one" + assert result[1].value == "two" + + assert ctx._bound_properties is None # noqa: SLF001 + assert ctx._invocation_properties is not None # noqa: SLF001 + + +def test_bound_state_with_error(): + @asset + def throws_error(context): + assert context.alias == "throws_error" + raise Failure("something bad happened!") + + ctx = build_asset_context() + + throws_error(ctx) + + with pytest.raises(Failure): + throws_error(ctx) + + # invocation pathway was interrupted, ctx is still in bound state + assert ctx._bound_properties is not None # noqa: SLF001 + + @asset + def no_error(context): + assert context.alias == "no_error" + + with pytest.raises(DagsterInvalidInvocationError): + no_error(ctx) From 9273e007aa41454f60637a357ac818266011c171 Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Fri, 1 Dec 2023 11:53:32 -0500 Subject: [PATCH 26/44] update comments --- .../dagster/_core/execution/context/invocation.py | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/python_modules/dagster/dagster/_core/execution/context/invocation.py b/python_modules/dagster/dagster/_core/execution/context/invocation.py index 6a34ce60a2d12..824393d834123 100644 --- a/python_modules/dagster/dagster/_core/execution/context/invocation.py +++ b/python_modules/dagster/dagster/_core/execution/context/invocation.py @@ -170,10 +170,18 @@ def __init__( # ... # ctx = build_op_context() # ctx._bound_properties is None # my_op(ctx) - # ctx._bound_properties.alias is "my_op", must call ctx.unbind() to unbind + # ctx._bound_properties is None # ctx is unbound at the end of invocation self._bound_properties = None # Maintains the properties on the context that are modified during invocation + # @op + # def my_op(context): + # # context._invocation_properties can be modified with output metadata etc. + # ... + # ctx = build_op_context() # ctx._invocation_properties is empty + # my_op(ctx) + # ctx._invocation_properties.output_metadata # information is retained after invocation + # my_op(ctx) # ctx._invocation_properties is cleared at the beginning of the next invocation self._invocation_properties = InvocationProperties() def __enter__(self): From 858c510119b5e3d8c0cc8234cc873797944592da Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Mon, 4 Dec 2023 11:12:09 -0500 Subject: [PATCH 27/44] handle raised errors --- .../_core/definitions/op_invocation.py | 63 ++++++++++----- .../core_tests/test_op_invocation.py | 78 +++++++++++++++++-- 2 files changed, 115 insertions(+), 26 deletions(-) diff --git a/python_modules/dagster/dagster/_core/definitions/op_invocation.py b/python_modules/dagster/dagster/_core/definitions/op_invocation.py index 49e9f49ed6174..8cbafad4509ab 100644 --- a/python_modules/dagster/dagster/_core/definitions/op_invocation.py +++ b/python_modules/dagster/dagster/_core/definitions/op_invocation.py @@ -206,20 +206,27 @@ def direct_invocation_result( ), ) - input_dict = _resolve_inputs(op_def, input_args, input_kwargs, bound_context) - - result = invoke_compute_fn( - fn=compute_fn.decorated_fn, - context=bound_context, - kwargs=input_dict, - context_arg_provided=compute_fn.has_context_arg(), - config_arg_cls=( - compute_fn.get_config_arg().annotation if compute_fn.has_config_arg() else None - ), - resource_args=resource_arg_mapping, - ) - - return _type_check_output_wrapper(op_def, result, bound_context) + try: + # if the compute function fails, we want to ensure we unbind the context. This + # try-except handles "vanilla" asset and op invocation (generators and async handled in + # _type_check_output_wrapper) + + input_dict = _resolve_inputs(op_def, input_args, input_kwargs, bound_context) + + result = invoke_compute_fn( + fn=compute_fn.decorated_fn, + context=bound_context, + kwargs=input_dict, + context_arg_provided=compute_fn.has_context_arg(), + config_arg_cls=( + compute_fn.get_config_arg().annotation if compute_fn.has_config_arg() else None + ), + resource_args=resource_arg_mapping, + ) + return _type_check_output_wrapper(op_def, result, bound_context) + except Exception as e: + bound_context.unbind() + raise e def _resolve_inputs( @@ -401,8 +408,14 @@ def _type_check_output_wrapper( async def to_gen(async_gen): outputs_seen = set() - async for event in async_gen: - yield _handle_gen_event(event, op_def, context, output_defs, outputs_seen) + try: + # if the compute function fails, we want to ensure we unbind the context. For + # async generators, the errors will only be surfaced here + async for event in async_gen: + yield _handle_gen_event(event, op_def, context, output_defs, outputs_seen) + except Exception as e: + context.unbind() + raise e for output_def in op_def.output_defs: if ( @@ -426,7 +439,13 @@ async def to_gen(async_gen): elif inspect.iscoroutine(result): async def type_check_coroutine(coro): - out = await coro + try: + # if the compute function fails, we want to ensure we unbind the context. For + # async, the errors will only be surfaced here + out = await coro + except Exception as e: + context.unbind() + raise e return _type_check_function_output(op_def, out, context) return type_check_coroutine(result) @@ -436,8 +455,14 @@ async def type_check_coroutine(coro): def type_check_gen(gen): outputs_seen = set() - for event in gen: - yield _handle_gen_event(event, op_def, context, output_defs, outputs_seen) + try: + # if the compute function fails, we want to ensure we unbind the context. For + # generators, the errors will only be surfaced here + for event in gen: + yield _handle_gen_event(event, op_def, context, output_defs, outputs_seen) + except Exception as e: + context.unbind() + raise e for output_def in op_def.output_defs: if ( diff --git a/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py b/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py index 14b251904ff08..3cb1d991144fb 100644 --- a/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py +++ b/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py @@ -1456,7 +1456,7 @@ async def get_results(): assert ctx._invocation_properties is not None # noqa: SLF001 -def test_bound_state_with_error(): +def test_bound_state_with_error_assets(): @asset def throws_error(context): assert context.alias == "throws_error" @@ -1464,17 +1464,81 @@ def throws_error(context): ctx = build_asset_context() - throws_error(ctx) - with pytest.raises(Failure): throws_error(ctx) - # invocation pathway was interrupted, ctx is still in bound state - assert ctx._bound_properties is not None # noqa: SLF001 + assert ctx._bound_properties is None # noqa: SLF001 @asset def no_error(context): assert context.alias == "no_error" - with pytest.raises(DagsterInvalidInvocationError): - no_error(ctx) + no_error(ctx) + + +def test_context_bound_state_with_error_ops(): + @op(out={"first": Out(), "second": Out()}) + def throws_error(context): + assert context._bound_properties is not None # noqa: SLF001 + raise Failure("something bad happened!") + + ctx = build_op_context() + + with pytest.raises(Failure): + throws_error(ctx) + + assert ctx._bound_properties is None # noqa: SLF001 + + +def test_context_bound_state_with_error_generator(): + @op(out={"first": Out(), "second": Out()}) + def generator(context): + assert context._bound_properties is not None # noqa: SLF001 + yield Output("one", output_name="first") + raise Failure("something bad happened!") + + ctx = build_op_context() + + with pytest.raises(Failure): + list(generator(ctx)) + + assert ctx._bound_properties is None # noqa: SLF001 + assert ctx._invocation_properties is not None # noqa: SLF001 + + +def test_context_bound_state_with_error_async(): + @asset + async def async_asset(context): + assert context._bound_properties is not None # noqa: SLF001 + await asyncio.sleep(0.01) + raise Failure("something bad happened!") + + ctx = build_asset_context() + + with pytest.raises(Failure): + asyncio.run(async_asset(ctx)) + + assert ctx._bound_properties is None # noqa: SLF001 + assert ctx._invocation_properties is not None # noqa: SLF001 + + +def test_context_bound_state_with_error_async_generator(): + @op(out={"first": Out(), "second": Out()}) + async def async_generator(context): + assert context._bound_properties is not None # noqa: SLF001 + yield Output("one", output_name="first") + await asyncio.sleep(0.01) + raise Failure("something bad happened!") + + ctx = build_op_context() + + async def get_results(): + res = [] + async for output in async_generator(ctx): + res.append(output) + return res + + with pytest.raises(Failure): + asyncio.run(get_results()) + + assert ctx._bound_properties is None # noqa: SLF001 From d737f5f37c2f939ab1f1fb2fcb36c3b3661df18b Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Mon, 4 Dec 2023 11:44:18 -0500 Subject: [PATCH 28/44] make pyright happy --- .../_core/execution/context/invocation.py | 61 +++++++++---------- 1 file changed, 30 insertions(+), 31 deletions(-) diff --git a/python_modules/dagster/dagster/_core/execution/context/invocation.py b/python_modules/dagster/dagster/_core/execution/context/invocation.py index 824393d834123..f0640cf47ae73 100644 --- a/python_modules/dagster/dagster/_core/execution/context/invocation.py +++ b/python_modules/dagster/dagster/_core/execution/context/invocation.py @@ -194,9 +194,12 @@ def __exit__(self, *exc): def __del__(self): self._exit_stack.close() - def _check_bound(self, fn_name: str, fn_type: str): + def _check_bound(self, fn_name: str, fn_type: str) -> BoundProperties: if self._bound_properties is None: raise DagsterInvalidPropertyError(_property_msg(fn_name, fn_type)) + # return self._bound_properties so that the calling function can access properties + # of self._bound_properties without causing pyright errors + return self._bound_properties def bind( self, @@ -209,14 +212,6 @@ def bind( from dagster._core.definitions.resource_invocation import resolve_bound_config if self._bound_properties is not None: - # warnings.warn( - # f"This context was already used to execute {self.alias}. The information about" - # f" {self.alias} will be cleared, including user events and output metadata." - # " If you would like to keep this information, you can create a new context" - # " using build_op_context() to invoke other ops. You can also manually clear the" - # f" information about {self.alias} using the unbind() method." - # ) - # self.unbind() raise DagsterInvalidInvocationError( f"This context is currently being used to execute {self.alias}. The context cannot be used to execute another op until {self.alias} has finished executing." ) @@ -355,12 +350,12 @@ def run_id(self) -> str: @property def run_config(self) -> dict: - self._check_bound(fn_name="run_config", fn_type="property") + bound_properties = self._check_bound(fn_name="run_config", fn_type="property") run_config: Dict[str, object] = {} - if self._op_config and self._bound_properties.op_def: + if self._op_config and bound_properties.op_def: run_config["ops"] = { - self._bound_properties.op_def.name: {"config": self._bound_properties.op_config} + bound_properties.op_def.name: {"config": bound_properties.op_config} } run_config["resources"] = self._resources_config return run_config @@ -392,23 +387,23 @@ def solid(self) -> Node: @property def op_def(self) -> OpDefinition: - self._check_bound(fn_name="op_def", fn_type="property") - return cast(OpDefinition, self._bound_properties.op_def) + bound_properties = self._check_bound(fn_name="op_def", fn_type="property") + return cast(OpDefinition, bound_properties.op_def) @property def has_assets_def(self) -> bool: - self._check_bound(fn_name="has_assets_def", fn_type="property") - return self._bound_properties.assets_def is not None + bound_properties = self._check_bound(fn_name="has_assets_def", fn_type="property") + return bound_properties.assets_def is not None @property def assets_def(self) -> AssetsDefinition: - self._check_bound(fn_name="assets_def", fn_type="property") + bound_properties = self._check_bound(fn_name="assets_def", fn_type="property") - if self._bound_properties.assets_def is None: + if bound_properties.assets_def is None: raise DagsterInvalidPropertyError( f"Op {self.op_def.name} does not have an assets definition." ) - return self._bound_properties.assets_def + return bound_properties.assets_def @property def has_partition_key(self) -> bool: @@ -438,17 +433,17 @@ def asset_partition_key_for_output(self, output_name: str = "result") -> str: return self.partition_key def has_tag(self, key: str) -> bool: - self._check_bound(fn_name="has_tag", fn_type="method") - return key in self._bound_properties.tags + bound_properties = self._check_bound(fn_name="has_tag", fn_type="method") + return key in bound_properties.tags def get_tag(self, key: str) -> Optional[str]: - self._check_bound(fn_name="get_tag", fn_type="method") - return self._bound_properties.tags.get(key) + bound_properties = self._check_bound(fn_name="get_tag", fn_type="method") + return bound_properties.tags.get(key) @property def alias(self) -> str: - self._check_bound(fn_name="alias", fn_type="property") - return cast(str, self._bound_properties.alias) + bound_properties = self._check_bound(fn_name="alias", fn_type="property") + return cast(str, bound_properties.alias) def get_step_execution_context(self) -> StepExecutionContext: raise DagsterInvalidPropertyError(_property_msg("get_step_execution_context", "method")) @@ -648,18 +643,22 @@ def add_metadata_two_outputs(context) -> Tuple[str, int]: # allowed. @property def requires_typed_event_stream(self) -> bool: - self._check_bound(fn_name="requires_typed_event_stream", fn_type="property") - return self._bound_properties.requires_typed_event_stream + bound_properties = self._check_bound( + fn_name="requires_typed_event_stream", fn_type="property" + ) + return bound_properties.requires_typed_event_stream @property def typed_event_stream_error_message(self) -> Optional[str]: - self._check_bound(fn_name="typed_event_stream_error_message", fn_type="property") - return self._bound_properties.typed_event_stream_error_message + bound_properties = self._check_bound( + fn_name="typed_event_stream_error_message", fn_type="property" + ) + return bound_properties.typed_event_stream_error_message def set_requires_typed_event_stream(self, *, error_message: Optional[str]) -> None: self._check_bound(fn_name="set_requires_typed_event_stream", fn_type="method") - self._bound_properties.requires_typed_event_stream = True - self._bound_properties.typed_event_stream_error_message = error_message + self._bound_properties.requires_typed_event_stream = True # type: ignore + self._bound_properties.typed_event_stream_error_message = error_message # type: ignore def _validate_resource_requirements( From eff0b6a1efb46ed9c08844a7df88ce79106867ef Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Mon, 4 Dec 2023 16:10:47 -0500 Subject: [PATCH 29/44] re-org to invocation props --- .../_core/definitions/op_invocation.py | 16 +- .../_core/execution/context/invocation.py | 240 ++++++++++-------- 2 files changed, 139 insertions(+), 117 deletions(-) diff --git a/python_modules/dagster/dagster/_core/definitions/op_invocation.py b/python_modules/dagster/dagster/_core/definitions/op_invocation.py index 8cbafad4509ab..485afda80027a 100644 --- a/python_modules/dagster/dagster/_core/definitions/op_invocation.py +++ b/python_modules/dagster/dagster/_core/definitions/op_invocation.py @@ -224,9 +224,9 @@ def direct_invocation_result( resource_args=resource_arg_mapping, ) return _type_check_output_wrapper(op_def, result, bound_context) - except Exception as e: + except Exception: bound_context.unbind() - raise e + raise def _resolve_inputs( @@ -413,9 +413,9 @@ async def to_gen(async_gen): # async generators, the errors will only be surfaced here async for event in async_gen: yield _handle_gen_event(event, op_def, context, output_defs, outputs_seen) - except Exception as e: + except Exception: context.unbind() - raise e + raise for output_def in op_def.output_defs: if ( @@ -443,9 +443,9 @@ async def type_check_coroutine(coro): # if the compute function fails, we want to ensure we unbind the context. For # async, the errors will only be surfaced here out = await coro - except Exception as e: + except Exception: context.unbind() - raise e + raise return _type_check_function_output(op_def, out, context) return type_check_coroutine(result) @@ -460,9 +460,9 @@ def type_check_gen(gen): # generators, the errors will only be surfaced here for event in gen: yield _handle_gen_event(event, op_def, context, output_defs, outputs_seen) - except Exception as e: + except Exception: context.unbind() - raise e + raise for output_def in op_def.output_defs: if ( diff --git a/python_modules/dagster/dagster/_core/execution/context/invocation.py b/python_modules/dagster/dagster/_core/execution/context/invocation.py index f0640cf47ae73..b24d310508db6 100644 --- a/python_modules/dagster/dagster/_core/execution/context/invocation.py +++ b/python_modules/dagster/dagster/_core/execution/context/invocation.py @@ -3,7 +3,6 @@ AbstractSet, Any, Dict, - List, Mapping, NamedTuple, Optional, @@ -66,9 +65,22 @@ def _property_msg(prop_name: str, method_name: str) -> str: ) -class BoundProperties: - def __init__( - self, +class BoundProperties( + NamedTuple( + "_BoundProperties", + [ + ("op_def", OpDefinition), + ("tags", Mapping[Any, Any]), + ("hook_defs", Optional[AbstractSet[HookDefinition]]), + ("alias", str), + ("assets_def", Optional[AssetsDefinition]), + ("resources", Resources), + ("op_config", Any), + ], + ) +): + def __new__( + cls, op_def: OpDefinition, tags: Mapping[Any, Any], hook_defs: Optional[AbstractSet[HookDefinition]], @@ -77,40 +89,116 @@ def __init__( resources: Resources, op_config: Any, ): - """Maintains the properties of the context that are provided at bind time. - This class is not implemented as a NamedTuple because requires_typed_event_stream - and type_event_stream_error_message should be mutable. - """ - self.op_def = check.inst_param(op_def, "op_def", OpDefinition) - self.tags = check.dict_param(tags, "tags") - self.hook_defs = check.opt_set_param(hook_defs, "hook_defs", HookDefinition) - self.alias = check.str_param(alias, "alias") - self.assets_def = check.opt_inst_param(assets_def, "assets_def", AssetsDefinition) - self.resources = check.inst_param(resources, "resources", Resources) - self.op_config = op_config - self.requires_typed_event_stream = False - self.typed_event_stream_error_message = None + """Maintains the properties of the context that are provided at bind time.""" + + def __new__(cls): + return super(BoundProperties, cls).__new__( + cls, + op_def=check.inst_param(op_def, "op_def", OpDefinition), + tags=check.dict_param(tags, "tags"), + hook_defs=check.opt_set_param(hook_defs, "hook_defs", HookDefinition), + alias=check.str_param(alias, "alias"), + assets_def=check.opt_inst_param(assets_def, "assets_def", AssetsDefinition), + resources=check.inst_param(resources, "resources", Resources), + op_config=op_config, + ) -class InvocationProperties( - NamedTuple( - "_InvocationProperties", - [ - ("user_events", List[UserEvent]), - ("seen_outputs", Dict[str, Union[Set[str], str]]), - ("output_metadata", Dict[str, Any]), - ], - ) -): +class InvocationProperties: """Maintains information about the invocation that is updated during execution time. This information needs to be available to the user once invocation is complete, so that they can assert on events and outputs. It needs to be cleared before the context is used for another invocation. """ - def __new__(cls): - return super(InvocationProperties, cls).__new__( - cls, user_events=[], seen_outputs={}, output_metadata={} + def __init__(self): + self.user_events = [] + self.seen_outputs = {} + self.output_metadata = {} + self.requires_typed_event_stream = False + self.typed_event_stream_error_message: Optional[str] = None + + def log_event(self, event: UserEvent) -> None: + check.inst_param( + event, + "event", + (AssetMaterialization, AssetObservation, ExpectationResult), ) + self.user_events.append(event) + + def observe_output(self, output_name: str, mapping_key: Optional[str] = None) -> None: + if mapping_key: + if output_name not in self.seen_outputs: + self.seen_outputs[output_name] = set() + cast(Set[str], self.seen_outputs[output_name]).add(mapping_key) + else: + self.seen_outputs[output_name] = "seen" + + def has_seen_output(self, output_name: str, mapping_key: Optional[str] = None) -> bool: + if mapping_key: + return ( + output_name in self.seen_outputs and mapping_key in self.seen_outputs[output_name] + ) + return output_name in self.seen_outputs + + def add_output_metadata( + self, + metadata: Mapping[str, Any], + op_def: OpDefinition, + output_name: Optional[str] = None, + mapping_key: Optional[str] = None, + ) -> None: + metadata = check.mapping_param(metadata, "metadata", key_type=str) + output_name = check.opt_str_param(output_name, "output_name") + mapping_key = check.opt_str_param(mapping_key, "mapping_key") + + if output_name is None and len(op_def.output_defs) == 1: + output_def = op_def.output_defs[0] + output_name = output_def.name + elif output_name is None: + raise DagsterInvariantViolationError( + "Attempted to log metadata without providing output_name, but multiple outputs" + " exist. Please provide an output_name to the invocation of" + " `context.add_output_metadata`." + ) + else: + output_def = op_def.output_def_named(output_name) + + if self.has_seen_output(output_name, mapping_key): + output_desc = ( + f"output '{output_def.name}'" + if not mapping_key + else f"output '{output_def.name}' with mapping_key '{mapping_key}'" + ) + raise DagsterInvariantViolationError( + f"In {op_def.node_type_str} '{op_def.name}', attempted to log output" + f" metadata for {output_desc} which has already been yielded. Metadata must be" + " logged before the output is yielded." + ) + if output_def.is_dynamic and not mapping_key: + raise DagsterInvariantViolationError( + f"In {op_def.node_type_str} '{op_def.name}', attempted to log metadata" + f" for dynamic output '{output_def.name}' without providing a mapping key. When" + " logging metadata for a dynamic output, it is necessary to provide a mapping key." + ) + + output_name = output_def.name + if output_name in self.output_metadata: + if not mapping_key or mapping_key in self.output_metadata[output_name]: + raise DagsterInvariantViolationError( + f"In {op_def.node_type_str} '{op_def.name}', attempted to log" + f" metadata for output '{output_name}' more than once." + ) + if mapping_key: + if output_name not in self.output_metadata: + self.output_metadata[output_name] = {} + self.output_metadata[output_name][mapping_key] = metadata + + else: + self.output_metadata[output_name] = metadata + + def set_requires_typed_event_stream(self, *, error_message: Optional[str]) -> None: + self.requires_typed_event_stream = True + self.typed_event_stream_error_message = error_message class DirectInvocationOpExecutionContext(OpExecutionContext): @@ -512,29 +600,16 @@ def describe_op(self) -> str: def log_event(self, event: UserEvent) -> None: self._check_bound(fn_name="log_event", fn_type="method") - check.inst_param( - event, - "event", - (AssetMaterialization, AssetObservation, ExpectationResult), - ) - self._invocation_properties.user_events.append(event) + self._invocation_properties.log_event(event) def observe_output(self, output_name: str, mapping_key: Optional[str] = None) -> None: self._check_bound(fn_name="observe_output", fn_type="method") - if mapping_key: - if output_name not in self._invocation_properties.seen_outputs: - self._invocation_properties.seen_outputs[output_name] = set() - cast(Set[str], self._invocation_properties.seen_outputs[output_name]).add(mapping_key) - else: - self._invocation_properties.seen_outputs[output_name] = "seen" + self._invocation_properties.observe_output(output_name=output_name, mapping_key=mapping_key) def has_seen_output(self, output_name: str, mapping_key: Optional[str] = None) -> bool: - if mapping_key: - return ( - output_name in self._invocation_properties.seen_outputs - and mapping_key in self._invocation_properties.seen_outputs[output_name] - ) - return output_name in self._invocation_properties.seen_outputs + return self._invocation_properties.has_seen_output( + output_name=output_name, mapping_key=mapping_key + ) def asset_partitions_time_window_for_output(self, output_name: str = "result") -> TimeWindow: self._check_bound(fn_name="asset_partitions_time_window_for_output", fn_type="method") @@ -587,78 +662,25 @@ def add_metadata_two_outputs(context) -> Tuple[str, int]: """ self._check_bound(fn_name="add_output_metadata", fn_type="method") - metadata = check.mapping_param(metadata, "metadata", key_type=str) - output_name = check.opt_str_param(output_name, "output_name") - mapping_key = check.opt_str_param(mapping_key, "mapping_key") - - if output_name is None and len(self.op_def.output_defs) == 1: - output_def = self.op_def.output_defs[0] - output_name = output_def.name - elif output_name is None: - raise DagsterInvariantViolationError( - "Attempted to log metadata without providing output_name, but multiple outputs" - " exist. Please provide an output_name to the invocation of" - " `context.add_output_metadata`." - ) - else: - output_def = self.op_def.output_def_named(output_name) - - if self.has_seen_output(output_name, mapping_key): - output_desc = ( - f"output '{output_def.name}'" - if not mapping_key - else f"output '{output_def.name}' with mapping_key '{mapping_key}'" - ) - raise DagsterInvariantViolationError( - f"In {self.op_def.node_type_str} '{self.op_def.name}', attempted to log output" - f" metadata for {output_desc} which has already been yielded. Metadata must be" - " logged before the output is yielded." - ) - if output_def.is_dynamic and not mapping_key: - raise DagsterInvariantViolationError( - f"In {self.op_def.node_type_str} '{self.op_def.name}', attempted to log metadata" - f" for dynamic output '{output_def.name}' without providing a mapping key. When" - " logging metadata for a dynamic output, it is necessary to provide a mapping key." - ) - - output_name = output_def.name - if output_name in self._invocation_properties.output_metadata: - if ( - not mapping_key - or mapping_key in self._invocation_properties.output_metadata[output_name] - ): - raise DagsterInvariantViolationError( - f"In {self.op_def.node_type_str} '{self.op_def.name}', attempted to log" - f" metadata for output '{output_name}' more than once." - ) - if mapping_key: - if output_name not in self._invocation_properties.output_metadata: - self._invocation_properties.output_metadata[output_name] = {} - self._invocation_properties.output_metadata[output_name][mapping_key] = metadata - - else: - self._invocation_properties.output_metadata[output_name] = metadata + self._invocation_properties.add_output_metadata( + metadata=metadata, op_def=self.op_def, output_name=output_name, mapping_key=mapping_key + ) # In bound mode no conversion is done on returned values and missing but expected outputs are not # allowed. @property def requires_typed_event_stream(self) -> bool: - bound_properties = self._check_bound( - fn_name="requires_typed_event_stream", fn_type="property" - ) - return bound_properties.requires_typed_event_stream + self._check_bound(fn_name="requires_typed_event_stream", fn_type="property") + return self._invocation_properties.requires_typed_event_stream @property def typed_event_stream_error_message(self) -> Optional[str]: - bound_properties = self._check_bound( - fn_name="typed_event_stream_error_message", fn_type="property" - ) - return bound_properties.typed_event_stream_error_message + self._check_bound(fn_name="typed_event_stream_error_message", fn_type="property") + return self._invocation_properties.typed_event_stream_error_message def set_requires_typed_event_stream(self, *, error_message: Optional[str]) -> None: self._check_bound(fn_name="set_requires_typed_event_stream", fn_type="method") - self._bound_properties.requires_typed_event_stream = True # type: ignore - self._bound_properties.typed_event_stream_error_message = error_message # type: ignore + self._invocation_properties.set_requires_typed_event_stream(error_message=error_message) def _validate_resource_requirements( From b725535dfc12626a4f0347a57a97d86aed525a9c Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Mon, 4 Dec 2023 16:29:23 -0500 Subject: [PATCH 30/44] clean up tests --- .../core_tests/test_op_invocation.py | 99 ++++++++++++++----- 1 file changed, 73 insertions(+), 26 deletions(-) diff --git a/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py b/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py index 3cb1d991144fb..91333baa3a4db 100644 --- a/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py +++ b/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py @@ -44,7 +44,10 @@ DagsterTypeCheckDidNotPass, ) from dagster._core.execution.context.compute import AssetExecutionContext, OpExecutionContext -from dagster._core.execution.context.invocation import build_asset_context +from dagster._core.execution.context.invocation import ( + DirectInvocationOpExecutionContext, + build_asset_context, +) from dagster._utils.test import wrap_op_in_graph_and_execute @@ -1349,7 +1352,7 @@ def my_other_asset(context): my_asset(ctx) assert ctx.get_output_metadata("result") == {"foo": "bar"} - # context in unbound when used in another invocation. This allows the metadata to be + # context is unbound when used in another invocation. This allows the metadata to be # added in my_other_asset my_other_asset(ctx) @@ -1385,23 +1388,51 @@ async def main(): asyncio.run(main()) +def assert_context_unbound(context: DirectInvocationOpExecutionContext): + # to assert that the context is correctly unbound after op invocation + assert context._bound_properties is None # noqa: SLF001 + + +def assert_context_bound(context: DirectInvocationOpExecutionContext): + # to assert that the context is correctly bound during op invocation + assert context._bound_properties is not None # noqa: SLF001 + + +def assert_invocation_properties_cleared(context: DirectInvocationOpExecutionContext): + # to assert that the invocation properties are reset at the beginning of op invocation + assert len(context._invocation_properties.output_metadata.keys()) == 0 # noqa: SLF001 + + +def assert_invocation_properties_exist(context: DirectInvocationOpExecutionContext): + # to assert that the invocation properties remain accessible after op invocation + assert len(context._invocation_properties.output_metadata.keys()) > 0 # noqa: SLF001 + + def test_context_bound_state_non_generator(): @asset def my_asset(context): - assert context._bound_properties is not None # noqa: SLF001 + assert_context_bound(context) + assert_invocation_properties_cleared(context) + context.add_output_metadata({"foo": "bar"}) ctx = build_op_context() - assert ctx._bound_properties is None # noqa: SLF001 + assert_context_unbound(ctx) my_asset(ctx) - assert ctx._bound_properties is None # noqa: SLF001 - assert ctx._invocation_properties is not None # noqa: SLF001 + assert_context_unbound(ctx) + assert_invocation_properties_exist(ctx) + + my_asset(ctx) + assert_context_unbound(ctx) + assert_invocation_properties_exist(ctx) def test_context_bound_state_generator(): @op(out={"first": Out(), "second": Out()}) def generator(context): - assert context._bound_properties is not None # noqa: SLF001 + assert_context_bound(context) + assert_invocation_properties_cleared(context) + context.add_output_metadata({"foo": "bar"}, output_name="one") yield Output("one", output_name="first") yield Output("two", output_name="second") @@ -1410,16 +1441,23 @@ def generator(context): result = list(generator(ctx)) assert result[0].value == "one" assert result[1].value == "two" + assert_context_unbound(ctx) + assert_invocation_properties_exist(ctx) - assert ctx._bound_properties is None # noqa: SLF001 - assert ctx._invocation_properties is not None # noqa: SLF001 + result = list(generator(ctx)) + assert result[0].value == "one" + assert result[1].value == "two" + assert_context_unbound(ctx) + assert_invocation_properties_exist(ctx) def test_context_bound_state_async(): @asset async def async_asset(context): - assert context._bound_properties is not None # noqa: SLF001 + assert_context_bound(context) + assert_invocation_properties_cleared(context) assert context.asset_key.to_user_string() == "async_asset" + context.add_output_metadata({"foo": "bar"}) await asyncio.sleep(0.01) return "one" @@ -1427,15 +1465,21 @@ async def async_asset(context): result = asyncio.run(async_asset(ctx)) assert result == "one" + assert_context_unbound(ctx) + assert_invocation_properties_exist(ctx) - assert ctx._bound_properties is None # noqa: SLF001 - assert ctx._invocation_properties is not None # noqa: SLF001 + result = asyncio.run(async_asset(ctx)) + assert result == "one" + assert_context_unbound(ctx) + assert_invocation_properties_exist(ctx) def test_context_bound_state_async_generator(): @op(out={"first": Out(), "second": Out()}) async def async_generator(context): - assert context._bound_properties is not None # noqa: SLF001 + assert_context_bound(context) + assert_invocation_properties_cleared(context) + context.add_output_metadata({"foo": "bar"}, output_name="one") yield Output("one", output_name="first") await asyncio.sleep(0.01) yield Output("two", output_name="second") @@ -1451,9 +1495,14 @@ async def get_results(): result = asyncio.run(get_results()) assert result[0].value == "one" assert result[1].value == "two" + assert_context_unbound(ctx) + assert_invocation_properties_exist(ctx) - assert ctx._bound_properties is None # noqa: SLF001 - assert ctx._invocation_properties is not None # noqa: SLF001 + result = asyncio.run(get_results()) + assert result[0].value == "one" + assert result[1].value == "two" + assert_context_unbound(ctx) + assert_invocation_properties_exist(ctx) def test_bound_state_with_error_assets(): @@ -1467,7 +1516,7 @@ def throws_error(context): with pytest.raises(Failure): throws_error(ctx) - assert ctx._bound_properties is None # noqa: SLF001 + assert_context_unbound(ctx) @asset def no_error(context): @@ -1479,7 +1528,7 @@ def no_error(context): def test_context_bound_state_with_error_ops(): @op(out={"first": Out(), "second": Out()}) def throws_error(context): - assert context._bound_properties is not None # noqa: SLF001 + assert_context_bound(ctx) raise Failure("something bad happened!") ctx = build_op_context() @@ -1487,13 +1536,13 @@ def throws_error(context): with pytest.raises(Failure): throws_error(ctx) - assert ctx._bound_properties is None # noqa: SLF001 + assert_context_unbound(ctx) def test_context_bound_state_with_error_generator(): @op(out={"first": Out(), "second": Out()}) def generator(context): - assert context._bound_properties is not None # noqa: SLF001 + assert_context_bound(ctx) yield Output("one", output_name="first") raise Failure("something bad happened!") @@ -1502,14 +1551,13 @@ def generator(context): with pytest.raises(Failure): list(generator(ctx)) - assert ctx._bound_properties is None # noqa: SLF001 - assert ctx._invocation_properties is not None # noqa: SLF001 + assert_context_unbound(ctx) def test_context_bound_state_with_error_async(): @asset async def async_asset(context): - assert context._bound_properties is not None # noqa: SLF001 + assert_context_bound(ctx) await asyncio.sleep(0.01) raise Failure("something bad happened!") @@ -1518,14 +1566,13 @@ async def async_asset(context): with pytest.raises(Failure): asyncio.run(async_asset(ctx)) - assert ctx._bound_properties is None # noqa: SLF001 - assert ctx._invocation_properties is not None # noqa: SLF001 + assert_context_unbound(ctx) def test_context_bound_state_with_error_async_generator(): @op(out={"first": Out(), "second": Out()}) async def async_generator(context): - assert context._bound_properties is not None # noqa: SLF001 + assert_context_bound(ctx) yield Output("one", output_name="first") await asyncio.sleep(0.01) raise Failure("something bad happened!") @@ -1541,4 +1588,4 @@ async def get_results(): with pytest.raises(Failure): asyncio.run(get_results()) - assert ctx._bound_properties is None # noqa: SLF001 + assert_context_unbound(ctx) From 7edf1b215616c8732426fb4ee293e536d791fc9c Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Tue, 5 Dec 2023 17:35:26 -0500 Subject: [PATCH 31/44] rename DirectInvocationOpExecutionContext to RunlessOpExecutionContext --- .../_core/definitions/op_invocation.py | 32 +++++++++---------- .../_core/execution/context/invocation.py | 8 ++--- .../dagster/dagster/_core/pipes/context.py | 10 ++---- .../core_tests/test_op_invocation.py | 10 +++--- 4 files changed, 27 insertions(+), 33 deletions(-) diff --git a/python_modules/dagster/dagster/_core/definitions/op_invocation.py b/python_modules/dagster/dagster/_core/definitions/op_invocation.py index 485afda80027a..80763e4240c9c 100644 --- a/python_modules/dagster/dagster/_core/definitions/op_invocation.py +++ b/python_modules/dagster/dagster/_core/definitions/op_invocation.py @@ -32,7 +32,7 @@ from .result import MaterializeResult if TYPE_CHECKING: - from ..execution.context.invocation import DirectInvocationOpExecutionContext + from ..execution.context.invocation import RunlessOpExecutionContext from .assets import AssetsDefinition from .composition import PendingNodeInvocation from .decorators.op_decorator import DecoratedOpFunction @@ -109,7 +109,7 @@ def direct_invocation_result( ) -> Any: from dagster._config.pythonic_config import Config from dagster._core.execution.context.invocation import ( - DirectInvocationOpExecutionContext, + RunlessOpExecutionContext, build_op_context, ) @@ -149,12 +149,12 @@ def direct_invocation_result( " no context was provided when invoking." ) if len(args) > 0: - if args[0] is not None and not isinstance(args[0], DirectInvocationOpExecutionContext): + if args[0] is not None and not isinstance(args[0], RunlessOpExecutionContext): raise DagsterInvalidInvocationError( f"Decorated function '{compute_fn.name}' has context argument, " "but no context was provided when invoking." ) - context = cast(DirectInvocationOpExecutionContext, args[0]) + context = cast(RunlessOpExecutionContext, args[0]) # update args to omit context args = args[1:] else: # context argument is provided under kwargs @@ -165,14 +165,14 @@ def direct_invocation_result( f"'{context_param_name}', but no value for '{context_param_name}' was " f"found when invoking. Provided kwargs: {kwargs}" ) - context = cast(DirectInvocationOpExecutionContext, kwargs[context_param_name]) + context = cast(RunlessOpExecutionContext, kwargs[context_param_name]) # update kwargs to remove context kwargs = { kwarg: val for kwarg, val in kwargs.items() if not kwarg == context_param_name } # allow passing context, even if the function doesn't have an arg for it - elif len(args) > 0 and isinstance(args[0], DirectInvocationOpExecutionContext): - context = cast(DirectInvocationOpExecutionContext, args[0]) + elif len(args) > 0 and isinstance(args[0], RunlessOpExecutionContext): + context = cast(RunlessOpExecutionContext, args[0]) args = args[1:] resource_arg_mapping = {arg.name: arg.name for arg in compute_fn.get_resource_args()} @@ -230,7 +230,7 @@ def direct_invocation_result( def _resolve_inputs( - op_def: "OpDefinition", args, kwargs, context: "DirectInvocationOpExecutionContext" + op_def: "OpDefinition", args, kwargs, context: "RunlessOpExecutionContext" ) -> Mapping[str, Any]: from dagster._core.execution.plan.execute_step import do_type_check @@ -333,9 +333,7 @@ def _resolve_inputs( return input_dict -def _key_for_result( - result: MaterializeResult, context: "DirectInvocationOpExecutionContext" -) -> AssetKey: +def _key_for_result(result: MaterializeResult, context: "RunlessOpExecutionContext") -> AssetKey: if result.asset_key: return result.asset_key @@ -350,7 +348,7 @@ def _key_for_result( def _output_name_for_result_obj( event: MaterializeResult, - context: "DirectInvocationOpExecutionContext", + context: "RunlessOpExecutionContext", ): asset_key = _key_for_result(event, context) return context.assets_def.get_output_name_for_asset_key(asset_key) @@ -359,7 +357,7 @@ def _output_name_for_result_obj( def _handle_gen_event( event: T, op_def: "OpDefinition", - context: "DirectInvocationOpExecutionContext", + context: "RunlessOpExecutionContext", output_defs: Mapping[str, OutputDefinition], outputs_seen: Set[str], ) -> T: @@ -393,7 +391,7 @@ def _handle_gen_event( def _type_check_output_wrapper( - op_def: "OpDefinition", result: Any, context: "DirectInvocationOpExecutionContext" + op_def: "OpDefinition", result: Any, context: "RunlessOpExecutionContext" ) -> Any: """Type checks and returns the result of a op. @@ -487,7 +485,7 @@ def type_check_gen(gen): def _type_check_function_output( - op_def: "OpDefinition", result: T, context: "DirectInvocationOpExecutionContext" + op_def: "OpDefinition", result: T, context: "RunlessOpExecutionContext" ) -> T: from ..execution.plan.compute_generator import validate_and_coerce_op_result_to_iterator @@ -506,14 +504,14 @@ def _type_check_function_output( def _type_check_output( output_def: "OutputDefinition", output: Union[Output, DynamicOutput], - context: "DirectInvocationOpExecutionContext", + context: "RunlessOpExecutionContext", ) -> None: """Validates and performs core type check on a provided output. Args: output_def (OutputDefinition): The output definition to validate against. output (Any): The output to validate. - context (DirectInvocationOpExecutionContext): Context containing resources to be used for type + context (RunlessOpExecutionContext): Context containing resources to be used for type check. """ from ..execution.plan.execute_step import do_type_check diff --git a/python_modules/dagster/dagster/_core/execution/context/invocation.py b/python_modules/dagster/dagster/_core/execution/context/invocation.py index b24d310508db6..569f79b524f4d 100644 --- a/python_modules/dagster/dagster/_core/execution/context/invocation.py +++ b/python_modules/dagster/dagster/_core/execution/context/invocation.py @@ -201,7 +201,7 @@ def set_requires_typed_event_stream(self, *, error_message: Optional[str]) -> No self.typed_event_stream_error_message = error_message -class DirectInvocationOpExecutionContext(OpExecutionContext): +class RunlessOpExecutionContext(OpExecutionContext): """The ``context`` object available as the first argument to an op's compute function when being invoked directly. Can also be used as a context manager. """ @@ -296,7 +296,7 @@ def bind( assets_def: Optional[AssetsDefinition], config_from_args: Optional[Mapping[str, Any]], resources_from_args: Optional[Mapping[str, Any]], - ) -> "DirectInvocationOpExecutionContext": + ) -> "RunlessOpExecutionContext": from dagster._core.definitions.resource_invocation import resolve_bound_config if self._bound_properties is not None: @@ -703,7 +703,7 @@ def build_op_context( partition_key_range: Optional[PartitionKeyRange] = None, mapping_key: Optional[str] = None, _assets_def: Optional[AssetsDefinition] = None, -) -> DirectInvocationOpExecutionContext: +) -> RunlessOpExecutionContext: """Builds op execution context from provided parameters. ``build_op_context`` can be used as either a function or context manager. If there is a @@ -751,7 +751,7 @@ def build_op_context( ) op_config = op_config if op_config else config - return DirectInvocationOpExecutionContext( + return RunlessOpExecutionContext( resources_dict=check.opt_mapping_param(resources, "resources", key_type=str), resources_config=check.opt_mapping_param( resources_config, "resources_config", key_type=str diff --git a/python_modules/dagster/dagster/_core/pipes/context.py b/python_modules/dagster/dagster/_core/pipes/context.py index 60dd3678f3eb0..f141048b8c76f 100644 --- a/python_modules/dagster/dagster/_core/pipes/context.py +++ b/python_modules/dagster/dagster/_core/pipes/context.py @@ -39,7 +39,7 @@ from dagster._core.errors import DagsterPipesExecutionError from dagster._core.events import EngineEventData from dagster._core.execution.context.compute import OpExecutionContext -from dagster._core.execution.context.invocation import DirectInvocationOpExecutionContext +from dagster._core.execution.context.invocation import RunlessOpExecutionContext from dagster._utils.error import ( ExceptionInfo, SerializableErrorInfo, @@ -406,12 +406,8 @@ def build_external_execution_context_data( _convert_time_window(partition_time_window) if partition_time_window else None ), run_id=context.run_id, - job_name=None - if isinstance(context, DirectInvocationOpExecutionContext) - else context.job_name, - retry_number=0 - if isinstance(context, DirectInvocationOpExecutionContext) - else context.retry_number, + job_name=None if isinstance(context, RunlessOpExecutionContext) else context.job_name, + retry_number=0 if isinstance(context, RunlessOpExecutionContext) else context.retry_number, extras=extras or {}, ) diff --git a/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py b/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py index 91333baa3a4db..e5bea9c5ffca4 100644 --- a/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py +++ b/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py @@ -45,7 +45,7 @@ ) from dagster._core.execution.context.compute import AssetExecutionContext, OpExecutionContext from dagster._core.execution.context.invocation import ( - DirectInvocationOpExecutionContext, + RunlessOpExecutionContext, build_asset_context, ) from dagster._utils.test import wrap_op_in_graph_and_execute @@ -1388,22 +1388,22 @@ async def main(): asyncio.run(main()) -def assert_context_unbound(context: DirectInvocationOpExecutionContext): +def assert_context_unbound(context: RunlessOpExecutionContext): # to assert that the context is correctly unbound after op invocation assert context._bound_properties is None # noqa: SLF001 -def assert_context_bound(context: DirectInvocationOpExecutionContext): +def assert_context_bound(context: RunlessOpExecutionContext): # to assert that the context is correctly bound during op invocation assert context._bound_properties is not None # noqa: SLF001 -def assert_invocation_properties_cleared(context: DirectInvocationOpExecutionContext): +def assert_invocation_properties_cleared(context: RunlessOpExecutionContext): # to assert that the invocation properties are reset at the beginning of op invocation assert len(context._invocation_properties.output_metadata.keys()) == 0 # noqa: SLF001 -def assert_invocation_properties_exist(context: DirectInvocationOpExecutionContext): +def assert_invocation_properties_exist(context: RunlessOpExecutionContext): # to assert that the invocation properties remain accessible after op invocation assert len(context._invocation_properties.output_metadata.keys()) > 0 # noqa: SLF001 From 1548074c7472542babffb196ed9bee0a8027b864 Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Tue, 5 Dec 2023 19:00:15 -0500 Subject: [PATCH 32/44] rename to runlessexecutionproperties --- .../_core/execution/context/invocation.py | 34 +++++++++---------- .../core_tests/test_op_invocation.py | 32 ++++++++--------- 2 files changed, 33 insertions(+), 33 deletions(-) diff --git a/python_modules/dagster/dagster/_core/execution/context/invocation.py b/python_modules/dagster/dagster/_core/execution/context/invocation.py index 569f79b524f4d..bf19309828344 100644 --- a/python_modules/dagster/dagster/_core/execution/context/invocation.py +++ b/python_modules/dagster/dagster/_core/execution/context/invocation.py @@ -104,7 +104,7 @@ def __new__(cls): ) -class InvocationProperties: +class RunlessExecutionProperties: """Maintains information about the invocation that is updated during execution time. This information needs to be available to the user once invocation is complete, so that they can assert on events and outputs. It needs to be cleared before the context is used for another invocation. @@ -264,13 +264,13 @@ def __init__( # Maintains the properties on the context that are modified during invocation # @op # def my_op(context): - # # context._invocation_properties can be modified with output metadata etc. + # # context._execution_properties can be modified with output metadata etc. # ... - # ctx = build_op_context() # ctx._invocation_properties is empty + # ctx = build_op_context() # ctx._execution_properties is empty # my_op(ctx) - # ctx._invocation_properties.output_metadata # information is retained after invocation - # my_op(ctx) # ctx._invocation_properties is cleared at the beginning of the next invocation - self._invocation_properties = InvocationProperties() + # ctx._execution_properties.output_metadata # information is retained after invocation + # my_op(ctx) # ctx._execution_properties is cleared at the beginning of the next invocation + self._execution_properties = RunlessExecutionProperties() def __enter__(self): self._cm_scope_entered = True @@ -304,8 +304,8 @@ def bind( f"This context is currently being used to execute {self.alias}. The context cannot be used to execute another op until {self.alias} has finished executing." ) - # reset invocation_properties - self._invocation_properties = InvocationProperties() + # reset execution_properties + self._execution_properties = RunlessExecutionProperties() # update the bound context with properties relevant to the execution of the op @@ -557,7 +557,7 @@ def test_my_op(): expectation_results = [event for event in all_user_events if isinstance(event, ExpectationResult)] ... """ - return self._invocation_properties.user_events + return self._execution_properties.user_events def get_output_metadata( self, output_name: str, mapping_key: Optional[str] = None @@ -573,7 +573,7 @@ def get_output_metadata( Returns: Optional[Mapping[str, Any]]: The metadata values present for the output_name/mapping_key combination, if present. """ - metadata = self._invocation_properties.output_metadata.get(output_name) + metadata = self._execution_properties.output_metadata.get(output_name) if mapping_key and metadata: return metadata.get(mapping_key) return metadata @@ -600,14 +600,14 @@ def describe_op(self) -> str: def log_event(self, event: UserEvent) -> None: self._check_bound(fn_name="log_event", fn_type="method") - self._invocation_properties.log_event(event) + self._execution_properties.log_event(event) def observe_output(self, output_name: str, mapping_key: Optional[str] = None) -> None: self._check_bound(fn_name="observe_output", fn_type="method") - self._invocation_properties.observe_output(output_name=output_name, mapping_key=mapping_key) + self._execution_properties.observe_output(output_name=output_name, mapping_key=mapping_key) def has_seen_output(self, output_name: str, mapping_key: Optional[str] = None) -> bool: - return self._invocation_properties.has_seen_output( + return self._execution_properties.has_seen_output( output_name=output_name, mapping_key=mapping_key ) @@ -662,7 +662,7 @@ def add_metadata_two_outputs(context) -> Tuple[str, int]: """ self._check_bound(fn_name="add_output_metadata", fn_type="method") - self._invocation_properties.add_output_metadata( + self._execution_properties.add_output_metadata( metadata=metadata, op_def=self.op_def, output_name=output_name, mapping_key=mapping_key ) @@ -671,16 +671,16 @@ def add_metadata_two_outputs(context) -> Tuple[str, int]: @property def requires_typed_event_stream(self) -> bool: self._check_bound(fn_name="requires_typed_event_stream", fn_type="property") - return self._invocation_properties.requires_typed_event_stream + return self._execution_properties.requires_typed_event_stream @property def typed_event_stream_error_message(self) -> Optional[str]: self._check_bound(fn_name="typed_event_stream_error_message", fn_type="property") - return self._invocation_properties.typed_event_stream_error_message + return self._execution_properties.typed_event_stream_error_message def set_requires_typed_event_stream(self, *, error_message: Optional[str]) -> None: self._check_bound(fn_name="set_requires_typed_event_stream", fn_type="method") - self._invocation_properties.set_requires_typed_event_stream(error_message=error_message) + self._execution_properties.set_requires_typed_event_stream(error_message=error_message) def _validate_resource_requirements( diff --git a/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py b/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py index e5bea9c5ffca4..74bb1673f60eb 100644 --- a/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py +++ b/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py @@ -1398,21 +1398,21 @@ def assert_context_bound(context: RunlessOpExecutionContext): assert context._bound_properties is not None # noqa: SLF001 -def assert_invocation_properties_cleared(context: RunlessOpExecutionContext): +def assert_execution_properties_cleared(context: RunlessOpExecutionContext): # to assert that the invocation properties are reset at the beginning of op invocation - assert len(context._invocation_properties.output_metadata.keys()) == 0 # noqa: SLF001 + assert len(context._execution_properties.output_metadata.keys()) == 0 # noqa: SLF001 -def assert_invocation_properties_exist(context: RunlessOpExecutionContext): +def assert_execution_properties_exist(context: RunlessOpExecutionContext): # to assert that the invocation properties remain accessible after op invocation - assert len(context._invocation_properties.output_metadata.keys()) > 0 # noqa: SLF001 + assert len(context._execution_properties.output_metadata.keys()) > 0 # noqa: SLF001 def test_context_bound_state_non_generator(): @asset def my_asset(context): assert_context_bound(context) - assert_invocation_properties_cleared(context) + assert_execution_properties_cleared(context) context.add_output_metadata({"foo": "bar"}) ctx = build_op_context() @@ -1420,18 +1420,18 @@ def my_asset(context): my_asset(ctx) assert_context_unbound(ctx) - assert_invocation_properties_exist(ctx) + assert_execution_properties_exist(ctx) my_asset(ctx) assert_context_unbound(ctx) - assert_invocation_properties_exist(ctx) + assert_execution_properties_exist(ctx) def test_context_bound_state_generator(): @op(out={"first": Out(), "second": Out()}) def generator(context): assert_context_bound(context) - assert_invocation_properties_cleared(context) + assert_execution_properties_cleared(context) context.add_output_metadata({"foo": "bar"}, output_name="one") yield Output("one", output_name="first") yield Output("two", output_name="second") @@ -1442,20 +1442,20 @@ def generator(context): assert result[0].value == "one" assert result[1].value == "two" assert_context_unbound(ctx) - assert_invocation_properties_exist(ctx) + assert_execution_properties_exist(ctx) result = list(generator(ctx)) assert result[0].value == "one" assert result[1].value == "two" assert_context_unbound(ctx) - assert_invocation_properties_exist(ctx) + assert_execution_properties_exist(ctx) def test_context_bound_state_async(): @asset async def async_asset(context): assert_context_bound(context) - assert_invocation_properties_cleared(context) + assert_execution_properties_cleared(context) assert context.asset_key.to_user_string() == "async_asset" context.add_output_metadata({"foo": "bar"}) await asyncio.sleep(0.01) @@ -1466,19 +1466,19 @@ async def async_asset(context): result = asyncio.run(async_asset(ctx)) assert result == "one" assert_context_unbound(ctx) - assert_invocation_properties_exist(ctx) + assert_execution_properties_exist(ctx) result = asyncio.run(async_asset(ctx)) assert result == "one" assert_context_unbound(ctx) - assert_invocation_properties_exist(ctx) + assert_execution_properties_exist(ctx) def test_context_bound_state_async_generator(): @op(out={"first": Out(), "second": Out()}) async def async_generator(context): assert_context_bound(context) - assert_invocation_properties_cleared(context) + assert_execution_properties_cleared(context) context.add_output_metadata({"foo": "bar"}, output_name="one") yield Output("one", output_name="first") await asyncio.sleep(0.01) @@ -1496,13 +1496,13 @@ async def get_results(): assert result[0].value == "one" assert result[1].value == "two" assert_context_unbound(ctx) - assert_invocation_properties_exist(ctx) + assert_execution_properties_exist(ctx) result = asyncio.run(get_results()) assert result[0].value == "one" assert result[1].value == "two" assert_context_unbound(ctx) - assert_invocation_properties_exist(ctx) + assert_execution_properties_exist(ctx) def test_bound_state_with_error_assets(): From fc1c2820c5f7e6f9f2fcfc3e065a3a87d58812b7 Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Tue, 5 Dec 2023 19:32:49 -0500 Subject: [PATCH 33/44] use bound properties in invocation --- .../_core/definitions/op_invocation.py | 30 ++++++++++++------- .../_core/execution/context/invocation.py | 24 +++++++++++---- 2 files changed, 38 insertions(+), 16 deletions(-) diff --git a/python_modules/dagster/dagster/_core/definitions/op_invocation.py b/python_modules/dagster/dagster/_core/definitions/op_invocation.py index 80763e4240c9c..115ccfcf55ae0 100644 --- a/python_modules/dagster/dagster/_core/definitions/op_invocation.py +++ b/python_modules/dagster/dagster/_core/definitions/op_invocation.py @@ -270,7 +270,7 @@ def _resolve_inputs( node_label = op_def.node_type_str raise DagsterInvalidInvocationError( - f"Too many input arguments were provided for {node_label} '{context.alias}'." + f"Too many input arguments were provided for {node_label} '{context.bound_properties.alias}'." f" {suggestion}" ) @@ -313,7 +313,7 @@ def _resolve_inputs( input_dict[k] = v # Type check inputs - op_label = context.describe_op() + op_label = context.bound_properties.step_description for input_name, val in input_dict.items(): input_def = input_defs_by_name[input_name] @@ -334,15 +334,19 @@ def _resolve_inputs( def _key_for_result(result: MaterializeResult, context: "RunlessOpExecutionContext") -> AssetKey: + if not context.bound_properties.assets_def: + raise DagsterInvariantViolationError( + f"Op {context.bound_properties.alias} does not have an assets definition." + ) if result.asset_key: return result.asset_key - if len(context.assets_def.keys) == 1: - return next(iter(context.assets_def.keys)) + if context.bound_properties.assets_def and len(context.bound_properties.assets_def.keys) == 1: + return next(iter(context.bound_properties.assets_def.keys)) raise DagsterInvariantViolationError( "MaterializeResult did not include asset_key and it can not be inferred. Specify which" - f" asset_key, options are: {context.assets_def.keys}" + f" asset_key, options are: {context.bound_properties.assets_def.keys}" ) @@ -350,8 +354,12 @@ def _output_name_for_result_obj( event: MaterializeResult, context: "RunlessOpExecutionContext", ): + if not context.bound_properties.assets_def: + raise DagsterInvariantViolationError( + f"Op {context.bound_properties.alias} does not have an assets definition." + ) asset_key = _key_for_result(event, context) - return context.assets_def.get_output_name_for_asset_key(asset_key) + return context.bound_properties.assets_def.get_output_name_for_asset_key(asset_key) def _handle_gen_event( @@ -383,7 +391,7 @@ def _handle_gen_event( output_def, DynamicOutputDefinition ): raise DagsterInvariantViolationError( - f"Invocation of {op_def.node_type_str} '{context.alias}' yielded" + f"Invocation of {op_def.node_type_str} '{context.bound_properties.alias}' yielded" f" an output '{output_def.name}' multiple times." ) outputs_seen.add(output_def.name) @@ -426,7 +434,7 @@ async def to_gen(async_gen): yield Output(output_name=output_def.name, value=None) else: raise DagsterInvariantViolationError( - f"Invocation of {op_def.node_type_str} '{context.alias}' did not" + f"Invocation of {op_def.node_type_str} '{context.bound_properties.alias}' did not" f" return an output for non-optional output '{output_def.name}'" ) context.unbind() @@ -473,7 +481,7 @@ def type_check_gen(gen): yield Output(output_name=output_def.name, value=None) else: raise DagsterInvariantViolationError( - f'Invocation of {op_def.node_type_str} "{context.alias}" did not' + f'Invocation of {op_def.node_type_str} "{context.bound_properties.alias}" did not' f' return an output for non-optional output "{output_def.name}"' ) context.unbind() @@ -516,7 +524,7 @@ def _type_check_output( """ from ..execution.plan.execute_step import do_type_check - op_label = context.describe_op() + op_label = context.bound_properties.step_description dagster_type = output_def.dagster_type type_check = do_type_check(context.for_type(dagster_type), dagster_type, output.value) if not type_check.success: @@ -530,6 +538,6 @@ def _type_check_output( dagster_type=dagster_type, ) - context.observe_output( + context.execution_properties.observe_output( output_def.name, output.mapping_key if isinstance(output, DynamicOutput) else None ) diff --git a/python_modules/dagster/dagster/_core/execution/context/invocation.py b/python_modules/dagster/dagster/_core/execution/context/invocation.py index bf19309828344..09b5dd6b259f0 100644 --- a/python_modules/dagster/dagster/_core/execution/context/invocation.py +++ b/python_modules/dagster/dagster/_core/execution/context/invocation.py @@ -76,6 +76,7 @@ class BoundProperties( ("assets_def", Optional[AssetsDefinition]), ("resources", Resources), ("op_config", Any), + ("step_description", str), ], ) ): @@ -88,6 +89,7 @@ def __new__( assets_def: Optional[AssetsDefinition], resources: Resources, op_config: Any, + step_description: str, ): """Maintains the properties of the context that are provided at bind time.""" @@ -101,6 +103,7 @@ def __new__(cls): assets_def=check.opt_inst_param(assets_def, "assets_def", AssetsDefinition), resources=check.inst_param(resources, "resources", Resources), op_config=op_config, + step_description=step_description, ) @@ -365,6 +368,11 @@ def bind( raise DagsterInvalidInvocationError("Cannot provide config in both context and kwargs") op_config = resolve_bound_config(config_from_args or self._op_config, op_def) + if isinstance(op_def, OpDefinition): + step_description = f'op "{op_def.name}"' + else: + step_description = f'solid "{op_def.name}"' + self._bound_properties = BoundProperties( op_def=op_def, tags=tags, @@ -373,6 +381,7 @@ def bind( assets_def=assets_def, resources=resources, op_config=op_config, + step_description=step_description, ) return self @@ -380,6 +389,14 @@ def bind( def unbind(self): self._bound_properties = None + @property + def execution_properties(self) -> RunlessExecutionProperties: + return self._execution_properties + + @property + def bound_properties(self) -> BoundProperties: + return self._check_bound(fn_name="bound_properties", fn_type="property") + @property def op_config(self) -> Any: if self._bound_properties is None: @@ -592,11 +609,8 @@ def for_type(self, dagster_type: DagsterType) -> TypeCheckContext: ) def describe_op(self) -> str: - self._check_bound(fn_name="describe_op", fn_type="method") - if isinstance(self.op_def, OpDefinition): - return f'op "{self.op_def.name}"' - - return f'solid "{self.op_def.name}"' + bound_properties = self._check_bound(fn_name="describe_op", fn_type="method") + return bound_properties.step_description def log_event(self, event: UserEvent) -> None: self._check_bound(fn_name="log_event", fn_type="method") From 051d74295cfe95bb0a6126426b668f963427ab67 Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Wed, 6 Dec 2023 12:22:18 -0500 Subject: [PATCH 34/44] make things properties --- .../_core/execution/context/invocation.py | 49 +++++++++++++------ 1 file changed, 35 insertions(+), 14 deletions(-) diff --git a/python_modules/dagster/dagster/_core/execution/context/invocation.py b/python_modules/dagster/dagster/_core/execution/context/invocation.py index 09b5dd6b259f0..09bb2c8b7925d 100644 --- a/python_modules/dagster/dagster/_core/execution/context/invocation.py +++ b/python_modules/dagster/dagster/_core/execution/context/invocation.py @@ -3,6 +3,7 @@ AbstractSet, Any, Dict, + List, Mapping, NamedTuple, Optional, @@ -114,11 +115,31 @@ class RunlessExecutionProperties: """ def __init__(self): - self.user_events = [] - self.seen_outputs = {} - self.output_metadata = {} - self.requires_typed_event_stream = False - self.typed_event_stream_error_message: Optional[str] = None + self._events: List[UserEvent] = [] + self._seen_outputs = {} + self._output_metadata = {} + self._requires_typed_event_stream = False + self._typed_event_stream_error_message = None + + @property + def user_events(self): + return self._events + + @property + def seen_outputs(self): + return self._seen_outputs + + @property + def output_metadata(self): + return self._output_metadata + + @property + def requires_typed_event_stream(self) -> bool: + return self._requires_typed_event_stream + + @property + def typed_event_stream_error_message(self) -> Optional[str]: + return self._typed_event_stream_error_message def log_event(self, event: UserEvent) -> None: check.inst_param( @@ -126,15 +147,15 @@ def log_event(self, event: UserEvent) -> None: "event", (AssetMaterialization, AssetObservation, ExpectationResult), ) - self.user_events.append(event) + self._events.append(event) def observe_output(self, output_name: str, mapping_key: Optional[str] = None) -> None: if mapping_key: if output_name not in self.seen_outputs: - self.seen_outputs[output_name] = set() - cast(Set[str], self.seen_outputs[output_name]).add(mapping_key) + self._seen_outputs[output_name] = set() + cast(Set[str], self._seen_outputs[output_name]).add(mapping_key) else: - self.seen_outputs[output_name] = "seen" + self._seen_outputs[output_name] = "seen" def has_seen_output(self, output_name: str, mapping_key: Optional[str] = None) -> bool: if mapping_key: @@ -193,15 +214,15 @@ def add_output_metadata( ) if mapping_key: if output_name not in self.output_metadata: - self.output_metadata[output_name] = {} - self.output_metadata[output_name][mapping_key] = metadata + self._output_metadata[output_name] = {} + self._output_metadata[output_name][mapping_key] = metadata else: - self.output_metadata[output_name] = metadata + self._output_metadata[output_name] = metadata def set_requires_typed_event_stream(self, *, error_message: Optional[str]) -> None: - self.requires_typed_event_stream = True - self.typed_event_stream_error_message = error_message + self._requires_typed_event_stream = True + self._typed_event_stream_error_message = error_message class RunlessOpExecutionContext(OpExecutionContext): From 101131139529b5a63521c18f229a045eb74d652b Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Wed, 6 Dec 2023 15:23:27 -0500 Subject: [PATCH 35/44] fix new fn callsite --- .../_core/execution/context/invocation.py | 24 +++++++++---------- 1 file changed, 11 insertions(+), 13 deletions(-) diff --git a/python_modules/dagster/dagster/_core/execution/context/invocation.py b/python_modules/dagster/dagster/_core/execution/context/invocation.py index 09bb2c8b7925d..49adc714175ff 100644 --- a/python_modules/dagster/dagster/_core/execution/context/invocation.py +++ b/python_modules/dagster/dagster/_core/execution/context/invocation.py @@ -93,19 +93,17 @@ def __new__( step_description: str, ): """Maintains the properties of the context that are provided at bind time.""" - - def __new__(cls): - return super(BoundProperties, cls).__new__( - cls, - op_def=check.inst_param(op_def, "op_def", OpDefinition), - tags=check.dict_param(tags, "tags"), - hook_defs=check.opt_set_param(hook_defs, "hook_defs", HookDefinition), - alias=check.str_param(alias, "alias"), - assets_def=check.opt_inst_param(assets_def, "assets_def", AssetsDefinition), - resources=check.inst_param(resources, "resources", Resources), - op_config=op_config, - step_description=step_description, - ) + return super(BoundProperties, cls).__new__( + cls, + op_def=check.inst_param(op_def, "op_def", OpDefinition), + tags=check.dict_param(tags, "tags"), + hook_defs=check.opt_set_param(hook_defs, "hook_defs", HookDefinition), + alias=check.str_param(alias, "alias"), + assets_def=check.opt_inst_param(assets_def, "assets_def", AssetsDefinition), + resources=check.inst_param(resources, "resources", Resources), + op_config=op_config, + step_description=step_description, + ) class RunlessExecutionProperties: From 9fa1d0e432ef32ca74c4dfb4baf19acb312a61f5 Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Wed, 6 Dec 2023 15:59:48 -0500 Subject: [PATCH 36/44] fix test --- .../dagster/dagster_tests/core_tests/test_op_invocation.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py b/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py index 74bb1673f60eb..65b78244935da 100644 --- a/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py +++ b/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py @@ -1432,7 +1432,7 @@ def test_context_bound_state_generator(): def generator(context): assert_context_bound(context) assert_execution_properties_cleared(context) - context.add_output_metadata({"foo": "bar"}, output_name="one") + context.add_output_metadata({"foo": "bar"}, output_name="first") yield Output("one", output_name="first") yield Output("two", output_name="second") @@ -1479,7 +1479,7 @@ def test_context_bound_state_async_generator(): async def async_generator(context): assert_context_bound(context) assert_execution_properties_cleared(context) - context.add_output_metadata({"foo": "bar"}, output_name="one") + context.add_output_metadata({"foo": "bar"}, output_name="first") yield Output("one", output_name="first") await asyncio.sleep(0.01) yield Output("two", output_name="second") From 129bfeec46bd199d7b4ebb904b1d979e83a68aef Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Fri, 8 Dec 2023 11:49:29 -0500 Subject: [PATCH 37/44] access via property --- .../dagster_tests/core_tests/test_op_invocation.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py b/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py index 65b78244935da..257889340910f 100644 --- a/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py +++ b/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py @@ -1390,22 +1390,22 @@ async def main(): def assert_context_unbound(context: RunlessOpExecutionContext): # to assert that the context is correctly unbound after op invocation - assert context._bound_properties is None # noqa: SLF001 + assert context.bound_properties is None def assert_context_bound(context: RunlessOpExecutionContext): # to assert that the context is correctly bound during op invocation - assert context._bound_properties is not None # noqa: SLF001 + assert context.bound_properties is not None def assert_execution_properties_cleared(context: RunlessOpExecutionContext): # to assert that the invocation properties are reset at the beginning of op invocation - assert len(context._execution_properties.output_metadata.keys()) == 0 # noqa: SLF001 + assert len(context.execution_properties.output_metadata.keys()) == 0 def assert_execution_properties_exist(context: RunlessOpExecutionContext): # to assert that the invocation properties remain accessible after op invocation - assert len(context._execution_properties.output_metadata.keys()) > 0 # noqa: SLF001 + assert len(context.execution_properties.output_metadata.keys()) > 0 def test_context_bound_state_non_generator(): From 9537bde53fa5825b04ab601ad901e7a89f9cb7e4 Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Fri, 8 Dec 2023 12:14:12 -0500 Subject: [PATCH 38/44] add is_bound prop --- .../dagster/dagster/_core/execution/context/invocation.py | 4 ++++ .../dagster/dagster_tests/core_tests/test_op_invocation.py | 4 ++-- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/python_modules/dagster/dagster/_core/execution/context/invocation.py b/python_modules/dagster/dagster/_core/execution/context/invocation.py index 49adc714175ff..2fe58e73cb7a7 100644 --- a/python_modules/dagster/dagster/_core/execution/context/invocation.py +++ b/python_modules/dagster/dagster/_core/execution/context/invocation.py @@ -408,6 +408,10 @@ def bind( def unbind(self): self._bound_properties = None + @property + def is_bound(self) -> bool: + return self._bound_properties is not None + @property def execution_properties(self) -> RunlessExecutionProperties: return self._execution_properties diff --git a/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py b/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py index 257889340910f..136c20c626f41 100644 --- a/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py +++ b/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py @@ -1390,12 +1390,12 @@ async def main(): def assert_context_unbound(context: RunlessOpExecutionContext): # to assert that the context is correctly unbound after op invocation - assert context.bound_properties is None + assert not context.is_bound def assert_context_bound(context: RunlessOpExecutionContext): # to assert that the context is correctly bound during op invocation - assert context.bound_properties is not None + assert context.is_bound def assert_execution_properties_cleared(context: RunlessOpExecutionContext): From 0371e8e00f15dcce601e58522e9cb6ce6f62c0f2 Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Fri, 8 Dec 2023 12:22:10 -0500 Subject: [PATCH 39/44] use a methods that's actually on the context --- .../dagster/dagster_tests/core_tests/test_op_invocation.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py b/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py index 136c20c626f41..b97e6f69938f3 100644 --- a/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py +++ b/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py @@ -1508,7 +1508,7 @@ async def get_results(): def test_bound_state_with_error_assets(): @asset def throws_error(context): - assert context.alias == "throws_error" + assert context.asset_key.to_user_string() == "throws_error" raise Failure("something bad happened!") ctx = build_asset_context() From deec5a32a5444df88ad839717cfa5ee87d62b583 Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Tue, 19 Dec 2023 14:30:32 -0500 Subject: [PATCH 40/44] rename --- .../_core/definitions/op_invocation.py | 30 +-- .../_core/execution/context/invocation.py | 212 +++++++----------- .../dagster/dagster/_core/pipes/context.py | 6 +- .../core_tests/test_op_invocation.py | 10 +- 4 files changed, 110 insertions(+), 148 deletions(-) diff --git a/python_modules/dagster/dagster/_core/definitions/op_invocation.py b/python_modules/dagster/dagster/_core/definitions/op_invocation.py index 115ccfcf55ae0..7a3254413eb69 100644 --- a/python_modules/dagster/dagster/_core/definitions/op_invocation.py +++ b/python_modules/dagster/dagster/_core/definitions/op_invocation.py @@ -32,7 +32,7 @@ from .result import MaterializeResult if TYPE_CHECKING: - from ..execution.context.invocation import RunlessOpExecutionContext + from ..execution.context.invocation import DirectOpExecutionContext from .assets import AssetsDefinition from .composition import PendingNodeInvocation from .decorators.op_decorator import DecoratedOpFunction @@ -109,7 +109,7 @@ def direct_invocation_result( ) -> Any: from dagster._config.pythonic_config import Config from dagster._core.execution.context.invocation import ( - RunlessOpExecutionContext, + DirectOpExecutionContext, build_op_context, ) @@ -149,12 +149,12 @@ def direct_invocation_result( " no context was provided when invoking." ) if len(args) > 0: - if args[0] is not None and not isinstance(args[0], RunlessOpExecutionContext): + if args[0] is not None and not isinstance(args[0], DirectOpExecutionContext): raise DagsterInvalidInvocationError( f"Decorated function '{compute_fn.name}' has context argument, " "but no context was provided when invoking." ) - context = cast(RunlessOpExecutionContext, args[0]) + context = cast(DirectOpExecutionContext, args[0]) # update args to omit context args = args[1:] else: # context argument is provided under kwargs @@ -165,14 +165,14 @@ def direct_invocation_result( f"'{context_param_name}', but no value for '{context_param_name}' was " f"found when invoking. Provided kwargs: {kwargs}" ) - context = cast(RunlessOpExecutionContext, kwargs[context_param_name]) + context = cast(DirectOpExecutionContext, kwargs[context_param_name]) # update kwargs to remove context kwargs = { kwarg: val for kwarg, val in kwargs.items() if not kwarg == context_param_name } # allow passing context, even if the function doesn't have an arg for it - elif len(args) > 0 and isinstance(args[0], RunlessOpExecutionContext): - context = cast(RunlessOpExecutionContext, args[0]) + elif len(args) > 0 and isinstance(args[0], DirectOpExecutionContext): + context = cast(DirectOpExecutionContext, args[0]) args = args[1:] resource_arg_mapping = {arg.name: arg.name for arg in compute_fn.get_resource_args()} @@ -230,7 +230,7 @@ def direct_invocation_result( def _resolve_inputs( - op_def: "OpDefinition", args, kwargs, context: "RunlessOpExecutionContext" + op_def: "OpDefinition", args, kwargs, context: "DirectOpExecutionContext" ) -> Mapping[str, Any]: from dagster._core.execution.plan.execute_step import do_type_check @@ -333,7 +333,7 @@ def _resolve_inputs( return input_dict -def _key_for_result(result: MaterializeResult, context: "RunlessOpExecutionContext") -> AssetKey: +def _key_for_result(result: MaterializeResult, context: "DirectOpExecutionContext") -> AssetKey: if not context.bound_properties.assets_def: raise DagsterInvariantViolationError( f"Op {context.bound_properties.alias} does not have an assets definition." @@ -352,7 +352,7 @@ def _key_for_result(result: MaterializeResult, context: "RunlessOpExecutionConte def _output_name_for_result_obj( event: MaterializeResult, - context: "RunlessOpExecutionContext", + context: "DirectOpExecutionContext", ): if not context.bound_properties.assets_def: raise DagsterInvariantViolationError( @@ -365,7 +365,7 @@ def _output_name_for_result_obj( def _handle_gen_event( event: T, op_def: "OpDefinition", - context: "RunlessOpExecutionContext", + context: "DirectOpExecutionContext", output_defs: Mapping[str, OutputDefinition], outputs_seen: Set[str], ) -> T: @@ -399,7 +399,7 @@ def _handle_gen_event( def _type_check_output_wrapper( - op_def: "OpDefinition", result: Any, context: "RunlessOpExecutionContext" + op_def: "OpDefinition", result: Any, context: "DirectOpExecutionContext" ) -> Any: """Type checks and returns the result of a op. @@ -493,7 +493,7 @@ def type_check_gen(gen): def _type_check_function_output( - op_def: "OpDefinition", result: T, context: "RunlessOpExecutionContext" + op_def: "OpDefinition", result: T, context: "DirectOpExecutionContext" ) -> T: from ..execution.plan.compute_generator import validate_and_coerce_op_result_to_iterator @@ -512,14 +512,14 @@ def _type_check_function_output( def _type_check_output( output_def: "OutputDefinition", output: Union[Output, DynamicOutput], - context: "RunlessOpExecutionContext", + context: "DirectOpExecutionContext", ) -> None: """Validates and performs core type check on a provided output. Args: output_def (OutputDefinition): The output definition to validate against. output (Any): The output to validate. - context (RunlessOpExecutionContext): Context containing resources to be used for type + context (DirectOpExecutionContext): Context containing resources to be used for type check. """ from ..execution.plan.execute_step import do_type_check diff --git a/python_modules/dagster/dagster/_core/execution/context/invocation.py b/python_modules/dagster/dagster/_core/execution/context/invocation.py index 2fe58e73cb7a7..bdd18be343880 100644 --- a/python_modules/dagster/dagster/_core/execution/context/invocation.py +++ b/python_modules/dagster/dagster/_core/execution/context/invocation.py @@ -106,124 +106,24 @@ def __new__( ) -class RunlessExecutionProperties: +class DirectExecutionProperties: """Maintains information about the invocation that is updated during execution time. This information needs to be available to the user once invocation is complete, so that they can assert on events and outputs. It needs to be cleared before the context is used for another invocation. + + This is not implemented as a NamedTuple because the various attributes will be mutated during + execution. """ def __init__(self): - self._events: List[UserEvent] = [] - self._seen_outputs = {} - self._output_metadata = {} - self._requires_typed_event_stream = False - self._typed_event_stream_error_message = None - - @property - def user_events(self): - return self._events - - @property - def seen_outputs(self): - return self._seen_outputs - - @property - def output_metadata(self): - return self._output_metadata - - @property - def requires_typed_event_stream(self) -> bool: - return self._requires_typed_event_stream - - @property - def typed_event_stream_error_message(self) -> Optional[str]: - return self._typed_event_stream_error_message - - def log_event(self, event: UserEvent) -> None: - check.inst_param( - event, - "event", - (AssetMaterialization, AssetObservation, ExpectationResult), - ) - self._events.append(event) - - def observe_output(self, output_name: str, mapping_key: Optional[str] = None) -> None: - if mapping_key: - if output_name not in self.seen_outputs: - self._seen_outputs[output_name] = set() - cast(Set[str], self._seen_outputs[output_name]).add(mapping_key) - else: - self._seen_outputs[output_name] = "seen" - - def has_seen_output(self, output_name: str, mapping_key: Optional[str] = None) -> bool: - if mapping_key: - return ( - output_name in self.seen_outputs and mapping_key in self.seen_outputs[output_name] - ) - return output_name in self.seen_outputs - - def add_output_metadata( - self, - metadata: Mapping[str, Any], - op_def: OpDefinition, - output_name: Optional[str] = None, - mapping_key: Optional[str] = None, - ) -> None: - metadata = check.mapping_param(metadata, "metadata", key_type=str) - output_name = check.opt_str_param(output_name, "output_name") - mapping_key = check.opt_str_param(mapping_key, "mapping_key") + self.user_events: List[UserEvent] = [] + self.seen_outputs: Dict[str, Union[str, Set[str]]] = {} + self.output_metadata: Dict[str, Dict[str, Union[Any, Mapping[str, Any]]]] = {} + self.requires_typed_event_stream: bool = False + self.typed_event_stream_error_message: Optional[str] = None - if output_name is None and len(op_def.output_defs) == 1: - output_def = op_def.output_defs[0] - output_name = output_def.name - elif output_name is None: - raise DagsterInvariantViolationError( - "Attempted to log metadata without providing output_name, but multiple outputs" - " exist. Please provide an output_name to the invocation of" - " `context.add_output_metadata`." - ) - else: - output_def = op_def.output_def_named(output_name) - if self.has_seen_output(output_name, mapping_key): - output_desc = ( - f"output '{output_def.name}'" - if not mapping_key - else f"output '{output_def.name}' with mapping_key '{mapping_key}'" - ) - raise DagsterInvariantViolationError( - f"In {op_def.node_type_str} '{op_def.name}', attempted to log output" - f" metadata for {output_desc} which has already been yielded. Metadata must be" - " logged before the output is yielded." - ) - if output_def.is_dynamic and not mapping_key: - raise DagsterInvariantViolationError( - f"In {op_def.node_type_str} '{op_def.name}', attempted to log metadata" - f" for dynamic output '{output_def.name}' without providing a mapping key. When" - " logging metadata for a dynamic output, it is necessary to provide a mapping key." - ) - - output_name = output_def.name - if output_name in self.output_metadata: - if not mapping_key or mapping_key in self.output_metadata[output_name]: - raise DagsterInvariantViolationError( - f"In {op_def.node_type_str} '{op_def.name}', attempted to log" - f" metadata for output '{output_name}' more than once." - ) - if mapping_key: - if output_name not in self.output_metadata: - self._output_metadata[output_name] = {} - self._output_metadata[output_name][mapping_key] = metadata - - else: - self._output_metadata[output_name] = metadata - - def set_requires_typed_event_stream(self, *, error_message: Optional[str]) -> None: - self._requires_typed_event_stream = True - self._typed_event_stream_error_message = error_message - - -class RunlessOpExecutionContext(OpExecutionContext): +class DirectOpExecutionContext(OpExecutionContext): """The ``context`` object available as the first argument to an op's compute function when being invoked directly. Can also be used as a context manager. """ @@ -292,7 +192,7 @@ def __init__( # my_op(ctx) # ctx._execution_properties.output_metadata # information is retained after invocation # my_op(ctx) # ctx._execution_properties is cleared at the beginning of the next invocation - self._execution_properties = RunlessExecutionProperties() + self._execution_properties = DirectExecutionProperties() def __enter__(self): self._cm_scope_entered = True @@ -318,7 +218,7 @@ def bind( assets_def: Optional[AssetsDefinition], config_from_args: Optional[Mapping[str, Any]], resources_from_args: Optional[Mapping[str, Any]], - ) -> "RunlessOpExecutionContext": + ) -> "DirectOpExecutionContext": from dagster._core.definitions.resource_invocation import resolve_bound_config if self._bound_properties is not None: @@ -327,7 +227,7 @@ def bind( ) # reset execution_properties - self._execution_properties = RunlessExecutionProperties() + self._execution_properties = DirectExecutionProperties() # update the bound context with properties relevant to the execution of the op @@ -413,7 +313,7 @@ def is_bound(self) -> bool: return self._bound_properties is not None @property - def execution_properties(self) -> RunlessExecutionProperties: + def execution_properties(self) -> DirectExecutionProperties: return self._execution_properties @property @@ -637,16 +537,29 @@ def describe_op(self) -> str: def log_event(self, event: UserEvent) -> None: self._check_bound(fn_name="log_event", fn_type="method") - self._execution_properties.log_event(event) + check.inst_param( + event, + "event", + (AssetMaterialization, AssetObservation, ExpectationResult), + ) + self._execution_properties.user_events.append(event) def observe_output(self, output_name: str, mapping_key: Optional[str] = None) -> None: self._check_bound(fn_name="observe_output", fn_type="method") - self._execution_properties.observe_output(output_name=output_name, mapping_key=mapping_key) + if mapping_key: + if output_name not in self._execution_properties.seen_outputs: + self._execution_properties.seen_outputs[output_name] = set() + cast(Set[str], self._execution_properties.seen_outputs[output_name]).add(mapping_key) + else: + self._execution_properties.seen_outputs[output_name] = "seen" def has_seen_output(self, output_name: str, mapping_key: Optional[str] = None) -> bool: - return self._execution_properties.has_seen_output( - output_name=output_name, mapping_key=mapping_key - ) + if mapping_key: + return ( + output_name in self._execution_properties.seen_outputs + and mapping_key in self._execution_properties.seen_outputs[output_name] + ) + return output_name in self._execution_properties.seen_outputs def asset_partitions_time_window_for_output(self, output_name: str = "result") -> TimeWindow: self._check_bound(fn_name="asset_partitions_time_window_for_output", fn_type="method") @@ -699,9 +612,57 @@ def add_metadata_two_outputs(context) -> Tuple[str, int]: """ self._check_bound(fn_name="add_output_metadata", fn_type="method") - self._execution_properties.add_output_metadata( - metadata=metadata, op_def=self.op_def, output_name=output_name, mapping_key=mapping_key - ) + metadata = check.mapping_param(metadata, "metadata", key_type=str) + output_name = check.opt_str_param(output_name, "output_name") + mapping_key = check.opt_str_param(mapping_key, "mapping_key") + + if output_name is None and len(self.op_def.output_defs) == 1: + output_def = self.op_def.output_defs[0] + output_name = output_def.name + elif output_name is None: + raise DagsterInvariantViolationError( + "Attempted to log metadata without providing output_name, but multiple outputs" + " exist. Please provide an output_name to the invocation of" + " `context.add_output_metadata`." + ) + else: + output_def = self.op_def.output_def_named(output_name) + + if self.has_seen_output(output_name, mapping_key): + output_desc = ( + f"output '{output_def.name}'" + if not mapping_key + else f"output '{output_def.name}' with mapping_key '{mapping_key}'" + ) + raise DagsterInvariantViolationError( + f"In {self.op_def.node_type_str} '{self.op_def.name}', attempted to log output" + f" metadata for {output_desc} which has already been yielded. Metadata must be" + " logged before the output is yielded." + ) + if output_def.is_dynamic and not mapping_key: + raise DagsterInvariantViolationError( + f"In {self.op_def.node_type_str} '{self.op_def.name}', attempted to log metadata" + f" for dynamic output '{output_def.name}' without providing a mapping key. When" + " logging metadata for a dynamic output, it is necessary to provide a mapping key." + ) + + output_name = output_def.name + if output_name in self._execution_properties.output_metadata: + if ( + not mapping_key + or mapping_key in self._execution_properties.output_metadata[output_name] + ): + raise DagsterInvariantViolationError( + f"In {self.op_def.node_type_str} '{self.op_def.name}', attempted to log" + f" metadata for output '{output_name}' more than once." + ) + if mapping_key: + if output_name not in self._execution_properties.output_metadata: + self._execution_properties.output_metadata[output_name] = {} + self._execution_properties.output_metadata[output_name][mapping_key] = metadata + + else: + self._execution_properties.output_metadata[output_name] = metadata # In bound mode no conversion is done on returned values and missing but expected outputs are not # allowed. @@ -717,7 +678,8 @@ def typed_event_stream_error_message(self) -> Optional[str]: def set_requires_typed_event_stream(self, *, error_message: Optional[str]) -> None: self._check_bound(fn_name="set_requires_typed_event_stream", fn_type="method") - self._execution_properties.set_requires_typed_event_stream(error_message=error_message) + self._execution_properties.requires_typed_event_stream = True + self._execution_properties.typed_event_stream_error_message = error_message def _validate_resource_requirements( @@ -740,7 +702,7 @@ def build_op_context( partition_key_range: Optional[PartitionKeyRange] = None, mapping_key: Optional[str] = None, _assets_def: Optional[AssetsDefinition] = None, -) -> RunlessOpExecutionContext: +) -> DirectOpExecutionContext: """Builds op execution context from provided parameters. ``build_op_context`` can be used as either a function or context manager. If there is a @@ -788,7 +750,7 @@ def build_op_context( ) op_config = op_config if op_config else config - return RunlessOpExecutionContext( + return DirectOpExecutionContext( resources_dict=check.opt_mapping_param(resources, "resources", key_type=str), resources_config=check.opt_mapping_param( resources_config, "resources_config", key_type=str diff --git a/python_modules/dagster/dagster/_core/pipes/context.py b/python_modules/dagster/dagster/_core/pipes/context.py index f141048b8c76f..5b61647bbf43b 100644 --- a/python_modules/dagster/dagster/_core/pipes/context.py +++ b/python_modules/dagster/dagster/_core/pipes/context.py @@ -39,7 +39,7 @@ from dagster._core.errors import DagsterPipesExecutionError from dagster._core.events import EngineEventData from dagster._core.execution.context.compute import OpExecutionContext -from dagster._core.execution.context.invocation import RunlessOpExecutionContext +from dagster._core.execution.context.invocation import DirectOpExecutionContext from dagster._utils.error import ( ExceptionInfo, SerializableErrorInfo, @@ -406,8 +406,8 @@ def build_external_execution_context_data( _convert_time_window(partition_time_window) if partition_time_window else None ), run_id=context.run_id, - job_name=None if isinstance(context, RunlessOpExecutionContext) else context.job_name, - retry_number=0 if isinstance(context, RunlessOpExecutionContext) else context.retry_number, + job_name=None if isinstance(context, DirectOpExecutionContext) else context.job_name, + retry_number=0 if isinstance(context, DirectOpExecutionContext) else context.retry_number, extras=extras or {}, ) diff --git a/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py b/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py index b97e6f69938f3..b41669e98b265 100644 --- a/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py +++ b/python_modules/dagster/dagster_tests/core_tests/test_op_invocation.py @@ -45,7 +45,7 @@ ) from dagster._core.execution.context.compute import AssetExecutionContext, OpExecutionContext from dagster._core.execution.context.invocation import ( - RunlessOpExecutionContext, + DirectOpExecutionContext, build_asset_context, ) from dagster._utils.test import wrap_op_in_graph_and_execute @@ -1388,22 +1388,22 @@ async def main(): asyncio.run(main()) -def assert_context_unbound(context: RunlessOpExecutionContext): +def assert_context_unbound(context: DirectOpExecutionContext): # to assert that the context is correctly unbound after op invocation assert not context.is_bound -def assert_context_bound(context: RunlessOpExecutionContext): +def assert_context_bound(context: DirectOpExecutionContext): # to assert that the context is correctly bound during op invocation assert context.is_bound -def assert_execution_properties_cleared(context: RunlessOpExecutionContext): +def assert_execution_properties_cleared(context: DirectOpExecutionContext): # to assert that the invocation properties are reset at the beginning of op invocation assert len(context.execution_properties.output_metadata.keys()) == 0 -def assert_execution_properties_exist(context: RunlessOpExecutionContext): +def assert_execution_properties_exist(context: DirectOpExecutionContext): # to assert that the invocation properties remain accessible after op invocation assert len(context.execution_properties.output_metadata.keys()) > 0 From 46328a509bd69cd5b537d834777e943d6a6099f0 Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Tue, 19 Dec 2023 14:37:44 -0500 Subject: [PATCH 41/44] comments --- .../dagster/_core/execution/context/invocation.py | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/python_modules/dagster/dagster/_core/execution/context/invocation.py b/python_modules/dagster/dagster/_core/execution/context/invocation.py index bdd18be343880..6277a930f395d 100644 --- a/python_modules/dagster/dagster/_core/execution/context/invocation.py +++ b/python_modules/dagster/dagster/_core/execution/context/invocation.py @@ -81,6 +81,11 @@ class BoundProperties( ], ) ): + """Maintains properties that are only available once the context has been bound to a particular + asset or op execution. By splitting these out into a separate object, it is easier to ensure that + all properties bound to an execution are cleared once the execution is complete. + """ + def __new__( cls, op_def: OpDefinition, @@ -92,7 +97,6 @@ def __new__( op_config: Any, step_description: str, ): - """Maintains the properties of the context that are provided at bind time.""" return super(BoundProperties, cls).__new__( cls, op_def=check.inst_param(op_def, "op_def", OpDefinition), @@ -107,9 +111,9 @@ def __new__( class DirectExecutionProperties: - """Maintains information about the invocation that is updated during execution time. This information - needs to be available to the user once invocation is complete, so that they can assert on events and - outputs. It needs to be cleared before the context is used for another invocation. + """Maintains information about the execution that can only be updated during execution (when + the context is bound), but can be read after execution is complete. It needs to be cleared before + the context is used for another execution. This is not implemented as a NamedTuple because the various attributes will be mutated during execution. From 92ea8d84f7cdb7899eebda81cf686c9c04567386 Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Tue, 19 Dec 2023 14:53:18 -0500 Subject: [PATCH 42/44] missed a name --- .../dagster/dagster/_core/definitions/op_invocation.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python_modules/dagster/dagster/_core/definitions/op_invocation.py b/python_modules/dagster/dagster/_core/definitions/op_invocation.py index 7a3254413eb69..64bb2d01c3a58 100644 --- a/python_modules/dagster/dagster/_core/definitions/op_invocation.py +++ b/python_modules/dagster/dagster/_core/definitions/op_invocation.py @@ -538,6 +538,6 @@ def _type_check_output( dagster_type=dagster_type, ) - context.execution_properties.observe_output( + context.observe_output( output_def.name, output.mapping_key if isinstance(output, DynamicOutput) else None ) From 5f174feba37734c3f0e5d57c2830ca2f192b96e5 Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Mon, 29 Jan 2024 11:29:38 -0500 Subject: [PATCH 43/44] final cleanup --- .../dagster/dagster/_core/execution/context/invocation.py | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/python_modules/dagster/dagster/_core/execution/context/invocation.py b/python_modules/dagster/dagster/_core/execution/context/invocation.py index 6277a930f395d..2325de5f08fd8 100644 --- a/python_modules/dagster/dagster/_core/execution/context/invocation.py +++ b/python_modules/dagster/dagster/_core/execution/context/invocation.py @@ -234,7 +234,6 @@ def bind( self._execution_properties = DirectExecutionProperties() # update the bound context with properties relevant to the execution of the op - invocation_tags = ( pending_invocation.tags if isinstance(pending_invocation, PendingNodeInvocation) @@ -291,10 +290,7 @@ def bind( raise DagsterInvalidInvocationError("Cannot provide config in both context and kwargs") op_config = resolve_bound_config(config_from_args or self._op_config, op_def) - if isinstance(op_def, OpDefinition): - step_description = f'op "{op_def.name}"' - else: - step_description = f'solid "{op_def.name}"' + step_description = f'op "{op_def.name}"' self._bound_properties = BoundProperties( op_def=op_def, From 8e4327d453ee3c3d43569cbffaf18c6d0e478d53 Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Mon, 29 Jan 2024 12:20:25 -0500 Subject: [PATCH 44/44] update boundproperties to perinvocationproperties --- .../_core/definitions/op_invocation.py | 31 +++-- .../_core/execution/context/invocation.py | 126 +++++++++++------- 2 files changed, 92 insertions(+), 65 deletions(-) diff --git a/python_modules/dagster/dagster/_core/definitions/op_invocation.py b/python_modules/dagster/dagster/_core/definitions/op_invocation.py index 64bb2d01c3a58..3dc8e5efc1a67 100644 --- a/python_modules/dagster/dagster/_core/definitions/op_invocation.py +++ b/python_modules/dagster/dagster/_core/definitions/op_invocation.py @@ -270,7 +270,7 @@ def _resolve_inputs( node_label = op_def.node_type_str raise DagsterInvalidInvocationError( - f"Too many input arguments were provided for {node_label} '{context.bound_properties.alias}'." + f"Too many input arguments were provided for {node_label} '{context.per_invocation_properties.alias}'." f" {suggestion}" ) @@ -313,7 +313,7 @@ def _resolve_inputs( input_dict[k] = v # Type check inputs - op_label = context.bound_properties.step_description + op_label = context.per_invocation_properties.step_description for input_name, val in input_dict.items(): input_def = input_defs_by_name[input_name] @@ -334,19 +334,22 @@ def _resolve_inputs( def _key_for_result(result: MaterializeResult, context: "DirectOpExecutionContext") -> AssetKey: - if not context.bound_properties.assets_def: + if not context.per_invocation_properties.assets_def: raise DagsterInvariantViolationError( - f"Op {context.bound_properties.alias} does not have an assets definition." + f"Op {context.per_invocation_properties.alias} does not have an assets definition." ) if result.asset_key: return result.asset_key - if context.bound_properties.assets_def and len(context.bound_properties.assets_def.keys) == 1: - return next(iter(context.bound_properties.assets_def.keys)) + if ( + context.per_invocation_properties.assets_def + and len(context.per_invocation_properties.assets_def.keys) == 1 + ): + return next(iter(context.per_invocation_properties.assets_def.keys)) raise DagsterInvariantViolationError( "MaterializeResult did not include asset_key and it can not be inferred. Specify which" - f" asset_key, options are: {context.bound_properties.assets_def.keys}" + f" asset_key, options are: {context.per_invocation_properties.assets_def.keys}" ) @@ -354,12 +357,12 @@ def _output_name_for_result_obj( event: MaterializeResult, context: "DirectOpExecutionContext", ): - if not context.bound_properties.assets_def: + if not context.per_invocation_properties.assets_def: raise DagsterInvariantViolationError( - f"Op {context.bound_properties.alias} does not have an assets definition." + f"Op {context.per_invocation_properties.alias} does not have an assets definition." ) asset_key = _key_for_result(event, context) - return context.bound_properties.assets_def.get_output_name_for_asset_key(asset_key) + return context.per_invocation_properties.assets_def.get_output_name_for_asset_key(asset_key) def _handle_gen_event( @@ -391,7 +394,7 @@ def _handle_gen_event( output_def, DynamicOutputDefinition ): raise DagsterInvariantViolationError( - f"Invocation of {op_def.node_type_str} '{context.bound_properties.alias}' yielded" + f"Invocation of {op_def.node_type_str} '{context.per_invocation_properties.alias}' yielded" f" an output '{output_def.name}' multiple times." ) outputs_seen.add(output_def.name) @@ -434,7 +437,7 @@ async def to_gen(async_gen): yield Output(output_name=output_def.name, value=None) else: raise DagsterInvariantViolationError( - f"Invocation of {op_def.node_type_str} '{context.bound_properties.alias}' did not" + f"Invocation of {op_def.node_type_str} '{context.per_invocation_properties.alias}' did not" f" return an output for non-optional output '{output_def.name}'" ) context.unbind() @@ -481,7 +484,7 @@ def type_check_gen(gen): yield Output(output_name=output_def.name, value=None) else: raise DagsterInvariantViolationError( - f'Invocation of {op_def.node_type_str} "{context.bound_properties.alias}" did not' + f'Invocation of {op_def.node_type_str} "{context.per_invocation_properties.alias}" did not' f' return an output for non-optional output "{output_def.name}"' ) context.unbind() @@ -524,7 +527,7 @@ def _type_check_output( """ from ..execution.plan.execute_step import do_type_check - op_label = context.bound_properties.step_description + op_label = context.per_invocation_properties.step_description dagster_type = output_def.dagster_type type_check = do_type_check(context.for_type(dagster_type), dagster_type, output.value) if not type_check.success: diff --git a/python_modules/dagster/dagster/_core/execution/context/invocation.py b/python_modules/dagster/dagster/_core/execution/context/invocation.py index 2325de5f08fd8..02038f15127ed 100644 --- a/python_modules/dagster/dagster/_core/execution/context/invocation.py +++ b/python_modules/dagster/dagster/_core/execution/context/invocation.py @@ -66,9 +66,9 @@ def _property_msg(prop_name: str, method_name: str) -> str: ) -class BoundProperties( +class PerInvocationProperties( NamedTuple( - "_BoundProperties", + "_PerInvocationProperties", [ ("op_def", OpDefinition), ("tags", Mapping[Any, Any]), @@ -82,8 +82,8 @@ class BoundProperties( ) ): """Maintains properties that are only available once the context has been bound to a particular - asset or op execution. By splitting these out into a separate object, it is easier to ensure that - all properties bound to an execution are cleared once the execution is complete. + asset or op invocation. By splitting these out into a separate object, it is easier to ensure that + all properties bound to an invocation are cleared once the execution is complete. """ def __new__( @@ -97,7 +97,7 @@ def __new__( op_config: Any, step_description: str, ): - return super(BoundProperties, cls).__new__( + return super(PerInvocationProperties, cls).__new__( cls, op_def=check.inst_param(op_def, "op_def", OpDefinition), tags=check.dict_param(tags, "tags"), @@ -180,12 +180,12 @@ def __init__( # of an op # @op # def my_op(context): - # # context._bound_properties.alias is "my_op" + # # context._per_invocation_properties.alias is "my_op" # ... - # ctx = build_op_context() # ctx._bound_properties is None + # ctx = build_op_context() # ctx._per_invocation_properties is None # my_op(ctx) - # ctx._bound_properties is None # ctx is unbound at the end of invocation - self._bound_properties = None + # ctx._per_invocation_properties is None # ctx is unbound at the end of invocation + self._per_invocation_properties = None # Maintains the properties on the context that are modified during invocation # @op @@ -208,12 +208,12 @@ def __exit__(self, *exc): def __del__(self): self._exit_stack.close() - def _check_bound(self, fn_name: str, fn_type: str) -> BoundProperties: - if self._bound_properties is None: + def _check_bound_to_invocation(self, fn_name: str, fn_type: str) -> PerInvocationProperties: + if self._per_invocation_properties is None: raise DagsterInvalidPropertyError(_property_msg(fn_name, fn_type)) - # return self._bound_properties so that the calling function can access properties - # of self._bound_properties without causing pyright errors - return self._bound_properties + # return self._per_invocation_properties so that the calling function can access properties + # of self._per_invocation_properties without causing pyright errors + return self._per_invocation_properties def bind( self, @@ -225,7 +225,7 @@ def bind( ) -> "DirectOpExecutionContext": from dagster._core.definitions.resource_invocation import resolve_bound_config - if self._bound_properties is not None: + if self._per_invocation_properties is not None: raise DagsterInvalidInvocationError( f"This context is currently being used to execute {self.alias}. The context cannot be used to execute another op until {self.alias} has finished executing." ) @@ -233,7 +233,7 @@ def bind( # reset execution_properties self._execution_properties = DirectExecutionProperties() - # update the bound context with properties relevant to the execution of the op + # update the bound context with properties relevant to the invocation of the op invocation_tags = ( pending_invocation.tags if isinstance(pending_invocation, PendingNodeInvocation) @@ -292,7 +292,7 @@ def bind( step_description = f'op "{op_def.name}"' - self._bound_properties = BoundProperties( + self._per_invocation_properties = PerInvocationProperties( op_def=op_def, tags=tags, hook_defs=hook_defs, @@ -306,25 +306,27 @@ def bind( return self def unbind(self): - self._bound_properties = None + self._per_invocation_properties = None @property def is_bound(self) -> bool: - return self._bound_properties is not None + return self._per_invocation_properties is not None @property def execution_properties(self) -> DirectExecutionProperties: return self._execution_properties @property - def bound_properties(self) -> BoundProperties: - return self._check_bound(fn_name="bound_properties", fn_type="property") + def per_invocation_properties(self) -> PerInvocationProperties: + return self._check_bound_to_invocation( + fn_name="_per_invocation_properties", fn_type="property" + ) @property def op_config(self) -> Any: - if self._bound_properties is None: + if self._per_invocation_properties is None: return self._op_config - return self._bound_properties.op_config + return self._per_invocation_properties.op_config @property def resource_keys(self) -> AbstractSet[str]: @@ -332,8 +334,8 @@ def resource_keys(self) -> AbstractSet[str]: @property def resources(self) -> Resources: - if self._bound_properties is not None: - return self._bound_properties.resources + if self._per_invocation_properties is not None: + return self._per_invocation_properties.resources if self._resources_contain_cm and not self._cm_scope_entered: raise DagsterInvariantViolationError( "At least one provided resource is a generator, but attempting to access " @@ -378,12 +380,16 @@ def run_id(self) -> str: @property def run_config(self) -> dict: - bound_properties = self._check_bound(fn_name="run_config", fn_type="property") + per_invocation_properties = self._check_bound_to_invocation( + fn_name="run_config", fn_type="property" + ) run_config: Dict[str, object] = {} - if self._op_config and bound_properties.op_def: + if self._op_config and per_invocation_properties.op_def: run_config["ops"] = { - bound_properties.op_def.name: {"config": bound_properties.op_config} + per_invocation_properties.op_def.name: { + "config": per_invocation_properties.op_config + } } run_config["resources"] = self._resources_config return run_config @@ -415,23 +421,29 @@ def solid(self) -> Node: @property def op_def(self) -> OpDefinition: - bound_properties = self._check_bound(fn_name="op_def", fn_type="property") - return cast(OpDefinition, bound_properties.op_def) + per_invocation_properties = self._check_bound_to_invocation( + fn_name="op_def", fn_type="property" + ) + return cast(OpDefinition, per_invocation_properties.op_def) @property def has_assets_def(self) -> bool: - bound_properties = self._check_bound(fn_name="has_assets_def", fn_type="property") - return bound_properties.assets_def is not None + per_invocation_properties = self._check_bound_to_invocation( + fn_name="has_assets_def", fn_type="property" + ) + return per_invocation_properties.assets_def is not None @property def assets_def(self) -> AssetsDefinition: - bound_properties = self._check_bound(fn_name="assets_def", fn_type="property") + per_invocation_properties = self._check_bound_to_invocation( + fn_name="assets_def", fn_type="property" + ) - if bound_properties.assets_def is None: + if per_invocation_properties.assets_def is None: raise DagsterInvalidPropertyError( f"Op {self.op_def.name} does not have an assets definition." ) - return bound_properties.assets_def + return per_invocation_properties.assets_def @property def has_partition_key(self) -> bool: @@ -461,17 +473,23 @@ def asset_partition_key_for_output(self, output_name: str = "result") -> str: return self.partition_key def has_tag(self, key: str) -> bool: - bound_properties = self._check_bound(fn_name="has_tag", fn_type="method") - return key in bound_properties.tags + per_invocation_properties = self._check_bound_to_invocation( + fn_name="has_tag", fn_type="method" + ) + return key in per_invocation_properties.tags def get_tag(self, key: str) -> Optional[str]: - bound_properties = self._check_bound(fn_name="get_tag", fn_type="method") - return bound_properties.tags.get(key) + per_invocation_properties = self._check_bound_to_invocation( + fn_name="get_tag", fn_type="method" + ) + return per_invocation_properties.tags.get(key) @property def alias(self) -> str: - bound_properties = self._check_bound(fn_name="alias", fn_type="property") - return cast(str, bound_properties.alias) + per_invocation_properties = self._check_bound_to_invocation( + fn_name="alias", fn_type="property" + ) + return cast(str, per_invocation_properties.alias) def get_step_execution_context(self) -> StepExecutionContext: raise DagsterInvalidPropertyError(_property_msg("get_step_execution_context", "method")) @@ -522,7 +540,7 @@ def get_mapping_key(self) -> Optional[str]: return self._mapping_key def for_type(self, dagster_type: DagsterType) -> TypeCheckContext: - self._check_bound(fn_name="for_type", fn_type="method") + self._check_bound_to_invocation(fn_name="for_type", fn_type="method") resources = cast(NamedTuple, self.resources) return TypeCheckContext( self.run_id, @@ -532,11 +550,13 @@ def for_type(self, dagster_type: DagsterType) -> TypeCheckContext: ) def describe_op(self) -> str: - bound_properties = self._check_bound(fn_name="describe_op", fn_type="method") - return bound_properties.step_description + per_invocation_properties = self._check_bound_to_invocation( + fn_name="describe_op", fn_type="method" + ) + return per_invocation_properties.step_description def log_event(self, event: UserEvent) -> None: - self._check_bound(fn_name="log_event", fn_type="method") + self._check_bound_to_invocation(fn_name="log_event", fn_type="method") check.inst_param( event, "event", @@ -545,7 +565,7 @@ def log_event(self, event: UserEvent) -> None: self._execution_properties.user_events.append(event) def observe_output(self, output_name: str, mapping_key: Optional[str] = None) -> None: - self._check_bound(fn_name="observe_output", fn_type="method") + self._check_bound_to_invocation(fn_name="observe_output", fn_type="method") if mapping_key: if output_name not in self._execution_properties.seen_outputs: self._execution_properties.seen_outputs[output_name] = set() @@ -562,7 +582,9 @@ def has_seen_output(self, output_name: str, mapping_key: Optional[str] = None) - return output_name in self._execution_properties.seen_outputs def asset_partitions_time_window_for_output(self, output_name: str = "result") -> TimeWindow: - self._check_bound(fn_name="asset_partitions_time_window_for_output", fn_type="method") + self._check_bound_to_invocation( + fn_name="asset_partitions_time_window_for_output", fn_type="method" + ) partitions_def = self.assets_def.partitions_def if partitions_def is None: check.failed("Tried to access partition_key for a non-partitioned asset") @@ -611,7 +633,7 @@ def add_metadata_two_outputs(context) -> Tuple[str, int]: return ("dog", 5) """ - self._check_bound(fn_name="add_output_metadata", fn_type="method") + self._check_bound_to_invocation(fn_name="add_output_metadata", fn_type="method") metadata = check.mapping_param(metadata, "metadata", key_type=str) output_name = check.opt_str_param(output_name, "output_name") mapping_key = check.opt_str_param(mapping_key, "mapping_key") @@ -668,16 +690,18 @@ def add_metadata_two_outputs(context) -> Tuple[str, int]: # allowed. @property def requires_typed_event_stream(self) -> bool: - self._check_bound(fn_name="requires_typed_event_stream", fn_type="property") + self._check_bound_to_invocation(fn_name="requires_typed_event_stream", fn_type="property") return self._execution_properties.requires_typed_event_stream @property def typed_event_stream_error_message(self) -> Optional[str]: - self._check_bound(fn_name="typed_event_stream_error_message", fn_type="property") + self._check_bound_to_invocation( + fn_name="typed_event_stream_error_message", fn_type="property" + ) return self._execution_properties.typed_event_stream_error_message def set_requires_typed_event_stream(self, *, error_message: Optional[str]) -> None: - self._check_bound(fn_name="set_requires_typed_event_stream", fn_type="method") + self._check_bound_to_invocation(fn_name="set_requires_typed_event_stream", fn_type="method") self._execution_properties.requires_typed_event_stream = True self._execution_properties.typed_event_stream_error_message = error_message