Skip to content

Commit

Permalink
rename .partial
Browse files Browse the repository at this point in the history
  • Loading branch information
benpankow committed Jan 20, 2023
1 parent c2c6500 commit 9e595b2
Show file tree
Hide file tree
Showing 3 changed files with 4 additions and 6 deletions.
6 changes: 2 additions & 4 deletions python_modules/dagster/dagster/_config/structured_config.py
Original file line number Diff line number Diff line change
Expand Up @@ -159,7 +159,7 @@ def create_object_to_pass_to_user_code(self, context) -> Any: # pylint: disable
return cast(T, self)

@classmethod
def partial(cls, **kwargs) -> "PartialResource[Self]":
def configure_at_launch(cls, **kwargs) -> "PartialResource[Self]":
"""
Returns a partially initialized copy of the resource, with remaining config fields
set at runtime.
Expand All @@ -184,8 +184,6 @@ class PartialResource(Generic[T], ResourceDefinition, MakeConfigCacheable):
def __init__(self, resource_cls: Type[Resource[T]], data: Dict[str, Any]):
check.invariant(data == {}, "PartialResource currently does not support config fields")

Config.__init__(self, data=data, resource_cls=resource_cls)

MakeConfigCacheable.__init__(self, data=data, resource_cls=resource_cls)

schema = infer_schema_from_config_class(
Expand Down Expand Up @@ -277,7 +275,7 @@ def create_io_manager_to_pass_to_user_code(self, context) -> IOManager:
raise NotImplementedError()

@classmethod
def partial(cls, **kwargs) -> "PartialIOManager":
def configure_at_launch(cls, **kwargs) -> "PartialIOManager":
"""
Returns a partially initialized copy of the resource, with remaining config fields
set at runtime.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -331,7 +331,7 @@ def hello_world_asset(writer: WriterResource):

defs = Definitions(
assets=[hello_world_asset],
resources={"writer": WriterResource.partial()},
resources={"writer": WriterResource.configure_at_launch()},
)

assert (
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -64,7 +64,7 @@ def hello_world_asset():

defs = Definitions(
assets=[hello_world_asset],
resources={"io_manager": MyIOManager.partial()},
resources={"io_manager": MyIOManager.configure_at_launch()},
)

assert (
Expand Down

0 comments on commit 9e595b2

Please sign in to comment.