From 7581aea6b87701e5b9362c2f6105819ff6cf896f Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Mon, 29 Jan 2024 12:25:30 -0500 Subject: [PATCH] update for rename --- .../_core/execution/context/invocation.py | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/python_modules/dagster/dagster/_core/execution/context/invocation.py b/python_modules/dagster/dagster/_core/execution/context/invocation.py index c479ae751fff3..9254cf5196642 100644 --- a/python_modules/dagster/dagster/_core/execution/context/invocation.py +++ b/python_modules/dagster/dagster/_core/execution/context/invocation.py @@ -83,8 +83,8 @@ def unbind(self): @property @abstractmethod - def bound_properties(self) -> "BoundProperties": - """Subclasses of BaseDirectExecutionContext must contain a BoundProperties object.""" + def per_invocation_properties(self) -> "PerInvocationProperties": + """Subclasses of BaseDirectExecutionContext must contain a PerInvocationProperties object.""" @property @abstractmethod @@ -754,8 +754,6 @@ class DirectAssetExecutionContext(AssetExecutionContext, BaseDirectExecutionCont def __init__(self, op_execution_context: DirectOpExecutionContext): self._op_execution_context = op_execution_context - self._run_props = None - def __enter__(self): self.op_execution_context._cm_scope_entered = True # noqa: SLF001 return self @@ -766,8 +764,8 @@ def __exit__(self, *exc): def __del__(self): self.op_execution_context._exit_stack.close() # noqa: SLF001 - def _check_bound(self, fn_name: str, fn_type: str): - if not self._op_execution_context._bound_properties: # noqa: SLF001 + def _check_bound_to_invocation(self, fn_name: str, fn_type: str): + if not self._op_execution_context._per_invocation_properties: # noqa: SLF001 raise DagsterInvalidPropertyError(_property_msg(fn_name, fn_type)) def bind( @@ -782,7 +780,7 @@ def bind( raise DagsterInvariantViolationError( "DirectAssetExecutionContext can only being used to invoke an asset." ) - if self._op_execution_context._bound_properties is not None: # noqa: SLF001 + if self._op_execution_context._per_invocation_properties is not None: # noqa: SLF001 raise DagsterInvalidInvocationError( f"This context is currently being used to execute {self.op_execution_context.alias}." " The context cannot be used to execute another asset until" @@ -803,8 +801,8 @@ def unbind(self): self._op_execution_context.unbind() @property - def bound_properties(self) -> BoundProperties: - return self.op_execution_context.bound_properties + def per_invocation_properties(self) -> PerInvocationProperties: + return self.op_execution_context.per_invocation_properties @property def is_bound(self) -> bool: