diff --git a/python_modules/dagster-graphql/dagster_graphql/implementation/asset_checks_loader.py b/python_modules/dagster-graphql/dagster_graphql/implementation/asset_checks_loader.py index dd5abdfb1ed6c..44e12c2b72a13 100644 --- a/python_modules/dagster-graphql/dagster_graphql/implementation/asset_checks_loader.py +++ b/python_modules/dagster-graphql/dagster_graphql/implementation/asset_checks_loader.py @@ -13,6 +13,7 @@ AssetCheckExecutionResolvedStatus, AssetCheckInstanceSupport, ) +from dagster._core.storage.event_log.base import AssetRecord from dagster._core.workspace.context import WorkspaceRequestContext from packaging import version @@ -153,7 +154,7 @@ def get_checks_for_asset( def _execution_targets_latest_materialization( instance: DagsterInstance, - check_key: AssetCheckKey, + asset_record: Optional[AssetRecord], execution: AssetCheckExecutionRecord, resolved_status: AssetCheckExecutionResolvedStatus, ) -> bool: @@ -161,8 +162,9 @@ def _execution_targets_latest_materialization( if resolved_status == AssetCheckExecutionResolvedStatus.IN_PROGRESS: return True - records = instance.get_asset_records([check_key.asset_key]) - latest_materialization = records[0].asset_entry.last_materialization_record if records else None + latest_materialization = ( + asset_record.asset_entry.last_materialization_record if asset_record else None + ) if not latest_materialization: # asset hasn't been materialized yet, so no reason to hide the check @@ -224,15 +226,21 @@ def __init__(self, instance: DagsterInstance, check_keys: List[AssetCheckKey]): def _fetch_executions(self) -> Mapping[AssetCheckKey, Optional[GrapheneAssetCheckExecution]]: from .fetch_asset_checks import get_asset_check_execution_status - latest_executions = ( + latest_executions_by_check_key = ( self._instance.event_log_storage.get_latest_asset_check_execution_by_key( self._check_keys ) ) + asset_records_by_asset_key = { + record.asset_entry.asset_key: record + for record in self._instance.get_asset_records( + list({ck.asset_key for ck in self._check_keys}) + ) + } self._executions = {} for check_key in self._check_keys: - execution = latest_executions.get(check_key) + execution = latest_executions_by_check_key.get(check_key) if not execution: self._executions[check_key] = None else: @@ -240,7 +248,10 @@ def _fetch_executions(self) -> Mapping[AssetCheckKey, Optional[GrapheneAssetChec self._executions[check_key] = ( GrapheneAssetCheckExecution(execution, resolved_status) if _execution_targets_latest_materialization( - self._instance, check_key, execution, resolved_status + instance=self._instance, + asset_record=asset_records_by_asset_key.get(check_key.asset_key), + execution=execution, + resolved_status=resolved_status, ) else None ) 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 44d9e5e7d5306..e98a3e89e8a7b 100644 --- a/python_modules/dagster-graphql/dagster_graphql/schema/asset_checks.py +++ b/python_modules/dagster-graphql/dagster_graphql/schema/asset_checks.py @@ -1,4 +1,4 @@ -from typing import TYPE_CHECKING, List, Optional, Union, cast +from typing import TYPE_CHECKING, Optional, Union, cast import dagster._check as check import graphene