Skip to content

[dask.order] Reduce memory pressure for multi array reductions by releasing splitter tasks more eagerly#10535

Merged
fjetter merged 17 commits intodask:mainfrom
fjetter:dask_order_root_tasks
Oct 10, 2023
Merged

[dask.order] Reduce memory pressure for multi array reductions by releasing splitter tasks more eagerly#10535
fjetter merged 17 commits intodask:mainfrom
fjetter:dask_order_root_tasks

Conversation

@fjetter
Copy link
Copy Markdown
Member

@fjetter fjetter commented Sep 26, 2023

Related #10505
Closes #10384
Closes #9995

  • It removes the ambiguity between dataframe and array computes by normalizing topologies by adding a final reducing node to all graphs (this requires a little more work)
  • It adds branched out dependents to the singles list. this is not exactly what it was designed for but this allows us to process those tasks before opening a new branch of computation
  • It removes a "change strategic goal" code path. Mostly because it is necessary to make the code work but also because I currently distrust the partition_keys target function

Edit: The above description is a little outdated. See comment farthest below and inline comments for an accurate description

Copy link
Copy Markdown
Member Author

@fjetter fjetter left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Currently, there are still failing tests

  • test_prefer_short_dependents requires me to adjust the "finish now" logic to account for the artificial root task
  • test_order_with_equal_dependents TODO
  • test_run_smaller_sections see comment

|
c3

Prefer b1 over c1 because it won't stick around waiting for d1 to complete
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I believe this explanation is actually wrong. I think c3 should be preferred to run before b2 because c3->a1 is the longest critical path.

I'm not even entirely convinced that d1 should run before b2. I believe the memory pressure is identical and b2 has a longer way to walk. Anyway, this is how it currently works and I think that's fine

/ \ /| | /
a c e cc

Prefer to run acb first because then we can get that out of the way
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is currently favoring finishing aa before anything else. I believe this is a matter of finding the appropriate target functions.

Considering that it was xfailed I'm not terribly concerned but I will have a look again.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This example is a bit ambiguous. There is a "best ordering" for single threaded execution and one for multi threaded execution.

Single threaded

This PR

Step 0 1 2 3 4 5 6 7 8
T1 e c d cc bb aa dd a b
Pressure 1 2 2 3 4 3 3 4 2

What the docs say

Step 0 1 2 3 4 5 6 7 8
T1 c a b e d cc dd bb aa
Pressure 1 2 2 3 2 3 3 3 2

Multi threaded

This PR

Step 0 1 2 3 4
T1 e d bb aa b
T2 c cc dd a x
Pressure 2 3 3 2 2

What the docs say

Step 0 1 2 3 4
T1 a b d bb aa
T2 c e cc dd x
Pressure 2 2 3 4 2

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Typically dask.order acts as if we were executing sequentially so what the docs say is not wrong. However, in practice, most will be running in a multi threaded environment. This behavior may, of course, look again very different for more threads...

@fjetter
Copy link
Copy Markdown
Member Author

fjetter commented Sep 26, 2023

At least on smaller graphs, this is fixing the root task problem mentioned in #9995 by ensuring to finish a computation branch before opening a new one
anom-mean-order-raw-colored

The xarray style reduction reported in #10384 now looks like
xarray-reduction-array-colored
the simulated pressure for dataframes and arrays are not identical due to the adjustment to the graph topologies. What's even better, the pressure with this PR is now smaller than the DF pressure before.

I will move to real world testing/benchmarking tomorrow. If that looks good, I will have to do a little refactoring since there is now plenty of dead code I'm effectively skipping due to the root task addition. 🤞

Comment on lines +137 to +143
if len(root_nodes) > 1:
root = "root-node"
dsk[root] = (object(), *root_nodes)
dependencies[root] = root_nodes
o = order(dsk, dependencies)
del o[root]
return o
Copy link
Copy Markdown
Member Author

