-
-
Notifications
You must be signed in to change notification settings - Fork 27
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Expr as singleton #798
Expr as singleton #798
Changes from all commits
0a9617f
8433deb
ed93aa7
a24c6c9
f8b9f4e
129aa7a
cb85ffc
4f8e0e3
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -36,7 +36,7 @@ class FromGraph(IO): | |
conversion from legacy dataframes. | ||
""" | ||
|
||
_parameters = ["layer", "_meta", "divisions", "_name"] | ||
_parameters = ["layer", "_meta", "divisions", "keys", "name_prefix"] | ||
|
||
@property | ||
def _meta(self): | ||
|
@@ -45,12 +45,19 @@ def _meta(self): | |
def _divisions(self): | ||
return self.operand("divisions") | ||
|
||
@property | ||
@functools.cached_property | ||
def _name(self): | ||
return self.operand("_name") | ||
return ( | ||
self.operand("name_prefix") + "-" + _tokenize_deterministic(*self.operands) | ||
) | ||
|
||
def _layer(self): | ||
return dict(self.operand("layer")) | ||
dsk = dict(self.operand("layer")) | ||
# The name may not actually match the layers name therefore rewrite this | ||
# using an alias | ||
for part, k in enumerate(self.operand("keys")): | ||
dsk[(self._name, part)] = k | ||
return dsk | ||
Comment on lines
54
to
+60
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The intuitive fix for me would've been to overwrite |
||
|
||
|
||
class BlockwiseIO(Blockwise, IO): | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -1,7 +1,6 @@ | ||
from __future__ import annotations | ||
|
||
import contextlib | ||
import functools | ||
import itertools | ||
import operator | ||
import warnings | ||
|
@@ -26,8 +25,9 @@ | |
from dask.dataframe.io.parquet.utils import _split_user_options | ||
from dask.dataframe.io.utils import _is_local_fs | ||
from dask.delayed import delayed | ||
from dask.utils import apply, natural_sort_key, typename | ||
from dask.utils import apply, funcname, natural_sort_key, typename | ||
from fsspec.utils import stringify_path | ||
from toolz import identity | ||
|
||
from dask_expr._expr import ( | ||
EQ, | ||
|
@@ -47,26 +47,15 @@ | |
determine_column_projection, | ||
) | ||
from dask_expr._reductions import Len | ||
from dask_expr._util import _convert_to_list | ||
from dask_expr._util import _convert_to_list, _tokenize_deterministic | ||
from dask_expr.io import BlockwiseIO, PartitionsFiltered | ||
|
||
NONE_LABEL = "__null_dask_index__" | ||
|
||
_cached_dataset_info = {} | ||
_CACHED_DATASET_SIZE = 10 | ||
_CACHED_PLAN_SIZE = 10 | ||
_cached_plan = {} | ||
|
||
|
||
def _control_cached_dataset_info(key): | ||
if ( | ||
len(_cached_dataset_info) > _CACHED_DATASET_SIZE | ||
and key not in _cached_dataset_info | ||
): | ||
key_to_pop = list(_cached_dataset_info.keys())[0] | ||
_cached_dataset_info.pop(key_to_pop) | ||
|
||
|
||
def _control_cached_plan(key): | ||
if len(_cached_plan) > _CACHED_PLAN_SIZE and key not in _cached_plan: | ||
key_to_pop = list(_cached_plan.keys())[0] | ||
|
@@ -121,7 +110,7 @@ def _lower(self): | |
class ToParquetData(Blockwise): | ||
_parameters = ToParquet._parameters | ||
|
||
@cached_property | ||
@property | ||
def io_func(self): | ||
return ToParquetFunctionWrapper( | ||
self.engine, | ||
|
@@ -257,7 +246,6 @@ def to_parquet( | |
|
||
# Clear read_parquet caches in case we are | ||
# also reading from the overwritten path | ||
_cached_dataset_info.clear() | ||
_cached_plan.clear() | ||
|
||
# Always skip divisions checks if divisions are unknown | ||
|
@@ -413,6 +401,7 @@ class ReadParquet(PartitionsFiltered, BlockwiseIO): | |
"kwargs", | ||
"_partitions", | ||
"_series", | ||
"_dataset_info_cache", | ||
] | ||
_defaults = { | ||
"columns": None, | ||
|
@@ -432,6 +421,7 @@ class ReadParquet(PartitionsFiltered, BlockwiseIO): | |
"kwargs": None, | ||
"_partitions": None, | ||
"_series": False, | ||
"_dataset_info_cache": None, | ||
} | ||
_pq_length_stats = None | ||
_absorb_projections = True | ||
|
@@ -474,7 +464,21 @@ def _simplify_up(self, parent, dependents): | |
return Literal(sum(_lengths)) | ||
|
||
@cached_property | ||
def _name(self): | ||
return ( | ||
funcname(type(self)).lower() | ||
+ "-" | ||
+ _tokenize_deterministic(self.checksum, *self.operands) | ||
) | ||
Comment on lines
+467
to
+472
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. this checksum is part of the |
||
|
||
@property | ||
def checksum(self): | ||
return self._dataset_info["checksum"] | ||
|
||
@property | ||
def _dataset_info(self): | ||
if rv := self.operand("_dataset_info_cache"): | ||
return rv | ||
# Process and split user options | ||
( | ||
dataset_options, | ||
|
@@ -536,13 +540,25 @@ def _dataset_info(self): | |
**other_options, | ||
}, | ||
) | ||
dataset_token = tokenize(*args) | ||
if dataset_token not in _cached_dataset_info: | ||
_control_cached_dataset_info(dataset_token) | ||
_cached_dataset_info[dataset_token] = self.engine._collect_dataset_info( | ||
*args | ||
) | ||
dataset_info = _cached_dataset_info[dataset_token].copy() | ||
dataset_info = self.engine._collect_dataset_info(*args) | ||
checksum = [] | ||
files_for_checksum = [] | ||
if dataset_info["has_metadata_file"]: | ||
if isinstance(self.path, list): | ||
files_for_checksum = [ | ||
next(path for path in self.path if path.endswith("_metadata")) | ||
] | ||
else: | ||
files_for_checksum = [self.path + fs.sep + "_metadata"] | ||
else: | ||
files_for_checksum = dataset_info["ds"].files | ||
|
||
for file in files_for_checksum: | ||
# The checksum / file info is usually already cached by the fsspec | ||
# FileSystem dir_cache since this info was already asked for in | ||
# _collect_dataset_info | ||
checksum.append(fs.checksum(file)) | ||
dataset_info["checksum"] = tokenize(checksum) | ||
Comment on lines
+556
to
+561
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. To deal with the cache consistency problem described in #800 I am calculating a checksum here. For s3 this falls back to using the ETag provided in the listdir response. This should not add any overhead since this stuff is already cached by fsspec. |
||
|
||
# Infer meta, accounting for index and columns arguments. | ||
meta = self.engine._create_dd_meta(dataset_info) | ||
|
@@ -558,6 +574,9 @@ def _dataset_info(self): | |
dataset_info["all_columns"] = all_columns | ||
dataset_info["calculate_divisions"] = self.calculate_divisions | ||
|
||
self.operands[ | ||
type(self)._parameters.index("_dataset_info_cache") | ||
] = dataset_info | ||
return dataset_info | ||
|
||
@property | ||
|
@@ -571,10 +590,10 @@ def _meta(self): | |
return meta[columns] | ||
return meta | ||
|
||
@cached_property | ||
@property | ||
def _io_func(self): | ||
if self._plan["empty"]: | ||
return lambda x: x | ||
return identity | ||
dataset_info = self._dataset_info | ||
return ParquetFunctionWrapper( | ||
self.engine, | ||
|
@@ -662,7 +681,7 @@ def _update_length_statistics(self): | |
stat["num-rows"] for stat in _collect_pq_statistics(self) | ||
) | ||
|
||
@functools.cached_property | ||
@property | ||
def _fusion_compression_factor(self): | ||
if self.operand("columns") is None: | ||
return 1 | ||
|
@@ -767,9 +786,11 @@ def _maybe_list(val): | |
return [val] | ||
|
||
return [ | ||
_maybe_list(val.to_list_tuple()) | ||
if hasattr(val, "to_list_tuple") | ||
else _maybe_list(val) | ||
( | ||
_maybe_list(val.to_list_tuple()) | ||
if hasattr(val, "to_list_tuple") | ||
else _maybe_list(val) | ||
) | ||
for val in self | ||
] | ||
|
||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The
FromGraph
thing got a little odd. For context, this thing is used in two waysThe previous implementation accepted the
_name
as an input argument. For the persisted dataframe, this was the name of the original expression. For the wrapped one, it is the name of the legacy dataframe.The problem is that the legacy dataframe does not have the same strict uniqueness guarantees as the expressions do so setting duplicate names is very easy. In fact, our tests where doing just that! (and still are). This caused the
Expr.__new__
to deduplicate and effectively ignore the second dataframe... oops.For persist it is also a little odd since if the
FromGraph
expression is inheriting the exact name of it's ancestor, there exist now two expressions of a different type with the same name. This is odd.Effectively, with this chosen singleton implementation, setting the
_name
explicitly instead of calculating it using a hash is a cardinal sin and will cause all sorts of weird things to happen.Now, this can be fixed but that has a rather big caveat. If I have to redefine the name of the expression, I actually also have to rewrite the graph! Many places in dask-expr are (imo incorrectly) assuming that output key names of a dataframe layer/expression are universally built as
(df._name, i)
and are hard coding this when implementing their own layer (instead of iterating overi
, iterating overdf.__dask_keys__()
would maintain the abstraction). This rewrite adds effectively another layer of keys. In reality this is really ugly since when computing something on top of a persisted dataframe, there will always be this dummy key in between.Alternatively, I could make the singleton deduplication type aware to give the
FromGraph
thing an excuse to overwrite the name. However, if we truly stick with singletons that are based on the name, I would prefer the name to actually be unique which required all implementations to stop hard coding keys of another expression/dataframe and iterate properly over the__dask_keys__
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is a good point, we should fix this instead of relying on df._name and i
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I will investigate if it is possible to just use
__dask_keys__
everywhere but I'd prefer doing this in a follow upThere was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes totally agree, this should definitely be a follow up (also doesn't have to be you, I could pick this up as well)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I got started on this, see
c3c01ed
That commit replaces all occurrences that match
\((\w+(_\w+)?)(?<!self)\._name,
but something appears to be still missing. It is possible and not as ugly as I thought it would be