Skip to content

Commit

Permalink
feat(dbt): suppress experimental warning when invoking @dbt_assets (d…
Browse files Browse the repository at this point in the history
…agster-io#22528)

## Summary & Motivation
When using `@dbt_assets`, I get a bunch of errors like:

```
2024-06-13 11:06:20 -0400 - dagster - WARNING - /Users/rexledesma/dagster-labs/dagster/python_modules/dagster/dagster/_core/decorator_utils.py:203: ExperimentalWarning: Parameter `tags` of initializer `AssetOut.__init__` is experimental. It may break in future versions, even between dot releases. To mute warnings for experimental functionality, invoke warnings.filterwarnings("ignore", category=dagster.ExperimentalWarning) or use one of the other methods described at https://docs.python.org/3/library/warnings.html#describing-warning-filters.
  return fn(*args, **kwargs)

2024-06-13 11:06:20 -0400 - dagster - WARNING - /Users/rexledesma/dagster-labs/dagster/python_modules/dagster/dagster/_core/decorator_utils.py:203: ExperimentalWarning: Parameter `tags` of initializer `AssetOut.__init__` is experimental. It may break in future versions, even between dot releases. To mute warnings for experimental functionality, invoke warnings.filterwarnings("ignore", category=dagster.ExperimentalWarning) or use one of the other methods described at https://docs.python.org/3/library/warnings.html#describing-warning-filters.
  return fn(*args, **kwargs)

2024-06-13 11:06:20 -0400 - dagster - WARNING - /Users/rexledesma/dagster-labs/dagster/python_modules/dagster/dagster/_core/decorator_utils.py:203: ExperimentalWarning: Parameter `tags` of initializer `AssetOut.__init__` is experimental. It may break in future versions, even between dot releases. To mute warnings for experimental functionality, invoke warnings.filterwarnings("ignore", category=dagster.ExperimentalWarning) or use one of the other methods described at https://docs.python.org/3/library/warnings.html#describing-warning-filters.
  return fn(*args, **kwargs)
```

This does not inspire confidence when I am using these APIs. My thought
process on this is:

1. If we use experimental APIs on behalf of the user in the default
case, they should not be shown.
2. Experimental warnings should only be shown once per call site, not
once per invocation.
3. If an entrypoint for an integration is experimental (e.g.
`DbtProject`), an experimental warning should be shown once, at
instantiation of the experimental entrypoint. If the entrypoint itself
is implemented using experimental APIs, those subsequent warnings should
be suppressed (i.e. we should follow (1)).

In this PR, we follow (1) for `@dbt_assets`.

## How I Tested These Changes
pytest
  • Loading branch information
rexledesma authored and danielgafni committed Jun 18, 2024
1 parent 053f07f commit 8884afc
Show file tree
Hide file tree
Showing 3 changed files with 7 additions and 8 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@
LocalFileCodeReference,
)
from dagster._core.definitions.tags import StorageKindTagSet
from dagster._utils.warnings import experimental_warning
from dagster._utils.warnings import suppress_dagster_warnings

from dagster_dbt.dbt_project import DbtProject

Expand Down Expand Up @@ -54,6 +54,7 @@
)


@suppress_dagster_warnings
def dbt_assets(
*,
manifest: DbtManifestParam,
Expand Down Expand Up @@ -550,9 +551,6 @@ def get_dbt_multi_asset_args(
parent_unique_ids_for_asset_key.add(parent_unique_id)
parent_resource_types_for_asset_key.add(dbt_parent_resource_props["resource_type"])

if parent_partition_mapping:
experimental_warning("DagsterDbtTranslator.get_partition_mapping")

# Add this parent as an internal dependency
output_internal_deps.add(parent_asset_key)

Expand All @@ -570,8 +568,6 @@ def get_dbt_multi_asset_args(
dbt_parent_resource_props=dbt_resource_props,
)
if self_partition_mapping and has_self_dependency(dbt_resource_props):
experimental_warning("+meta.dagster.has_self_dependency")

deps.add(
AssetDep(
asset=asset_key,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -364,7 +364,7 @@ def to_default_asset_events(
"This test was included in Dagster's asset check"
" selection, and was likely executed due to dbt indirect selection."
)
logger.warn(message)
logger.warning(message)

yield from self._yield_observation_events_for_test(
dagster_dbt_translator=dagster_dbt_translator,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,7 @@
PartitionMapping,
_check as check,
)
from dagster._annotations import public
from dagster._annotations import experimental, public
from dagster._core.definitions.asset_key import (
CoercibleToAssetKeyPrefix,
check_opt_coercible_to_asset_key_prefix_param,
Expand Down Expand Up @@ -125,6 +125,7 @@ def get_asset_key(self, dbt_resource_props: Mapping[str, Any]) -> AssetKey:
return default_asset_key_fn(dbt_resource_props)

@public
@experimental(emit_runtime_warning=False)
def get_partition_mapping(
self,
dbt_resource_props: Mapping[str, Any],
Expand Down Expand Up @@ -323,6 +324,7 @@ def get_owners(self, dbt_resource_props: Mapping[str, Any]) -> Optional[Sequence
return default_owners_from_dbt_resource_props(dbt_resource_props)

@public
@experimental(emit_runtime_warning=False)
def get_freshness_policy(
self, dbt_resource_props: Mapping[str, Any]
) -> Optional[FreshnessPolicy]:
Expand Down Expand Up @@ -376,6 +378,7 @@ def get_freshness_policy(self, dbt_resource_props: Mapping[str, Any]) -> Optiona
return default_freshness_policy_fn(dbt_resource_props)

@public
@experimental(emit_runtime_warning=False)
def get_auto_materialize_policy(
self, dbt_resource_props: Mapping[str, Any]
) -> Optional[AutoMaterializePolicy]:
Expand Down

0 comments on commit 8884afc

Please sign in to comment.