Skip to content

Commit

Permalink
Add tests
Browse files Browse the repository at this point in the history
  • Loading branch information
briantu committed Nov 14, 2024
1 parent e857b20 commit 054f3ad
Show file tree
Hide file tree
Showing 2 changed files with 50 additions and 22 deletions.
43 changes: 22 additions & 21 deletions python_modules/dagster/dagster/_core/definitions/asset_selection.py
Original file line number Diff line number Diff line change
Expand Up @@ -502,27 +502,28 @@ def from_string(cls, string: str) -> "AssetSelection":
try:
return AntlrAssetSelectionParser(string).asset_selection
except:
if string == "*":
return cls.all()

parts = parse_clause(string)
if parts is not None:
key_selection = cls.assets(parts.item_name)
if parts.up_depth and parts.down_depth:
selection = key_selection.upstream(parts.up_depth) | key_selection.downstream(
parts.down_depth
)
elif parts.up_depth:
selection = key_selection.upstream(parts.up_depth)
elif parts.down_depth:
selection = key_selection.downstream(parts.down_depth)
else:
selection = key_selection
return selection

elif string.startswith("tag:"):
tag_str = string[len("tag:") :]
return cls.tag_string(tag_str)
pass
if string == "*":
return cls.all()

parts = parse_clause(string)
if parts is not None:
key_selection = cls.assets(parts.item_name)
if parts.up_depth and parts.down_depth:
selection = key_selection.upstream(parts.up_depth) | key_selection.downstream(
parts.down_depth
)
elif parts.up_depth:
selection = key_selection.upstream(parts.up_depth)
elif parts.down_depth:
selection = key_selection.downstream(parts.down_depth)
else:
selection = key_selection
return selection

elif string.startswith("tag:"):
tag_str = string[len("tag:") :]
return cls.tag_string(tag_str)

check.failed(f"Invalid selection string: {string}")

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@
from dagster._core.definitions.assets import AssetsDefinition
from dagster._core.definitions.base_asset_graph import BaseAssetGraph
from dagster._core.definitions.events import AssetKey
from dagster._core.selector.subset_selector import MAX_NUM
from dagster._serdes import deserialize_value
from dagster._serdes.serdes import _WHITELIST_MAP
from typing_extensions import TypeAlias
Expand Down Expand Up @@ -804,7 +805,33 @@ def test_deserialize_old_all_asset_selection():
assert not new_unserialized_value.include_sources


def test_from_string_tag():
def test_from_string():
assert AssetSelection.from_string("*") == AssetSelection.all(include_sources=True)
assert AssetSelection.from_string("my_asset") == AssetSelection.assets("my_asset")
assert AssetSelection.from_string("*my_asset") == AssetSelection.assets("my_asset").upstream(
depth=MAX_NUM, include_self=True
)
assert AssetSelection.from_string("+my_asset") == AssetSelection.assets("my_asset").upstream(
depth=1, include_self=True
)
assert AssetSelection.from_string("++my_asset") == AssetSelection.assets("my_asset").upstream(
depth=2, include_self=True
)
assert AssetSelection.from_string("my_asset*") == AssetSelection.assets("my_asset").downstream(
depth=MAX_NUM, include_self=True
)
assert AssetSelection.from_string("my_asset+") == AssetSelection.assets("my_asset").downstream(
depth=1, include_self=True
)
assert AssetSelection.from_string("my_asset++") == AssetSelection.assets("my_asset").downstream(
depth=2, include_self=True
)
assert AssetSelection.from_string("+my_asset+") == AssetSelection.assets("my_asset").downstream(
depth=1, include_self=True
) | AssetSelection.assets("my_asset").upstream(depth=1, include_self=True)
assert AssetSelection.from_string("*my_asset*") == AssetSelection.assets("my_asset").downstream(
depth=MAX_NUM, include_self=True
) | AssetSelection.assets("my_asset").upstream(depth=MAX_NUM, include_self=True)
assert AssetSelection.from_string("tag:foo=bar") == AssetSelection.tag("foo", "bar")
assert AssetSelection.from_string("tag:foo") == AssetSelection.tag("foo", "")

Expand Down

0 comments on commit 054f3ad

Please sign in to comment.