Skip to content

Commit

Permalink
[refactor] *ExecutionContext.solid_config -> op_config (#12372)
Browse files Browse the repository at this point in the history
### Summary & Motivation

- Delete
`{OpExecutionContext,DagstermillExecutionContext,DagstermillRuntimeExecutionContext}.solid_config`
(update refs to use `op_config`)

### How I Tested These Changes

BK
  • Loading branch information
smackesey committed Feb 17, 2023
1 parent 82901d3 commit c132dce
Show file tree
Hide file tree
Showing 36 changed files with 121 additions and 137 deletions.
2 changes: 1 addition & 1 deletion examples/deploy_k8s/example_project/example_repo/repo.py
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,7 @@

@op(ins={"word": In(str)}, config_schema={"factor": int})
def multiply_the_word(context, word):
return word * context.solid_config["factor"]
return word * context.op_config["factor"]


@op(ins={"word": In(str)})
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@

@op(config_schema={"fail": bool})
def foo(context):
if context.solid_config["fail"]:
if context.op_config["fail"]:
raise Exception("This will always fail!")


Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@

@solid(input_defs=[InputDefinition("word", str)], config_schema={"factor": int})
def multiply_the_word(context, word):
return word * context.solid_config["factor"]
return word * context.op_config["factor"]


@solid(input_defs=[InputDefinition("word")])
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ def sleeper(context, units):
],
)
def giver(context):
units = context.solid_config
units = context.op_config
queues = [[], [], [], []]
for i, sec in enumerate(units):
queues[i % 4].append(sec)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -216,7 +216,7 @@ def csv_hello_world_solids_config():

@solid(config_schema={"file": Field(String)})
def loop(context):
with open(context.solid_config["file"], "w", encoding="utf8") as ff:
with open(context.op_config["file"], "w", encoding="utf8") as ff:
ff.write("yup")

while True:
Expand Down Expand Up @@ -817,7 +817,7 @@ def hard_failer():
output_defs=[OutputDefinition(Int)],
)
def hard_fail_or_0(context):
if context.solid_config["fail"]:
if context.op_config["fail"]:
segfault()
return 0

Expand Down Expand Up @@ -876,7 +876,7 @@ def retry_resource_pipeline():
],
)
def can_fail(context, inp): # pylint: disable=unused-argument
if context.solid_config["fail"]:
if context.op_config["fail"]:
raise Exception("blah")

yield Output("okay perfect", "start_fail")
Expand Down Expand Up @@ -950,7 +950,7 @@ def return_one():
required_resource_keys={"disable_gc"},
)
def get_input_one(context, one):
if context.solid_config["wait_to_terminate"]:
if context.op_config["wait_to_terminate"]:
while True:
time.sleep(0.1)
return one
Expand All @@ -962,7 +962,7 @@ def get_input_one(context, one):
required_resource_keys={"disable_gc"},
)
def get_input_two(context, one):
if context.solid_config["wait_to_terminate"]:
if context.op_config["wait_to_terminate"]:
while True:
time.sleep(0.1)
return one
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ def math():

@solid(config_schema={"gimme": str})
def needs_config(context):
return context.solid_config["gimme"]
return context.op_config["gimme"]


@lambda_solid
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -111,18 +111,18 @@ def docker_mode_defs():
},
)
def multiply_the_word(context, word):
if context.solid_config.get("should_segfault"):
if context.op_config.get("should_segfault"):
segfault()
return word * context.solid_config["factor"]
return word * context.op_config["factor"]


@solid(
input_defs=[InputDefinition("word", String)],
config_schema={"factor": IntSource, "sleep_time": IntSource},
)
def multiply_the_word_slow(context, word):
time.sleep(context.solid_config["sleep_time"])
return word * context.solid_config["factor"]
time.sleep(context.op_config["sleep_time"])
return word * context.op_config["factor"]


@lambda_solid(input_defs=[InputDefinition("word")])
Expand All @@ -145,7 +145,7 @@ def always_fail(context, word):
config_schema={"factor": IntSource},
)
def multiply_the_word_op(context, word):
return word * context.solid_config["factor"]
return word * context.op_config["factor"]


@op(ins={"word": In()})
Expand All @@ -169,7 +169,7 @@ def hanging_solid(_):