@fjetter fjetter Sep 27, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is by far the most interesting change in this PR since it is effectively disabling a lot of logic. For one, this is disabling the entire skip_root_node and also specialized finish_now things.

What's even more critical is that this effectively disables many root-node dependent graph metrics (stuff like max_dependencies, min_dependencies) since all nodes now share the same long term goal

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There is an interesting effect, though. The partition_keys function is depending on min_dependencies, or rather the difference between min_dependencies and total_dependencies. If min_dependencies was removed entirely, this function would become negative, reverting every decision.

@fjetter
Copy link
Copy Markdown
Member Author

fjetter commented Sep 27, 2023

test_prefer_short_dependents actually tests multiple, slightly different variations of the same graph. For the original, as reported and shown in #5859, this PR is producing identical results as the main branch. However, the main branch is already diverging from the optimal solution depicted in #5859 (comment) but this wasn't caught by tests. However, I believe that the current behavior is fine even if it isn't perfect.

For the first variation in dsk2, an additional node is appended at the leafs to generate some asymmetry. This PR differs from main in which branch of the asymmetric end is walked first. This branch favors the longer branch while main favors the shorter one.
This is in fact the asserted behavior although I do not understand why that is. In terms of memory pressure, the two choices are identical. In terms of computational efficiency, I would make the case for the longest critical path since a multi threaded executor will be able to finish more quickly this way. The same argument holds for the next variation dsk3 where instead of an additional leaf, one is removed. Same principle topology and same problem.

image

The fourth variation removes an additional leaf and only keeps a single node after the reduction

image

