Skip to content

Commit e427f2f

Browse files
committed
feedback
1 parent 505f991 commit e427f2f

File tree

2 files changed

+63
-0
lines changed

2 files changed

+63
-0
lines changed

python_modules/dagster/dagster/_core/definitions/asset_selection.py

Lines changed: 47 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -204,6 +204,30 @@ def key_prefixes(
204204
include_sources=include_sources,
205205
)
206206

207+
@staticmethod
208+
@beta_param(param="include_sources")
209+
def key_substring(
210+
key_substring: str, include_sources: bool = False
211+
) -> "KeySubstringAssetSelection":
212+
"""Returns a selection that includes assets whose string representation contains the provided substring and all the asset checks that target it.
213+
214+
Args:
215+
include_sources (bool): If True, then include external assets matching the substring
216+
in the selection.
217+
218+
Examples:
219+
.. code-block:: python
220+
# match any asset key containing "bc"
221+
# e.g. AssetKey(["a", "bcd"]) would match, but not AssetKey(["ab", "cd"]).
222+
AssetSelection.key_substring("bc")
223+
# match any asset key containing "b/c"
224+
# e.g. AssetKey(["ab", "cd"]) would match.
225+
AssetSelection.key_substring("b/c")
226+
"""
227+
return KeySubstringAssetSelection(
228+
selected_key_substring=key_substring, include_sources=include_sources
229+
)
230+
207231
@public
208232
@staticmethod
209233
@beta_param(param="include_sources")
@@ -1172,6 +1196,29 @@ def to_serializable_asset_selection(self, asset_graph: BaseAssetGraph) -> "Asset
11721196
return self
11731197

11741198

1199+
@whitelist_for_serdes
1200+
@record
1201+
class KeySubstringAssetSelection(AssetSelection):
1202+
selected_key_substring: str
1203+
include_sources: bool
1204+
1205+
def resolve_inner(
1206+
self, asset_graph: BaseAssetGraph, allow_missing: bool
1207+
) -> AbstractSet[AssetKey]:
1208+
base_set = (
1209+
asset_graph.get_all_asset_keys()
1210+
if self.include_sources
1211+
else asset_graph.materializable_asset_keys
1212+
)
1213+
return {key for key in base_set if self.selected_key_substring in key.to_user_string()}
1214+
1215+
def to_serializable_asset_selection(self, asset_graph: BaseAssetGraph) -> "AssetSelection":
1216+
return self
1217+
1218+
def to_selection_str(self) -> str:
1219+
return f'key_substring:"{self.selected_key_substring}"'
1220+
1221+
11751222
@whitelist_for_serdes
11761223
@record
11771224
class KeyWildCardAssetSelection(AssetSelection):

python_modules/dagster/dagster_tests/asset_defs_tests/test_asset_selection.py

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -211,6 +211,22 @@ def test_asset_selection_key_prefixes(all_assets: _AssetList):
211211
assert sel.resolve(all_assets) == {earth.key}
212212

213213

214+
def test_asset_selection_key_substring(all_assets: _AssetList):
215+
sel = AssetSelection.key_substring("alice")
216+
assert sel.resolve(all_assets) == _asset_keys_of({alice})
217+
218+
sel = AssetSelection.key_substring("ls/ze")
219+
assert sel.resolve(all_assets) == _asset_keys_of({zebra})
220+
221+
# does not include source assets by default
222+
sel = AssetSelection.key_substring("celestial")
223+
assert sel.resolve(all_assets) == set()
224+
225+
# includes source assets if flag set
226+
sel = AssetSelection.key_substring("celestial/e", include_sources=True)
227+
assert sel.resolve(all_assets) == {earth.key}
228+
229+
214230
def test_select_source_asset_keys():
215231
a = SourceAsset("a")
216232
selection = AssetSelection.keys(a.key)

0 commit comments

Comments
 (0)