@solid(config_schema={"looking_for": str})
def get_environment_solid(context):
return os.environ.get(context.solid_config["looking_for"])
return os.environ.get(context.op_config["looking_for"])


@pipeline(
Expand Down Expand Up @@ -634,7 +634,7 @@ def hard_failer():
output_defs=[OutputDefinition(Int)],
)
def hard_fail_or_0(context):
if context.solid_config["fail"]:
if context.op_config["fail"]:
segfault()
return 0

Expand Down
2 changes: 1 addition & 1 deletion python_modules/dagster/dagster/_core/definitions/assets.py
Original file line number Diff line number Diff line change
Expand Up @@ -1074,7 +1074,7 @@ def _build_invocation_context_with_included_resources(
# pylint: disable=protected-access
return build_op_context(
resources=all_resources,
config=context.solid_config,
config=context.op_config,
resources_config=context._resources_config,
instance=context._instance,
partition_key=context._partition_key,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -313,8 +313,8 @@ def hello_world(context, foo):
config_schema={'str_value' : Field(str)}
)
def hello_world(context, foo):
# context.solid_config is a dictionary with 'str_value' key
return foo + context.solid_config['str_value']
# context.op_config is a dictionary with 'str_value' key
return foo + context.op_config['str_value']
"""
# This case is for when decorator is used bare, without arguments. e.g. @solid versus @solid()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -112,14 +112,10 @@ def __init__(self, step_execution_context: StepExecutionContext):
self._events: List[DagsterEvent] = []
self._output_metadata: Dict[str, Any] = {}

@property
def solid_config(self) -> Any:
return self._step_execution_context.op_config

@public
@property
def op_config(self) -> Any:
return self.solid_config
return self._step_execution_context.op_config

@property
def pipeline_run(self) -> DagsterRun:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,7 @@ class UnboundOpExecutionContext(OpExecutionContext):

def __init__(
self,
solid_config: Any,
op_config: Any,
resources_dict: Mapping[str, Any],
resources_config: Mapping[str, Any],
instance: Optional[DagsterInstance],
Expand All @@ -76,7 +76,7 @@ def __init__(
from dagster._core.execution.api import ephemeral_instance_if_missing
from dagster._core.execution.context_creation_pipeline import initialize_console_manager

self._solid_config = solid_config
self._op_config = op_config
self._mapping_key = mapping_key

self._instance_provided = (
Expand Down Expand Up @@ -123,8 +123,8 @@ def __del__(self):
self._instance_cm.__exit__(None, None, None) # pylint: disable=no-member

@property
def solid_config(self) -> Any:
return self._solid_config
def op_config(self) -> Any:
return self._op_config

@property
def resources(self) -> Resources:
Expand Down Expand Up @@ -239,11 +239,11 @@ def bind(

from dagster._core.definitions.resource_invocation import resolve_bound_config

solid_config = resolve_bound_config(self.solid_config, op_def)
op_config = resolve_bound_config(self.op_config, op_def)

return BoundOpExecutionContext(
op_def=op_def,
op_config=solid_config,
op_config=op_config,
resources=self.resources,
resources_config=self._resources_config,
instance=self.instance,
Expand Down Expand Up @@ -377,7 +377,7 @@ def __init__(
self._partition_key = partition_key

@property
def solid_config(self) -> Any:
def op_config(self) -> Any:
return self._op_config

@property
Expand Down Expand Up @@ -671,7 +671,7 @@ def build_solid_context(
resources (Optional[Dict[str, Any]]): The resources to provide to the context. These can be
either values or resource definitions.
solid_config (Optional[Any]): The solid config to provide to the context. The value provided
here will be available as ``context.solid_config``.
here will be available as ``context.op_config``.
resources_config (Optional[Dict[str, Any]]): Configuration for any resource definitions
provided to the resources arg. The configuration under a specific key should match the
resource under a specific key in the resources dictionary.
Expand Down Expand Up @@ -700,7 +700,7 @@ def build_solid_context(
resources_config=check.opt_mapping_param(
resources_config, "resources_config", key_type=str
),
solid_config=solid_config,
op_config=solid_config,
instance=check.opt_inst_param(instance, "instance", DagsterInstance),
partition_key=check.opt_str_param(partition_key, "partition_key"),
mapping_key=check.opt_str_param(mapping_key, "mapping_key"),
Expand Down

0 comments on commit c132dce

Please sign in to comment.