From 3ca5a8293b33c65753bf47fd9658feee9c3446cf Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Fri, 8 Dec 2023 14:31:13 -0500 Subject: [PATCH] add run prop test --- .../dagster/_core/execution/context/invocation.py | 15 +++++++++++---- .../core_tests/test_op_invocation.py | 14 ++++++++++++++ 2 files changed, 25 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 4ce9843eacdc5..6481b5ee239a8 100644 --- a/python_modules/dagster/dagster/_core/execution/context/invocation.py +++ b/python_modules/dagster/dagster/_core/execution/context/invocation.py @@ -256,6 +256,12 @@ def set_requires_typed_event_stream(self, *, error_message: Optional[str]) -> No self._typed_event_stream_error_message = error_message +class RunlessRunProperties(RunProperties): + @property + def dagster_run(self): + raise DagsterInvalidPropertyError(_property_msg("dagster_run", "property")) + + class RunlessOpExecutionContext(OpExecutionContext, BaseRunlessContext): """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. @@ -829,7 +835,7 @@ 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, + self.run_properties.run_id, self.log, ScopedResourcesBuilder(resources._asdict()), dagster_type, @@ -842,11 +848,12 @@ def observe_output(self, output_name: str, mapping_key: Optional[str] = None) -> def run_properties(self) -> RunProperties: self._check_bound(fn_name="run_properties", fn_type="property") if self._run_props is None: - self._run_props = RunProperties( + self._run_props = RunlessRunProperties( run_id=self.op_execution_context.run_id, run_config=self.op_execution_context.run_config, - dagster_run=self.op_execution_context.run, - retry_number=self.op_execution_context.retry_number, + # pass None for dagster_run, since RunlessRunProperties raises an exception for this attr + dagster_run=None, # type: ignore + retry_number=0, ) return self._run_props 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 2adfe12337e36..1a231a6b960c4 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 @@ -1589,3 +1589,17 @@ async def get_results(): asyncio.run(get_results()) assert_context_unbound(ctx) + + +def test_run_properties_access(): + @asset + def access_run_properties(context: AssetExecutionContext): + assert context.run_properties.run_id == "EPHEMERAL" + assert context.run_properties.retry_number == 0 + + with pytest.raises(DagsterInvalidPropertyError): + context.run_properties.dagster_run # noqa:B018 + + ctx = build_asset_context() + + access_run_properties(ctx)