diff --git a/modin/core/dataframe/pandas/partitioning/partition_manager.py b/modin/core/dataframe/pandas/partitioning/partition_manager.py index 5df9b79b907..da1eb4a17c8 100644 --- a/modin/core/dataframe/pandas/partitioning/partition_manager.py +++ b/modin/core/dataframe/pandas/partitioning/partition_manager.py @@ -1234,13 +1234,13 @@ def apply_func_to_indices_both_axis( all_parts = np.array( [ [ - partitions_copy[row_blk_idx, col_blk_idx] - for row_blk_idx, _ in row_partitions + partition_copy[row_blk_idx, col_blk_idx] + for row_blk_idx, _ in row_partitions_list ] for col_blk_idx, _ in col_partitions_list ] ).flatten() - self._update_partition_dimension_caches(all_parts) + cls._update_partition_dimension_caches(all_parts) def compute_part_size(indexer, remote_part, part_idx, axis): """Compute indexer length along the specified axis for the passed partition.""" diff --git a/modin/core/execution/dask/implementations/pandas_on_dask/dataframe/dataframe.py b/modin/core/execution/dask/implementations/pandas_on_dask/dataframe/dataframe.py index ab9ddb54d83..77b4e468a47 100644 --- a/modin/core/execution/dask/implementations/pandas_on_dask/dataframe/dataframe.py +++ b/modin/core/execution/dask/implementations/pandas_on_dask/dataframe/dataframe.py @@ -72,14 +72,9 @@ def len_fn(df): return [partition.apply(len_fn)._data] elif partition.axis == axis: return [ - ptn.apply(len_fn)._data - for ptn in partition.list_of_block_partitions + ptn.apply(len_fn)._data for ptn in partition.list_of_block_partitions ] - return [ - partition.list_of_block_partitions[0] - .apply(len_fn) - ._data - ] + return [partition.list_of_block_partitions[0].apply(len_fn)._data] @property def _row_lengths(self): diff --git a/modin/core/execution/dask/implementations/pandas_on_dask/partitioning/virtual_partition.py b/modin/core/execution/dask/implementations/pandas_on_dask/partitioning/virtual_partition.py index d33f6169956..050cf44160f 100644 --- a/modin/core/execution/dask/implementations/pandas_on_dask/partitioning/virtual_partition.py +++ b/modin/core/execution/dask/implementations/pandas_on_dask/partitioning/virtual_partition.py @@ -421,7 +421,7 @@ def width(self): for obj in self.list_of_partitions_to_combine ] new_widths = DaskWrapper.materialize( - [cache for cache in caches if isinstance(promise, Future)] + [cache for cache in caches if isinstance(cache, Future)] ) dask_idx = 0 for i, cache in enumerate(caches): diff --git a/modin/core/execution/ray/implementations/pandas_on_ray/partitioning/virtual_partition.py b/modin/core/execution/ray/implementations/pandas_on_ray/partitioning/virtual_partition.py index 18f47544717..54e62dfb700 100644 --- a/modin/core/execution/ray/implementations/pandas_on_ray/partitioning/virtual_partition.py +++ b/modin/core/execution/ray/implementations/pandas_on_ray/partitioning/virtual_partition.py @@ -427,7 +427,7 @@ def width(self): for obj in self.list_of_partitions_to_combine ] new_widths = ray.get( - [cache for cache in caches if isinstance(promise, ObjectIDType)] + [cache for cache in caches if isinstance(cache, ObjectIDType)] ) ray_idx = 0 for i, cache in enumerate(caches):