dagster
dagster copied to clipboard
prevent aliased config attributes shadowing
Summary & Motivation
As pointed out by #21289, our team is currently facing weird behaviour when using the PolarsParquetIOManager
.
The s3 custom configuration (endpoint, access key, secret key set up via the storage_options
attribute) is visible when creating the PolarsParquetIOManager
but after being given to the Definitions
constructor, it loses its storage_options
attribute.
After a bit of investigation, @ShootingStarD and I managed to locate the place where the vanishing appeared.
The extra attributes of the PolarsParquetIOManager
, via the ConfigurableIOManager
, are parsed by the constructor of the ConfigurableResourceFactory
.
Then I only followed the functions lead:
-
ConfigurableResourceFactory._convert_to_config_dictionary
-
ConfigurableResourceFactory._get_non_default_public_field_values
-
ConfigurableResourceFactory._get_non_default_public_field_values_cls
This function loops over the different attributes of the given object, cast as a dictionary. Here's a reminder of what's inside this function.
@classmethod
def _get_non_default_public_field_values_cls(cls, items: Dict[str, Any]) -> Mapping[str, Any]:
"""Returns a dictionary representation of this config object,
ignoring any private fields, and any defaulted fields which are equal to the default value.
Inner fields are returned as-is in the dictionary,
meaning any nested config objects will be returned as config objects, not dictionaries.
"""
output = {}
for key, value in items.items():
if _is_field_internal(key):
continue
field = model_fields(cls).get(key)
if field:
if (
not is_literal(field.annotation)
and not safe_is_subclass(field.annotation, Enum)
and value == field.default
):
continue
resolved_field_name = field.alias or key
output[resolved_field_name] = value
else:
output[key] = value
return output
So at this point, the items
dictionary has both cloud_storage_options
and storage_options
.
At first, the key cloud_storage_options
is treated by this loop, and the value is the dictionary passed to the PolarsParquetIOManager
constructor
- it's not an internal field,
-
field = model_fields(cls).get(key)
withkey='cloud_storage_opions'
is notNone
- it passes the tests
is_litteral
,safe_is_subclass
andvalue != default
- since it's an aliased field, the value is stored to the output as its alias:
"storage_options"
Then the key storage_options
is treated, and the value is {}
- it's not an internal field
-
field = model_fields(cls).get(key)
isNone
- The value
output[key]
is overridden
Solution
So the solution I found to avoid that behaviour is to prevent the treatment of the items if their key is the same as a field's alias.
I am not saying it's the best, but it's a good start.
For example, I don't understand why there is key/value for the aliased name in items
, and why is this value {}
instead of None
.
And if for any reason, the aliased fields with bad values are treated before the real Field
s, they would be overridden without us noticing.
How I Tested These Changes
I've used the code provided in #21289
from dagster_polars import PolarsParquetIOManager
from dagster import Definitions
io_manager = PolarsParquetIOManager(storage_options={"key":"value"})
foo = Definitions(resources={"parquet_io_manager":io_manager})._created_pending_or_normal_repo.get_top_level_resources()['parquet_io_manager'].get_config_field()
assert io_manager.cloud_storage_options == foo.default_value['storage_options']
This assertion gives an error for dagster 1.7.13 After the patch, the test succeeds.
Please let me know if you think it's the correct way of fixing this behaviour, and if the test is rigorous enough to validate the desired behaviour.
Have a nice day!