It is asserted that those two nodes run exactly after each other (asserting that the two branches run consecutively would've been the better and correct assertion for the above variations as well and would've maintained best ordering for #5859 (comment))
This branch is currently not preserving this property. Main can only hold this because there is a specialized code branch (that is currently disabled on this branch) that will "finish dangling tasks". I can see the appeal of such a logic, particularly in the space of optimized/fused graphs, but I would of course prefer it if the ordering algorithm would be agnostic to this and just finish this branch off regardless of its length because it is reducing pressure

@fjetter
Copy link
Copy Markdown
Member Author

fjetter commented Sep 27, 2023

Ok, I got through the unit tests and think everything is in order. I got sidetracked a little when looking at the cost functions because the graph_metrics confused me a little. Turns out (obvious in hindsight) they are only accurate for trees and are double counting quite a bit. Oh well, now there are a couple of tests. I also changed the definition slightly in a way that makes more sense to me. I may break this out into a dedicated PR, this is quite unrelated to the important stuff in here

@mrocklin
Copy link
Copy Markdown
Member

Cool. How excited should I be about this? "Solves all Xarray pain?" "Solves a couple of problems we've seen recently, but still lots more to do?"

@fjetter
Copy link
Copy Markdown
Member Author

fjetter commented Sep 27, 2023

Cool. How excited should I be about this? "Solves all Xarray pain?" "Solves a couple of problems we've seen recently, but still lots more to do?"

Let's wait for the first round of benchmarks before I answer :)

@fjetter
Copy link
Copy Markdown
Member Author

fjetter commented Sep 27, 2023

OK, so benchmark results will be a little delayed. There is some kind of issue with our CI.

From manual testing I can say that the xarray example that's been reported in pangeo-data/distributed-array-examples#2 now runs smoothly with constant memory.
However, if I dial up the size, we are running again into a bad ordering problem. I suspect something with the graph metrics is off.

(Legend below: scaling the time dimension / number of tasks by that factor; The green line is only constant because we're spilling)

image

The up side is that on main, the 1.0 measure was already saturating memory fully... Looks like I have some more work to do

@fjetter
Copy link
Copy Markdown
Member Author

fjetter commented Sep 27, 2023

Well, at the very least simulated memory pressure, i.e. the number of tasks a single threaded scheduler would have to hold in memory, is behaving reasonably well. Not constant but no longer linear and even better than the DF version before.

(This goes up to 100_000, i.e. 20x more than the reported example)

image

@fjetter
Copy link
Copy Markdown
Member Author

fjetter commented Sep 27, 2023

I'm pretty sure that what I'm seeing right now is not an ordering problem anymore. The behavior appears to be quite flaky. I assume this there is some kind distributed scheduler state that is impacting the worker placement decision.

For instance, I now managed to get another round of runs through where the small dataset actually misbehaved but I didn't have an issue scaling up to 5 times the data (i.e. total input data is 10x more than the cluster memory)

image

@fjetter
Copy link
Copy Markdown
Member Author

fjetter commented Sep 27, 2023

OK, turns out it is an ordering problem after all. Order appears to not be stable.

I patched the scheduler to compute a hash of the order result and compute the simulated pressure and log this. The hash is always different. The pressure is sometimes bad and when it is, the run is horrible.

Just running the same thing in a for loop for a couple of times

image

At least I know what to look for now... kind of

@fjetter
Copy link
Copy Markdown
Member Author

fjetter commented Sep 27, 2023

Well... good and bad news. Order seems to be fine (ish?). If I disable optimization, everything seems to be very, very stable. However, once optimization is on, this becomes flaky!
going through the code, once I remove/comment out optimize_blockwise everything works fine. With optimize_blockwise this is flaky

@fjetter
Copy link
Copy Markdown
Member Author

fjetter commented Sep 27, 2023

By best guess is that this is connected to keys somehow. This issue is only flaky if I regenerate the graph, i.e. regenerate the keys. We probably no longer have sufficient structural information to break ties w/out str comp after the fusion.

@fjetter
Copy link
Copy Markdown
Member Author

fjetter commented Sep 28, 2023

TLDR I shouldn't have touch the graph_metrics. However, everything is sooo fragile that there is more work to do still

Who was I to think touching the graph_metrics definition was a good idea? Stupid me! The two diverging orderings were indeed triggered by a str comparison but the fundamental bug was introduced by me changing the graph_metrics. I don't know if this will ever be relevant but for my own sanity, I want to document what happened. Interested readers may skip this safely.

I was looking at why these two graphs diverged

raw graph
first

snapshot good ordering
image

snapshot bad ordering

image

The good ordering properly targeted the first reducer and (not shown here) would clean up after itself. That's perfect. The bad ordering regressed into a breadth first search that was releasing roots relatively quickly but wouldn't go deep, such that we cannot truly release anything until rather late.

The divergence happens already at the very fist decision point. The initial root node is random (str compared) because the graph is pretty much symmetrical. It would then decide which one of the dependents to pick by using dependents_key which is identical for both after my graph_metrics changes (because with my changes, coincidentally min_heights and total_dependents is equal causing this to zero out https://github.com/dask/dask/blob/2b45b215e13e6d7893853ef33cb4b1796c477a9e/dask/order.py#L224C47-L224C47)
This decision here

dask/dask/order.py

Lines 593 to 599 in 2b45b21

if (
key2 < key
or key == key2
and dependents_key(dep2) < dependents_key(dep)
):
dep, dep2 = dep2, dep
key, key2 = key2, key

which leads us to these if/elif/else clauses

dask/dask/order.py

Lines 640 to 652 in 2b45b21

if not num_needed[dep2]:
inner_stacks_append(inner_stack)
inner_stack = [dep]
inner_stack_pop = inner_stack.pop
seen_add(dep)
singles[dep2] = item
elif key == key2 and 5 * partition_keys[item] > 22 * key:
inner_stacks_append(inner_stack)
inner_stacks_append([dep2])
inner_stack = [dep]
inner_stack_pop = inner_stack.pop
seen_update(deps)
else:

The thing is, these dependents are not identical since one of them can already be run while the other has a dependency to load first. Depending on which one of the two is dep2 the if not num_needed[dep2] clause will evaluate differently.
The bad case actually runs in to the num_needed[dep2] == 0 zero case and puts a "single" relation on the other dependency

singles[dep2] = item
which forces order to work on this "single" before loading any more dependents. Effectively these singles is a deviation from the depth first search (which is sometimes necessary to free memory).
The good path actually picks the key == key2 and 5 * partition_keys[item] > 22 * key path (where do these magical numbers come from) https://github.com/fjetter/dask/blob/e71033821ecedab254750bd6f158714ba589acf8/dask/order.py#L650
which does almost the same thing but doesn't put a "single" relationship on the other dep.

Alas, reverting the graph_metrics change ensure that both graphs are now producing identical orderings. The chosen path is actually the bad one... but once we move to larger graphs it appears to work again. This just shows that there is still some work to do...

@fjetter
Copy link
Copy Markdown
Member Author

fjetter commented Sep 28, 2023

alright, the above example now works nicely with constant memory. The below shows runs with increasingly more data/tasks. Runtime is increasing linearily and memory is constant. Just the way it's supposed to.

image

@mrocklin
Copy link
Copy Markdown
Member

mrocklin commented Sep 28, 2023 via email

@dcherian
Copy link
Copy Markdown
Collaborator

❤️

Comment on lines +468 to +475
def test_favor_longest_critical_path(abcde):
r"""

a
|
d b e
\ | /
c
a
|
d b e
\ | /
c
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I believe the argument of the earlier version is wrong. Writing again out what the pressure is over time, yields

Single threaded

This PR

Step 0 1 2 3 4
T1 c b a e d
Pressure 1 2 2 3 3

Main

Step 0 1 2 3 4
T1 c d e b a
Pressure 1 2 3 3 3

Multi threaded

This PR

Step 0 1 2
T1 c b a
T2 x d e
Pressure 1 3 3

Main

Step 0 1 2 3
T1 c d b a
T2 x e x x
Pressure 1 3 3 3

Of course, this assumes that all tasks weigh the same. If we assumed that source tasks weight more, this is different but this is not universally true (often but not always).

Favoring the longest critical path, however, benefits execution times in multi threaded environments (fewer steps, again assuming equal weight / runtime)

@fjetter fjetter changed the title [WIP] allow computations branches to finish properly before opening a new one [dask.order] Reduce memory pressure for multi array reductions by releasing splitter tasks more eagerly Oct 5, 2023
@fjetter fjetter marked this pull request as ready for review October 5, 2023 08:27
Copy link
Copy Markdown
Member Author

@fjetter fjetter left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I left inline comments to explain all the changes that happened here. This can either guide a reviewer or help a forensic analysis in case something blows up after all

Comment on lines +137 to +149
if len(root_nodes) > 1:
# This is also nice because it makes us robust to difference when
# computing vs persisting collections
root = object()

def _f(*args, **kwargs):
pass

dsk[root] = (_f, *root_nodes)
dependencies[root] = root_nodes
o = order(dsk, dependencies)
del o[root]
return o
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is an important but still poorly tested change. It ensures that all graphs that are orderd will be reduced to a single task, i.e. even embarrassingly parallel graphs will virtually reduce into a single sink node.
Initially, this was motivated because I noticed different behavior between an array and a dataframe workload that I could track back to exactly this change in the topology.

However, zooming out, this is quite an important thing in general considering that distributed is frequently appending such a sink task whenever a compute is called on a collection. I believe a compute (vs persist) should not alter the way graphs are treated. Therefore, normalizing every graph like this seems to be the right approach (it could also be normalized in the other direction, of course)

Comment on lines +168 to +169
_,
_,
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The "single sink node" normalization above also causes all min/max_dependencies to be equal for all tasks since they all reduce to the same node. From what I can tell, this is the strongest driver for the difference we've seen between the array vs dataframe reduction.

A follow-up PR can change the graph_metrics to no longer compute this. I'll leave this in for now.

Comment on lines 217 to 232

def finish_now_key(x):
"""Determine the order of dependents that are ready to run and be released"""
return (-len(dependencies[x]), StrComparable(x))

root_total_dependencies = total_dependencies[list(root_nodes)[0]]
# Computing this for all keys can sometimes be relatively expensive :(
partition_keys = {
key: (
(min_dependencies - total_dependencies[key] + 1)
(root_total_dependencies - total_dependencies[key] + 1)
* (total_dependents - min_heights)
)
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This partition_keys cost function is from what I can tell a little fragile but it is extremely important. Without min_dependencies here, this needed a different baseline and taking the largest value of total_dependencies ensure that the first scalar of this product is strictly positive. If this is not guaranteed, this cost function will no longer be stable, particularly not if one of the two scalars is allowed to become zero.

This cost function makes me nervous because I don't fully understand the reasoning behind it. It's doing a decent job so I didn't modify it further.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we add an in-line comment for the requirement to be strictly positive for the cost funtion to be stable?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't understand this cost function sufficiently well to say whether this must be strictly positive. I merely discovered cases where zero caused very bad behavior. I think this PR is a better documentation than whatever I can write in the code.

# These dicts use `partition_keys` as keys. We process them by placing the values
# in `outer_stack` so that the smallest keys will be processed first.
next_nodes = defaultdict(list)
later_nodes = defaultdict(list)
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I couldn't find an actual use for later_nodes and couldn't find a workload or test that would be sensitive to this. Both next and later nodes are ordered by partition keys and if we trust partition_keys, I don't see a reason to further divide into next/later. In the interest of removing complexity, I deleted this code path.

Comment on lines -282 to -285
if skip_root_node:
seen = set(root_nodes)
else:
seen = set() # seen in an inner_stack (and has dependencies)
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Due to the topology normalization we're now always in the skip_root_node branch which is a little confusing given the normalization approach.
This toggle ensure that when we're walking the graph, we're actually ignoring this root node, i.e. during metrics calculation we act as if it was there but when walking we're ignoring it.
I haven't found a decent way to remove this and decided to keep it. This logic allows us to keep otherwise connected subgraphs from not mixing which is overall a nice property to have.
Mildly inconsistent but effective.


# NOTE: If this was too slow, LIFO would be a decent
# approximation
for single in sorted(singles_keys, key=lambda x: partition_keys[x]):
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I didn't perform any measurements but empirically, I found that many singles are already part of result unless we're working in very dense all-to-all graphs.

In practice, this means that singles should be small here and sorting is feasible. In earlier versions I was simply iterating reversely which also provided really nice results but sorting it is much better.

This is mostly replacing the finish_now logic that I removed above. Cherry on top is that this may now even finish dangling linear branches instead of just single tasks. The downside is that it is no longer guaranteed

Comment on lines 447 to 455
if (
add_to_inner_stack
and len(set_difference(dependents[parent], result)) > 1
len(
set_difference(
set_difference(dependents[parent], result),
seen,
)
)
> 1
):
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is what makes the singles processing actually a little more eager than before. Apart from the topological normalization, probably the most impactful change in here.
We're typically only allowing singles to be processed if they are guaranteed to release their parent. this condition now also allows to run a single to run if the parent would be freed after the next tactical goal is reached (i.e. inner_stack is emptied).
This is very close to the idea of later_singles that is already implemented here but there are multiple code paths that create a new inner_stack without triggering a singles compute. I found this logic to be more reliable.

Comment on lines +521 to +526
if add_to_inner_stack and not inner_stack:
inner_stack = [dep]
inner_stack_pop = inner_stack.pop
seen_add(dep)
continue
key = partition_keys[dep]
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No functional change, should just be a reduction of code duplication

Comment on lines +619 to +623
psingles = possible_singles[key]
for s in psingles:
singles[s] = item
vals -= psingles
next_nodes[key].append(vals)
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is another big-ish change. Previously, singles would only be considered for nodes with two dependents. This now allows us to detect dependents of nodes that split into many tasks to be treated as singles as well.

dask/order.py Outdated
Comment on lines +759 to +763
0
|
2 1
1 0
\ /
4
3
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

TODO: revert

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Has this been addressed?

@mrocklin
Copy link
Copy Markdown
Member

mrocklin commented Oct 5, 2023

Ok, I think this is mostly good to go

This has been fun to read every morning (certainly more interesting and promising than the daily news). Thanks!

It's a shame we no longer write papers. This would be an interesting topic.

@mrocklin
Copy link
Copy Markdown
Member

mrocklin commented Oct 5, 2023

It seems like the relative nature of the AB tests may bemisleading sometimes. Might be worth rethinking at some point.

@fjetter
Copy link
Copy Markdown
Member Author

fjetter commented Oct 5, 2023

I intend to follow up here with with a smaller non-functional PR that cleans this up a little further like...

  • a couple of variable renames
  • maybe typing if I feel like it
  • Another pass on the doc strings
  • if I feel lucky, I'll refactor some duplicated code 🤞

but I think all of this is better left for a follow up PR

@d-v-b
Copy link
Copy Markdown
Member

d-v-b commented Oct 5, 2023

I have loved reading this, and it's very exciting!

@mrocklin
Copy link
Copy Markdown
Member

mrocklin commented Oct 5, 2023

@hendrikmakait hendrikmakait self-requested a review October 7, 2023 10:55
@hendrikmakait hendrikmakait added needs review Needs review from a contributor. enhancement Improve existing functionality or make things work better labels Oct 7, 2023
@fjetter
Copy link
Copy Markdown
Member Author

fjetter commented Oct 9, 2023

I intend to merge in about 24hrs unless there is a valid reason not to. If a review comes in after this, I will address any raised issues in a follow-up PR.

Merging early allows this to sit on main (and be tested by dask engineers and our benchmarking suite) for a couple of days before the next release.

@mrocklin
Copy link
Copy Markdown
Member

mrocklin commented Oct 9, 2023 via email

Copy link
Copy Markdown
Member

@hendrikmakait hendrikmakait left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for adding the comments. They make it easier feasible to understand what's going on. Generally, the commentary and testing seem reasonable to me. I haven't done an in-depth check of the functional changes, but at least there is nothing obvious going awry. This is good enough for me to merge 👍

Comment on lines 217 to 232

def finish_now_key(x):
"""Determine the order of dependents that are ready to run and be released"""
return (-len(dependencies[x]), StrComparable(x))

root_total_dependencies = total_dependencies[list(root_nodes)[0]]
# Computing this for all keys can sometimes be relatively expensive :(
partition_keys = {
key: (
(min_dependencies - total_dependencies[key] + 1)
(root_total_dependencies - total_dependencies[key] + 1)
* (total_dependents - min_heights)
)
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we add an in-line comment for the requirement to be strictly positive for the cost funtion to be stable?

return task


def sanitize_dsk(dsk):
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: Adding some type hints to new functions (and order) would be a nice addition. The code is already difficult enough to understand if you know what data types to expect.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: Adding some type hints to new functions (and order) would be a nice addition. The code is already difficult enough to understand if you know what data types to expect.

I intend to follow up with this (also some variable renaming) but this is definitely out of scope for this PR

@fjetter
Copy link
Copy Markdown
Member Author

fjetter commented Dec 13, 2023

In case anybody is still stumbling over this PR, we recently merged a follow up #10660 that is rewriting this entire code section. If you are encountering issues with this, please open a new ticket

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

enhancement Improve existing functionality or make things work better needs review Needs review from a contributor.

Projects

None yet

5 participants