Skip to content

Commit

Permalink
AssetGroup.from_current_module (#6959)
Browse files Browse the repository at this point in the history
  • Loading branch information
sryza committed Mar 7, 2022
1 parent 803063e commit 4262be1
Show file tree
Hide file tree
Showing 2 changed files with 43 additions and 0 deletions.
25 changes: 25 additions & 0 deletions python_modules/dagster/dagster/core/asset_defs/asset_group.py
Original file line number Diff line number Diff line change
@@ -1,3 +1,4 @@
import inspect
import os
import pkgutil
import re
Expand Down Expand Up @@ -440,6 +441,30 @@ def from_modules(
executor_def=executor_def,
)

@staticmethod
def from_current_module(
resource_defs: Optional[Mapping[str, ResourceDefinition]] = None,
executor_def: Optional[ExecutorDefinition] = None,
) -> "AssetGroup":
"""
Constructs an AssetGroup that includes all asset definitions and source assets in the module
where this is called from.
Args:
resource_defs (Optional[Mapping[str, ResourceDefinition]]): A dictionary of resource
definitions to include on the returned asset group.
executor_def (Optional[ExecutorDefinition]): An executor to include on the returned
asset group.
Returns:
AssetGroup: An asset group with all the assets defined in the module.
"""
caller = inspect.stack()[1]
module = inspect.getmodule(caller[0])
if module is None:
check.failed("Could not find a module for the caller")
return AssetGroup.from_modules([module], resource_defs, executor_def)


def _find_assets_in_module(
module: ModuleType,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -378,6 +378,24 @@ def test_asset_group_from_modules():
}


@asset
def asset_in_current_module():
pass


source_asset_in_current_module = SourceAsset(AssetKey("source_asset_in_current_module"))


def test_asset_group_from_current_module():
group = AssetGroup.from_current_module()
assert {asset.op.name for asset in group.assets} == {"asset_in_current_module"}
assert len(group.assets) == 1
assert {source_asset.key for source_asset in group.source_assets} == {
AssetKey("source_asset_in_current_module")
}
assert len(group.source_assets) == 1


def test_default_io_manager():
@asset
def asset_foo():
Expand Down

0 comments on commit 4262be1

Please sign in to comment.