Skip to content

Commit

Permalink
[refactor] ResolvedRunConfig.solids -> ops (#12373)
Browse files Browse the repository at this point in the history
### Summary & Motivation

- Rename ResolvedRunConfig.solids -> ops

### How I Tested These Changes

BK
  • Loading branch information
smackesey committed Feb 17, 2023
1 parent c132dce commit fa3418e
Show file tree
Hide file tree
Showing 12 changed files with 29 additions and 29 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -119,7 +119,7 @@ def resources(self) -> "Resources":

@property
def solid_config(self) -> Any:
solid_config = self._step_execution_context.resolved_run_config.solids.get(
solid_config = self._step_execution_context.resolved_run_config.ops.get(
str(self._step_execution_context.step.node_handle)
)
return solid_config.config if solid_config else None
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -725,7 +725,7 @@ def get_output_context(
"""
step = execution_plan.get_step_by_key(step_output_handle.step_key)
# get config
op_config = resolved_run_config.solids[step.node_handle.to_string()]
op_config = resolved_run_config.ops[step.node_handle.to_string()]
outputs_config = op_config.outputs

if outputs_config:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -816,7 +816,7 @@ def previous_attempt_count(self) -> int:

@property
def op_config(self) -> Any:
op_config = self.resolved_run_config.solids.get(str(self.solid_handle))
op_config = self.resolved_run_config.ops.get(str(self.solid_handle))
return op_config.config if op_config else None

@property
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,7 @@ def create_step_outputs(
config_output_names: Set[str] = set()
current_handle = handle
while current_handle:
solid_config = resolved_run_config.solids[current_handle.to_string()]
solid_config = resolved_run_config.ops[current_handle.to_string()]
current_handle = current_handle.parent
config_output_names = config_output_names.union(solid_config.outputs.output_names)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -688,7 +688,7 @@ def _create_type_materializations(

# check for output mappings at every point up the composition hierarchy
while current_handle:
solid_config = step_context.resolved_run_config.solids.get(current_handle.to_string())
solid_config = step_context.resolved_run_config.ops.get(current_handle.to_string())
current_handle = current_handle.parent

if solid_config is None:
Expand Down
12 changes: 6 additions & 6 deletions python_modules/dagster/dagster/_core/execution/plan/inputs.py
Original file line number Diff line number Diff line change
Expand Up @@ -161,7 +161,7 @@ def load_input_object(
else asset_layer.io_manager_key_for_asset(input_asset_key)
)

op_config = step_context.resolved_run_config.solids.get(str(self.solid_handle))
op_config = step_context.resolved_run_config.ops.get(str(self.solid_handle))
config_data = op_config.inputs.get(self.input_name) if op_config else None

loader = getattr(step_context.resources, input_manager_key)
Expand Down Expand Up @@ -214,7 +214,7 @@ def compute_version(
input_manager_key
]

op_config = check.not_none(resolved_run_config.solids.get(op.name))
op_config = check.not_none(resolved_run_config.ops.get(op.name))
input_config = op_config.inputs.get(self.input_name)
resource_entry = check.not_none(resolved_run_config.resources.get(input_manager_key))
resource_config = resource_entry.config
Expand Down Expand Up @@ -304,7 +304,7 @@ def load_input_object(

input_def = step_context.solid_def.input_def_named(input_def.name)

solid_config = step_context.resolved_run_config.solids.get(str(self.solid_handle))
solid_config = step_context.resolved_run_config.ops.get(str(self.solid_handle))
config_data = solid_config.inputs.get(self.input_name) if solid_config else None

input_manager_key = check.not_none(
Expand Down Expand Up @@ -355,7 +355,7 @@ def compute_version(
resolved_run_config.mode
).resource_defs[input_manager_key]

solid_config = resolved_run_config.solids[solid.name]
solid_config = resolved_run_config.ops[solid.name]
input_config = solid_config.inputs.get(self.input_name)
resource_config = check.not_none(
resolved_run_config.resources.get(input_manager_key)
Expand Down Expand Up @@ -464,7 +464,7 @@ def get_load_context(
resource_config = step_context.resolved_run_config.resources[resolved_io_manager_key].config
resources = build_resources_for_manager(resolved_io_manager_key, step_context)

solid_config = step_context.resolved_run_config.solids.get(str(step_context.solid_handle))
solid_config = step_context.resolved_run_config.ops.get(str(step_context.solid_handle))
config_data = solid_config.inputs.get(input_def.name) if solid_config else None

return step_context.for_input_manager(
Expand Down Expand Up @@ -592,7 +592,7 @@ def get_associated_config(self, resolved_run_config: ResolvedRunConfig):
including the root.
"""
if self.solid_handle:
op_config = resolved_run_config.solids.get(str(self.solid_handle))
op_config = resolved_run_config.ops.get(str(self.solid_handle))
return op_config.inputs.get(self.input_name) if op_config else None
else:
input_config = resolved_run_config.inputs
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -485,7 +485,7 @@ def get_step_input_source(
)

input_handle = node.get_input(input_name)
node_config = plan_builder.resolved_run_config.solids.get(str(handle))
node_config = plan_builder.resolved_run_config.ops.get(str(handle))

input_def = node.definition.input_def_named(input_name)
asset_layer = plan_builder.pipeline.get_definition().asset_layer
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -107,7 +107,7 @@ def resolve_step_versions(

solid_name = str(step.node_handle)

solid_config = resolved_run_config.solids[solid_name].config
solid_config = resolved_run_config.ops[solid_name].config

solid_def_version = None
if solid_def.version is not None: # type: ignore # (should be OpDefinition)
Expand Down
10 changes: 5 additions & 5 deletions python_modules/dagster/dagster/_core/system_config/objects.py
Original file line number Diff line number Diff line change
Expand Up @@ -99,7 +99,7 @@ class ResolvedRunConfig(
NamedTuple(
"_ResolvedRunConfig",
[
("solids", Mapping[str, OpConfig]),
("ops", Mapping[str, OpConfig]),
("execution", "ExecutionConfig"),
("resources", Mapping[str, ResourceConfig]),
("loggers", Mapping[str, Mapping[str, object]]),
Expand All @@ -111,7 +111,7 @@ class ResolvedRunConfig(
):
def __new__(
cls,
solids: Optional[Mapping[str, OpConfig]] = None,
ops: Optional[Mapping[str, OpConfig]] = None,
execution: Optional["ExecutionConfig"] = None,
resources: Optional[Mapping[str, ResourceConfig]] = None,
loggers: Optional[Mapping[str, Mapping[str, object]]] = None,
Expand All @@ -130,7 +130,7 @@ def __new__(

return super(ResolvedRunConfig, cls).__new__(
cls,
solids=check.opt_mapping_param(solids, "solids", key_type=str, value_type=OpConfig),
ops=check.opt_mapping_param(ops, "ops", key_type=str, value_type=OpConfig),
execution=execution,
resources=resources,
loggers=check.opt_mapping_param(loggers, "loggers", key_type=str, value_type=Mapping),
Expand Down Expand Up @@ -216,7 +216,7 @@ def build(
input_configs = config_value.get("inputs", {})

return ResolvedRunConfig(
solids=solid_config_dict,
ops=solid_config_dict,
execution=ExecutionConfig.from_dict(config_mapped_execution_configs),
loggers=config_mapped_logger_configs,
original_config_dict=run_config,
Expand All @@ -229,7 +229,7 @@ def to_dict(self) -> Mapping[str, Mapping[str, object]]:
env_dict: Dict[str, Mapping[str, object]] = {}

solid_configs: Dict[str, object] = {}
for solid_name, solid_config in self.solids.items():
for solid_name, solid_config in self.ops.items():
solid_configs[solid_name] = {
"config": solid_config.config,
"inputs": solid_config.inputs,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -145,7 +145,7 @@ def test_solid_dictionary_type():
},
)

value = env_obj.solids
value = env_obj.ops

assert set(["int_config_op", "string_config_op"]) == set(value.keys())
assert value == {
Expand Down Expand Up @@ -223,8 +223,8 @@ def job_def():

env = ResolvedRunConfig.build(job_def, {"ops": {"int_config_op": {"config": 1}}})

assert {"int_config_op", "no_config_op"} == set(env.solids.keys())
assert env.solids == {
assert {"int_config_op", "no_config_op"} == set(env.ops.keys())
assert env.ops == {
"int_config_op": OpConfig.from_dict({"config": 1}),
"no_config_op": OpConfig.from_dict({}),
}
Expand Down Expand Up @@ -264,7 +264,7 @@ def test_whole_environment():
)

assert isinstance(env, ResolvedRunConfig)
assert env.solids == {
assert env.ops == {
"int_config_op": OpConfig.from_dict({"config": 123}),
"no_config_op": OpConfig.from_dict({}),
}
Expand Down Expand Up @@ -350,7 +350,7 @@ def _assert_config_none(context, value):

env_obj = ResolvedRunConfig.build(job_def, {})

assert env_obj.solids["int_config_op"].config is None
assert env_obj.ops["int_config_op"].config is None


def test_optional_solid_with_required_scalar_config():
Expand Down Expand Up @@ -420,7 +420,7 @@ def test_required_solid_with_required_subfield():
{"ops": {"int_config_op": {"config": {"required_field": "foobar"}}}},
)

assert env_obj.solids["int_config_op"].config["required_field"] == "foobar"
assert env_obj.ops["int_config_op"].config["required_field"] == "foobar"

res = process_config(env_type, {"ops": {}})
assert not res.success
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -91,8 +91,8 @@ def test_basic_json_default_output_config_schema():
{"solids": {"return_one": {"outputs": [{"result": {"json": {"path": "foo"}}}]}}},
)

assert env.solids["return_one"]
assert env.solids["return_one"].outputs.type_materializer_specs == [
assert env.ops["return_one"]
assert env.ops["return_one"].outputs.type_materializer_specs == [
{"result": {"json": {"path": "foo"}}}
]

Expand All @@ -103,8 +103,8 @@ def test_basic_json_named_output_config_schema():
{"solids": {"return_named_one": {"outputs": [{"named": {"json": {"path": "foo"}}}]}}},
)

assert env.solids["return_named_one"]
assert env.solids["return_named_one"].outputs.type_materializer_specs == [
assert env.ops["return_named_one"]
assert env.ops["return_named_one"].outputs.type_materializer_specs == [
{"named": {"json": {"path": "foo"}}}
]

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -211,7 +211,7 @@ def op_config(self) -> Any:
if self._op_config:
return self._op_config

op_config = self.resolved_run_config.solids.get(self.solid_name)
op_config = self.resolved_run_config.ops.get(self.solid_name)
return op_config.config if op_config else None


Expand Down

0 comments on commit fa3418e

Please sign in to comment.