diff --git a/python_modules/libraries/dagster-dbt/dagster_dbt/asset_decorator.py b/python_modules/libraries/dagster-dbt/dagster_dbt/asset_decorator.py index 6a2646b05f8a5..11c52d762d293 100644 --- a/python_modules/libraries/dagster-dbt/dagster_dbt/asset_decorator.py +++ b/python_modules/libraries/dagster-dbt/dagster_dbt/asset_decorator.py @@ -20,6 +20,7 @@ DagsterInvalidDefinitionError, Nothing, PartitionsDefinition, + RetryPolicy, multi_asset, ) from dagster._utils.warnings import ( @@ -55,6 +56,7 @@ def dbt_assets( partitions_def: Optional[PartitionsDefinition] = None, dagster_dbt_translator: DagsterDbtTranslator = DagsterDbtTranslator(), backfill_policy: Optional[BackfillPolicy] = None, + retry_policy: Optional[RetryPolicy] = None, op_tags: Optional[Mapping[str, Any]] = None, ) -> Callable[..., AssetsDefinition]: """Create a definition for how to compute a set of dbt resources, described by a manifest.json. @@ -81,6 +83,7 @@ def dbt_assets( dbt models, seeds, etc. to asset keys and asset metadata. backfill_policy (Optional[BackfillPolicy]): If a partitions_def is defined, this determines how to execute backfills that target multiple partitions. + retry_policy (Optional[RetryPolicy]): The retry policy for the underlying asset function. op_tags (Optional[Dict[str, Any]]): A dictionary of tags for the op that computes the assets. Frameworks may expect and require certain metadata to be attached to a op. Values that are not strings will be json encoded and must meet the criteria that @@ -311,6 +314,7 @@ def inner(fn) -> AssetsDefinition: op_tags=resolved_op_tags, check_specs=check_specs, backfill_policy=backfill_policy, + retry_policy=retry_policy, )(fn) return asset_definition diff --git a/python_modules/libraries/dagster-dbt/dagster_dbt_tests/test_asset_decorator.py b/python_modules/libraries/dagster-dbt/dagster_dbt_tests/test_asset_decorator.py index 0944afde1cbb8..5e95d168ce4ff 100644 --- a/python_modules/libraries/dagster-dbt/dagster_dbt_tests/test_asset_decorator.py +++ b/python_modules/libraries/dagster-dbt/dagster_dbt_tests/test_asset_decorator.py @@ -17,6 +17,7 @@ NodeInvocation, PartitionMapping, PartitionsDefinition, + RetryPolicy, TimeWindowPartitionMapping, asset, ) @@ -231,6 +232,19 @@ def my_dbt_assets(): assert my_dbt_assets.backfill_policy == backfill_policy +def test_retry_policy(): + retry_policy = RetryPolicy(max_retries=2) + + @dbt_assets( + manifest=manifest, + retry_policy=retry_policy, + ) + def my_dbt_assets(): + ... + + assert my_dbt_assets.op.retry_policy == retry_policy + + def test_op_tags(): @dbt_assets(manifest=manifest, op_tags={"a": "b", "c": "d"}) def my_dbt_assets():