Skip to content

Commit a04cf72

Browse files
[dagster-dbt] Support asset checks in dbt_cloud_assets
1 parent 004e3cc commit a04cf72

File tree

3 files changed

+31
-3
lines changed

3 files changed

+31
-3
lines changed

python_modules/libraries/dagster-dbt/dagster_dbt/cloud_v2/asset_decorator.py

+5
Original file line numberDiff line numberDiff line change
@@ -53,4 +53,9 @@ def dbt_cloud_assets(
5353
exclude=exclude,
5454
),
5555
op_tags=op_tags,
56+
check_specs=workspace.load_check_specs(
57+
dagster_dbt_translator=dagster_dbt_translator,
58+
select=select,
59+
exclude=exclude,
60+
)
5661
)

python_modules/libraries/dagster-dbt/dagster_dbt/cloud_v2/run_handler.py

+20-1
Original file line numberDiff line numberDiff line change
@@ -8,6 +8,7 @@
88
AssetMaterialization,
99
MetadataValue,
1010
Output,
11+
AssetCheckResult,
1112
)
1213
from dagster._annotations import preview
1314
from dagster._record import record
@@ -195,7 +196,25 @@ def to_default_asset_events(
195196
test_unique_id=unique_id,
196197
)
197198

198-
if asset_check_key is not None:
199+
if (
200+
context
201+
and has_asset_def
202+
and asset_check_key is not None
203+
and asset_check_key in context.selected_asset_check_keys
204+
):
205+
# The test is an asset check in an asset, so yield an `AssetCheckResult`.
206+
yield AssetCheckResult(
207+
passed=result_status == TestStatus.Pass,
208+
asset_key=asset_check_key.asset_key,
209+
check_name=asset_check_key.name,
210+
metadata=metadata,
211+
severity=(
212+
AssetCheckSeverity.WARN
213+
if result_status == TestStatus.Warn
214+
else AssetCheckSeverity.ERROR
215+
),
216+
)
217+
elif asset_check_key is not None:
199218
yield AssetCheckEvaluation(
200219
passed=result_status == TestStatus.Pass,
201220
asset_key=asset_check_key.asset_key,

python_modules/libraries/dagster-dbt/dagster_dbt_tests/cloud_v2/test_resources.py

+6-2
Original file line numberDiff line numberDiff line change
@@ -209,9 +209,13 @@ def my_dbt_cloud_assets(context: AssetExecutionContext, dbt_cloud: DbtCloudWorks
209209
asset_materialization_events = result.get_asset_materialization_events()
210210
asset_check_evaluation = result.get_asset_check_evaluations()
211211

212-
# materializations are successful outputs, asset check evaluations are not outputs
212+
# materializations and check results are successful outputs
213213
outputs = [event for event in result.all_events if event.is_successful_output]
214-
assert len(outputs) == 8
214+
assert len(outputs) == 28
215+
216+
# materialization outputs have metadata, asset check outputs don't
217+
outputs_with_metadata = [output for output in outputs if output.step_output_data.metadata]
218+
assert len(outputs_with_metadata) == 8
215219

216220
# 8 asset materializations
217221
assert len(asset_materialization_events) == 8

0 commit comments

Comments
 (0)