Skip to content

Commit

Permalink
api docs
Browse files Browse the repository at this point in the history
  • Loading branch information
johannkm committed Sep 19, 2023
1 parent 0ae578b commit 1ccc7b2
Show file tree
Hide file tree
Showing 5 changed files with 31 additions and 27 deletions.
2 changes: 1 addition & 1 deletion docs/content/api/modules.json

Large diffs are not rendered by default.

2 changes: 1 addition & 1 deletion docs/content/api/searchindex.json

Large diffs are not rendered by default.

2 changes: 1 addition & 1 deletion docs/content/api/sections.json

Large diffs are not rendered by default.

Binary file modified docs/next/public/objects.inv
Binary file not shown.
52 changes: 28 additions & 24 deletions python_modules/dagster/dagster/_core/definitions/asset_selection.py
Original file line number Diff line number Diff line change
Expand Up @@ -38,13 +38,14 @@


class AssetSelection(ABC):
"""An AssetSelection defines a query over a set of assets, normally all the assets in a code location.
"""An AssetSelection defines a query over a set of assets and asset checks, normally all that are defined in a code location.
You can use the "|", "&", and "-" operators to create unions, intersections, and differences of
asset selections, respectively.
You can use the "|", "&", and "-" operators to create unions, intersections, and differences of selections, respectively.
AssetSelections are typically used with :py:func:`define_asset_job`.
By default, selecting assets will also select all of the asset checks that target those assets.
Examples:
.. code-block:: python
Expand All @@ -57,14 +58,24 @@ class AssetSelection(ABC):
# Select all assets in group "marketing" that are downstream of asset "leads":
AssetSelection.groups("marketing") & AssetSelection.keys("leads").downstream()
# Select all assets in a list of assets:
# Select a list of assets:
AssetSelection.assets(*my_assets_list)
# Select all assets except for those in group "marketing"
AssetSelection.all() - AssetSelection.groups("marketing")
# Select all assets which are materialized by the same op as "projections":
AssetSelection.keys("projections").required_multi_asset_neighbors()
# Select all assets in group "marketing" and exclude their asset checks:
AssetSelection.groups("marketing") - AssetSelection.all_asset_checks()
# Select all asset checks that target a list of assets:
AssetSelection.asset_checks_for_assets(*my_assets_list)
# Select a specific asset check:
AssetSelection.asset_checks(my_asset_check)
"""

@public
Expand All @@ -82,13 +93,13 @@ def all_asset_checks() -> "AllAssetCheckSelection":
@public
@staticmethod
def assets(*assets_defs: AssetsDefinition) -> "KeysAssetSelection":
"""Returns a selection that includes all of the provided assets."""
"""Returns a selection that includes all of the provided assets and asset checks that target them."""
return KeysAssetSelection(*(key for assets_def in assets_defs for key in assets_def.keys))

@public
@staticmethod
def keys(*asset_keys: CoercibleToAssetKey) -> "KeysAssetSelection":
"""Returns a selection that includes assets with any of the provided keys.
"""Returns a selection that includes assets with any of the provided keys and all asset checks that target them.
Examples:
.. code-block:: python
Expand All @@ -115,7 +126,7 @@ def keys(*asset_keys: CoercibleToAssetKey) -> "KeysAssetSelection":
def key_prefixes(
*key_prefixes: CoercibleToAssetKeyPrefix, include_sources: bool = False
) -> "KeyPrefixesAssetSelection":
"""Returns a selection that includes assets that match any of the provided key prefixes.
"""Returns a selection that includes assets that match any of the provided key prefixes and all the asset checks that target them.
Args:
include_sources (bool): If True, then include source assets matching the key prefix(es)
Expand All @@ -138,7 +149,7 @@ def key_prefixes(
@staticmethod
def groups(*group_strs, include_sources: bool = False) -> "GroupsAssetSelection":
"""Returns a selection that includes materializable assets that belong to any of the
provided groups.
provided groups and all the asset checks that target them.
Args:
include_sources (bool): If True, then include source assets matching the group in the
Expand All @@ -152,7 +163,7 @@ def downstream(
self, depth: Optional[int] = None, include_self: bool = True
) -> "DownstreamAssetSelection":
"""Returns a selection that includes all assets that are downstream of any of the assets in
this selection, selecting the assets in this selection by default. Iterates through each
this selection, selecting the assets in this selection by default. Includes the asset checks targeting the returned assets. Iterates through each
asset in this selection and returns the union of all downstream assets.
depth (Optional[int]): If provided, then only include assets to the given depth. A depth
Expand All @@ -171,7 +182,7 @@ def upstream(
self, depth: Optional[int] = None, include_self: bool = True
) -> "UpstreamAssetSelection":
"""Returns a selection that includes all materializable assets that are upstream of any of
the assets in this selection, selecting the assets in this selection by default. Iterates
the assets in this selection, selecting the assets in this selection by default. Includes the asset checks targeting the returned assets. Iterates
through each asset in this selection and returns the union of all upstream assets.
Because mixed selections of source and materializable assets are currently not supported,
Expand All @@ -192,7 +203,7 @@ def upstream(
@public
def sinks(self) -> "SinkAssetSelection":
"""Given an asset selection, returns a new asset selection that contains all of the sink
assets within the original asset selection.
assets within the original asset selection. Includes the asset checks targeting the returned assets.
A sink asset is an asset that has no downstream dependencies within the asset selection.
The sink asset can have downstream dependencies outside of the asset selection.
Expand All @@ -203,14 +214,14 @@ def sinks(self) -> "SinkAssetSelection":
def required_multi_asset_neighbors(self) -> "RequiredNeighborsAssetSelection":
"""Given an asset selection in which some assets are output from a multi-asset compute op
which cannot be subset, returns a new asset selection that contains all of the assets
required to execute the original asset selection.
required to execute the original asset selection. Includes the asset checks targeting the returned assets.
"""
return RequiredNeighborsAssetSelection(self)

@public
def roots(self) -> "RootAssetSelection":
"""Given an asset selection, returns a new asset selection that contains all of the root
assets within the original asset selection.
assets within the original asset selection. Includes the asset checks targeting the returned assets.
A root asset is an asset that has no upstream dependencies within the asset selection.
The root asset can have downstream dependencies outside of the asset selection.
Expand All @@ -225,7 +236,7 @@ def roots(self) -> "RootAssetSelection":
@deprecated(breaking_version="2.0", additional_warn_text="Use AssetSelection.roots instead.")
def sources(self) -> "RootAssetSelection":
"""Given an asset selection, returns a new asset selection that contains all of the root
assets within the original asset selection.
assets within the original asset selection. Includes the asset checks targeting the returned assets.
A root asset is a materializable asset that has no upstream dependencies within the asset
selection. The root asset can have downstream dependencies outside of the asset selection.
Expand All @@ -239,22 +250,22 @@ def sources(self) -> "RootAssetSelection":
@public
def upstream_source_assets(self) -> "SourceAssetSelection":
"""Given an asset selection, returns a new asset selection that contains all of the source
assets upstream of assets in the original selection.
assets upstream of assets in the original selection. Includes the asset checks targeting the returned assets.
"""
return SourceAssetSelection(self)

@public
@staticmethod
def asset_checks_for_assets(*assets_defs: AssetsDefinition) -> "KeysAssetSelection":
"""Returns a selection that includes all of the provided assets."""
"""Returns a selection with the asset checks that target the provided assets."""
return AssetChecksForAssetKeys(
[key for assets_def in assets_defs for key in assets_def.keys]
)

@public
@staticmethod
def asset_checks(*asset_checks: AssetChecksDefinition) -> "KeysAssetSelection":
"""Returns a selection that includes all of the provided assets."""
"""Returns a selection that includes all of the provided asset checks."""
return AssetChecksForHandles(
[
AssetCheckHandle(asset_key=AssetKey.from_coercible(spec.asset_key), name=spec.name)
Expand Down Expand Up @@ -369,13 +380,6 @@ def resolve_inner(self, asset_graph: AssetGraph) -> AbstractSet[AssetKey]:
return asset_graph.materializable_asset_keys


class AllAssetSelection(AssetSelection):
def resolve_inner(self, asset_graph: AssetGraph) -> AbstractSet[AssetKey]:
return asset_graph.materializable_asset_keys

def resolve_checks_inner(self, asset_graph: AssetGraph) -> AbstractSet[AssetCheckHandle]:
return set()


class AllAssetCheckSelection(AssetSelection):
def resolve_inner(self, asset_graph: AssetGraph) -> AbstractSet[AssetKey]:
Expand Down

0 comments on commit 1ccc7b2

Please sign in to comment.