diff --git a/python_modules/dagster-graphql/dagster_graphql/schema/asset_checks.py b/python_modules/dagster-graphql/dagster_graphql/schema/asset_checks.py index 20ae2de0a6b30..fef57e244d621 100644 --- a/python_modules/dagster-graphql/dagster_graphql/schema/asset_checks.py +++ b/python_modules/dagster-graphql/dagster_graphql/schema/asset_checks.py @@ -158,7 +158,7 @@ async def resolve_executionForLatestMaterialization( record = await AssetCheckExecutionRecord.gen(graphene_info.context, self._asset_check.key) return ( GrapheneAssetCheckExecution(record) - if record and record.targets_latest_materialization(graphene_info.context) + if record and await record.targets_latest_materialization(graphene_info.context) else None ) diff --git a/python_modules/dagster/dagster/_core/asset_graph_view/asset_graph_view.py b/python_modules/dagster/dagster/_core/asset_graph_view/asset_graph_view.py index 9f0e7fc91e15d..3807af1834df3 100644 --- a/python_modules/dagster/dagster/_core/asset_graph_view/asset_graph_view.py +++ b/python_modules/dagster/dagster/_core/asset_graph_view/asset_graph_view.py @@ -367,7 +367,7 @@ async def compute_subset_with_status( latest_record = summary.last_check_execution_record if summary else None resolved_status = ( latest_record.resolve_status(self) - if latest_record and latest_record.targets_latest_materialization(self) + if latest_record and await latest_record.targets_latest_materialization(self) else None ) if resolved_status == status: diff --git a/python_modules/dagster/dagster/_core/storage/asset_check_execution_record.py b/python_modules/dagster/dagster/_core/storage/asset_check_execution_record.py index 0424d2ddb1b56..7d7e94496d20e 100644 --- a/python_modules/dagster/dagster/_core/storage/asset_check_execution_record.py +++ b/python_modules/dagster/dagster/_core/storage/asset_check_execution_record.py @@ -156,7 +156,7 @@ def resolve_status(self, loading_context: LoadingContext) -> AssetCheckExecution else: check.failed(f"Unexpected status {self.status}") - def targets_latest_materialization(self, loading_context: LoadingContext) -> bool: + async def targets_latest_materialization(self, loading_context: LoadingContext) -> bool: from dagster._core.storage.event_log.base import AssetRecord resolved_status = self.resolve_status(loading_context) @@ -164,7 +164,7 @@ def targets_latest_materialization(self, loading_context: LoadingContext) -> boo # all in-progress checks execute against the latest version return True - asset_record = AssetRecord.blocking_get(loading_context, self.key.asset_key) + asset_record = await AssetRecord.gen(loading_context, self.key.asset_key) latest_materialization = ( asset_record.asset_entry.last_materialization_record if asset_record else None ) @@ -197,10 +197,10 @@ def targets_latest_materialization(self, loading_context: LoadingContext) -> boo ]: # the evaluation didn't complete, so we don't have target_materialization_data, so check if # the check's run executed after the materializations as a fallback - latest_materialization_run_record = RunRecord.blocking_get( + latest_materialization_run_record = await RunRecord.gen( loading_context, latest_materialization_run_id ) - check_run_record = RunRecord.blocking_get(loading_context, self.run_id) + check_run_record = await RunRecord.gen(loading_context, self.run_id) return bool( latest_materialization_run_record and check_run_record