Skip to content

Commit

Permalink
[core-api][experimental] exclude_subruns (#26687)
Browse files Browse the repository at this point in the history
## Summary & Motivation

decision: experimental -> public
reason: we rely on this functionality fairly heavily internally, making
it unlikely that we could change it even if we wanted to (beyond a name
change, which is easy enough to do in a non-breaking way in the future)
docs exist: yes (api only, which is ok)

## How I Tested These Changes

## Changelog

> Insert changelog entry or delete this section.
  • Loading branch information
OwenKephart authored Dec 31, 2024
1 parent 734f308 commit 6c7e648
Show file tree
Hide file tree
Showing 14 changed files with 17 additions and 36 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,7 @@
from typing import Any, Dict, Iterable, Optional, Sequence, Union, cast

from dagster import _check as check
from dagster._annotations import experimental
from dagster._annotations import beta
from dagster._core.definitions.asset_check_factories.utils import (
DEADLINE_CRON_PARAM_KEY,
DEFAULT_FRESHNESS_SEVERITY,
Expand Down Expand Up @@ -40,7 +40,7 @@
)


@experimental
@beta
def build_last_update_freshness_checks(
*,
assets: Sequence[Union[CoercibleToAssetKey, AssetsDefinition, SourceAsset]],
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,7 @@
from typing import Iterator, Optional, Sequence, Tuple, Union, cast

from dagster import _check as check
from dagster._annotations import experimental
from dagster._annotations import beta
from dagster._core.definitions.asset_check_factories.utils import (
FRESH_UNTIL_METADATA_KEY,
ensure_no_duplicate_asset_checks,
Expand Down Expand Up @@ -33,7 +33,7 @@
"""


@experimental
@beta
def build_sensor_for_freshness_checks(
*,
freshness_checks: Sequence[AssetChecksDefinition],
Expand Down
Original file line number Diff line number Diff line change
@@ -1,7 +1,7 @@
from typing import Any, Dict, Iterable, Sequence, Union

from dagster import _check as check
from dagster._annotations import experimental
from dagster._annotations import beta
from dagster._core.definitions.asset_check_factories.utils import (
DEADLINE_CRON_PARAM_KEY,
DEFAULT_FRESHNESS_SEVERITY,
Expand Down Expand Up @@ -37,7 +37,7 @@
)


@experimental
@beta
def build_time_partition_freshness_checks(
*,
assets: Sequence[Union[SourceAsset, CoercibleToAssetKey, AssetsDefinition]],
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,7 @@
from typing import Optional, Sequence, Tuple, Union, cast

import dagster._check as check
from dagster._annotations import experimental
from dagster._annotations import beta
from dagster._core.definitions.asset_check_factories.utils import (
assets_to_keys,
build_multi_asset_check,
Expand All @@ -21,7 +21,7 @@
from dagster._core.instance import DagsterInstance


@experimental
@beta
def build_metadata_bounds_checks(
*,
assets: Sequence[Union[CoercibleToAssetKey, AssetsDefinition, SourceAsset]],
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,7 @@

from pydantic import BaseModel

from dagster._annotations import experimental
from dagster._annotations import beta
from dagster._core.definitions.asset_check_factories.utils import build_multi_asset_check
from dagster._core.definitions.asset_check_spec import (
AssetCheckKey,
Expand All @@ -17,7 +17,7 @@
from dagster._core.instance import DagsterInstance


@experimental
@beta
def build_column_schema_change_checks(
*,
assets: Sequence[Union[CoercibleToAssetKey, AssetsDefinition, SourceAsset]],
Expand Down
Original file line number Diff line number Diff line change
@@ -1,12 +1,7 @@
from typing import Any, Mapping, Optional, Sequence, Type, Union

import dagster._check as check
from dagster._annotations import (
experimental_param,
hidden_param,
only_allow_hidden_params_in_kwargs,
public,
)
from dagster._annotations import hidden_param, only_allow_hidden_params_in_kwargs, public
from dagster._core.definitions.asset_dep import AssetDep
from dagster._core.definitions.asset_spec import AssetSpec
from dagster._core.definitions.auto_materialize_policy import AutoMaterializePolicy
Expand Down Expand Up @@ -35,8 +30,6 @@
EMPTY_ASSET_KEY_SENTINEL = AssetKey([])


@experimental_param(param="owners")
@experimental_param(param="tags")
@hidden_param(
param="freshness_policy",
breaking_version="1.10.0",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
import dagster._check as check
from dagster._annotations import (
PublicAttr,
experimental_param,
hidden_param,
only_allow_hidden_params_in_kwargs,
public,
Expand Down Expand Up @@ -96,7 +95,6 @@ def validate_kind_tags(kinds: Optional[AbstractSet[str]]) -> None:
raise DagsterInvalidDefinitionError("Assets can have at most three kinds currently.")


@experimental_param(param="owners")
@hidden_param(
param="freshness_policy",
breaking_version="1.10.0",
Expand Down
1 change: 0 additions & 1 deletion python_modules/dagster/dagster/_core/definitions/assets.py
Original file line number Diff line number Diff line change
Expand Up @@ -119,7 +119,6 @@ class AssetsDefinition(ResourceAddable, IHasInternalInit):
_specs_by_key: Mapping[AssetKey, AssetSpec]
_computation: Optional[AssetGraphComputation]

@experimental_param(param="specs")
@experimental_param(param="execution_type")
def __init__(
self,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -134,7 +134,6 @@ def _validate_hidden_non_argument_dep_param(
@experimental_param(param="resource_defs")
@experimental_param(param="io_manager_def")
@experimental_param(param="backfill_policy")
@experimental_param(param="owners")
@hidden_param(
param="non_argument_deps",
breaking_version="2.0.0",
Expand Down Expand Up @@ -736,7 +735,6 @@ def graph_asset(
) -> Callable[[Callable[..., Any]], AssetsDefinition]: ...


@experimental_param(param="owners")
@hidden_param(
param="freshness_policy",
breaking_version="1.10.0",
Expand Down
Original file line number Diff line number Diff line change
@@ -1,7 +1,7 @@
from typing import List, Sequence

from dagster import _check as check
from dagster._annotations import deprecated, experimental
from dagster._annotations import deprecated
from dagster._core.definitions.asset_spec import (
SYSTEM_METADATA_KEY_AUTO_OBSERVE_INTERVAL_MINUTES,
SYSTEM_METADATA_KEY_IO_MANAGER_KEY,
Expand All @@ -23,7 +23,6 @@ def external_asset_from_spec(spec: AssetSpec) -> AssetsDefinition:


@deprecated(breaking_version="1.9.0", additional_warn_text="Directly use the AssetSpecs instead.")
@experimental
def external_assets_from_specs(specs: Sequence[AssetSpec]) -> List[AssetsDefinition]:
"""Create an external assets definition from a sequence of asset specs.
Expand Down
5 changes: 2 additions & 3 deletions python_modules/dagster/dagster/_core/definitions/input.py
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@
)

import dagster._check as check
from dagster._annotations import PublicAttr, deprecated_param, experimental_param
from dagster._annotations import PublicAttr, deprecated_param, superseded
from dagster._core.definitions.events import AssetKey
from dagster._core.definitions.inference import InferredInputProps
from dagster._core.definitions.metadata import (
Expand Down Expand Up @@ -59,8 +59,7 @@ def _check_default_value(input_name: str, dagster_type: DagsterType, default_val
return default_value # type: ignore # (pyright bug)


@experimental_param(param="asset_key")
@experimental_param(param="asset_partitions")
@superseded(additional_warn_text="Use `In` instead", emit_runtime_warning=False)
class InputDefinition:
"""Defines an argument to an op's compute function.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
)

import dagster._check as check
from dagster._annotations import PublicAttr, experimental_param
from dagster._annotations import PublicAttr
from dagster._core.definitions.asset_check_evaluation import AssetCheckEvaluation
from dagster._core.definitions.asset_check_spec import AssetCheckKey
from dagster._core.definitions.asset_graph_subset import AssetGraphSubset
Expand Down Expand Up @@ -354,9 +354,6 @@ def __new__(
)


@experimental_param(
param="asset_events", additional_warn_text="Runless asset events are experimental"
)
class SensorResult(
NamedTuple(
"_SensorResult",
Expand Down
3 changes: 1 addition & 2 deletions python_modules/dagster/dagster/_core/instance/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@
from typing_extensions import Protocol, Self, TypeAlias, TypeVar, runtime_checkable

import dagster._check as check
from dagster._annotations import deprecated, experimental, public
from dagster._annotations import deprecated, public
from dagster._core.definitions.asset_check_evaluation import (
AssetCheckEvaluation,
AssetCheckEvaluationPlanned,
Expand Down Expand Up @@ -3261,7 +3261,6 @@ def get_latest_materialization_code_versions(

return result

@experimental
@public
def report_runless_asset_event(
self,
Expand Down
3 changes: 1 addition & 2 deletions python_modules/dagster/dagster/_core/storage/dagster_run.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@
from typing_extensions import Self

import dagster._check as check
from dagster._annotations import PublicAttr, experimental_param, public
from dagster._annotations import PublicAttr, public
from dagster._core.definitions.asset_check_spec import AssetCheckKey
from dagster._core.definitions.events import AssetKey
from dagster._core.loader import LoadableBy, LoadingContext
Expand Down Expand Up @@ -615,7 +615,6 @@ class RunsFilter(IHaveNew):
created_before: Optional[datetime]
exclude_subruns: Optional[bool]

@experimental_param(param="exclude_subruns")
def __new__(
cls,
run_ids: Optional[Sequence[str]] = None,
Expand Down

0 comments on commit 6c7e648

Please sign in to comment.