Skip to content
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

FEAT-#4419: Extend virtual partitioning API to pandas on Dask #4420

Merged
Merged
Show file tree
Hide file tree
Changes from 10 commits
Commits
Show all changes
30 commits
Select commit Hold shift + click to select a range
28cdc19
FEAT-#4419: Extend virtual partitioning API to pandas on Dask
RehanSD Apr 25, 2022
15b0072
Format code
RehanSD Apr 25, 2022
ae4eb6c
Fix list_of_blocks add list_of_ips
RehanSD Apr 26, 2022
026bd14
Fix documentation
RehanSD Apr 26, 2022
fa377a8
Fix docs
RehanSD Apr 26, 2022
b9d68f7
Update modin/core/execution/dask/implementations/pandas_on_dask/parti…
RehanSD May 2, 2022
17a9c65
Merge remote-tracking branch 'upstream/master' into rehan/virtual_par…
RehanSD Jun 13, 2022
348272d
Merge remote-tracking branch 'upstream/master' into rehan/virtual_par…
RehanSD Jun 15, 2022
6784b4b
Add comments to virtual partition, and update drain call queue to acc…
RehanSD Jun 15, 2022
4efd83b
lint
RehanSD Jun 15, 2022
af4edbc
copy rebalance_partitions implementation to dask virtual partitions
RehanSD Jun 15, 2022
0ac6de3
Fix rebalance_partitions, update groupby test, and add test for virtu…
RehanSD Jun 16, 2022
8c44eb1
Add name to release notes
RehanSD Jun 16, 2022
a5af1ff
lint
RehanSD Jun 16, 2022
971e00a
Refactor to reduce code redundancy
RehanSD Jun 16, 2022
87cad7c
fix docs
RehanSD Jun 16, 2022
0de19cd
Apply suggestions from code review
RehanSD Jun 16, 2022
4f5dd50
update tests
RehanSD Jun 16, 2022
f5d3eb1
Merge branch 'rehan/virtual_partitioning_dask' of https://github.com/…
RehanSD Jun 16, 2022
5e748cd
Fix typo in tests
RehanSD Jun 16, 2022
0cc25c4
Add more comprehensive test
RehanSD Jun 16, 2022
43296b2
Merge remote-tracking branch 'upstream/master' into rehan/virtual_par…
RehanSD Jun 17, 2022
66c4360
Resolve col width issue
RehanSD Jun 19, 2022
9f48f0d
lint
RehanSD Jun 19, 2022
8dab0d1
flake8
RehanSD Jun 19, 2022
2d9f150
Resolve col width issue without hurting perf
RehanSD Jun 20, 2022
9a945e8
Move concat into testr
RehanSD Jun 27, 2022
5b322ee
Address review comments
RehanSD Jul 12, 2022
a761182
Merge remote-tracking branch 'upstream/master' into rehan/virtual_par…
RehanSD Jul 12, 2022
6a9855c
fix typos
RehanSD Jul 12, 2022
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions docs/release_notes/release_notes-0.16.0.rst
Original file line number Diff line number Diff line change
Expand Up @@ -39,4 +39,5 @@ Contributors
@mvashishtha
@NickCrews
@prutskov
@vnlitvinov
@RehanSD
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@

from modin.core.dataframe.pandas.dataframe.dataframe import PandasDataframe
from ..partitioning.partition_manager import PandasOnDaskDataframePartitionManager
from modin.core.execution.dask.common.engine_wrapper import DaskWrapper


class PandasOnDaskDataframe(PandasDataframe):
Expand All @@ -41,22 +42,63 @@ class PandasOnDaskDataframe(PandasDataframe):

_partition_mgr_cls = PandasOnDaskDataframePartitionManager

def _get_partition_size_along_axis(self, partition, axis=0):
Copy link
Collaborator

Choose a reason for hiding this comment

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

Per #4494, we are currently calculating partition shapes serially in other places than the one you changed here, e.g. here for Ray, in Ray virtual partitioning length, and in the same place length for the new dask virtual partition class. The dask blocking means that this function actually blocks on inner partitions if the Modin frame consists of virtual partitions that are themselves made of virtual partitions.

In my opinion, we should parallelize getting all the partition shapes correctly in a separate fix for #4494.

Copy link
Collaborator

Choose a reason for hiding this comment

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

Ah, I'm wrong about this function blocking on inner partitions, because it doesn't call length or width. So I think it's correct. Still, this part seems outside the scope of this PR. I'd rather worry about this fix in a separate PR for #4494.

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

I think it makes sense to include here, since the original code to get length + width in parallel breaks when applied to axis partitions so I'll need to fix that code anyways, and if I'm doing that, I may as well fix the code to just do it all in parallel right?

Copy link
Collaborator

Choose a reason for hiding this comment

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

If the original code was broken, I think it's okay to include the partial fix for #4494 here.

