diff --git a/python_modules/dagster/dagster/_core/execution/context/compute.py b/python_modules/dagster/dagster/_core/execution/context/compute.py index ae8cad1f8ac60..f7e4f78003631 100644 --- a/python_modules/dagster/dagster/_core/execution/context/compute.py +++ b/python_modules/dagster/dagster/_core/execution/context/compute.py @@ -1468,10 +1468,9 @@ def __init__(self, op_execution_context: OpExecutionContext) -> None: retry_number=self._op_execution_context.retry_number, ) - self._execution_props = ExecutionProperties( - step_description=f"asset {self.assets_def.node_def.name}", - op_execution_context=self._op_execution_context, - ) + # start execution_props as None since enter_execution_context builds an AssetExecutionContext + # for all steps (including ops) and ops will fail on self.assets_def call + self._execution_props = None @staticmethod def get() -> "AssetExecutionContext": @@ -1490,6 +1489,11 @@ def run_properties(self) -> RunProperties: @property def execution_properties(self) -> ExecutionProperties: + if self._execution_props is None: + self._execution_props = ExecutionProperties( + step_description=f"asset {self.assets_def.node_def.name}", + op_execution_context=self._op_execution_context, + ) return self._execution_props ######## Deprecated methods