"""
Compute the length along the specified axis of the specified partition.

Parameters
----------
partition : ``PandasOnDaskDataframeVirtualPartition`` or ``PandasOnDaskDataframePartition``
The partition whose size to compute.
axis : int, default: 0
The axis along which to compute size.

Returns
-------
list
A list of lengths along the specified axis that sum to the overall length of the partition
along the specified axis.

Notes
-----
This utility function is used to ensure that computation occurs asynchronously across all partitions
whether the partitions are virtual or physical partitions.
"""
if isinstance(partition, self._partition_mgr_cls._partition_class):
return [
partition.apply(
lambda df: len(df) if not axis else len(df.columns)
)._data
]
elif partition.axis == axis:
return [
ptn.apply(lambda df: len(df) if not axis else len(df.columns))._data
for ptn in partition.list_of_partitions_to_combine
]
return [
partition.list_of_partitions_to_combine[0]
.apply(lambda df: len(df) if not axis else (len(df.columns)))
._data
]

@property
def _row_lengths(self):
"""
Compute the row partitions lengths if they are not cached.
Compute ther row partitions lengths if they are not cached.
Copy link
Collaborator

Choose a reason for hiding this comment

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

Suggested change
Compute ther row partitions lengths if they are not cached.
Compute the row partition lengths if they are not cached.

Copy link
Collaborator

Choose a reason for hiding this comment

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

not fixed


Returns
-------
list
A list of row partitions lengths.
"""
if self._row_lengths_cache is None:
self._row_lengths_cache = (
self._partition_mgr_cls.get_objects_from_partitions(
[obj.apply(lambda df: len(df)) for obj in self._partitions.T[0]]
)
row_lengths_list = DaskWrapper.materialize(
[
YarShev marked this conversation as resolved.
Show resolved Hide resolved
self._get_partition_size_along_axis(obj, axis=0)
for obj in self._partitions.T[0]
]
)
self._row_lengths_cache = [sum(len_list) for len_list in row_lengths_list]
return self._row_lengths_cache

@property
Expand All @@ -70,12 +112,13 @@ def _column_widths(self):
A list of column partitions widths.
"""
if self._column_widths_cache is None:
self._column_widths_cache = (
self._partition_mgr_cls.get_objects_from_partitions(
[
obj.apply(lambda df: len(df.columns))
for obj in self._partitions[0]
]
)
col_widths_list = DaskWrapper.materialize(
YarShev marked this conversation as resolved.
Show resolved Hide resolved
[
self._get_partition_size_along_axis(obj, axis=1)
for obj in self._partitions[0]
]
)
self._column_widths_cache = [
sum(width_list) for width_list in col_widths_list
]
return self._column_widths_cache
2 changes: 1 addition & 1 deletion modin/pandas/test/test_groupby.py
Original file line number Diff line number Diff line change
Expand Up @@ -1982,7 +1982,7 @@ def test_groupby_with_virtual_partitions():

# Check that the constructed Modin DataFrame has virtual partitions when
# using Ray or Dask, and doesn't when using another execution engines.
if Engine.get() == "Ray" or Engine.get() == "Dask":
if Engine.get() in ["Ray", "Dask"]:
assert issubclass(
type(big_modin_df._query_compiler._modin_frame._partitions[0][0]),
PandasDataframeAxisPartition,
Expand Down
81 changes: 61 additions & 20 deletions modin/test/storage_formats/pandas/test_internals.py
Original file line number Diff line number Diff line change
Expand Up @@ -112,36 +112,77 @@ def func_to_apply(partition, row_internal_indices, col_internal_indices, item):
df_equals(md_df, pd_df)


small_dfs = [
pd.DataFrame(
[[i + j for j in range(0, 1000)]],
columns=[f"col{j}" for j in range(1, 1001)],
index=pd.Index([i - 1]),
)
for i in range(1, 100001, 1000)
]
Copy link
Collaborator

Choose a reason for hiding this comment

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

Suggested change
small_dfs = [
pd.DataFrame(
[[i + j for j in range(0, 1000)]],
columns=[f"col{j}" for j in range(1, 1001)],
index=pd.Index([i - 1]),
)
for i in range(1, 100001, 1000)
]
small_dfs = [
pd.DataFrame([[i + j for j in range(0, 1000)]]).add_prefix('col')
for i in range(100 * 1000, 1000)
]

I prefer to have the values start from 0. Then we can create the dataframe like this. Let me know if you prefer the current way.

large_df = pd.DataFrame(
RehanSD marked this conversation as resolved.
Show resolved Hide resolved
[[i + j for j in range(1, 1000)] for i in range(0, 100000, 1000)],
columns=[f"col{j}" for j in range(1, 1000)],
index=pd.Index(list(range(0, 100000, 1000))),
)
Copy link
Collaborator

Choose a reason for hiding this comment

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

Suggested change
large_df = pd.DataFrame(
[[i + j for j in range(1, 1000)] for i in range(0, 100000, 1000)],
columns=[f"col{j}" for j in range(1, 1000)],
index=pd.Index(list(range(0, 100000, 1000))),
)
large_df = pd.DataFrame(
[[i + j for j in range(1000)] for i in range(0, 100 * 1000, 1000)]]
).add_prefix('col')

Unless you see a good reason for the 1-indexing and a custom index, I prefer this.



@pytest.mark.skipif(
Engine.get() != "Dask" and Engine.get() != "Ray",
Engine.get() not in ("Dask", "Ray"),
reason="Rebalancing partitions is only supported for Dask and Ray engines",
)
def test_rebalance_partitions():
small_dfs = [
pd.DataFrame(
[[i + j for j in range(0, 100)]],
columns=[f"col{j}" for j in range(1, 101)],
index=pd.Index([i - 1]),
)
for i in range(1, 10001, 100)
]
large_df = pd.concat(small_dfs)
@pytest.mark.parametrize(
"large_df,col_length",
[
(pd.concat(small_dfs), 100),
(pd.concat([pd.concat(small_dfs)] + small_dfs[:3]), 103),
Copy link
Collaborator

Choose a reason for hiding this comment

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

Does this really trigger two rebalances?

(pd.concat([large_df] + small_dfs[:3]), 103),
RehanSD marked this conversation as resolved.
Show resolved Hide resolved
],
)
def test_rebalance_partitions(large_df, col_length):
large_modin_frame = large_df._query_compiler._modin_frame
assert large_modin_frame._partitions.shape == (
4,
4,
NPartitions.get(),
NPartitions.get(),
), "Partitions were not rebalanced after concat."
assert all(
isinstance(ptn, large_modin_frame._partition_mgr_cls._column_partitions_class)
for ptn in large_modin_frame._partitions.flatten()
)
large_df = large_df.apply(lambda x: x + 1)
large_modin_frame = large_df._query_compiler._modin_frame
assert large_modin_frame._partitions.shape == (
# The following check tests that we can correctly form full-axis virtual partitions
# over the orthogonal axis from non-full-axis virtual partitions.

def col_apply_func(col):
assert len(col) == col_length, "Partial axis partition detected."
Copy link
Collaborator

Choose a reason for hiding this comment

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

Why do we need this assert? Why isn't it enough to check that the column-wise apply does what pandas would do?

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

We want to ensure that this is a full-axis partition, and not just a partial axis partition - the function as I've written it will run correctly even if it is a partial axis partition, so I just thought this assert would be a good way to keep the function simple, but still test that it is full axis.

Copy link
Collaborator

Choose a reason for hiding this comment

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

I think this test should check for functional correctness. We already check that we get the apply result correct when the starting partitions are non-full-axis virtual partitions. I don't think it's right to check for internal implementation details here.

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

Hmm - I'm not sure that I agree with that - this file is called test_internals which would imply its testing internal details, and it makes sense to ensure that the internal partitioning code didn't incorrectly make a full axis partition.

Copy link
Collaborator

Choose a reason for hiding this comment

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

I don't think it's appropriate to test any internal detail here just because it's test_internals. Why not do something like have this function return col[::-1]? That way we can be pretty confident that modin is really applying the function across the full axis. I think testing the intermediate representations during the function call is outside the scope of this particular test case.

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

I could do that, but then I would have to go into each partition to determine what the last row is and then double check the output, so this feels "less intrusive" open to other suggestions though! Perhaps the solution may be to add unit testing for partition layer stuff into the codebase like @pyrito suggested, and move this code to there, so that test_internals just tests the overall internals as its supposed to, and implementation details can be tested at that level?

return col + 1

large_df = large_df.apply(col_apply_func)
new_large_modin_frame = large_df._query_compiler._modin_frame
assert new_large_modin_frame._partitions.shape == (
NPartitions.get(),
NPartitions.get(),
), "Partitions list shape is incorrect."
assert all(
isinstance(ptn, new_large_modin_frame._partition_mgr_cls._partition_class)
for ptn in new_large_modin_frame._partitions.flatten()
), "Partitions are not block partitioned after apply."
large_df = pd.DataFrame(
query_compiler=large_df._query_compiler.__constructor__(large_modin_frame)
)
# The following check tests that we can correctly form full-axis virtual partitions
# over the same axis from non-full-axis virtual partitions.

def row_apply_func(row):
assert len(row) == 1000, "Partial axis partition detected."
Copy link
Collaborator

Choose a reason for hiding this comment

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

ditto about the assert

return row + 1

large_df = large_df.apply(row_apply_func, axis=1)
new_large_modin_frame = large_df._query_compiler._modin_frame
assert new_large_modin_frame._partitions.shape == (
4,
4,
), "Partitions are not block partitioned after apply."
), "Partitions list shape is incorrect."
assert all(
isinstance(ptn, large_modin_frame._partition_mgr_cls._partition_class)
for ptn in large_modin_frame._partitions.flatten()
)
isinstance(ptn, new_large_modin_frame._partition_mgr_cls._partition_class)
for ptn in new_large_modin_frame._partitions.flatten()
), "Partitions are not block partitioned after apply."