From 3cae9e797e3ea33c5095a38ce57f3f774ca89225 Mon Sep 17 00:00:00 2001 From: "saksham.kumar" Date: Tue, 19 Apr 2022 15:13:25 -0400 Subject: [PATCH 01/32] PoC complete. need to test --- mars/dataframe/groupby/aggregation.py | 210 +++++++++++++++- mars/dataframe/groupby/sort.py | 332 ++++++++++++++++++++++++++ mars/opcodes.py | 3 + 3 files changed, 538 insertions(+), 7 deletions(-) create mode 100644 mars/dataframe/groupby/sort.py diff --git a/mars/dataframe/groupby/aggregation.py b/mars/dataframe/groupby/aggregation.py index 2630f084c5..063c7f3c09 100644 --- a/mars/dataframe/groupby/aggregation.py +++ b/mars/dataframe/groupby/aggregation.py @@ -14,6 +14,7 @@ import functools import itertools +import threading import typing import uuid from typing import List @@ -22,6 +23,7 @@ import pandas as pd from scipy.stats import variation +from .sort import DataFramePSRSGroupbySample, DataFrameGroupbyConcatPivot, DataFrameGroupbySortShuffle from ... import opcodes as OperandDef from ...config import options from ...core.custom_log import redirect_custom_log @@ -44,7 +46,7 @@ pd_release_version, estimate_pandas_size, ) -from ..core import GROUPBY_TYPE +from ..core import GROUPBY_TYPE, IndexValue from ..merge import DataFrameConcat from ..operands import DataFrameOperand, DataFrameOperandMixin, DataFrameShuffleProxy from ..reduction.core import ( @@ -220,7 +222,7 @@ def _call_dataframe(self, groupby, input_df): shape = (np.nan, agg_df.shape[1]) index_value = parse_index(agg_df.index, groupby.key, groupby.index_value.key) - index_value.value.should_be_monotonic = True + # index_value.value.should_be_monotonic = True # make sure if as_index=False takes effect self._fix_as_index(agg_df.index) @@ -248,7 +250,7 @@ def _call_series(self, groupby, in_series): index_value = parse_index( agg_result.index, groupby.key, groupby.index_value.key ) - index_value.value.should_be_monotonic = True + # index_value.value.should_be_monotonic = True inputs = self._get_inputs([in_series]) @@ -293,6 +295,105 @@ def __call__(self, groupby): else: return self._call_series(groupby, df) + @classmethod + def partition_merge_data( + cls, op, partition_chunks, proxy_chunk, in_df + ): + # stage 4: all *ith* classes are gathered and merged + partition_sort_chunks = [] + properties = dict( + by=op.groupby_params['by'], + gpu=op.is_gpu() + ) + + for i, partition_chunk in enumerate(partition_chunks): + partition_shuffle_reduce = DataFrameGroupbySortShuffle( + stage=OperandStage.reduce, + reducer_index=(i, 0), + output_types=op.output_types, + **properties + ) + chunk_shape = list(partition_chunk.shape) + chunk_shape[0] = np.nan + + kw = dict( + shape=tuple(chunk_shape), + index=partition_chunk.index, + index_value=partition_chunk.index_value, + ) + if op.outputs[0].ndim == 2: + kw.update( + dict( + columns_value=partition_chunk.columns_value, + dtypes=partition_chunk.dtypes + ) + ) + else: + kw.update(dict(dtype=partition_chunk.dtype, name=partition_chunk.name)) + cs = partition_shuffle_reduce.new_chunks([proxy_chunk], **kw) + + partition_sort_chunks.append(cs[0]) + return partition_sort_chunks + + @classmethod + def partition_local_data( + cls, op, sorted_chunks, concat_pivot_chunk, in_df + ): + properties = dict( + by=op.groupby_params['by'], + gpu=op.is_gpu() + ) + + map_chunks = [] + chunk_shape = (in_df.chunk_shape[0], 1) + for chunk in sorted_chunks: + chunk_inputs = [chunk, concat_pivot_chunk] + map_chunk_op = DataFrameGroupbySortShuffle( + shuffle_size=chunk_shape[0], + stage=OperandStage.map, + n_partition = len(sorted_chunks), + output_types=op.output_types, + **properties + ) + kw = dict() + if op.outputs[0].ndim == 2: + kw.update( + dict( + columns_value=chunk_inputs[0].columns_value, + dtypes=chunk_inputs[0].dtypes + ) + ) + else: + kw.update(dict(dtype=chunk_inputs[0].dtype, name=chunk_inputs[0].name)) + + map_chunks.append( + map_chunk_op.new_chunk( + chunk_inputs, + shape=chunk_shape, + index=chunk.index, + index_value=chunk_inputs[0].index_value, + **kw + ) + ) + + return map_chunks + + @classmethod + def _gen_shuffle_chunks_with_pivot(cls, op, in_df, chunks, pivot): + map_chunks = cls.partition_local_data( + op, chunks, pivot, in_df + ) + + proxy_chunk = DataFrameShuffleProxy(output_types=op.output_types).new_chunk( + map_chunks, shape=() + ) + + partition_sort_chunks = cls.partition_merge_data( + op, map_chunks, proxy_chunk, in_df + ) + + return partition_sort_chunks + @classmethod def _gen_shuffle_chunks(cls, op, in_df, chunks): # generate map chunks @@ -333,9 +434,20 @@ def _gen_shuffle_chunks(cls, op, in_df, chunks): index_value=None, ) ) - return reduce_chunks + @classmethod + def _gen_pivot_chunks( + cls, + op: "DataFrameGroupByAgg", + in_chunks: List[ChunkType], + func_infos: ReductionSteps, + ): + # find groups -> dataframe group by operand + # stage 1: local sort and regular samples collected + # stage 2: gather and merge samples, choose and broadcast p-1 pivots + pass + @classmethod def _gen_map_chunks( cls, @@ -375,6 +487,7 @@ def _gen_map_chunks( index=new_index, index_value=out_df.index_value, columns_value=out_df.columns_value, + dtypes=chunk.dtypes, ) else: map_chunk = map_op.new_chunk( @@ -382,6 +495,7 @@ def _gen_map_chunks( shape=(out_df.shape[0], 1), index=new_index, index_value=out_df.index_value, + dtype=chunk.dtype, ) map_chunks.append(map_chunk) return map_chunks @@ -422,7 +536,85 @@ def _tile_with_shuffle( ): # First, perform groupby and aggregation on each chunk. agg_chunks = cls._gen_map_chunks(op, in_df.chunks, out_df, func_infos) - return cls._perform_shuffle(op, agg_chunks, in_df, out_df, func_infos) + pivot_chunk = None + if op.groupby_params['sort'] and len(in_df.chunks) > 1: + out_idx = (0,) if in_df.ndim == 2 else (), + agg_chunk_len = len(agg_chunks) + sample_chunks = cls._sample_chunks(op, agg_chunks) + pivot_chunk = cls._gen_pivot_chunk(op, sample_chunks, out_idx, agg_chunk_len) + + # agg_chunks = agg_chunks + sample_chunks + return cls._perform_shuffle(op, agg_chunks, in_df, out_df, func_infos, pivot_chunk) + + @classmethod + def _gen_pivot_chunk(cls, op, sample_chunks, out_idx, agg_chunk_len): + + properties = dict( + by=op.groupby_params['by'], + gpu=op.is_gpu(), + ) + + # stage 2: gather and merge samples, choose and broadcast p-1 pivots + kind = 'quicksort' + output_types = [OutputType.tensor] + + concat_pivot_op = DataFrameGroupbyConcatPivot( + kind=kind, + n_partition=agg_chunk_len, + output_types=output_types, + **properties + ) + + # concat_pivot_index = out_idx[: 0] + (0,) + out_idx[0:] + concat_pivot_chunk = concat_pivot_op.new_chunk( + sample_chunks, + shape=(agg_chunk_len,), + dtype=object, + # shape=concat_pivot_shape, + # index=concat_pivot_index, + ) + return concat_pivot_chunk + + @classmethod + def _sample_chunks(cls, op, agg_chunks): + chunk_shape = len(agg_chunks) + sampled_chunks = [] + + properties = dict( + by=op.groupby_params['by'], + gpu=op.is_gpu(), + ) + + for i, chunk in enumerate(agg_chunks): + chunk_op = DataFramePSRSGroupbySample ( + kind='quicksort', + n_partition=chunk_shape, + output_types=op.output_types, + **properties + ) + kws = [] + sampled_shape = ( + (chunk_shape, len(op.groupby_params['by'])) if op.groupby_params['by'] else (chunk_shape,) + ) + kws.append( + { + "shape": sampled_shape, + "index_value": chunk.index_value, + "index": (i, 0), + "type": "regular_sampled", + } + ) + if op.outputs[0].ndim == 2: + kws[0].update( + {"columns_value": chunk.columns_value, "dtypes": chunk.dtypes} + ) + else: + kws[0].update({"dtype": chunk.dtype}) + + chunk = chunk_op.new_chunk([chunk], kws=kws) + sampled_chunks.append(chunk) + + return sampled_chunks @classmethod def _perform_shuffle( @@ -432,9 +624,13 @@ def _perform_shuffle( in_df: TileableType, out_df: TileableType, func_infos: ReductionSteps, + pivot_chunk ): # Shuffle the aggregation chunk. - reduce_chunks = cls._gen_shuffle_chunks(op, in_df, agg_chunks) + if op.groupby_params["sort"] and pivot_chunk is not None: + reduce_chunks = cls._gen_shuffle_chunks_with_pivot(op, in_df, agg_chunks, pivot_chunk) + else: + reduce_chunks = cls._gen_shuffle_chunks(op, in_df, agg_chunks) # Combine groups agg_chunks = [] @@ -670,7 +866,7 @@ def _get_grouped(cls, op: "DataFrameGroupByAgg", df, ctx, copy=False, grouper=No grouped = df.groupby(**params) else: # for the intermediate phases, do not sort - params["sort"] = False + # params["sort"] = False grouped = df.groupby(**params) if selection is not None: diff --git a/mars/dataframe/groupby/sort.py b/mars/dataframe/groupby/sort.py new file mode 100644 index 0000000000..99099a6876 --- /dev/null +++ b/mars/dataframe/groupby/sort.py @@ -0,0 +1,332 @@ +import numpy as np +import pandas as pd + +from mars.dataframe.operands import DataFrameOperandMixin +from mars.dataframe.sort.psrs import DataFramePSRSChunkOperand +from mars.utils import lazy_import +from ..utils import is_cudf + +from ... import opcodes as OperandDef +from ...core.operand import MapReduceOperand, OperandStage +from ...serialization.serializables import StringField, Int32Field, BoolField, ListField + +cudf = lazy_import("cudf", globals=globals()) + +class _Largest: + """ + This util class resolve TypeError when + comparing strings with None values + """ + + def __lt__(self, other): + return False + + def __gt__(self, other): + return self is not other + + +_largest = _Largest() + +class DataFrameGroupbyConcatPivot(DataFramePSRSChunkOperand, DataFrameOperandMixin): + _op_type_ = OperandDef.GROUPBY_SORT_PIVOT + + @property + def output_limit(self): + return 1 + + @classmethod + def execute(cls, ctx, op): + inputs = [ctx[c.key] for c in op.inputs if len(ctx[c.key]) > 0] + if len(inputs) == 0: + # corner case: nothing sampled, we need to do nothing + ctx[op.outputs[-1].key] = ctx[op.inputs[0].key] + return + + xdf = pd if isinstance(inputs[0], (pd.DataFrame, pd.Series)) else cudf + + a = xdf.concat(inputs, axis=0) + # a = a.reset_index(level=[op.by])[op.by] + a = a.sort_index() + index = a.index.drop_duplicates() + + p = len(inputs) + if len(index) < p: + num = p // len(index) + 1 + index = index.append([index] * (num-1)) + # assert a.shape[op.axis] == p * len(op.inputs) + + index = index.sort_values() + + values = index.values + slc = np.linspace( + p - 1, len(index) - 1, num=len(op.inputs) - 1, endpoint=False + ).astype(int) + out = values[slc] + ctx[op.outputs[-1].key] = out + +class DataFramePSRSGroupbySample(DataFramePSRSChunkOperand, DataFrameOperandMixin): + _op_type_ = OperandDef.GROUPBY_SORT_REGULAR_SAMPLE + + @property + def output_limit(self): + return 1 + + @classmethod + def execute(cls, ctx, op): + a = ctx[op.inputs[0].key][0] + xdf = pd if isinstance(a, (pd.DataFrame, pd.Series)) else cudf + if len(a) == 0: + # when chunk is empty, return the empty chunk itself + ctx[op.outputs[0].key] = a + return + + # if op.sort_type == "sort_values": + # ctx[op.outputs[0].key] = res = execute_sort_values(a, op) + # else: + # ctx[op.outputs[0].key] = res = execute_sort_index(a, op) + + by = op.by + # add_distinct_col = bool(int(os.environ.get("PSRS_DISTINCT_COL", "0"))) + # if ( + # add_distinct_col + # and isinstance(a, xdf.DataFrame) + # and op.sort_type == "sort_values" + # ): + # # when running under distributed mode, we introduce an extra column + # # to make sure pivots are distinct + # chunk_idx = op.inputs[0].index[0] + # distinct_col = ( + # _PSRS_DISTINCT_COL + # if a.columns.nlevels == 1 + # else (_PSRS_DISTINCT_COL,) + ("",) * (a.columns.nlevels - 1) + # ) + # res[distinct_col] = np.arange( + # chunk_idx << 32, (chunk_idx << 32) + len(a), dtype=np.int64 + # ) + # by = list(by) + [distinct_col] + + n = op.n_partition + if a.shape[0] < n: + num = n // a.shape[0] + 1 + a = xdf.concat([a] * num).sort_values(by=op.by) + + w = a.shape[0] * 1.0 / (n + 1) + + slc = np.linspace( + max(w - 1, 0), a.shape[0] - 1, num=n, endpoint=False + ).astype(int) + + out = a.iloc[slc] + + ctx[op.outputs[-1].key] = out + +class DataFrameGroupbySortShuffle(MapReduceOperand, DataFrameOperandMixin): + _op_type_ = OperandDef.GROUPBY_SORT_SHUFFLE + + _sort_type = StringField("sort_type") + + # for shuffle map + _axis = Int32Field("axis") + _by = ListField("by") + _ascending = BoolField("ascending") + _inplace = BoolField("inplace") + _na_position = StringField("na_position") + _n_partition = Int32Field("n_partition") + + # for sort_index + _level = ListField("level") + _sort_remaining = BoolField("sort_remaining") + + # for shuffle reduce + _kind = StringField("kind") + + def __init__( + self, + sort_type=None, + by=None, + axis=None, + ascending=None, + n_partition=None, + na_position=None, + inplace=None, + kind=None, + level=None, + sort_remaining=None, + output_types=None, + **kw + ): + super().__init__( + _sort_type=sort_type, + _by=by, + _axis=axis, + _ascending=ascending, + _n_partition=n_partition, + _na_position=na_position, + _inplace=inplace, + _kind=kind, + _level=level, + _sort_remaining=sort_remaining, + _output_types=output_types, + **kw + ) + + @property + def sort_type(self): + return self._sort_type + + @property + def by(self): + return self._by + + @property + def axis(self): + return self._axis + + @property + def ascending(self): + return self._ascending + + @property + def inplace(self): + return self._inplace + + @property + def na_position(self): + return self._na_position + + @property + def level(self): + return self._level + + @property + def sort_remaining(self): + return self._sort_remaining + + @property + def n_partition(self): + return self._n_partition + + @property + def kind(self): + return self._kind + + @property + def output_limit(self): + return 1 + + @staticmethod + def _calc_poses(src_cols, pivots, ascending=True): + records = src_cols.to_records(index=False) + p_records = pivots.to_records(index=False) + if ascending: + poses = records.searchsorted(p_records, side="right") + else: + poses = len(records) - records[::-1].searchsorted(p_records, side="right") + del records, p_records + return poses + + @classmethod + def _execute_dataframe_map(cls, ctx, op): + df, pivots = [ctx[c.key] for c in op.inputs] + out = op.outputs[0] + + def _get_out_df(p_index, in_df): + if p_index == 0: + out_df = in_df.loc[:pivots[p_index]].iloc[:-1] + elif p_index == op.n_partition - 1: + out_df = in_df.loc[pivots[p_index-1]:] + else: + out_df = in_df.loc[pivots[p_index - 1]:pivots[p_index]].iloc[:-1] + return out_df + + for i in range(op.n_partition): + index = (i, 0) + if isinstance(df, tuple): + out_df = tuple(_get_out_df(i, x) for x in df) + else: + out_df = _get_out_df(i, df) + ctx[out.key, index] = out_df + + + @classmethod + def _calc_series_poses(cls, s, pivots, ascending=True): + if ascending: + poses = s.searchsorted(pivots, side="right") + else: + poses = len(s) - s.iloc[::-1].searchsorted(pivots, side="right") + return poses + + @classmethod + def _execute_series_map(cls, ctx, op): + a, pivots = [ctx[c.key] for c in op.inputs] + out = op.outputs[0] + + if len(a) == 0: + # when the chunk is empty, no slices can be produced + for i in range(op.n_partition): + ctx[out.key, (i,)] = a + return + + if isinstance(a, pd.Series): + try: + poses = cls._calc_series_poses(a, pivots) + except TypeError: + filled_a = a.fillna(_largest) + filled_pivots = pivots.fillna(_largest) + poses = cls._calc_series_poses( + filled_a, filled_pivots + ) + poses = (None,) + tuple(poses) + (None,) + for i in range(op.n_partition): + values = a.iloc[poses[i] : poses[i + 1]] + ctx[out.key, (i,)] = values + + @classmethod + def _execute_map(cls, ctx, op): + a = [ctx[c.key] for c in op.inputs][0] + if isinstance(a, tuple): + a = a[0] + if len(a.shape) == 2: + # DataFrame type + cls._execute_dataframe_map(ctx, op) + else: + # Series type + cls._execute_series_map(ctx, op) + + @classmethod + def _execute_reduce(cls, ctx, op: "DataFramePSRSShuffle"): + out_chunk = op.outputs[0] + raw_inputs = list(op.iter_mapper_data(ctx, pop=False)) + by = op.by + xdf = cudf if op.gpu else pd + + r = [] + + if isinstance(raw_inputs[0], tuple): + tuple_len = len(raw_inputs[0]) + for i in range(tuple_len): + r.append(xdf.concat([inp[i] for inp in raw_inputs], axis=0)) + r = tuple(r) + else: + r = xdf.concat(raw_inputs, axis=0) + + if isinstance(r, tuple): + ctx[op.outputs[0].key] = r + (by,) + else: + ctx[op.outputs[0].key] = (r, by) + + @classmethod + def estimate_size(cls, ctx, op): + super().estimate_size(ctx, op) + result = ctx[op.outputs[0].key] + if op.stage == OperandStage.reduce: + ctx[op.outputs[0].key] = (result[0], result[1] * 1.5) + else: + ctx[op.outputs[0].key] = result + + @classmethod + def execute(cls, ctx, op): + if op.stage == OperandStage.map: + cls._execute_map(ctx, op) + else: + cls._execute_reduce(ctx, op) diff --git a/mars/opcodes.py b/mars/opcodes.py index b5be7b4f6e..c601c94b17 100644 --- a/mars/opcodes.py +++ b/mars/opcodes.py @@ -422,6 +422,9 @@ GROUPBY_CONCAT = 2034 GROUPBY_HEAD = 2035 GROUPBY_SAMPLE_ILOC = 2036 +GROUPBY_SORT_REGULAR_SAMPLE = 2037 +GROUPBY_SORT_PIVOT = 2038 +GROUPBY_SORT_SHUFFLE = 2039 # parallel sorting by regular sampling PSRS_SORT_REGULAR_SMAPLE = 2040 From aa394938ed56ee118eb306f6c85881e09ff500f2 Mon Sep 17 00:00:00 2001 From: "saksham.kumar" Date: Wed, 20 Apr 2022 16:11:41 -0400 Subject: [PATCH 02/32] method auto will also use sort. added changes so that if partition doesnt exist in current chunk still we all data --- mars/dataframe/groupby/aggregation.py | 9 ++++++++- mars/dataframe/groupby/sort.py | 23 +++++++++++++++++++++-- 2 files changed, 29 insertions(+), 3 deletions(-) diff --git a/mars/dataframe/groupby/aggregation.py b/mars/dataframe/groupby/aggregation.py index 03163fdd29..b5869e7f49 100644 --- a/mars/dataframe/groupby/aggregation.py +++ b/mars/dataframe/groupby/aggregation.py @@ -816,8 +816,15 @@ def _tile_auto( return cls._combine_tree(op, chunks + left_chunks, out_df, func_infos) else: # otherwise, use shuffle + pivot_chunk = None + if op.groupby_params['sort'] and len(in_df.chunks) > 1: + out_idx = (0,) if in_df.ndim == 2 else (), + agg_chunk_len = len(chunks + left_chunks) + sample_chunks = cls._sample_chunks(op, chunks + left_chunks) + pivot_chunk = cls._gen_pivot_chunk(op, sample_chunks, out_idx, agg_chunk_len) + return cls._perform_shuffle( - op, chunks + left_chunks, in_df, out_df, func_infos + op, chunks + left_chunks, in_df, out_df, func_infos, pivot_chunk ) @classmethod diff --git a/mars/dataframe/groupby/sort.py b/mars/dataframe/groupby/sort.py index 99099a6876..b843f74189 100644 --- a/mars/dataframe/groupby/sort.py +++ b/mars/dataframe/groupby/sort.py @@ -47,7 +47,9 @@ def execute(cls, ctx, op): a = xdf.concat(inputs, axis=0) # a = a.reset_index(level=[op.by])[op.by] a = a.sort_index() + # print("a " + str(a)) index = a.index.drop_duplicates() + # print("index " + str(index)) p = len(inputs) if len(index) < p: @@ -56,12 +58,17 @@ def execute(cls, ctx, op): # assert a.shape[op.axis] == p * len(op.inputs) index = index.sort_values() + # print("index " + str(index)) values = index.values + # print("values " + str(values)) + slc = np.linspace( p - 1, len(index) - 1, num=len(op.inputs) - 1, endpoint=False ).astype(int) + # print("slc " + str(slc)) out = values[slc] + # print(out) ctx[op.outputs[-1].key] = out class DataFramePSRSGroupbySample(DataFramePSRSChunkOperand, DataFrameOperandMixin): @@ -117,6 +124,7 @@ def execute(cls, ctx, op): ).astype(int) out = a.iloc[slc] + # print(out) ctx[op.outputs[-1].key] = out @@ -232,19 +240,30 @@ def _execute_dataframe_map(cls, ctx, op): def _get_out_df(p_index, in_df): if p_index == 0: - out_df = in_df.loc[:pivots[p_index]].iloc[:-1] + if pivots[p_index] in in_df.index.values: + out_df = in_df.loc[:pivots[p_index]].iloc[:-1] + else: + out_df = in_df.loc[:pivots[p_index]] + # print("here " + str(out_df)) elif p_index == op.n_partition - 1: out_df = in_df.loc[pivots[p_index-1]:] else: - out_df = in_df.loc[pivots[p_index - 1]:pivots[p_index]].iloc[:-1] + if pivots[p_index] in in_df.index.values: + out_df = in_df.loc[pivots[p_index - 1]:pivots[p_index]].iloc[:-1] + else: + out_df = in_df.loc[pivots[p_index - 1]:pivots[p_index]] + # print("here " + str(out_df)) return out_df + # print("index " + str(out.key) + " " + str(df)) + for i in range(op.n_partition): index = (i, 0) if isinstance(df, tuple): out_df = tuple(_get_out_df(i, x) for x in df) else: out_df = _get_out_df(i, df) + # print("index " + str(out.key) + " " + str(index) + " out df " + str(out_df)) ctx[out.key, index] = out_df From 98b995a5fa9c4d71fcffbfa2b8b4d3e748c32ec4 Mon Sep 17 00:00:00 2001 From: "saksham.kumar" Date: Thu, 21 Apr 2022 21:59:39 -0400 Subject: [PATCH 03/32] fixing unit tests --- mars/dataframe/groupby/aggregation.py | 73 ++++++++++----------------- mars/dataframe/groupby/sort.py | 16 ++---- 2 files changed, 32 insertions(+), 57 deletions(-) diff --git a/mars/dataframe/groupby/aggregation.py b/mars/dataframe/groupby/aggregation.py index b5869e7f49..44909374a3 100644 --- a/mars/dataframe/groupby/aggregation.py +++ b/mars/dataframe/groupby/aggregation.py @@ -310,7 +310,7 @@ def partition_merge_data( partition_shuffle_reduce = DataFrameGroupbySortShuffle( stage=OperandStage.reduce, reducer_index=(i, 0), - output_types=op.output_types, + output_types=[OutputType.dataframe_groupby], **properties ) chunk_shape = list(partition_chunk.shape) @@ -321,17 +321,16 @@ def partition_merge_data( index=partition_chunk.index, index_value=partition_chunk.index_value, ) - if op.outputs[0].ndim == 2: - kw.update( - dict( - columns_value=partition_chunk.columns_value, - dtypes=partition_chunk.dtypes - ) - ) - else: - kw.update(dict(dtype=partition_chunk.dtype, name=partition_chunk.name)) + # if op.outputs[0].ndim == 2: + # kw.update( + # dict( + # columns_value=partition_chunk.columns_value, + # # dtypes=partition_chunk.dtypes + # ) + # ) + # else: + # kw.update(dict(dtype=partition_chunk.dtype, name=partition_chunk.name)) cs = partition_shuffle_reduce.new_chunks([proxy_chunk], **kw) - partition_sort_chunks.append(cs[0]) return partition_sort_chunks @@ -352,19 +351,21 @@ def partition_local_data( shuffle_size=chunk_shape[0], stage=OperandStage.map, n_partition = len(sorted_chunks), - output_types=op.output_types, - **properties + output_types=[OutputType.dataframe_groupby], + # columns_value=chunk_inputs[0].columns_value, ) kw = dict() - if op.outputs[0].ndim == 2: - kw.update( - dict( - columns_value=chunk_inputs[0].columns_value, - dtypes=chunk_inputs[0].dtypes - ) - ) - else: - kw.update(dict(dtype=chunk_inputs[0].dtype, name=chunk_inputs[0].name)) + # if op.outputs[0].ndim == 2: + # kw.update( + # dict( + # columns_value=chunk_inputs[0].columns_value, + # # dtypes=chunk_inputs[0].dtypes + # ) + # ) + # else: + # kw.update(dict(dtype=chunk_inputs[0].dtype, name=chunk_inputs[0].name)) + # kw.update(dict(name=chunk_inputs[0].name)) + map_chunks.append( map_chunk_op.new_chunk( @@ -372,7 +373,7 @@ def partition_local_data( shape=chunk_shape, index=chunk.index, index_value=chunk_inputs[0].index_value, - **kw + # **kw ) ) @@ -384,7 +385,7 @@ def _gen_shuffle_chunks_with_pivot(cls, op, in_df, chunks, pivot): op, chunks, pivot, in_df ) - proxy_chunk = DataFrameShuffleProxy(output_types=op.output_types).new_chunk( + proxy_chunk = DataFrameShuffleProxy(output_types=[OutputType.dataframe]).new_chunk( map_chunks, shape=() ) @@ -436,18 +437,6 @@ def _gen_shuffle_chunks(cls, op, in_df, chunks): ) return reduce_chunks - @classmethod - def _gen_pivot_chunks( - cls, - op: "DataFrameGroupByAgg", - in_chunks: List[ChunkType], - func_infos: ReductionSteps, - ): - # find groups -> dataframe group by operand - # stage 1: local sort and regular samples collected - # stage 2: gather and merge samples, choose and broadcast p-1 pivots - pass - @classmethod def _gen_map_chunks( cls, @@ -487,7 +476,6 @@ def _gen_map_chunks( index=new_index, index_value=out_df.index_value, columns_value=out_df.columns_value, - dtypes=chunk.dtypes, ) else: map_chunk = map_op.new_chunk( @@ -495,7 +483,6 @@ def _gen_map_chunks( shape=(out_df.shape[0], 1), index=new_index, index_value=out_df.index_value, - dtype=chunk.dtype, ) map_chunks.append(map_chunk) return map_chunks @@ -589,13 +576,14 @@ def _sample_chunks(cls, op, agg_chunks): chunk_op = DataFramePSRSGroupbySample ( kind='quicksort', n_partition=chunk_shape, - output_types=op.output_types, + output_types=[OutputType.dataframe], **properties ) kws = [] sampled_shape = ( (chunk_shape, len(op.groupby_params['by'])) if op.groupby_params['by'] else (chunk_shape,) ) + print(chunk) kws.append( { "shape": sampled_shape, @@ -604,12 +592,6 @@ def _sample_chunks(cls, op, agg_chunks): "type": "regular_sampled", } ) - if op.outputs[0].ndim == 2: - kws[0].update( - {"columns_value": chunk.columns_value, "dtypes": chunk.dtypes} - ) - else: - kws[0].update({"dtype": chunk.dtype}) chunk = chunk_op.new_chunk([chunk], kws=kws) sampled_chunks.append(chunk) @@ -1061,6 +1043,7 @@ def _wrapped_func(col): size_recorder = ctx.get_remote_object(op.size_recorder_name) size_recorder.record(raw_size, agg_size) + # print(tuple(agg_dfs)) ctx[op.outputs[0].key] = tuple(agg_dfs) @classmethod diff --git a/mars/dataframe/groupby/sort.py b/mars/dataframe/groupby/sort.py index b843f74189..f3fe1b612f 100644 --- a/mars/dataframe/groupby/sort.py +++ b/mars/dataframe/groupby/sort.py @@ -115,7 +115,7 @@ def execute(cls, ctx, op): n = op.n_partition if a.shape[0] < n: num = n // a.shape[0] + 1 - a = xdf.concat([a] * num).sort_values(by=op.by) + a = xdf.concat([a] * num).sort_index() w = a.shape[0] * 1.0 / (n + 1) @@ -240,19 +240,11 @@ def _execute_dataframe_map(cls, ctx, op): def _get_out_df(p_index, in_df): if p_index == 0: - if pivots[p_index] in in_df.index.values: - out_df = in_df.loc[:pivots[p_index]].iloc[:-1] - else: - out_df = in_df.loc[:pivots[p_index]] - # print("here " + str(out_df)) + out_df = in_df.loc[:pivots[p_index]] elif p_index == op.n_partition - 1: - out_df = in_df.loc[pivots[p_index-1]:] + out_df = in_df.loc[pivots[p_index-1]:].drop(index=pivots[p_index-1], errors="ignore") else: - if pivots[p_index] in in_df.index.values: - out_df = in_df.loc[pivots[p_index - 1]:pivots[p_index]].iloc[:-1] - else: - out_df = in_df.loc[pivots[p_index - 1]:pivots[p_index]] - # print("here " + str(out_df)) + out_df = in_df.loc[pivots[p_index - 1]:pivots[p_index]].drop(index=pivots[p_index-1], errors="ignore") return out_df # print("index " + str(out.key) + " " + str(df)) From 0fcbc474737b4fe7caf9a61f2229f9106adb936a Mon Sep 17 00:00:00 2001 From: "saksham.kumar" Date: Fri, 22 Apr 2022 13:25:08 -0400 Subject: [PATCH 04/32] added unit tests for group by sort --- .../groupby/tests/test_groupby_execution.py | 106 ++++++++++++++++++ 1 file changed, 106 insertions(+) diff --git a/mars/dataframe/groupby/tests/test_groupby_execution.py b/mars/dataframe/groupby/tests/test_groupby_execution.py index 4debb75bb3..423eda8d38 100644 --- a/mars/dataframe/groupby/tests/test_groupby_execution.py +++ b/mars/dataframe/groupby/tests/test_groupby_execution.py @@ -483,6 +483,111 @@ def test_dataframe_groupby_agg(setup): ), ) +def test_dataframe_groupby_agg_sort(setup): + agg_funs = [ + "std", + "mean", + "var", + "max", + "count", + "size", + "all", + "any", + "skew", + "kurt", + "sem", + ] + + rs = np.random.RandomState(0) + raw = pd.DataFrame( + { + "c1": np.arange(100).astype(np.int64), + "c2": rs.choice(["a", "b", "c"], (100,)), + "c3": rs.rand(100), + } + ) + mdf = md.DataFrame(raw, chunk_size=13) + + for method in ["tree", "shuffle"]: + r = mdf.groupby("c2").agg("size", method=method) + pd.testing.assert_series_equal( + r.execute().fetch(), raw.groupby("c2").agg("size") + ) + + for agg_fun in agg_funs: + if agg_fun == "size": + continue + r = mdf.groupby("c2").agg(agg_fun, method=method) + pd.testing.assert_frame_equal( + r.execute().fetch(), + raw.groupby("c2").agg(agg_fun), + ) + + r = mdf.groupby("c2").agg(agg_funs, method=method) + pd.testing.assert_frame_equal( + r.execute().fetch(), + raw.groupby("c2").agg(agg_funs), + ) + + agg = OrderedDict([("c1", ["min", "mean"]), ("c3", "std")]) + r = mdf.groupby("c2").agg(agg, method=method) + pd.testing.assert_frame_equal( + r.execute().fetch(), raw.groupby("c2").agg(agg) + ) + + agg = OrderedDict([("c1", "min"), ("c3", "sum")]) + r = mdf.groupby("c2").agg(agg, method=method) + pd.testing.assert_frame_equal( + r.execute().fetch(), raw.groupby("c2").agg(agg) + ) + + r = mdf.groupby("c2").agg({"c1": "min", "c3": "min"}, method=method) + pd.testing.assert_frame_equal( + r.execute().fetch(), + raw.groupby("c2").agg({"c1": "min", "c3": "min"}), + ) + + r = mdf.groupby("c2").agg({"c1": "min"}, method=method) + pd.testing.assert_frame_equal( + r.execute().fetch(), + raw.groupby("c2").agg({"c1": "min"}), + ) + + # test groupby series + r = mdf.groupby(mdf["c2"]).sum(method=method) + pd.testing.assert_frame_equal( + r.execute().fetch(), raw.groupby(raw["c2"]).sum() + ) + + r = mdf.groupby("c2").size(method="tree") + pd.testing.assert_series_equal(r.execute().fetch(), raw.groupby("c2").size()) + + # test inserted kurt method + r = mdf.groupby("c2").kurtosis(method="tree") + pd.testing.assert_frame_equal(r.execute().fetch(), raw.groupby("c2").kurtosis()) + + for agg_fun in agg_funs: + if agg_fun == "size" or callable(agg_fun): + continue + r = getattr(mdf.groupby("c2"), agg_fun)(method="tree") + pd.testing.assert_frame_equal( + r.execute().fetch(), getattr(raw.groupby("c2"), agg_fun)() + ) + + # test as_index=False takes no effect + r = mdf.groupby(["c1", "c2"], as_index=False).agg(["mean", "count"]) + pd.testing.assert_frame_equal( + r.execute().fetch(), + raw.groupby(["c1", "c2"], as_index=False).agg(["mean", "count"]), + ) + assert r.op.groupby_params["as_index"] is True + + # r = mdf.groupby("c2").agg(["cumsum", "cumcount"]) + # pd.testing.assert_frame_equal( + # r.execute().fetch(), + # raw.groupby("c2").agg(["cumsum", "cumcount"]), + # ) + def test_series_groupby_agg(setup): rs = np.random.RandomState(0) @@ -1251,3 +1356,4 @@ def test_groupby_nunique(setup): .nunique() .sort_values(by="b", ignore_index=True), ) + From a7684d51c15a025e99b1896b495f856e65d40a2a Mon Sep 17 00:00:00 2001 From: "saksham.kumar" Date: Mon, 25 Apr 2022 02:44:48 -0400 Subject: [PATCH 05/32] fixed conflicts --- mars/dataframe/groupby/aggregation.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/mars/dataframe/groupby/aggregation.py b/mars/dataframe/groupby/aggregation.py index 2e82e2ff67..f33757716c 100644 --- a/mars/dataframe/groupby/aggregation.py +++ b/mars/dataframe/groupby/aggregation.py @@ -224,7 +224,6 @@ def _call_dataframe(self, groupby, input_df): shape = (np.nan, agg_df.shape[1]) index_value = parse_index(agg_df.index, groupby.key, groupby.index_value.key) - # index_value.value.should_be_monotonic = True # make sure if as_index=False takes effect self._fix_as_index(agg_df.index) @@ -252,7 +251,6 @@ def _call_series(self, groupby, in_series): index_value = parse_index( agg_result.index, groupby.key, groupby.index_value.key ) - # index_value.value.should_be_monotonic = True inputs = self._get_inputs([in_series]) From f9f4e9c8da39e8b608b2cb2b8e0e525dc7ca0ef6 Mon Sep 17 00:00:00 2001 From: Saksham Kumar Date: Tue, 26 Apr 2022 14:52:18 -0400 Subject: [PATCH 06/32] sample output chunk shape does not rely on by --- mars/dataframe/groupby/aggregation.py | 160 +++++++++---------- mars/dataframe/groupby/tests/test_groupby.py | 24 ++- 2 files changed, 102 insertions(+), 82 deletions(-) diff --git a/mars/dataframe/groupby/aggregation.py b/mars/dataframe/groupby/aggregation.py index f33757716c..1ad6ef63b8 100644 --- a/mars/dataframe/groupby/aggregation.py +++ b/mars/dataframe/groupby/aggregation.py @@ -129,22 +129,22 @@ def _group_kurt(x, *args, **kwargs): def build_mock_agg_result( - groupby: GROUPBY_TYPE, - groupby_params: typing.Dict, - raw_func: typing.Callable, - **raw_func_kw, + groupby: GROUPBY_TYPE, + groupby_params: typing.Dict, + raw_func: typing.Callable, + **raw_func_kw, ): try: agg_result = groupby.op.build_mock_groupby().aggregate(raw_func, **raw_func_kw) except ValueError: if ( - groupby_params.get("as_index") or _support_get_group_without_as_index + groupby_params.get("as_index") or _support_get_group_without_as_index ): # pragma: no cover raise agg_result = ( groupby.op.build_mock_groupby(as_index=True) - .aggregate(raw_func, **raw_func_kw) - .to_frame() + .aggregate(raw_func, **raw_func_kw) + .to_frame() ) agg_result.index.names = [None] * agg_result.index.nlevels return agg_result @@ -350,7 +350,7 @@ def partition_local_data( map_chunk_op = DataFrameGroupbySortShuffle( shuffle_size=chunk_shape[0], stage=OperandStage.map, - n_partition = len(sorted_chunks), + n_partition=len(sorted_chunks), output_types=[OutputType.dataframe_groupby], # columns_value=chunk_inputs[0].columns_value, ) @@ -363,9 +363,8 @@ def partition_local_data( # ) # ) # else: - # kw.update(dict(dtype=chunk_inputs[0].dtype, name=chunk_inputs[0].name)) - # kw.update(dict(name=chunk_inputs[0].name)) - + # kw.update(dict(dtype=chunk_inputs[0].dtype, name=chunk_inputs[0].name)) + # kw.update(dict(name=chunk_inputs[0].name)) map_chunks.append( map_chunk_op.new_chunk( @@ -439,11 +438,11 @@ def _gen_shuffle_chunks(cls, op, in_df, chunks): @classmethod def _gen_map_chunks( - cls, - op: "DataFrameGroupByAgg", - in_chunks: List[ChunkType], - out_df: TileableType, - func_infos: ReductionSteps, + cls, + op: "DataFrameGroupByAgg", + in_chunks: List[ChunkType], + out_df: TileableType, + func_infos: ReductionSteps, ): map_chunks = [] for chunk in in_chunks: @@ -515,11 +514,11 @@ def _compile_funcs(cls, op: "DataFrameGroupByAgg", in_df) -> ReductionSteps: @classmethod def _tile_with_shuffle( - cls, - op: "DataFrameGroupByAgg", - in_df: TileableType, - out_df: TileableType, - func_infos: ReductionSteps, + cls, + op: "DataFrameGroupByAgg", + in_df: TileableType, + out_df: TileableType, + func_infos: ReductionSteps, ): # First, perform groupby and aggregation on each chunk. agg_chunks = cls._gen_map_chunks(op, in_df.chunks, out_df, func_infos) @@ -573,7 +572,7 @@ def _sample_chunks(cls, op, agg_chunks): ) for i, chunk in enumerate(agg_chunks): - chunk_op = DataFramePSRSGroupbySample ( + chunk_op = DataFramePSRSGroupbySample( kind='quicksort', n_partition=chunk_shape, output_types=[OutputType.dataframe], @@ -581,9 +580,8 @@ def _sample_chunks(cls, op, agg_chunks): ) kws = [] sampled_shape = ( - (chunk_shape, len(op.groupby_params['by'])) if op.groupby_params['by'] else (chunk_shape,) + (chunk_shape, chunk.shape[1]) if len(chunk.shape) == 2 else (chunk_shape,) ) - print(chunk) kws.append( { "shape": sampled_shape, @@ -600,13 +598,13 @@ def _sample_chunks(cls, op, agg_chunks): @classmethod def _perform_shuffle( - cls, - op: "DataFrameGroupByAgg", - agg_chunks: List[ChunkType], - in_df: TileableType, - out_df: TileableType, - func_infos: ReductionSteps, - pivot_chunk + cls, + op: "DataFrameGroupByAgg", + agg_chunks: List[ChunkType], + in_df: TileableType, + out_df: TileableType, + func_infos: ReductionSteps, + pivot_chunk ): # Shuffle the aggregation chunk. if op.groupby_params["sort"] and pivot_chunk is not None: @@ -658,28 +656,28 @@ def _perform_shuffle( @classmethod def _tile_with_tree( - cls, - op: "DataFrameGroupByAgg", - in_df: TileableType, - out_df: TileableType, - func_infos: ReductionSteps, + cls, + op: "DataFrameGroupByAgg", + in_df: TileableType, + out_df: TileableType, + func_infos: ReductionSteps, ): chunks = cls._gen_map_chunks(op, in_df.chunks, out_df, func_infos) return cls._combine_tree(op, chunks, out_df, func_infos) @classmethod def _combine_tree( - cls, - op: "DataFrameGroupByAgg", - chunks: List[ChunkType], - out_df: TileableType, - func_infos: ReductionSteps, + cls, + op: "DataFrameGroupByAgg", + chunks: List[ChunkType], + out_df: TileableType, + func_infos: ReductionSteps, ): combine_size = op.combine_size while len(chunks) > combine_size: new_chunks = [] for idx, i in enumerate(range(0, len(chunks), combine_size)): - chks = chunks[i : i + combine_size] + chks = chunks[i: i + combine_size] if len(chks) == 1: chk = chks[0] else: @@ -741,7 +739,7 @@ def _combine_tree( @classmethod def _choose_tree_method( - cls, raw_sizes, agg_sizes, sample_count, total_count, chunk_store_limit + cls, raw_sizes, agg_sizes, sample_count, total_count, chunk_store_limit ): # calculate the coefficient of variation of aggregation sizes, # if the CV is less than 0.2 and the mean of agg_size/raw_size @@ -766,11 +764,11 @@ def _choose_tree_method( @classmethod def _tile_auto( - cls, - op: "DataFrameGroupByAgg", - in_df: TileableType, - out_df: TileableType, - func_infos: ReductionSteps, + cls, + op: "DataFrameGroupByAgg", + in_df: TileableType, + out_df: TileableType, + func_infos: ReductionSteps, ): ctx = get_context() combine_size = op.combine_size @@ -793,7 +791,7 @@ def _tile_auto( left_chunks = in_df.chunks[combine_size:] left_chunks = cls._gen_map_chunks(op, left_chunks, out_df, func_infos) if cls._choose_tree_method( - raw_sizes, agg_sizes, len(chunks), len(in_df.chunks), op.chunk_store_limit + raw_sizes, agg_sizes, len(chunks), len(in_df.chunks), op.chunk_store_limit ): logger.debug("Choose tree method for groupby operand %s", op) return cls._combine_tree(op, chunks + left_chunks, out_df, func_infos) @@ -873,7 +871,7 @@ def _pack_inputs(agg_funcs: List[ReductionAggStep], in_data): out_dict[step.output_key] = in_data[pos] else: out_dict[step.output_key] = tuple( - in_data[pos : pos + step.output_limit] + in_data[pos: pos + step.output_limit] ) pos += step.output_limit return out_dict @@ -968,8 +966,8 @@ def _execute_map(cls, ctx, op: "DataFrameGroupByAgg"): in_data = ctx[op.inputs[0].key] if ( - isinstance(in_data, xdf.Series) - and op.output_types[0] == OutputType.dataframe + isinstance(in_data, xdf.Series) + and op.output_types[0] == OutputType.dataframe ): in_data = cls._series_to_df(in_data, op.gpu) @@ -1019,13 +1017,13 @@ def _wrapped_func(col): agg_dfs = [] for ( - input_key, - map_func_name, - _agg_func_name, - custom_reduction, - _output_key, - _output_limit, - kwds, + input_key, + map_func_name, + _agg_func_name, + custom_reduction, + _output_key, + _output_limit, + kwds, ) in op.agg_funcs: input_obj = ret_map_groupbys[input_key] if map_func_name == "custom_reduction": @@ -1056,8 +1054,8 @@ def _execute_combine(cls, ctx, op: "DataFrameGroupByAgg"): in_data_list = [] for in_data in in_data_tuple: if ( - isinstance(in_data, xdf.Series) - and op.output_types[0] == OutputType.dataframe + isinstance(in_data, xdf.Series) + and op.output_types[0] == OutputType.dataframe ): in_data = cls._series_to_df(in_data, op.gpu) in_data_list.append(cls._get_grouped(op, in_data, ctx)) @@ -1066,13 +1064,13 @@ def _execute_combine(cls, ctx, op: "DataFrameGroupByAgg"): combines = [] for ( - _input_key, - _map_func_name, - agg_func_name, - custom_reduction, - output_key, - _output_limit, - kwds, + _input_key, + _map_func_name, + agg_func_name, + custom_reduction, + output_key, + _output_limit, + kwds, ) in op.agg_funcs: input_obj = in_data_dict[output_key] if agg_func_name == "custom_reduction": @@ -1097,8 +1095,8 @@ def _execute_agg(cls, ctx, op: "DataFrameGroupByAgg"): in_data_list = [] for in_data in in_data_tuple: if ( - isinstance(in_data, xdf.Series) - and op.output_types[0] == OutputType.dataframe + isinstance(in_data, xdf.Series) + and op.output_types[0] == OutputType.dataframe ): in_data = cls._series_to_df(in_data, op.gpu) in_data_list.append(in_data) @@ -1106,13 +1104,13 @@ def _execute_agg(cls, ctx, op: "DataFrameGroupByAgg"): in_data_dict = cls._pack_inputs(op.agg_funcs, in_data_tuple) for ( - _input_key, - _map_func_name, - agg_func_name, - custom_reduction, - output_key, - _output_limit, - kwds, + _input_key, + _map_func_name, + agg_func_name, + custom_reduction, + output_key, + _output_limit, + kwds, ) in op.agg_funcs: if agg_func_name == "custom_reduction": input_obj = tuple( @@ -1135,8 +1133,8 @@ def _execute_agg(cls, ctx, op: "DataFrameGroupByAgg"): func_inputs = [in_data_dict[k][cols] for k in input_keys] if ( - func_inputs[0].ndim == 2 - and len(set(inp.shape[1] for inp in func_inputs)) > 1 + func_inputs[0].ndim == 2 + and len(set(inp.shape[1] for inp in func_inputs)) > 1 ): common_cols = func_inputs[0].columns for inp in func_inputs[1:]: @@ -1165,8 +1163,8 @@ def _execute_agg(cls, ctx, op: "DataFrameGroupByAgg"): if out_chunk.ndim == 2: result = xdf.concat(aggs, axis=1) if ( - not op.groupby_params.get("as_index", True) - and col_value.nlevels == result.columns.nlevels + not op.groupby_params.get("as_index", True) + and col_value.nlevels == result.columns.nlevels ): result.reset_index( inplace=True, drop=result.index.name in result.columns diff --git a/mars/dataframe/groupby/tests/test_groupby.py b/mars/dataframe/groupby/tests/test_groupby.py index 336564d4ae..b548eeb2bb 100644 --- a/mars/dataframe/groupby/tests/test_groupby.py +++ b/mars/dataframe/groupby/tests/test_groupby.py @@ -18,6 +18,7 @@ import pandas as pd import pytest +from ..sort import DataFrameGroupbySortShuffle from .... import dataframe as md from .... import opcodes from ....core import OutputType, tile @@ -118,7 +119,7 @@ def test_groupby_agg(): } ) mdf = md.DataFrame(df, chunk_size=2) - r = mdf.groupby("c2").sum(method="shuffle") + r = mdf.groupby("c2", sort=False).sum(method="shuffle") assert isinstance(r.op, DataFrameGroupByAgg) assert isinstance(r, DataFrame) @@ -139,6 +140,27 @@ def test_groupby_agg(): agg_chunk = chunk.inputs[0].inputs[0].inputs[0].inputs[0] assert agg_chunk.op.stage == OperandStage.map + r = mdf.groupby("c2", ).sum(method="shuffle") + + assert isinstance(r.op, DataFrameGroupByAgg) + assert isinstance(r, DataFrame) + + r = tile(r) + assert len(r.chunks) == 5 + for chunk in r.chunks: + assert isinstance(chunk.op, DataFrameGroupByAgg) + assert chunk.op.stage == OperandStage.agg + assert isinstance(chunk.inputs[0].op, DataFrameGroupbySortShuffle) + assert chunk.inputs[0].op.stage == OperandStage.reduce + assert isinstance(chunk.inputs[0].inputs[0].op, DataFrameShuffleProxy) + assert isinstance( + chunk.inputs[0].inputs[0].inputs[0].op, DataFrameGroupbySortShuffle + ) + assert chunk.inputs[0].inputs[0].inputs[0].op.stage == OperandStage.map + + agg_chunk = chunk.inputs[0].inputs[0].inputs[0].inputs[0] + assert agg_chunk.op.stage == OperandStage.map + # test unknown method with pytest.raises(ValueError): mdf.groupby("c2").sum(method="not_exist") From 2f2f4d919fe608b9f1b313d3f63296d418b22af6 Mon Sep 17 00:00:00 2001 From: "saksham.kumar" Date: Mon, 25 Apr 2022 02:35:14 -0400 Subject: [PATCH 07/32] order preserve poc --- mars/dataframe/groupby/aggregation.py | 3 ++- mars/dataframe/groupby/core.py | 14 +++++++++++++- 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/mars/dataframe/groupby/aggregation.py b/mars/dataframe/groupby/aggregation.py index 1ad6ef63b8..5dfafbbf0e 100644 --- a/mars/dataframe/groupby/aggregation.py +++ b/mars/dataframe/groupby/aggregation.py @@ -218,8 +218,9 @@ def _fix_as_index(self, result_index: pd.Index): self.groupby_params["as_index"] = True def _call_dataframe(self, groupby, input_df): + print(groupby.op.preserve_order) agg_df = build_mock_agg_result( - groupby, self.groupby_params, self.raw_func, **self.raw_func_kw + groupby, groupby.op.groupby_params, self.raw_func, **self.raw_func_kw ) shape = (np.nan, agg_df.shape[1]) diff --git a/mars/dataframe/groupby/core.py b/mars/dataframe/groupby/core.py index 0209a4c950..72e75dd6b8 100644 --- a/mars/dataframe/groupby/core.py +++ b/mars/dataframe/groupby/core.py @@ -48,6 +48,7 @@ class DataFrameGroupByOperand(MapReduceOperand, DataFrameOperandMixin): _level = AnyField("level") _as_index = BoolField("as_index") _sort = BoolField("sort") + _preserve_order = BoolField("preserve_order") _group_keys = BoolField("group_keys") _shuffle_size = Int32Field("shuffle_size") @@ -61,6 +62,7 @@ def __init__( group_keys=None, shuffle_size=None, output_types=None, + preserve_order=None, **kw ): super().__init__( @@ -71,8 +73,13 @@ def __init__( _group_keys=group_keys, _shuffle_size=shuffle_size, _output_types=output_types, + _preserve_order=preserve_order, **kw ) + if sort: + self._preserve_order = False + else: + self._preserve_order = preserve_order if output_types: if self.stage in (OperandStage.map, OperandStage.reduce): if output_types[0] in ( @@ -108,6 +115,10 @@ def as_index(self): def sort(self): return self._sort + @property + def preserve_order(self): + return self._preserve_order + @property def group_keys(self): return self._group_keys @@ -485,7 +496,7 @@ def execute(cls, ctx, op: "DataFrameGroupByOperand"): ) -def groupby(df, by=None, level=None, as_index=True, sort=True, group_keys=True): +def groupby(df, by=None, level=None, as_index=True, sort=True, group_keys=True, preserve_order=False): if not as_index and df.op.output_types[0] == OutputType.series: raise TypeError("as_index=False only valid with DataFrame") @@ -505,5 +516,6 @@ def groupby(df, by=None, level=None, as_index=True, sort=True, group_keys=True): sort=sort, group_keys=group_keys, output_types=output_types, + preserve_order=preserve_order, ) return op(df) From c93adb0a621831174cf581d9f943e6645b70a3e1 Mon Sep 17 00:00:00 2001 From: Saksham Kumar Date: Thu, 28 Apr 2022 17:07:53 -0400 Subject: [PATCH 08/32] impl execute for indexing phase --- mars/dataframe/groupby/aggregation.py | 54 ++++++++++++++++++++++++--- 1 file changed, 48 insertions(+), 6 deletions(-) diff --git a/mars/dataframe/groupby/aggregation.py b/mars/dataframe/groupby/aggregation.py index 5dfafbbf0e..97f9b6dda2 100644 --- a/mars/dataframe/groupby/aggregation.py +++ b/mars/dataframe/groupby/aggregation.py @@ -23,6 +23,7 @@ import pandas as pd from scipy.stats import variation +from .preserve_order import DataFrameOrderPreserveIndexOperand from .sort import DataFramePSRSGroupbySample, DataFrameGroupbyConcatPivot, DataFrameGroupbySortShuffle from ... import opcodes as OperandDef from ...config import options @@ -159,6 +160,7 @@ class DataFrameGroupByAgg(DataFrameOperand, DataFrameOperandMixin): func_rename = ListField("func_rename") groupby_params = DictField("groupby_params") + preserve_order = BoolField("preserve_order") method = StringField("method") use_inf_as_na = BoolField("use_inf_as_na") @@ -218,7 +220,6 @@ def _fix_as_index(self, result_index: pd.Index): self.groupby_params["as_index"] = True def _call_dataframe(self, groupby, input_df): - print(groupby.op.preserve_order) agg_df = build_mock_agg_result( groupby, groupby.op.groupby_params, self.raw_func, **self.raw_func_kw ) @@ -277,6 +278,7 @@ def _call_series(self, groupby, in_series): ) def __call__(self, groupby): + self.preserve_order = groupby.op.preserve_order normalize_reduction_funcs(self, ndim=groupby.ndim) df = groupby while df.op.output_types[0] not in (OutputType.dataframe, OutputType.series): @@ -525,16 +527,56 @@ def _tile_with_shuffle( agg_chunks = cls._gen_map_chunks(op, in_df.chunks, out_df, func_infos) pivot_chunk = None if op.groupby_params['sort'] and len(in_df.chunks) > 1: - out_idx = (0,) if in_df.ndim == 2 else (), agg_chunk_len = len(agg_chunks) sample_chunks = cls._sample_chunks(op, agg_chunks) - pivot_chunk = cls._gen_pivot_chunk(op, sample_chunks, out_idx, agg_chunk_len) + pivot_chunk = cls._gen_pivot_chunk(op, sample_chunks, agg_chunk_len) + + if op.preserve_order and len(in_df.chunks) > 1: + # add min col to in_df + index_chunks = cls._gen_index_chunks(op, agg_chunks) + # concat and get table and pivot + index_table, pivot_chunk = cls._find_index_table_and_pivot(op, index_chunks, agg_chunk_len) + # join the concat table with in_dfs + pass # agg_chunks = agg_chunks + sample_chunks return cls._perform_shuffle(op, agg_chunks, in_df, out_df, func_infos, pivot_chunk) @classmethod - def _gen_pivot_chunk(cls, op, sample_chunks, out_idx, agg_chunk_len): + def _gen_index_chunks(cls, op, agg_chunks): + chunk_shape = len(agg_chunks) + index_chunks = [] + + properties = dict( + by=op.groupby_params['by'], + gpu=op.is_gpu(), + ) + + for i, chunk in enumerate(agg_chunks): + chunk_op = DataFrameOrderPreserveIndexOperand( + output_types=[OutputType.dataframe], + **properties + ) + kws = [] + sampled_shape = ( + (chunk_shape, len(op.groupby_params['by'])) if op.groupby_params['by'] else (chunk_shape,) + ) + print(chunk) + kws.append( + { + "shape": sampled_shape, + "index_value": chunk.index_value, + "index": (i, 0), + } + ) + + chunk = chunk_op.new_chunk([chunk], kws=kws) + index_chunks.append(chunk) + + return index_chunks + + @classmethod + def _gen_pivot_chunk(cls, op, sample_chunks, agg_chunk_len): properties = dict( by=op.groupby_params['by'], @@ -800,10 +842,9 @@ def _tile_auto( # otherwise, use shuffle pivot_chunk = None if op.groupby_params['sort'] and len(in_df.chunks) > 1: - out_idx = (0,) if in_df.ndim == 2 else (), agg_chunk_len = len(chunks + left_chunks) sample_chunks = cls._sample_chunks(op, chunks + left_chunks) - pivot_chunk = cls._gen_pivot_chunk(op, sample_chunks, out_idx, agg_chunk_len) + pivot_chunk = cls._gen_pivot_chunk(op, sample_chunks, agg_chunk_len) logger.debug("Choose shuffle method for groupby operand %s", op) return cls._perform_shuffle( @@ -1265,5 +1306,6 @@ def agg(groupby, func=None, method="auto", combine_size=None, *args, **kwargs): combine_size=combine_size or options.combine_size, chunk_store_limit=options.chunk_store_limit, use_inf_as_na=use_inf_as_na, + preserve_order=groupby.op.preserve_order, ) return agg_op(groupby) From e75bfcf81e474c3fbcc7d3849fba256b48359b23 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=BB=A7=E7=9B=9B?= Date: Thu, 5 May 2022 16:02:01 +0800 Subject: [PATCH 09/32] Fix output_type of DataFrameGroupbyAgg map --- mars/dataframe/groupby/aggregation.py | 237 +++++++++--------- mars/dataframe/groupby/sort.py | 27 +- mars/dataframe/groupby/tests/test_groupby.py | 4 +- .../groupby/tests/test_groupby_execution.py | 14 +- 4 files changed, 144 insertions(+), 138 deletions(-) diff --git a/mars/dataframe/groupby/aggregation.py b/mars/dataframe/groupby/aggregation.py index 04fc9be0b4..3bfca72cf4 100644 --- a/mars/dataframe/groupby/aggregation.py +++ b/mars/dataframe/groupby/aggregation.py @@ -22,7 +22,11 @@ import pandas as pd from scipy.stats import variation -from .sort import DataFramePSRSGroupbySample, DataFrameGroupbyConcatPivot, DataFrameGroupbySortShuffle +from .sort import ( + DataFramePSRSGroupbySample, + DataFrameGroupbyConcatPivot, + DataFrameGroupbySortShuffle, +) from ... import opcodes as OperandDef from ...config import options from ...core.custom_log import redirect_custom_log @@ -128,22 +132,22 @@ def _group_kurt(x, *args, **kwargs): def build_mock_agg_result( - groupby: GROUPBY_TYPE, - groupby_params: Dict, - raw_func: Callable, - **raw_func_kw, + groupby: GROUPBY_TYPE, + groupby_params: Dict, + raw_func: Callable, + **raw_func_kw, ): try: agg_result = groupby.op.build_mock_groupby().aggregate(raw_func, **raw_func_kw) except ValueError: if ( - groupby_params.get("as_index") or _support_get_group_without_as_index + groupby_params.get("as_index") or _support_get_group_without_as_index ): # pragma: no cover raise agg_result = ( groupby.op.build_mock_groupby(as_index=True) - .aggregate(raw_func, **raw_func_kw) - .to_frame() + .aggregate(raw_func, **raw_func_kw) + .to_frame() ) agg_result.index.names = [None] * agg_result.index.nlevels return agg_result @@ -295,22 +299,17 @@ def __call__(self, groupby): return self._call_series(groupby, df) @classmethod - def partition_merge_data( - cls, op, partition_chunks, proxy_chunk, in_df - ): + def partition_merge_data(cls, op, partition_chunks, proxy_chunk, in_df): # stage 4: all *ith* classes are gathered and merged partition_sort_chunks = [] - properties = dict( - by=op.groupby_params['by'], - gpu=op.is_gpu() - ) + properties = dict(by=op.groupby_params["by"], gpu=op.is_gpu()) for i, partition_chunk in enumerate(partition_chunks): partition_shuffle_reduce = DataFrameGroupbySortShuffle( stage=OperandStage.reduce, reducer_index=(i, 0), output_types=[OutputType.dataframe_groupby], - **properties + **properties, ) chunk_shape = list(partition_chunk.shape) chunk_shape[0] = np.nan @@ -334,13 +333,8 @@ def partition_merge_data( return partition_sort_chunks @classmethod - def partition_local_data( - cls, op, sorted_chunks, concat_pivot_chunk, in_df - ): - properties = dict( - by=op.groupby_params['by'], - gpu=op.is_gpu() - ) + def partition_local_data(cls, op, sorted_chunks, concat_pivot_chunk, in_df): + properties = dict(by=op.groupby_params["by"], gpu=op.is_gpu()) map_chunks = [] chunk_shape = (in_df.chunk_shape[0], 1) @@ -379,13 +373,11 @@ def partition_local_data( @classmethod def _gen_shuffle_chunks_with_pivot(cls, op, in_df, chunks, pivot): - map_chunks = cls.partition_local_data( - op, chunks, pivot, in_df - ) + map_chunks = cls.partition_local_data(op, chunks, pivot, in_df) - proxy_chunk = DataFrameShuffleProxy(output_types=[OutputType.dataframe]).new_chunk( - map_chunks, shape=() - ) + proxy_chunk = DataFrameShuffleProxy( + output_types=[OutputType.dataframe] + ).new_chunk(map_chunks, shape=()) partition_sort_chunks = cls.partition_merge_data( op, map_chunks, proxy_chunk, in_df @@ -437,18 +429,18 @@ def _gen_shuffle_chunks(cls, op, in_df, chunks): @classmethod def _gen_map_chunks( - cls, - op: "DataFrameGroupByAgg", - in_chunks: List[ChunkType], - out_df: TileableType, - func_infos: ReductionSteps, + cls, + op: "DataFrameGroupByAgg", + in_chunks: List[ChunkType], + out_df: TileableType, + func_infos: ReductionSteps, ): map_chunks = [] for chunk in in_chunks: chunk_inputs = [chunk] map_op = op.copy().reset_key() # force as_index=True for map phase - map_op.output_types = [OutputType.dataframe] + map_op.output_types = op.output_types map_op.groupby_params = map_op.groupby_params.copy() map_op.groupby_params["as_index"] = True if isinstance(map_op.groupby_params["by"], list): @@ -466,7 +458,7 @@ def _gen_map_chunks( map_op.stage = OperandStage.map map_op.pre_funcs = func_infos.pre_funcs map_op.agg_funcs = func_infos.agg_funcs - new_index = chunk.index if len(chunk.index) == 2 else (chunk.index[0], 0) + new_index = chunk.index if len(chunk.index) == 2 else (chunk.index[0],) if op.output_types[0] == OutputType.dataframe: map_chunk = map_op.new_chunk( chunk_inputs, @@ -478,9 +470,10 @@ def _gen_map_chunks( else: map_chunk = map_op.new_chunk( chunk_inputs, - shape=(out_df.shape[0], 1), + shape=(out_df.shape[0],), index=new_index, index_value=out_df.index_value, + dtype=out_df.dtype, ) map_chunks.append(map_chunk) return map_chunks @@ -513,41 +506,45 @@ def _compile_funcs(cls, op: "DataFrameGroupByAgg", in_df) -> ReductionSteps: @classmethod def _tile_with_shuffle( - cls, - op: "DataFrameGroupByAgg", - in_df: TileableType, - out_df: TileableType, - func_infos: ReductionSteps, + cls, + op: "DataFrameGroupByAgg", + in_df: TileableType, + out_df: TileableType, + func_infos: ReductionSteps, ): # First, perform groupby and aggregation on each chunk. agg_chunks = cls._gen_map_chunks(op, in_df.chunks, out_df, func_infos) pivot_chunk = None - if op.groupby_params['sort'] and len(in_df.chunks) > 1: - out_idx = (0,) if in_df.ndim == 2 else (), + if op.groupby_params["sort"] and len(in_df.chunks) > 1: + out_idx = ((0,) if in_df.ndim == 2 else (),) agg_chunk_len = len(agg_chunks) sample_chunks = cls._sample_chunks(op, agg_chunks) - pivot_chunk = cls._gen_pivot_chunk(op, sample_chunks, out_idx, agg_chunk_len) + pivot_chunk = cls._gen_pivot_chunk( + op, sample_chunks, out_idx, agg_chunk_len + ) # agg_chunks = agg_chunks + sample_chunks - return cls._perform_shuffle(op, agg_chunks, in_df, out_df, func_infos, pivot_chunk) + return cls._perform_shuffle( + op, agg_chunks, in_df, out_df, func_infos, pivot_chunk + ) @classmethod def _gen_pivot_chunk(cls, op, sample_chunks, out_idx, agg_chunk_len): properties = dict( - by=op.groupby_params['by'], + by=op.groupby_params["by"], gpu=op.is_gpu(), ) # stage 2: gather and merge samples, choose and broadcast p-1 pivots - kind = 'quicksort' + kind = "quicksort" output_types = [OutputType.tensor] concat_pivot_op = DataFrameGroupbyConcatPivot( kind=kind, n_partition=agg_chunk_len, output_types=output_types, - **properties + **properties, ) # concat_pivot_index = out_idx[: 0] + (0,) + out_idx[0:] @@ -566,26 +563,30 @@ def _sample_chunks(cls, op, agg_chunks): sampled_chunks = [] properties = dict( - by=op.groupby_params['by'], + by=op.groupby_params["by"], gpu=op.is_gpu(), ) for i, chunk in enumerate(agg_chunks): + output_types = ( + [OutputType.dataframe] if chunk.ndim == 2 else [OutputType.series] + ) chunk_op = DataFramePSRSGroupbySample( - kind='quicksort', + kind="quicksort", n_partition=chunk_shape, - output_types=[OutputType.dataframe], - **properties + output_types=output_types, + **properties, ) kws = [] sampled_shape = ( - (chunk_shape, chunk.shape[1]) if len(chunk.shape) == 2 else (chunk_shape,) + (chunk_shape, chunk.shape[1]) if chunk.ndim == 2 else (chunk_shape,) ) + chunk_index = (i, 0) if chunk.ndim == 2 else (i,) kws.append( { "shape": sampled_shape, "index_value": chunk.index_value, - "index": (i, 0), + "index": chunk_index, "type": "regular_sampled", } ) @@ -597,17 +598,19 @@ def _sample_chunks(cls, op, agg_chunks): @classmethod def _perform_shuffle( - cls, - op: "DataFrameGroupByAgg", - agg_chunks: List[ChunkType], - in_df: TileableType, - out_df: TileableType, - func_infos: ReductionSteps, - pivot_chunk + cls, + op: "DataFrameGroupByAgg", + agg_chunks: List[ChunkType], + in_df: TileableType, + out_df: TileableType, + func_infos: ReductionSteps, + pivot_chunk, ): # Shuffle the aggregation chunk. if op.groupby_params["sort"] and pivot_chunk is not None: - reduce_chunks = cls._gen_shuffle_chunks_with_pivot(op, in_df, agg_chunks, pivot_chunk) + reduce_chunks = cls._gen_shuffle_chunks_with_pivot( + op, in_df, agg_chunks, pivot_chunk + ) else: reduce_chunks = cls._gen_shuffle_chunks(op, in_df, agg_chunks) @@ -655,28 +658,28 @@ def _perform_shuffle( @classmethod def _tile_with_tree( - cls, - op: "DataFrameGroupByAgg", - in_df: TileableType, - out_df: TileableType, - func_infos: ReductionSteps, + cls, + op: "DataFrameGroupByAgg", + in_df: TileableType, + out_df: TileableType, + func_infos: ReductionSteps, ): chunks = cls._gen_map_chunks(op, in_df.chunks, out_df, func_infos) return cls._combine_tree(op, chunks, out_df, func_infos) @classmethod def _combine_tree( - cls, - op: "DataFrameGroupByAgg", - chunks: List[ChunkType], - out_df: TileableType, - func_infos: ReductionSteps, + cls, + op: "DataFrameGroupByAgg", + chunks: List[ChunkType], + out_df: TileableType, + func_infos: ReductionSteps, ): combine_size = op.combine_size while len(chunks) > combine_size: new_chunks = [] for idx, i in enumerate(range(0, len(chunks), combine_size)): - chks = chunks[i: i + combine_size] + chks = chunks[i : i + combine_size] if len(chks) == 1: chk = chks[0] else: @@ -738,7 +741,7 @@ def _combine_tree( @classmethod def _choose_tree_method( - cls, raw_sizes, agg_sizes, sample_count, total_count, chunk_store_limit + cls, raw_sizes, agg_sizes, sample_count, total_count, chunk_store_limit ): # calculate the coefficient of variation of aggregation sizes, # if the CV is less than 0.2 and the mean of agg_size/raw_size @@ -763,11 +766,11 @@ def _choose_tree_method( @classmethod def _tile_auto( - cls, - op: "DataFrameGroupByAgg", - in_df: TileableType, - out_df: TileableType, - func_infos: ReductionSteps, + cls, + op: "DataFrameGroupByAgg", + in_df: TileableType, + out_df: TileableType, + func_infos: ReductionSteps, ): ctx = get_context() combine_size = op.combine_size @@ -790,18 +793,20 @@ def _tile_auto( left_chunks = in_df.chunks[combine_size:] left_chunks = cls._gen_map_chunks(op, left_chunks, out_df, func_infos) if cls._choose_tree_method( - raw_sizes, agg_sizes, len(chunks), len(in_df.chunks), op.chunk_store_limit + raw_sizes, agg_sizes, len(chunks), len(in_df.chunks), op.chunk_store_limit ): logger.debug("Choose tree method for groupby operand %s", op) return cls._combine_tree(op, chunks + left_chunks, out_df, func_infos) else: # otherwise, use shuffle pivot_chunk = None - if op.groupby_params['sort'] and len(in_df.chunks) > 1: - out_idx = (0,) if in_df.ndim == 2 else (), + if op.groupby_params["sort"] and len(in_df.chunks) > 1: + out_idx = ((0,) if in_df.ndim == 2 else (),) agg_chunk_len = len(chunks + left_chunks) sample_chunks = cls._sample_chunks(op, chunks + left_chunks) - pivot_chunk = cls._gen_pivot_chunk(op, sample_chunks, out_idx, agg_chunk_len) + pivot_chunk = cls._gen_pivot_chunk( + op, sample_chunks, out_idx, agg_chunk_len + ) logger.debug("Choose shuffle method for groupby operand %s", op) return cls._perform_shuffle( @@ -870,7 +875,7 @@ def _pack_inputs(agg_funcs: List[ReductionAggStep], in_data): out_dict[step.output_key] = in_data[pos] else: out_dict[step.output_key] = tuple( - in_data[pos: pos + step.output_limit] + in_data[pos : pos + step.output_limit] ) pos += step.output_limit return out_dict @@ -965,8 +970,8 @@ def _execute_map(cls, ctx, op: "DataFrameGroupByAgg"): in_data = ctx[op.inputs[0].key] if ( - isinstance(in_data, xdf.Series) - and op.output_types[0] == OutputType.dataframe + isinstance(in_data, xdf.Series) + and op.output_types[0] == OutputType.dataframe ): in_data = cls._series_to_df(in_data, op.gpu) @@ -1016,13 +1021,13 @@ def _wrapped_func(col): agg_dfs = [] for ( - input_key, - map_func_name, - _agg_func_name, - custom_reduction, - _output_key, - _output_limit, - kwds, + input_key, + map_func_name, + _agg_func_name, + custom_reduction, + _output_key, + _output_limit, + kwds, ) in op.agg_funcs: input_obj = ret_map_groupbys[input_key] if map_func_name == "custom_reduction": @@ -1053,8 +1058,8 @@ def _execute_combine(cls, ctx, op: "DataFrameGroupByAgg"): in_data_list = [] for in_data in in_data_tuple: if ( - isinstance(in_data, xdf.Series) - and op.output_types[0] == OutputType.dataframe + isinstance(in_data, xdf.Series) + and op.output_types[0] == OutputType.dataframe ): in_data = cls._series_to_df(in_data, op.gpu) in_data_list.append(cls._get_grouped(op, in_data, ctx)) @@ -1063,13 +1068,13 @@ def _execute_combine(cls, ctx, op: "DataFrameGroupByAgg"): combines = [] for ( - _input_key, - _map_func_name, - agg_func_name, - custom_reduction, - output_key, - _output_limit, - kwds, + _input_key, + _map_func_name, + agg_func_name, + custom_reduction, + output_key, + _output_limit, + kwds, ) in op.agg_funcs: input_obj = in_data_dict[output_key] if agg_func_name == "custom_reduction": @@ -1094,8 +1099,8 @@ def _execute_agg(cls, ctx, op: "DataFrameGroupByAgg"): in_data_list = [] for in_data in in_data_tuple: if ( - isinstance(in_data, xdf.Series) - and op.output_types[0] == OutputType.dataframe + isinstance(in_data, xdf.Series) + and op.output_types[0] == OutputType.dataframe ): in_data = cls._series_to_df(in_data, op.gpu) in_data_list.append(in_data) @@ -1103,13 +1108,13 @@ def _execute_agg(cls, ctx, op: "DataFrameGroupByAgg"): in_data_dict = cls._pack_inputs(op.agg_funcs, in_data_tuple) for ( - _input_key, - _map_func_name, - agg_func_name, - custom_reduction, - output_key, - _output_limit, - kwds, + _input_key, + _map_func_name, + agg_func_name, + custom_reduction, + output_key, + _output_limit, + kwds, ) in op.agg_funcs: if agg_func_name == "custom_reduction": input_obj = tuple( @@ -1132,8 +1137,8 @@ def _execute_agg(cls, ctx, op: "DataFrameGroupByAgg"): func_inputs = [in_data_dict[k][cols] for k in input_keys] if ( - func_inputs[0].ndim == 2 - and len(set(inp.shape[1] for inp in func_inputs)) > 1 + func_inputs[0].ndim == 2 + and len(set(inp.shape[1] for inp in func_inputs)) > 1 ): common_cols = func_inputs[0].columns for inp in func_inputs[1:]: @@ -1162,8 +1167,8 @@ def _execute_agg(cls, ctx, op: "DataFrameGroupByAgg"): if out_chunk.ndim == 2: result = xdf.concat(aggs, axis=1) if ( - not op.groupby_params.get("as_index", True) - and col_value.nlevels == result.columns.nlevels + not op.groupby_params.get("as_index", True) + and col_value.nlevels == result.columns.nlevels ): result.reset_index( inplace=True, drop=result.index.name in result.columns diff --git a/mars/dataframe/groupby/sort.py b/mars/dataframe/groupby/sort.py index f3fe1b612f..1583cd97ab 100644 --- a/mars/dataframe/groupby/sort.py +++ b/mars/dataframe/groupby/sort.py @@ -12,6 +12,7 @@ cudf = lazy_import("cudf", globals=globals()) + class _Largest: """ This util class resolve TypeError when @@ -27,6 +28,7 @@ def __gt__(self, other): _largest = _Largest() + class DataFrameGroupbyConcatPivot(DataFramePSRSChunkOperand, DataFrameOperandMixin): _op_type_ = OperandDef.GROUPBY_SORT_PIVOT @@ -54,7 +56,7 @@ def execute(cls, ctx, op): p = len(inputs) if len(index) < p: num = p // len(index) + 1 - index = index.append([index] * (num-1)) + index = index.append([index] * (num - 1)) # assert a.shape[op.axis] == p * len(op.inputs) index = index.sort_values() @@ -71,6 +73,7 @@ def execute(cls, ctx, op): # print(out) ctx[op.outputs[-1].key] = out + class DataFramePSRSGroupbySample(DataFramePSRSChunkOperand, DataFrameOperandMixin): _op_type_ = OperandDef.GROUPBY_SORT_REGULAR_SAMPLE @@ -119,15 +122,16 @@ def execute(cls, ctx, op): w = a.shape[0] * 1.0 / (n + 1) - slc = np.linspace( - max(w - 1, 0), a.shape[0] - 1, num=n, endpoint=False - ).astype(int) + slc = np.linspace(max(w - 1, 0), a.shape[0] - 1, num=n, endpoint=False).astype( + int + ) out = a.iloc[slc] # print(out) ctx[op.outputs[-1].key] = out + class DataFrameGroupbySortShuffle(MapReduceOperand, DataFrameOperandMixin): _op_type_ = OperandDef.GROUPBY_SORT_SHUFFLE @@ -240,11 +244,15 @@ def _execute_dataframe_map(cls, ctx, op): def _get_out_df(p_index, in_df): if p_index == 0: - out_df = in_df.loc[:pivots[p_index]] + out_df = in_df.loc[: pivots[p_index]] elif p_index == op.n_partition - 1: - out_df = in_df.loc[pivots[p_index-1]:].drop(index=pivots[p_index-1], errors="ignore") + out_df = in_df.loc[pivots[p_index - 1] :].drop( + index=pivots[p_index - 1], errors="ignore" + ) else: - out_df = in_df.loc[pivots[p_index - 1]:pivots[p_index]].drop(index=pivots[p_index-1], errors="ignore") + out_df = in_df.loc[pivots[p_index - 1] : pivots[p_index]].drop( + index=pivots[p_index - 1], errors="ignore" + ) return out_df # print("index " + str(out.key) + " " + str(df)) @@ -258,7 +266,6 @@ def _get_out_df(p_index, in_df): # print("index " + str(out.key) + " " + str(index) + " out df " + str(out_df)) ctx[out.key, index] = out_df - @classmethod def _calc_series_poses(cls, s, pivots, ascending=True): if ascending: @@ -284,9 +291,7 @@ def _execute_series_map(cls, ctx, op): except TypeError: filled_a = a.fillna(_largest) filled_pivots = pivots.fillna(_largest) - poses = cls._calc_series_poses( - filled_a, filled_pivots - ) + poses = cls._calc_series_poses(filled_a, filled_pivots) poses = (None,) + tuple(poses) + (None,) for i in range(op.n_partition): values = a.iloc[poses[i] : poses[i + 1]] diff --git a/mars/dataframe/groupby/tests/test_groupby.py b/mars/dataframe/groupby/tests/test_groupby.py index 3e7bb8911b..3c05d16595 100644 --- a/mars/dataframe/groupby/tests/test_groupby.py +++ b/mars/dataframe/groupby/tests/test_groupby.py @@ -140,7 +140,9 @@ def test_groupby_agg(): agg_chunk = chunk.inputs[0].inputs[0].inputs[0].inputs[0] assert agg_chunk.op.stage == OperandStage.map - r = mdf.groupby("c2", ).sum(method="shuffle") + r = mdf.groupby( + "c2", + ).sum(method="shuffle") assert isinstance(r.op, DataFrameGroupByAgg) assert isinstance(r, DataFrame) diff --git a/mars/dataframe/groupby/tests/test_groupby_execution.py b/mars/dataframe/groupby/tests/test_groupby_execution.py index 479c95d9ce..e2e601dd4b 100644 --- a/mars/dataframe/groupby/tests/test_groupby_execution.py +++ b/mars/dataframe/groupby/tests/test_groupby_execution.py @@ -483,6 +483,7 @@ def test_dataframe_groupby_agg(setup): ), ) + def test_dataframe_groupby_agg_sort(setup): agg_funs = [ "std", @@ -531,15 +532,11 @@ def test_dataframe_groupby_agg_sort(setup): agg = OrderedDict([("c1", ["min", "mean"]), ("c3", "std")]) r = mdf.groupby("c2").agg(agg, method=method) - pd.testing.assert_frame_equal( - r.execute().fetch(), raw.groupby("c2").agg(agg) - ) + pd.testing.assert_frame_equal(r.execute().fetch(), raw.groupby("c2").agg(agg)) agg = OrderedDict([("c1", "min"), ("c3", "sum")]) r = mdf.groupby("c2").agg(agg, method=method) - pd.testing.assert_frame_equal( - r.execute().fetch(), raw.groupby("c2").agg(agg) - ) + pd.testing.assert_frame_equal(r.execute().fetch(), raw.groupby("c2").agg(agg)) r = mdf.groupby("c2").agg({"c1": "min", "c3": "min"}, method=method) pd.testing.assert_frame_equal( @@ -555,9 +552,7 @@ def test_dataframe_groupby_agg_sort(setup): # test groupby series r = mdf.groupby(mdf["c2"]).sum(method=method) - pd.testing.assert_frame_equal( - r.execute().fetch(), raw.groupby(raw["c2"]).sum() - ) + pd.testing.assert_frame_equal(r.execute().fetch(), raw.groupby(raw["c2"]).sum()) r = mdf.groupby("c2").size(method="tree") pd.testing.assert_series_equal(r.execute().fetch(), raw.groupby("c2").size()) @@ -1356,4 +1351,3 @@ def test_groupby_nunique(setup): .nunique() .sort_values(by="b", ignore_index=True), ) - From 3966d0518358fda34f62c3c491fb7ef094cc13b2 Mon Sep 17 00:00:00 2001 From: Saksham Kumar Date: Thu, 5 May 2022 11:57:08 -0400 Subject: [PATCH 10/32] removed redundant comments --- mars/dataframe/groupby/preserve_order.py | 254 +++++++++++++++++++++++ mars/dataframe/groupby/sort.py | 38 +--- mars/opcodes.py | 3 + 3 files changed, 259 insertions(+), 36 deletions(-) create mode 100644 mars/dataframe/groupby/preserve_order.py diff --git a/mars/dataframe/groupby/preserve_order.py b/mars/dataframe/groupby/preserve_order.py new file mode 100644 index 0000000000..9bfb021fa0 --- /dev/null +++ b/mars/dataframe/groupby/preserve_order.py @@ -0,0 +1,254 @@ +import numpy as np +import pandas as pd +from pandas import MultiIndex + +from ... import opcodes as OperandDef +from mars.dataframe.operands import DataFrameOperandMixin, DataFrameOperand +from mars.utils import lazy_import +from ...core.operand import OperandStage, MapReduceOperand +from ...serialization.serializables import Int32Field, AnyField, StringField, ListField, BoolField + +cudf = lazy_import("cudf", globals=globals()) + + +class DataFrameOrderPreserveIndexOperand(DataFrameOperand, DataFrameOperandMixin): + _op_type_ = OperandDef.GROUPBY_SORT_ORDER_INDEX + + _index_prefix = Int32Field("index_prefix") + + def __init__(self, output_types=None, index_prefix=None, *args, **kwargs): + super().__init__(_output_types=output_types, _index_prefix=index_prefix, *args, **kwargs) + + @property + def index_prefix(self): + return self._index_prefix + + @property + def output_limit(self): + return 1 + + @classmethod + def execute(cls, ctx, op): + a = ctx[op.inputs[0].key][0] + xdf = pd if isinstance(a, (pd.DataFrame, pd.Series)) else cudf + if len(a) == 0: + # when chunk is empty, return the empty chunk itself + ctx[op.outputs[0].key] = a + return + + min_table = xdf.DataFrame({"min_col": np.arange(0, len((a))), "index": op.index_prefix} , index=a.index) + + ctx[op.outputs[-1].key] = min_table + + +class DataFrameOrderPreservePivotOperand(DataFrameOperand, DataFrameOperandMixin): + _op_type_ = OperandDef.GROUPBY_SORT_ORDER_PIVOT + + _n_partition = Int32Field("n_partition") + _by = AnyField("by") + + def __init__(self, n_partition=None, output_types=None, by=None, *args, **kwargs): + super().__init__(_n_partition=n_partition, _output_types=output_types, _by=by, *args, **kwargs) + + @property + def by(self): + return self._by + + @property + def output_limit(self): + return 2 + + @classmethod + def execute(cls, ctx, op): + inputs = [ctx[c.key] for c in op.inputs if len(ctx[c.key]) > 0] + if len(inputs) == 0: + # corner case: nothing sampled, we need to do nothing + ctx[op.outputs[0].key] = ctx[op.outputs[-1].key] = ctx[op.inputs[0].key] + return + + xdf = pd if isinstance(inputs[0], (pd.DataFrame, pd.Series)) else cudf + + a = xdf.concat(inputs, axis=0) + a = a.sort_index() + # a = a.groupby(op.by).min(['index', 'min_col']) + a_group = a.groupby(op.by).groups + a_list = [] + for g in a_group: + group_df = a.loc[g] + group_min_index = group_df['index'].min() + group_min_col = group_df.loc[group_df['index'] == group_min_index]['min_col'].min() + if isinstance(a.axes[0], MultiIndex): + index = pd.MultiIndex.from_tuples([g], names=group_df.index.names) + else: + index = pd.Index([g], name=group_df.index.names) + a_list_df = pd.DataFrame({"index" : group_min_index, "min_col" : group_min_col}, index=index) + a_list.append(a_list_df) + + a = pd.concat(a_list) + + ctx[op.outputs[0].key] = a + + sort_values_df = a.sort_values(['index', 'min_col']) + + p = len(inputs) + if len(sort_values_df) < p: + num = p // len(a) + 1 + sort_values_df = sort_values_df.append([sort_values_df] * (num - 1)) + + sort_values_df = sort_values_df.sort_values(['index', 'min_col']) + + w = sort_values_df.shape[0] * 1.0 / (p + 1) + + values = sort_values_df[['index', 'min_col']].values + + slc = np.linspace( + max(w-1, 0), len(sort_values_df) - 1, num=len(op.inputs) - 1, endpoint=False + ).astype(int) + out = values[slc] + ctx[op.outputs[-1].key] = out + +class DataFrameGroupbyOrderPresShuffle(MapReduceOperand, DataFrameOperandMixin): + _op_type_ = OperandDef.GROUPBY_SORT_SHUFFLE + + _by = ListField("by") + _n_partition = Int32Field("n_partition") + + def __init__( + self, + by=None, + n_partition=None, + output_types=None, + **kw + ): + super().__init__( + _by=by, + _n_partition=n_partition, + _output_types=output_types, + **kw + ) + + @property + def by(self): + return self._by + + @property + def n_partition(self): + return self._n_partition + + @property + def output_limit(self): + return 1 + + + @classmethod + def _execute_dataframe_map(cls, ctx, op): + df, pivots, min_table = [ctx[c.key] for c in op.inputs] + out = op.outputs[0] + if isinstance(df, tuple): + ijoin_df = tuple(x.join(min_table, how="inner") for x in df) + else: + ijoin_df = df.join(min_table, how="inner") + + if isinstance(df, tuple): + for i in range(len(df)): + ijoin_df[i].index = ijoin_df[i].index.rename(df[i].index.names) if isinstance(df[i].index, MultiIndex) else ijoin_df[i].index.rename(df[i].index.name) + else: + ijoin_df.index = ijoin_df.index.rename(df.index.names) if isinstance(df.index, MultiIndex) else ijoin_df.index.rename(df.index.name) + + def _get_out_df(p_index, in_df): + if p_index == 0: + index_upper = pivots[p_index][0]+1 + intermediary_dfs = [] + for i in range(0, index_upper): + if i == index_upper-1: + intermediary_dfs.append(in_df.loc[in_df['index'] == i].loc[in_df['min_col'] < pivots[p_index][1]]) + else: + intermediary_dfs.append(in_df.loc[in_df['index'] == i]) + elif p_index == op.n_partition - 1: + intermediary_dfs = [] + index_lower = pivots[p_index-1][0] + index_upper = in_df['index'].max() + 1 + for i in range(index_lower, index_upper): + if i == index_lower: + intermediary_dfs.append(in_df.loc[in_df['index'] == i].loc[in_df['min_col'] >= pivots[p_index-1][1]]) + else: + intermediary_dfs.append(in_df.loc[in_df['index'] == i]) + else: + intermediary_dfs = [] + index_lower = pivots[p_index - 1][0] + index_upper = pivots[p_index][0]+1 + if index_upper == index_lower + 1: + intermediary_dfs.append( + in_df.loc[in_df['index'] == index_lower].loc[ + (in_df['min_col'] >= pivots[p_index - 1][1]) & (in_df['min_col'] < pivots[p_index][1])]) + else: + for i in range(index_lower, index_upper): + if i == index_lower: + if index_lower != index_upper: + intermediary_dfs.append(in_df.loc[in_df['index'] == i].loc[in_df['min_col'] >= pivots[p_index-1][1]]) + elif i == index_upper-1: + intermediary_dfs.append(in_df.loc[in_df['index'] == i].loc[in_df['min_col'] < pivots[p_index][1]]) + else: + intermediary_dfs.append(in_df.loc[in_df['index'] == i]) + if len(intermediary_dfs) > 0: + out_df = pd.concat(intermediary_dfs) + else: + # out_df = pd.DataFrame(columns=in_df.columns) + # out_df.index = out_df.index.rename(in_df.index.names) if isinstance(in_df.index, MultiIndex) else out_df.index.rename(in_df.index.name) + out_df = None + return out_df + + for i in range(op.n_partition): + index = (i, 0) + if isinstance(df, tuple): + out_df = tuple(_get_out_df(i, x) for x in ijoin_df) + else: + out_df = _get_out_df(i, ijoin_df) + if out_df is not None: + ctx[out.key, index] = out_df + + @classmethod + def _execute_map(cls, ctx, op): + cls._execute_dataframe_map(ctx, op) + + @classmethod + def _execute_reduce(cls, ctx, op: "DataFramePSRSShuffle"): + out_chunk = op.outputs[0] + raw_inputs = list(op.iter_mapper_data(ctx, pop=False)) + by = op.by + xdf = cudf if op.gpu else pd + + r = [] + + if isinstance(raw_inputs[0], tuple): + tuple_len = len(raw_inputs[0]) + for i in range(tuple_len): + concat_df = xdf.concat([inp[i] for inp in raw_inputs], axis=0) + concat_df = concat_df.sort_values(["index", "min_col"]).drop(columns=["index", "min_col"]) + r.append(concat_df) + r = tuple(r) + else: + concat_df = xdf.concat(raw_inputs, axis=0) + concat_df = concat_df.sort_values(["index", "min_col"]).drop(columns=["index", "min_col"]) + r = concat_df + + if isinstance(r, tuple): + ctx[op.outputs[0].key] = r + (by,) + else: + ctx[op.outputs[0].key] = (r, by) + + @classmethod + def estimate_size(cls, ctx, op): + super().estimate_size(ctx, op) + result = ctx[op.outputs[0].key] + if op.stage == OperandStage.reduce: + ctx[op.outputs[0].key] = (result[0], result[1] * 1.5) + else: + ctx[op.outputs[0].key] = result + + @classmethod + def execute(cls, ctx, op): + if op.stage == OperandStage.map: + cls._execute_map(ctx, op) + else: + cls._execute_reduce(ctx, op) diff --git a/mars/dataframe/groupby/sort.py b/mars/dataframe/groupby/sort.py index f3fe1b612f..4629d297a0 100644 --- a/mars/dataframe/groupby/sort.py +++ b/mars/dataframe/groupby/sort.py @@ -27,6 +27,7 @@ def __gt__(self, other): _largest = _Largest() + class DataFrameGroupbyConcatPivot(DataFramePSRSChunkOperand, DataFrameOperandMixin): _op_type_ = OperandDef.GROUPBY_SORT_PIVOT @@ -45,32 +46,25 @@ def execute(cls, ctx, op): xdf = pd if isinstance(inputs[0], (pd.DataFrame, pd.Series)) else cudf a = xdf.concat(inputs, axis=0) - # a = a.reset_index(level=[op.by])[op.by] a = a.sort_index() - # print("a " + str(a)) index = a.index.drop_duplicates() - # print("index " + str(index)) p = len(inputs) if len(index) < p: num = p // len(index) + 1 index = index.append([index] * (num-1)) - # assert a.shape[op.axis] == p * len(op.inputs) index = index.sort_values() - # print("index " + str(index)) values = index.values - # print("values " + str(values)) slc = np.linspace( p - 1, len(index) - 1, num=len(op.inputs) - 1, endpoint=False ).astype(int) - # print("slc " + str(slc)) out = values[slc] - # print(out) ctx[op.outputs[-1].key] = out + class DataFramePSRSGroupbySample(DataFramePSRSChunkOperand, DataFrameOperandMixin): _op_type_ = OperandDef.GROUPBY_SORT_REGULAR_SAMPLE @@ -87,31 +81,6 @@ def execute(cls, ctx, op): ctx[op.outputs[0].key] = a return - # if op.sort_type == "sort_values": - # ctx[op.outputs[0].key] = res = execute_sort_values(a, op) - # else: - # ctx[op.outputs[0].key] = res = execute_sort_index(a, op) - - by = op.by - # add_distinct_col = bool(int(os.environ.get("PSRS_DISTINCT_COL", "0"))) - # if ( - # add_distinct_col - # and isinstance(a, xdf.DataFrame) - # and op.sort_type == "sort_values" - # ): - # # when running under distributed mode, we introduce an extra column - # # to make sure pivots are distinct - # chunk_idx = op.inputs[0].index[0] - # distinct_col = ( - # _PSRS_DISTINCT_COL - # if a.columns.nlevels == 1 - # else (_PSRS_DISTINCT_COL,) + ("",) * (a.columns.nlevels - 1) - # ) - # res[distinct_col] = np.arange( - # chunk_idx << 32, (chunk_idx << 32) + len(a), dtype=np.int64 - # ) - # by = list(by) + [distinct_col] - n = op.n_partition if a.shape[0] < n: num = n // a.shape[0] + 1 @@ -124,7 +93,6 @@ def execute(cls, ctx, op): ).astype(int) out = a.iloc[slc] - # print(out) ctx[op.outputs[-1].key] = out @@ -247,7 +215,6 @@ def _get_out_df(p_index, in_df): out_df = in_df.loc[pivots[p_index - 1]:pivots[p_index]].drop(index=pivots[p_index-1], errors="ignore") return out_df - # print("index " + str(out.key) + " " + str(df)) for i in range(op.n_partition): index = (i, 0) @@ -255,7 +222,6 @@ def _get_out_df(p_index, in_df): out_df = tuple(_get_out_df(i, x) for x in df) else: out_df = _get_out_df(i, df) - # print("index " + str(out.key) + " " + str(index) + " out df " + str(out_df)) ctx[out.key, index] = out_df diff --git a/mars/opcodes.py b/mars/opcodes.py index 561a2359d7..7fb46ac060 100644 --- a/mars/opcodes.py +++ b/mars/opcodes.py @@ -429,6 +429,9 @@ GROUPBY_SORT_REGULAR_SAMPLE = 2037 GROUPBY_SORT_PIVOT = 2038 GROUPBY_SORT_SHUFFLE = 2039 +GROUPBY_SORT_ORDER_INDEX = 2130 +GROUPBY_SORT_ORDER_PIVOT = 2131 +GROUPBY_SORT_ORDER_SHUFFLE = 2132 # parallel sorting by regular sampling PSRS_SORT_REGULAR_SMAPLE = 2040 From dd3772404f24f3e58865f5d67b679667d6e550e9 Mon Sep 17 00:00:00 2001 From: Saksham Kumar Date: Thu, 5 May 2022 12:01:49 -0400 Subject: [PATCH 11/32] changes for order preservation --- mars/dataframe/groupby/aggregation.py | 147 +++++++++++++++++++------- 1 file changed, 111 insertions(+), 36 deletions(-) diff --git a/mars/dataframe/groupby/aggregation.py b/mars/dataframe/groupby/aggregation.py index 97f9b6dda2..93d788d963 100644 --- a/mars/dataframe/groupby/aggregation.py +++ b/mars/dataframe/groupby/aggregation.py @@ -23,7 +23,8 @@ import pandas as pd from scipy.stats import variation -from .preserve_order import DataFrameOrderPreserveIndexOperand +from .preserve_order import DataFrameOrderPreserveIndexOperand, DataFrameOrderPreservePivotOperand, \ + DataFrameGroupbyOrderPresShuffle from .sort import DataFramePSRSGroupbySample, DataFrameGroupbyConcatPivot, DataFrameGroupbySortShuffle from ... import opcodes as OperandDef from ...config import options @@ -324,15 +325,6 @@ def partition_merge_data( index=partition_chunk.index, index_value=partition_chunk.index_value, ) - # if op.outputs[0].ndim == 2: - # kw.update( - # dict( - # columns_value=partition_chunk.columns_value, - # # dtypes=partition_chunk.dtypes - # ) - # ) - # else: - # kw.update(dict(dtype=partition_chunk.dtype, name=partition_chunk.name)) cs = partition_shuffle_reduce.new_chunks([proxy_chunk], **kw) partition_sort_chunks.append(cs[0]) return partition_sort_chunks @@ -357,17 +349,6 @@ def partition_local_data( output_types=[OutputType.dataframe_groupby], # columns_value=chunk_inputs[0].columns_value, ) - kw = dict() - # if op.outputs[0].ndim == 2: - # kw.update( - # dict( - # columns_value=chunk_inputs[0].columns_value, - # # dtypes=chunk_inputs[0].dtypes - # ) - # ) - # else: - # kw.update(dict(dtype=chunk_inputs[0].dtype, name=chunk_inputs[0].name)) - # kw.update(dict(name=chunk_inputs[0].name)) map_chunks.append( map_chunk_op.new_chunk( @@ -381,6 +362,65 @@ def partition_local_data( return map_chunks + @classmethod + def _gen_shuffle_chunks_order_preserve(cls, op, in_df, chunks, pivot, index_table): + properties = dict( + by=op.groupby_params['by'], + gpu=op.is_gpu() + ) + + map_chunks = [] + chunk_shape = (in_df.chunk_shape[0], 1) + for chunk in chunks: + chunk_inputs = [chunk, pivot, index_table] + map_chunk_op = DataFrameGroupbyOrderPresShuffle( + shuffle_size=chunk_shape[0], + stage=OperandStage.map, + n_partition=len(chunks), + output_types=[OutputType.dataframe_groupby], + # columns_value=chunk_inputs[0].columns_value, + ) + + map_chunks.append( + map_chunk_op.new_chunk( + chunk_inputs, + shape=chunk_shape, + index=chunk.index, + index_value=chunk.index_value, + # **kw + ) + ) + + proxy_chunk = DataFrameShuffleProxy(output_types=[OutputType.dataframe]).new_chunk( + map_chunks, shape=() + ) + + partition_sort_chunks = [] + properties = dict( + by=op.groupby_params['by'], + gpu=op.is_gpu() + ) + + for i, partition_chunk in enumerate(map_chunks): + partition_shuffle_reduce = DataFrameGroupbyOrderPresShuffle( + stage=OperandStage.reduce, + reducer_index=(i, 0), + output_types=[OutputType.dataframe_groupby], + **properties + ) + chunk_shape = list(partition_chunk.shape) + chunk_shape[0] = np.nan + + kw = dict( + shape=tuple(chunk_shape), + index=partition_chunk.index, + index_value=partition_chunk.index_value, + ) + cs = partition_shuffle_reduce.new_chunks([proxy_chunk], **kw) + partition_sort_chunks.append(cs[0]) + + return partition_sort_chunks + @classmethod def _gen_shuffle_chunks_with_pivot(cls, op, in_df, chunks, pivot): map_chunks = cls.partition_local_data( @@ -526,8 +566,10 @@ def _tile_with_shuffle( # First, perform groupby and aggregation on each chunk. agg_chunks = cls._gen_map_chunks(op, in_df.chunks, out_df, func_infos) pivot_chunk = None + index_table = None + agg_chunk_len = len(agg_chunks) + if op.groupby_params['sort'] and len(in_df.chunks) > 1: - agg_chunk_len = len(agg_chunks) sample_chunks = cls._sample_chunks(op, agg_chunks) pivot_chunk = cls._gen_pivot_chunk(op, sample_chunks, agg_chunk_len) @@ -536,11 +578,47 @@ def _tile_with_shuffle( index_chunks = cls._gen_index_chunks(op, agg_chunks) # concat and get table and pivot index_table, pivot_chunk = cls._find_index_table_and_pivot(op, index_chunks, agg_chunk_len) - # join the concat table with in_dfs - pass - # agg_chunks = agg_chunks + sample_chunks - return cls._perform_shuffle(op, agg_chunks, in_df, out_df, func_infos, pivot_chunk) + return cls._perform_shuffle(op, agg_chunks, in_df, out_df, func_infos, pivot_chunk, index_table) + + @classmethod + def _find_index_table_and_pivot(cls, op, chunks, agg_chunk_len): + output_types = [OutputType.dataframe, OutputType.tensor] + properties = dict( + gpu=op.is_gpu(), + ) + pivot_op = DataFrameOrderPreservePivotOperand( + n_partition=agg_chunk_len, + output_types=output_types, + by=op.groupby_params['by'], + **properties + ) + + kws = [] + shape = 0 + for c in chunks: + shape += c.shape[0] + kws.append( + { + "shape": (shape, c.shape[1]) if c.shape[1] is not None else (shape, ) + } + ) + kws.append( + { + "shape": (agg_chunk_len,), + "dtype": object + } + ) + # if op.outputs[0].ndim == 2: + # kws[0].update( + # {"columns_value": chunks[0].columns_value, "dtypes": chunks[0].dtypes} + # ) + # else: + # kws[0].update(({"dtype": chunks[0].dtype, "name": chunks[0].name})) + + chunks = pivot_op.new_chunks(chunks, kws=kws, output_limit=2) + index_table, pivot_chunk = chunks + return index_table, pivot_chunk @classmethod def _gen_index_chunks(cls, op, agg_chunks): @@ -548,23 +626,20 @@ def _gen_index_chunks(cls, op, agg_chunks): index_chunks = [] properties = dict( - by=op.groupby_params['by'], gpu=op.is_gpu(), ) for i, chunk in enumerate(agg_chunks): chunk_op = DataFrameOrderPreserveIndexOperand( output_types=[OutputType.dataframe], + index_prefix=i, **properties ) kws = [] - sampled_shape = ( - (chunk_shape, len(op.groupby_params['by'])) if op.groupby_params['by'] else (chunk_shape,) - ) - print(chunk) + shape = (chunk_shape, 1) kws.append( { - "shape": sampled_shape, + "shape": shape, "index_value": chunk.index_value, "index": (i, 0), } @@ -647,11 +722,14 @@ def _perform_shuffle( in_df: TileableType, out_df: TileableType, func_infos: ReductionSteps, - pivot_chunk + pivot_chunk, + index_table, ): # Shuffle the aggregation chunk. if op.groupby_params["sort"] and pivot_chunk is not None: reduce_chunks = cls._gen_shuffle_chunks_with_pivot(op, in_df, agg_chunks, pivot_chunk) + elif op.preserve_order and index_table is not None: + reduce_chunks = cls._gen_shuffle_chunks_order_preserve(op, in_df, agg_chunks, pivot_chunk, index_table) else: reduce_chunks = cls._gen_shuffle_chunks(op, in_df, agg_chunks) @@ -896,8 +974,6 @@ def _get_grouped(cls, op: "DataFrameGroupByAgg", df, ctx, copy=False, grouper=No if op.stage == OperandStage.agg: grouped = df.groupby(**params) else: - # for the intermediate phases, do not sort - # params["sort"] = False grouped = df.groupby(**params) if selection is not None: @@ -1085,7 +1161,6 @@ def _wrapped_func(col): size_recorder = ctx.get_remote_object(op.size_recorder_name) size_recorder.record(raw_size, agg_size) - # print(tuple(agg_dfs)) ctx[op.outputs[0].key] = tuple(agg_dfs) @classmethod From dfbb6da08109fbe6d4ba3cb62acdf4cfdb127a66 Mon Sep 17 00:00:00 2001 From: Saksham Kumar Date: Thu, 5 May 2022 16:26:47 -0400 Subject: [PATCH 12/32] Made changes for sampling on the basis of series or dataframe output --- mars/dataframe/groupby/aggregation.py | 42 +++++++++++--------- mars/dataframe/groupby/sort.py | 55 +++++---------------------- 2 files changed, 34 insertions(+), 63 deletions(-) diff --git a/mars/dataframe/groupby/aggregation.py b/mars/dataframe/groupby/aggregation.py index 3bfca72cf4..d9220c4aa3 100644 --- a/mars/dataframe/groupby/aggregation.py +++ b/mars/dataframe/groupby/aggregation.py @@ -568,29 +568,36 @@ def _sample_chunks(cls, op, agg_chunks): ) for i, chunk in enumerate(agg_chunks): - output_types = ( - [OutputType.dataframe] if chunk.ndim == 2 else [OutputType.series] - ) - chunk_op = DataFramePSRSGroupbySample( - kind="quicksort", - n_partition=chunk_shape, - output_types=output_types, - **properties, - ) kws = [] sampled_shape = ( (chunk_shape, chunk.shape[1]) if chunk.ndim == 2 else (chunk_shape,) ) chunk_index = (i, 0) if chunk.ndim == 2 else (i,) - kws.append( - { - "shape": sampled_shape, - "index_value": chunk.index_value, - "index": chunk_index, - "type": "regular_sampled", - } + chunk_op = DataFramePSRSGroupbySample( + kind="quicksort", + n_partition=chunk_shape, + output_types=op.output_types, + **properties, ) - + if op.output_types[0] == OutputType.dataframe: + kws.append( + { + "shape": sampled_shape, + "index_value": chunk.index_value, + "index": chunk_index, + "type": "regular_sampled", + } + ) + else: + kws.append( + { + "shape": sampled_shape, + "index_value": chunk.index_value, + "index": chunk_index, + "type": "regular_sampled", + "dtype": chunk.dtype, + } + ) chunk = chunk_op.new_chunk([chunk], kws=kws) sampled_chunks.append(chunk) @@ -1047,7 +1054,6 @@ def _wrapped_func(col): size_recorder = ctx.get_remote_object(op.size_recorder_name) size_recorder.record(raw_size, agg_size) - # print(tuple(agg_dfs)) ctx[op.outputs[0].key] = tuple(agg_dfs) @classmethod diff --git a/mars/dataframe/groupby/sort.py b/mars/dataframe/groupby/sort.py index 1583cd97ab..810a505222 100644 --- a/mars/dataframe/groupby/sort.py +++ b/mars/dataframe/groupby/sort.py @@ -7,8 +7,9 @@ from ..utils import is_cudf from ... import opcodes as OperandDef +from ...core import OutputType from ...core.operand import MapReduceOperand, OperandStage -from ...serialization.serializables import StringField, Int32Field, BoolField, ListField +from ...serialization.serializables import StringField, Int32Field, BoolField, ListField, FieldTypes cudf = lazy_import("cudf", globals=globals()) @@ -28,6 +29,11 @@ def __gt__(self, other): _largest = _Largest() +def _series_to_df(in_series, xdf): + in_df = in_series.to_frame() + if in_series.name is not None: + in_df.columns = xdf.Index([in_series.name]) + return in_df class DataFrameGroupbyConcatPivot(DataFramePSRSChunkOperand, DataFrameOperandMixin): _op_type_ = OperandDef.GROUPBY_SORT_PIVOT @@ -47,30 +53,22 @@ def execute(cls, ctx, op): xdf = pd if isinstance(inputs[0], (pd.DataFrame, pd.Series)) else cudf a = xdf.concat(inputs, axis=0) - # a = a.reset_index(level=[op.by])[op.by] a = a.sort_index() - # print("a " + str(a)) index = a.index.drop_duplicates() - # print("index " + str(index)) p = len(inputs) if len(index) < p: num = p // len(index) + 1 index = index.append([index] * (num - 1)) - # assert a.shape[op.axis] == p * len(op.inputs) index = index.sort_values() - # print("index " + str(index)) values = index.values - # print("values " + str(values)) slc = np.linspace( p - 1, len(index) - 1, num=len(op.inputs) - 1, endpoint=False ).astype(int) - # print("slc " + str(slc)) out = values[slc] - # print(out) ctx[op.outputs[-1].key] = out @@ -89,31 +87,8 @@ def execute(cls, ctx, op): # when chunk is empty, return the empty chunk itself ctx[op.outputs[0].key] = a return - - # if op.sort_type == "sort_values": - # ctx[op.outputs[0].key] = res = execute_sort_values(a, op) - # else: - # ctx[op.outputs[0].key] = res = execute_sort_index(a, op) - - by = op.by - # add_distinct_col = bool(int(os.environ.get("PSRS_DISTINCT_COL", "0"))) - # if ( - # add_distinct_col - # and isinstance(a, xdf.DataFrame) - # and op.sort_type == "sort_values" - # ): - # # when running under distributed mode, we introduce an extra column - # # to make sure pivots are distinct - # chunk_idx = op.inputs[0].index[0] - # distinct_col = ( - # _PSRS_DISTINCT_COL - # if a.columns.nlevels == 1 - # else (_PSRS_DISTINCT_COL,) + ("",) * (a.columns.nlevels - 1) - # ) - # res[distinct_col] = np.arange( - # chunk_idx << 32, (chunk_idx << 32) + len(a), dtype=np.int64 - # ) - # by = list(by) + [distinct_col] + if isinstance(a, xdf.Series) and op.output_types[0] == OutputType.dataframe: + a = _series_to_df(a, xdf) n = op.n_partition if a.shape[0] < n: @@ -127,8 +102,6 @@ def execute(cls, ctx, op): ) out = a.iloc[slc] - # print(out) - ctx[op.outputs[-1].key] = out @@ -255,15 +228,12 @@ def _get_out_df(p_index, in_df): ) return out_df - # print("index " + str(out.key) + " " + str(df)) - for i in range(op.n_partition): index = (i, 0) if isinstance(df, tuple): out_df = tuple(_get_out_df(i, x) for x in df) else: out_df = _get_out_df(i, df) - # print("index " + str(out.key) + " " + str(index) + " out df " + str(out_df)) ctx[out.key, index] = out_df @classmethod @@ -302,12 +272,7 @@ def _execute_map(cls, ctx, op): a = [ctx[c.key] for c in op.inputs][0] if isinstance(a, tuple): a = a[0] - if len(a.shape) == 2: - # DataFrame type - cls._execute_dataframe_map(ctx, op) - else: - # Series type - cls._execute_series_map(ctx, op) + cls._execute_dataframe_map(ctx, op) @classmethod def _execute_reduce(cls, ctx, op: "DataFramePSRSShuffle"): From 03fb0e1f3053c05983c5ca6afdb305e7e5a1b299 Mon Sep 17 00:00:00 2001 From: Saksham Kumar Date: Thu, 5 May 2022 16:38:35 -0400 Subject: [PATCH 13/32] code cleanup --- mars/dataframe/groupby/aggregation.py | 6 --- mars/dataframe/groupby/sort.py | 74 ++++++--------------------- 2 files changed, 17 insertions(+), 63 deletions(-) diff --git a/mars/dataframe/groupby/aggregation.py b/mars/dataframe/groupby/aggregation.py index d9220c4aa3..752a81e868 100644 --- a/mars/dataframe/groupby/aggregation.py +++ b/mars/dataframe/groupby/aggregation.py @@ -523,7 +523,6 @@ def _tile_with_shuffle( op, sample_chunks, out_idx, agg_chunk_len ) - # agg_chunks = agg_chunks + sample_chunks return cls._perform_shuffle( op, agg_chunks, in_df, out_df, func_infos, pivot_chunk ) @@ -547,13 +546,10 @@ def _gen_pivot_chunk(cls, op, sample_chunks, out_idx, agg_chunk_len): **properties, ) - # concat_pivot_index = out_idx[: 0] + (0,) + out_idx[0:] concat_pivot_chunk = concat_pivot_op.new_chunk( sample_chunks, shape=(agg_chunk_len,), dtype=object, - # shape=concat_pivot_shape, - # index=concat_pivot_index, ) return concat_pivot_chunk @@ -865,8 +861,6 @@ def _get_grouped(cls, op: "DataFrameGroupByAgg", df, ctx, copy=False, grouper=No if op.stage == OperandStage.agg: grouped = df.groupby(**params) else: - # for the intermediate phases, do not sort - # params["sort"] = False grouped = df.groupby(**params) if selection is not None: diff --git a/mars/dataframe/groupby/sort.py b/mars/dataframe/groupby/sort.py index 810a505222..21d8a77744 100644 --- a/mars/dataframe/groupby/sort.py +++ b/mars/dataframe/groupby/sort.py @@ -29,12 +29,14 @@ def __gt__(self, other): _largest = _Largest() + def _series_to_df(in_series, xdf): in_df = in_series.to_frame() if in_series.name is not None: in_df.columns = xdf.Index([in_series.name]) return in_df + class DataFrameGroupbyConcatPivot(DataFramePSRSChunkOperand, DataFrameOperandMixin): _op_type_ = OperandDef.GROUPBY_SORT_PIVOT @@ -126,19 +128,19 @@ class DataFrameGroupbySortShuffle(MapReduceOperand, DataFrameOperandMixin): _kind = StringField("kind") def __init__( - self, - sort_type=None, - by=None, - axis=None, - ascending=None, - n_partition=None, - na_position=None, - inplace=None, - kind=None, - level=None, - sort_remaining=None, - output_types=None, - **kw + self, + sort_type=None, + by=None, + axis=None, + ascending=None, + n_partition=None, + na_position=None, + inplace=None, + kind=None, + level=None, + sort_remaining=None, + output_types=None, + **kw ): super().__init__( _sort_type=sort_type, @@ -199,17 +201,6 @@ def kind(self): def output_limit(self): return 1 - @staticmethod - def _calc_poses(src_cols, pivots, ascending=True): - records = src_cols.to_records(index=False) - p_records = pivots.to_records(index=False) - if ascending: - poses = records.searchsorted(p_records, side="right") - else: - poses = len(records) - records[::-1].searchsorted(p_records, side="right") - del records, p_records - return poses - @classmethod def _execute_dataframe_map(cls, ctx, op): df, pivots = [ctx[c.key] for c in op.inputs] @@ -219,11 +210,11 @@ def _get_out_df(p_index, in_df): if p_index == 0: out_df = in_df.loc[: pivots[p_index]] elif p_index == op.n_partition - 1: - out_df = in_df.loc[pivots[p_index - 1] :].drop( + out_df = in_df.loc[pivots[p_index - 1]:].drop( index=pivots[p_index - 1], errors="ignore" ) else: - out_df = in_df.loc[pivots[p_index - 1] : pivots[p_index]].drop( + out_df = in_df.loc[pivots[p_index - 1]: pivots[p_index]].drop( index=pivots[p_index - 1], errors="ignore" ) return out_df @@ -236,37 +227,6 @@ def _get_out_df(p_index, in_df): out_df = _get_out_df(i, df) ctx[out.key, index] = out_df - @classmethod - def _calc_series_poses(cls, s, pivots, ascending=True): - if ascending: - poses = s.searchsorted(pivots, side="right") - else: - poses = len(s) - s.iloc[::-1].searchsorted(pivots, side="right") - return poses - - @classmethod - def _execute_series_map(cls, ctx, op): - a, pivots = [ctx[c.key] for c in op.inputs] - out = op.outputs[0] - - if len(a) == 0: - # when the chunk is empty, no slices can be produced - for i in range(op.n_partition): - ctx[out.key, (i,)] = a - return - - if isinstance(a, pd.Series): - try: - poses = cls._calc_series_poses(a, pivots) - except TypeError: - filled_a = a.fillna(_largest) - filled_pivots = pivots.fillna(_largest) - poses = cls._calc_series_poses(filled_a, filled_pivots) - poses = (None,) + tuple(poses) + (None,) - for i in range(op.n_partition): - values = a.iloc[poses[i] : poses[i + 1]] - ctx[out.key, (i,)] = values - @classmethod def _execute_map(cls, ctx, op): a = [ctx[c.key] for c in op.inputs][0] From 59c447f933b1b61a15696997594733fe2bd8f2b6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=BB=A7=E7=9B=9B?= Date: Fri, 6 May 2022 14:47:55 +0800 Subject: [PATCH 14/32] Fix ut --- mars/dataframe/groupby/aggregation.py | 82 ++++++++++++++++----------- mars/dataframe/groupby/sort.py | 44 +++++++------- 2 files changed, 75 insertions(+), 51 deletions(-) diff --git a/mars/dataframe/groupby/aggregation.py b/mars/dataframe/groupby/aggregation.py index 752a81e868..28a9dc2653 100644 --- a/mars/dataframe/groupby/aggregation.py +++ b/mars/dataframe/groupby/aggregation.py @@ -49,7 +49,7 @@ pd_release_version, estimate_pandas_size, ) -from ..core import GROUPBY_TYPE, IndexValue +from ..core import GROUPBY_TYPE from ..merge import DataFrameConcat from ..operands import DataFrameOperand, DataFrameOperandMixin, DataFrameShuffleProxy from ..reduction.core import ( @@ -303,12 +303,18 @@ def partition_merge_data(cls, op, partition_chunks, proxy_chunk, in_df): # stage 4: all *ith* classes are gathered and merged partition_sort_chunks = [] properties = dict(by=op.groupby_params["by"], gpu=op.is_gpu()) + out_df = op.outputs[0] for i, partition_chunk in enumerate(partition_chunks): + output_types = ( + [OutputType.dataframe_groupby] + if out_df.ndim == 2 + else [OutputType.series_groupby] + ) partition_shuffle_reduce = DataFrameGroupbySortShuffle( stage=OperandStage.reduce, reducer_index=(i, 0), - output_types=[OutputType.dataframe_groupby], + output_types=output_types, **properties, ) chunk_shape = list(partition_chunk.shape) @@ -319,45 +325,48 @@ def partition_merge_data(cls, op, partition_chunks, proxy_chunk, in_df): index=partition_chunk.index, index_value=partition_chunk.index_value, ) - # if op.outputs[0].ndim == 2: - # kw.update( - # dict( - # columns_value=partition_chunk.columns_value, - # # dtypes=partition_chunk.dtypes - # ) - # ) - # else: - # kw.update(dict(dtype=partition_chunk.dtype, name=partition_chunk.name)) + if op.outputs[0].ndim == 2: + kw.update( + dict( + columns_value=partition_chunk.columns_value, + dtypes=partition_chunk.dtypes, + ) + ) + else: + kw.update(dict(dtype=partition_chunk.dtype, name=partition_chunk.name)) cs = partition_shuffle_reduce.new_chunks([proxy_chunk], **kw) partition_sort_chunks.append(cs[0]) return partition_sort_chunks @classmethod def partition_local_data(cls, op, sorted_chunks, concat_pivot_chunk, in_df): - properties = dict(by=op.groupby_params["by"], gpu=op.is_gpu()) - + # properties = dict(by=op.groupby_params["by"], gpu=op.is_gpu()) + out_df = op.outputs[0] map_chunks = [] chunk_shape = (in_df.chunk_shape[0], 1) for chunk in sorted_chunks: chunk_inputs = [chunk, concat_pivot_chunk] + output_types = ( + [OutputType.dataframe_groupby] + if out_df.ndim == 2 + else [OutputType.series_groupby] + ) map_chunk_op = DataFrameGroupbySortShuffle( shuffle_size=chunk_shape[0], stage=OperandStage.map, n_partition=len(sorted_chunks), - output_types=[OutputType.dataframe_groupby], - # columns_value=chunk_inputs[0].columns_value, + output_types=output_types, ) kw = dict() - # if op.outputs[0].ndim == 2: - # kw.update( - # dict( - # columns_value=chunk_inputs[0].columns_value, - # # dtypes=chunk_inputs[0].dtypes - # ) - # ) - # else: - # kw.update(dict(dtype=chunk_inputs[0].dtype, name=chunk_inputs[0].name)) - # kw.update(dict(name=chunk_inputs[0].name)) + if out_df.ndim == 2: + kw.update( + dict( + columns_value=chunk_inputs[0].columns_value, + dtypes=chunk_inputs[0].dtypes, + ) + ) + else: + kw.update(dict(dtype=chunk_inputs[0].dtype, name=chunk_inputs[0].name)) map_chunks.append( map_chunk_op.new_chunk( @@ -459,15 +468,18 @@ def _gen_map_chunks( map_op.pre_funcs = func_infos.pre_funcs map_op.agg_funcs = func_infos.agg_funcs new_index = chunk.index if len(chunk.index) == 2 else (chunk.index[0],) - if op.output_types[0] == OutputType.dataframe: + if out_df.ndim == 2: + new_index = (new_index[0], 0) if len(new_index) == 1 else new_index map_chunk = map_op.new_chunk( chunk_inputs, shape=out_df.shape, index=new_index, index_value=out_df.index_value, columns_value=out_df.columns_value, + dtypes=out_df.dtypes, ) else: + new_index = new_index[:1] if len(new_index) == 2 else new_index map_chunk = map_op.new_chunk( chunk_inputs, shape=(out_df.shape[0],), @@ -577,7 +589,7 @@ def _sample_chunks(cls, op, agg_chunks): ) if op.output_types[0] == OutputType.dataframe: kws.append( - { + { "shape": sampled_shape, "index_value": chunk.index_value, "index": chunk_index, @@ -686,14 +698,17 @@ def _combine_tree( if len(chks) == 1: chk = chks[0] else: - concat_op = DataFrameConcat(output_types=[OutputType.dataframe]) + concat_op = DataFrameConcat(output_types=out_df.op.output_types) # Change index for concatenate for j, c in enumerate(chks): c._index = (j, 0) - chk = concat_op.new_chunk(chks, dtypes=chks[0].dtypes) + if out_df.ndim == 2: + chk = concat_op.new_chunk(chks, dtypes=chks[0].dtypes) + else: + chk = concat_op.new_chunk(chks, dtype=chunks[0].dtype) chunk_op = op.copy().reset_key() chunk_op.tileable_op_key = None - chunk_op.output_types = [OutputType.dataframe] + chunk_op.output_types = out_df.op.output_types chunk_op.stage = OperandStage.combine chunk_op.groupby_params = chunk_op.groupby_params.copy() chunk_op.groupby_params.pop("selection", None) @@ -717,8 +732,11 @@ def _combine_tree( ) chunks = new_chunks - concat_op = DataFrameConcat(output_types=[OutputType.dataframe]) - chk = concat_op.new_chunk(chunks, dtypes=chunks[0].dtypes) + concat_op = DataFrameConcat(output_types=out_df.op.output_types) + if out_df.ndim == 2: + chk = concat_op.new_chunk(chunks, dtypes=chunks[0].dtypes) + else: + chk = concat_op.new_chunk(chunks, dtype=chunks[0].dtype) chunk_op = op.copy().reset_key() chunk_op.tileable_op_key = op.key chunk_op.stage = OperandStage.agg diff --git a/mars/dataframe/groupby/sort.py b/mars/dataframe/groupby/sort.py index 21d8a77744..012faa42d5 100644 --- a/mars/dataframe/groupby/sort.py +++ b/mars/dataframe/groupby/sort.py @@ -4,12 +4,16 @@ from mars.dataframe.operands import DataFrameOperandMixin from mars.dataframe.sort.psrs import DataFramePSRSChunkOperand from mars.utils import lazy_import -from ..utils import is_cudf from ... import opcodes as OperandDef from ...core import OutputType from ...core.operand import MapReduceOperand, OperandStage -from ...serialization.serializables import StringField, Int32Field, BoolField, ListField, FieldTypes +from ...serialization.serializables import ( + StringField, + Int32Field, + BoolField, + ListField, +) cudf = lazy_import("cudf", globals=globals()) @@ -104,6 +108,9 @@ def execute(cls, ctx, op): ) out = a.iloc[slc] + if op.output_types[0] == OutputType.series and out.ndim == 2: + assert out.shape[1] == 1 + out = out.iloc[:, 0] ctx[op.outputs[-1].key] = out @@ -128,19 +135,19 @@ class DataFrameGroupbySortShuffle(MapReduceOperand, DataFrameOperandMixin): _kind = StringField("kind") def __init__( - self, - sort_type=None, - by=None, - axis=None, - ascending=None, - n_partition=None, - na_position=None, - inplace=None, - kind=None, - level=None, - sort_remaining=None, - output_types=None, - **kw + self, + sort_type=None, + by=None, + axis=None, + ascending=None, + n_partition=None, + na_position=None, + inplace=None, + kind=None, + level=None, + sort_remaining=None, + output_types=None, + **kw ): super().__init__( _sort_type=sort_type, @@ -210,11 +217,11 @@ def _get_out_df(p_index, in_df): if p_index == 0: out_df = in_df.loc[: pivots[p_index]] elif p_index == op.n_partition - 1: - out_df = in_df.loc[pivots[p_index - 1]:].drop( + out_df = in_df.loc[pivots[p_index - 1] :].drop( index=pivots[p_index - 1], errors="ignore" ) else: - out_df = in_df.loc[pivots[p_index - 1]: pivots[p_index]].drop( + out_df = in_df.loc[pivots[p_index - 1] : pivots[p_index]].drop( index=pivots[p_index - 1], errors="ignore" ) return out_df @@ -235,8 +242,7 @@ def _execute_map(cls, ctx, op): cls._execute_dataframe_map(ctx, op) @classmethod - def _execute_reduce(cls, ctx, op: "DataFramePSRSShuffle"): - out_chunk = op.outputs[0] + def _execute_reduce(cls, ctx, op: "DataFrameGroupbySortShuffle"): raw_inputs = list(op.iter_mapper_data(ctx, pop=False)) by = op.by xdf = cudf if op.gpu else pd From e658810b3febf62a7047c7b320ee0f4c9a4778dd Mon Sep 17 00:00:00 2001 From: Saksham Kumar Date: Sun, 8 May 2022 03:24:52 -0400 Subject: [PATCH 15/32] fixed imports. removed redundant code --- mars/dataframe/groupby/sort.py | 77 +++------------------------------- 1 file changed, 5 insertions(+), 72 deletions(-) diff --git a/mars/dataframe/groupby/sort.py b/mars/dataframe/groupby/sort.py index 012faa42d5..add9c0cabd 100644 --- a/mars/dataframe/groupby/sort.py +++ b/mars/dataframe/groupby/sort.py @@ -1,9 +1,11 @@ import numpy as np import pandas as pd -from mars.dataframe.operands import DataFrameOperandMixin -from mars.dataframe.sort.psrs import DataFramePSRSChunkOperand -from mars.utils import lazy_import +from ..operands import DataFrameOperandMixin +from ..sort.psrs import DataFramePSRSChunkOperand +from ...utils import ( + lazy_import, +) from ... import opcodes as OperandDef from ...core import OutputType @@ -18,22 +20,6 @@ cudf = lazy_import("cudf", globals=globals()) -class _Largest: - """ - This util class resolve TypeError when - comparing strings with None values - """ - - def __lt__(self, other): - return False - - def __gt__(self, other): - return self is not other - - -_largest = _Largest() - - def _series_to_df(in_series, xdf): in_df = in_series.to_frame() if in_series.name is not None: @@ -117,93 +103,40 @@ def execute(cls, ctx, op): class DataFrameGroupbySortShuffle(MapReduceOperand, DataFrameOperandMixin): _op_type_ = OperandDef.GROUPBY_SORT_SHUFFLE - _sort_type = StringField("sort_type") - # for shuffle map - _axis = Int32Field("axis") _by = ListField("by") - _ascending = BoolField("ascending") _inplace = BoolField("inplace") - _na_position = StringField("na_position") _n_partition = Int32Field("n_partition") - # for sort_index - _level = ListField("level") - _sort_remaining = BoolField("sort_remaining") - - # for shuffle reduce - _kind = StringField("kind") def __init__( self, - sort_type=None, by=None, - axis=None, - ascending=None, n_partition=None, - na_position=None, inplace=None, - kind=None, - level=None, - sort_remaining=None, output_types=None, **kw ): super().__init__( - _sort_type=sort_type, _by=by, - _axis=axis, - _ascending=ascending, _n_partition=n_partition, - _na_position=na_position, _inplace=inplace, - _kind=kind, - _level=level, - _sort_remaining=sort_remaining, _output_types=output_types, **kw ) - @property - def sort_type(self): - return self._sort_type - @property def by(self): return self._by - @property - def axis(self): - return self._axis - - @property - def ascending(self): - return self._ascending - @property def inplace(self): return self._inplace - @property - def na_position(self): - return self._na_position - - @property - def level(self): - return self._level - - @property - def sort_remaining(self): - return self._sort_remaining - @property def n_partition(self): return self._n_partition - @property - def kind(self): - return self._kind - @property def output_limit(self): return 1 From 1126278b759a7716db6f2311fef01037af4a9510 Mon Sep 17 00:00:00 2001 From: Saksham Kumar Date: Sun, 8 May 2022 03:47:34 -0400 Subject: [PATCH 16/32] reformat code --- mars/dataframe/groupby/sort.py | 25 +++++++++++-------------- 1 file changed, 11 insertions(+), 14 deletions(-) diff --git a/mars/dataframe/groupby/sort.py b/mars/dataframe/groupby/sort.py index add9c0cabd..cf7d8e7f84 100644 --- a/mars/dataframe/groupby/sort.py +++ b/mars/dataframe/groupby/sort.py @@ -3,19 +3,17 @@ from ..operands import DataFrameOperandMixin from ..sort.psrs import DataFramePSRSChunkOperand -from ...utils import ( - lazy_import, -) - from ... import opcodes as OperandDef from ...core import OutputType from ...core.operand import MapReduceOperand, OperandStage from ...serialization.serializables import ( - StringField, Int32Field, BoolField, ListField, ) +from ...utils import ( + lazy_import, +) cudf = lazy_import("cudf", globals=globals()) @@ -108,14 +106,13 @@ class DataFrameGroupbySortShuffle(MapReduceOperand, DataFrameOperandMixin): _inplace = BoolField("inplace") _n_partition = Int32Field("n_partition") - def __init__( - self, - by=None, - n_partition=None, - inplace=None, - output_types=None, - **kw + self, + by=None, + n_partition=None, + inplace=None, + output_types=None, + **kw ): super().__init__( _by=by, @@ -150,11 +147,11 @@ def _get_out_df(p_index, in_df): if p_index == 0: out_df = in_df.loc[: pivots[p_index]] elif p_index == op.n_partition - 1: - out_df = in_df.loc[pivots[p_index - 1] :].drop( + out_df = in_df.loc[pivots[p_index - 1]:].drop( index=pivots[p_index - 1], errors="ignore" ) else: - out_df = in_df.loc[pivots[p_index - 1] : pivots[p_index]].drop( + out_df = in_df.loc[pivots[p_index - 1]: pivots[p_index]].drop( index=pivots[p_index - 1], errors="ignore" ) return out_df From 49ee66f9227a88b2c02131938ded13dd40b46fa8 Mon Sep 17 00:00:00 2001 From: Saksham Kumar Date: Sun, 8 May 2022 04:04:09 -0400 Subject: [PATCH 17/32] code style check --- mars/dataframe/groupby/sort.py | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/mars/dataframe/groupby/sort.py b/mars/dataframe/groupby/sort.py index cf7d8e7f84..5385b9c2ec 100644 --- a/mars/dataframe/groupby/sort.py +++ b/mars/dataframe/groupby/sort.py @@ -107,12 +107,7 @@ class DataFrameGroupbySortShuffle(MapReduceOperand, DataFrameOperandMixin): _n_partition = Int32Field("n_partition") def __init__( - self, - by=None, - n_partition=None, - inplace=None, - output_types=None, - **kw + self, by=None, n_partition=None, inplace=None, output_types=None, **kw ): super().__init__( _by=by, @@ -147,11 +142,11 @@ def _get_out_df(p_index, in_df): if p_index == 0: out_df = in_df.loc[: pivots[p_index]] elif p_index == op.n_partition - 1: - out_df = in_df.loc[pivots[p_index - 1]:].drop( + out_df = in_df.loc[pivots[p_index - 1] :].drop( index=pivots[p_index - 1], errors="ignore" ) else: - out_df = in_df.loc[pivots[p_index - 1]: pivots[p_index]].drop( + out_df = in_df.loc[pivots[p_index - 1] : pivots[p_index]].drop( index=pivots[p_index - 1], errors="ignore" ) return out_df From 3bd71ea9c0ea91009ee77c0ca64163d9500822d8 Mon Sep 17 00:00:00 2001 From: Saksham Kumar Date: Sun, 8 May 2022 04:13:06 -0400 Subject: [PATCH 18/32] code style check --- mars/dataframe/groupby/sort.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mars/dataframe/groupby/sort.py b/mars/dataframe/groupby/sort.py index 5385b9c2ec..ad1557aa8a 100644 --- a/mars/dataframe/groupby/sort.py +++ b/mars/dataframe/groupby/sort.py @@ -107,7 +107,7 @@ class DataFrameGroupbySortShuffle(MapReduceOperand, DataFrameOperandMixin): _n_partition = Int32Field("n_partition") def __init__( - self, by=None, n_partition=None, inplace=None, output_types=None, **kw + self, by=None, n_partition=None, inplace=None, output_types=None, **kw ): super().__init__( _by=by, From 10591b6d3bb4cfc420be844cb27d8a81b00bf5e4 Mon Sep 17 00:00:00 2001 From: Saksham Kumar Date: Sun, 8 May 2022 04:23:12 -0400 Subject: [PATCH 19/32] added apache 2.0 licence --- mars/dataframe/groupby/sort.py | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/mars/dataframe/groupby/sort.py b/mars/dataframe/groupby/sort.py index ad1557aa8a..263d864217 100644 --- a/mars/dataframe/groupby/sort.py +++ b/mars/dataframe/groupby/sort.py @@ -1,3 +1,17 @@ +# Copyright 1999-2021 Alibaba Group Holding Ltd. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + import numpy as np import pandas as pd From 19cedfbb4b5faf952f2e8f5aa6bb8bc59cbe3972 Mon Sep 17 00:00:00 2001 From: Saksham Kumar Date: Sun, 8 May 2022 05:43:58 -0400 Subject: [PATCH 20/32] code cleanup --- mars/dataframe/groupby/sort.py | 19 +++---------------- 1 file changed, 3 insertions(+), 16 deletions(-) diff --git a/mars/dataframe/groupby/sort.py b/mars/dataframe/groupby/sort.py index 263d864217..fc2f6443e9 100644 --- a/mars/dataframe/groupby/sort.py +++ b/mars/dataframe/groupby/sort.py @@ -117,7 +117,6 @@ class DataFrameGroupbySortShuffle(MapReduceOperand, DataFrameOperandMixin): # for shuffle map _by = ListField("by") - _inplace = BoolField("inplace") _n_partition = Int32Field("n_partition") def __init__( @@ -126,7 +125,6 @@ def __init__( super().__init__( _by=by, _n_partition=n_partition, - _inplace=inplace, _output_types=output_types, **kw ) @@ -135,10 +133,6 @@ def __init__( def by(self): return self._by - @property - def inplace(self): - return self._inplace - @property def n_partition(self): return self._n_partition @@ -148,7 +142,7 @@ def output_limit(self): return 1 @classmethod - def _execute_dataframe_map(cls, ctx, op): + def _execute_map(cls, ctx, op): df, pivots = [ctx[c.key] for c in op.inputs] out = op.outputs[0] @@ -156,11 +150,11 @@ def _get_out_df(p_index, in_df): if p_index == 0: out_df = in_df.loc[: pivots[p_index]] elif p_index == op.n_partition - 1: - out_df = in_df.loc[pivots[p_index - 1] :].drop( + out_df = in_df.loc[pivots[p_index - 1]:].drop( index=pivots[p_index - 1], errors="ignore" ) else: - out_df = in_df.loc[pivots[p_index - 1] : pivots[p_index]].drop( + out_df = in_df.loc[pivots[p_index - 1]: pivots[p_index]].drop( index=pivots[p_index - 1], errors="ignore" ) return out_df @@ -173,13 +167,6 @@ def _get_out_df(p_index, in_df): out_df = _get_out_df(i, df) ctx[out.key, index] = out_df - @classmethod - def _execute_map(cls, ctx, op): - a = [ctx[c.key] for c in op.inputs][0] - if isinstance(a, tuple): - a = a[0] - cls._execute_dataframe_map(ctx, op) - @classmethod def _execute_reduce(cls, ctx, op: "DataFrameGroupbySortShuffle"): raw_inputs = list(op.iter_mapper_data(ctx, pop=False)) From df6f6fac65cdfe8ddcb43803afbb148b98913b28 Mon Sep 17 00:00:00 2001 From: Saksham Kumar Date: Sun, 8 May 2022 06:07:56 -0400 Subject: [PATCH 21/32] Added test for empty df --- .../groupby/tests/test_groupby_execution.py | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/mars/dataframe/groupby/tests/test_groupby_execution.py b/mars/dataframe/groupby/tests/test_groupby_execution.py index e2e601dd4b..3999636811 100644 --- a/mars/dataframe/groupby/tests/test_groupby_execution.py +++ b/mars/dataframe/groupby/tests/test_groupby_execution.py @@ -577,11 +577,14 @@ def test_dataframe_groupby_agg_sort(setup): ) assert r.op.groupby_params["as_index"] is True - # r = mdf.groupby("c2").agg(["cumsum", "cumcount"]) - # pd.testing.assert_frame_equal( - # r.execute().fetch(), - # raw.groupby("c2").agg(["cumsum", "cumcount"]), - # ) + # test empty dataframe + e_df = pd.DataFrame(columns=["A", "B"]) + e_mars_df = md.DataFrame(e_df, chunk_size=10) + for method in ["tree", "shuffle"]: + pd.testing.assert_frame_equal( + e_mars_df.groupby(["A"]).mean(method=method).execute().fetch(), + e_df.groupby("A").mean() + ) def test_series_groupby_agg(setup): From 86bab10748912baa3481f4b2d576c1ce9c9fdb91 Mon Sep 17 00:00:00 2001 From: Saksham Kumar Date: Sun, 8 May 2022 06:14:16 -0400 Subject: [PATCH 22/32] removed unused import --- mars/dataframe/groupby/sort.py | 1 - 1 file changed, 1 deletion(-) diff --git a/mars/dataframe/groupby/sort.py b/mars/dataframe/groupby/sort.py index fc2f6443e9..54d0330614 100644 --- a/mars/dataframe/groupby/sort.py +++ b/mars/dataframe/groupby/sort.py @@ -22,7 +22,6 @@ from ...core.operand import MapReduceOperand, OperandStage from ...serialization.serializables import ( Int32Field, - BoolField, ListField, ) from ...utils import ( From 68ef1aca14c79e8d3a99b757be42efd4a7427856 Mon Sep 17 00:00:00 2001 From: Saksham Kumar Date: Sun, 8 May 2022 06:28:00 -0400 Subject: [PATCH 23/32] fixed code style issues --- mars/dataframe/groupby/sort.py | 9 +++------ mars/dataframe/groupby/tests/test_groupby_execution.py | 2 +- 2 files changed, 4 insertions(+), 7 deletions(-) diff --git a/mars/dataframe/groupby/sort.py b/mars/dataframe/groupby/sort.py index 54d0330614..c6a7dc0d42 100644 --- a/mars/dataframe/groupby/sort.py +++ b/mars/dataframe/groupby/sort.py @@ -122,10 +122,7 @@ def __init__( self, by=None, n_partition=None, inplace=None, output_types=None, **kw ): super().__init__( - _by=by, - _n_partition=n_partition, - _output_types=output_types, - **kw + _by=by, _n_partition=n_partition, _output_types=output_types, **kw ) @property @@ -149,11 +146,11 @@ def _get_out_df(p_index, in_df): if p_index == 0: out_df = in_df.loc[: pivots[p_index]] elif p_index == op.n_partition - 1: - out_df = in_df.loc[pivots[p_index - 1]:].drop( + out_df = in_df.loc[pivots[p_index - 1] :].drop( index=pivots[p_index - 1], errors="ignore" ) else: - out_df = in_df.loc[pivots[p_index - 1]: pivots[p_index]].drop( + out_df = in_df.loc[pivots[p_index - 1] : pivots[p_index]].drop( index=pivots[p_index - 1], errors="ignore" ) return out_df diff --git a/mars/dataframe/groupby/tests/test_groupby_execution.py b/mars/dataframe/groupby/tests/test_groupby_execution.py index 3999636811..5ee6f9926a 100644 --- a/mars/dataframe/groupby/tests/test_groupby_execution.py +++ b/mars/dataframe/groupby/tests/test_groupby_execution.py @@ -583,7 +583,7 @@ def test_dataframe_groupby_agg_sort(setup): for method in ["tree", "shuffle"]: pd.testing.assert_frame_equal( e_mars_df.groupby(["A"]).mean(method=method).execute().fetch(), - e_df.groupby("A").mean() + e_df.groupby("A").mean(), ) From f893fc3ffd1632f419178e9d0cadfc64039d255f Mon Sep 17 00:00:00 2001 From: Saksham Kumar Date: Sun, 8 May 2022 14:03:32 -0400 Subject: [PATCH 24/32] removed redundant code --- mars/dataframe/groupby/sort.py | 29 ++++--------------- .../groupby/tests/test_groupby_execution.py | 9 ------ 2 files changed, 6 insertions(+), 32 deletions(-) diff --git a/mars/dataframe/groupby/sort.py b/mars/dataframe/groupby/sort.py index c6a7dc0d42..975297ecf5 100644 --- a/mars/dataframe/groupby/sort.py +++ b/mars/dataframe/groupby/sort.py @@ -48,10 +48,6 @@ def output_limit(self): @classmethod def execute(cls, ctx, op): inputs = [ctx[c.key] for c in op.inputs if len(ctx[c.key]) > 0] - if len(inputs) == 0: - # corner case: nothing sampled, we need to do nothing - ctx[op.outputs[-1].key] = ctx[op.inputs[0].key] - return xdf = pd if isinstance(inputs[0], (pd.DataFrame, pd.Series)) else cudf @@ -86,10 +82,6 @@ def output_limit(self): def execute(cls, ctx, op): a = ctx[op.inputs[0].key][0] xdf = pd if isinstance(a, (pd.DataFrame, pd.Series)) else cudf - if len(a) == 0: - # when chunk is empty, return the empty chunk itself - ctx[op.outputs[0].key] = a - return if isinstance(a, xdf.Series) and op.output_types[0] == OutputType.dataframe: a = _series_to_df(a, xdf) @@ -157,10 +149,7 @@ def _get_out_df(p_index, in_df): for i in range(op.n_partition): index = (i, 0) - if isinstance(df, tuple): - out_df = tuple(_get_out_df(i, x) for x in df) - else: - out_df = _get_out_df(i, df) + out_df = tuple(_get_out_df(i, x) for x in df) ctx[out.key, index] = out_df @classmethod @@ -171,18 +160,12 @@ def _execute_reduce(cls, ctx, op: "DataFrameGroupbySortShuffle"): r = [] - if isinstance(raw_inputs[0], tuple): - tuple_len = len(raw_inputs[0]) - for i in range(tuple_len): - r.append(xdf.concat([inp[i] for inp in raw_inputs], axis=0)) - r = tuple(r) - else: - r = xdf.concat(raw_inputs, axis=0) + tuple_len = len(raw_inputs[0]) + for i in range(tuple_len): + r.append(xdf.concat([inp[i] for inp in raw_inputs], axis=0)) + r = tuple(r) - if isinstance(r, tuple): - ctx[op.outputs[0].key] = r + (by,) - else: - ctx[op.outputs[0].key] = (r, by) + ctx[op.outputs[0].key] = r + (by,) @classmethod def estimate_size(cls, ctx, op): diff --git a/mars/dataframe/groupby/tests/test_groupby_execution.py b/mars/dataframe/groupby/tests/test_groupby_execution.py index 5ee6f9926a..57ce84db43 100644 --- a/mars/dataframe/groupby/tests/test_groupby_execution.py +++ b/mars/dataframe/groupby/tests/test_groupby_execution.py @@ -577,15 +577,6 @@ def test_dataframe_groupby_agg_sort(setup): ) assert r.op.groupby_params["as_index"] is True - # test empty dataframe - e_df = pd.DataFrame(columns=["A", "B"]) - e_mars_df = md.DataFrame(e_df, chunk_size=10) - for method in ["tree", "shuffle"]: - pd.testing.assert_frame_equal( - e_mars_df.groupby(["A"]).mean(method=method).execute().fetch(), - e_df.groupby("A").mean(), - ) - def test_series_groupby_agg(setup): rs = np.random.RandomState(0) From 515f692faf7c465ffd39444b69fe5904ccb3ff72 Mon Sep 17 00:00:00 2001 From: Saksham Kumar Date: Mon, 9 May 2022 00:53:33 -0400 Subject: [PATCH 25/32] made changes to map and reduce stage to support series and dataframe --- mars/dataframe/groupby/aggregation.py | 78 ++++++++++++++----- .../groupby/tests/test_groupby_execution.py | 6 -- 2 files changed, 57 insertions(+), 27 deletions(-) diff --git a/mars/dataframe/groupby/aggregation.py b/mars/dataframe/groupby/aggregation.py index a5483561fd..8872c3a454 100644 --- a/mars/dataframe/groupby/aggregation.py +++ b/mars/dataframe/groupby/aggregation.py @@ -386,49 +386,64 @@ def partition_local_data(cls, op, sorted_chunks, concat_pivot_chunk, in_df): return map_chunks @classmethod - def _gen_shuffle_chunks_order_preserve(cls, op, in_df, chunks, pivot, index_table): - properties = dict( - by=op.groupby_params['by'], - gpu=op.is_gpu() - ) - + def partition_local_data_op(cls, op, sorted_chunks, concat_pivot_chunk, in_df, index_table): + out_df = op.outputs[0] map_chunks = [] chunk_shape = (in_df.chunk_shape[0], 1) - for chunk in chunks: - chunk_inputs = [chunk, pivot, index_table] + for chunk in sorted_chunks: + chunk_inputs = [chunk, concat_pivot_chunk, index_table] + output_types = ( + [OutputType.dataframe_groupby] + if out_df.ndim == 2 + else [OutputType.series_groupby] + ) map_chunk_op = DataFrameGroupbyOrderPresShuffle( shuffle_size=chunk_shape[0], stage=OperandStage.map, - n_partition=len(chunks), - output_types=[OutputType.dataframe_groupby], - # columns_value=chunk_inputs[0].columns_value, + n_partition=len(sorted_chunks), + output_types=output_types, ) + kw = dict() + if out_df.ndim == 2: + kw.update( + dict( + columns_value=chunk_inputs[0].columns_value, + dtypes=chunk_inputs[0].dtypes, + ) + ) + else: + kw.update(dict(dtype=chunk_inputs[0].dtype, name=chunk_inputs[0].name)) + map_chunks.append( map_chunk_op.new_chunk( chunk_inputs, shape=chunk_shape, index=chunk.index, - index_value=chunk.index_value, + index_value=chunk_inputs[0].index_value, # **kw ) ) - proxy_chunk = DataFrameShuffleProxy(output_types=[OutputType.dataframe]).new_chunk( - map_chunks, shape=() - ) + return map_chunks + @classmethod + def partition_merge_data_op(cls, op, partition_chunks, proxy_chunk, in_df): + # stage 4: all *ith* classes are gathered and merged partition_sort_chunks = [] - properties = dict( - by=op.groupby_params['by'], - gpu=op.is_gpu() - ) + properties = dict(by=op.groupby_params["by"], gpu=op.is_gpu()) + out_df = op.outputs[0] - for i, partition_chunk in enumerate(map_chunks): + for i, partition_chunk in enumerate(partition_chunks): + output_types = ( + [OutputType.dataframe_groupby] + if out_df.ndim == 2 + else [OutputType.series_groupby] + ) partition_shuffle_reduce = DataFrameGroupbyOrderPresShuffle( stage=OperandStage.reduce, reducer_index=(i, 0), - output_types=[OutputType.dataframe_groupby], + output_types=output_types, **properties ) chunk_shape = list(partition_chunk.shape) @@ -439,8 +454,29 @@ def _gen_shuffle_chunks_order_preserve(cls, op, in_df, chunks, pivot, index_tabl index=partition_chunk.index, index_value=partition_chunk.index_value, ) + if op.outputs[0].ndim == 2: + kw.update( + dict( + columns_value=partition_chunk.columns_value, + dtypes=partition_chunk.dtypes, + ) + ) + else: + kw.update(dict(dtype=partition_chunk.dtype, name=partition_chunk.name)) cs = partition_shuffle_reduce.new_chunks([proxy_chunk], **kw) partition_sort_chunks.append(cs[0]) + return partition_sort_chunks + + @classmethod + def _gen_shuffle_chunks_order_preserve(cls, op, in_df, chunks, pivot, index_table): + # properties = dict(by=op.groupby_params['by'], gpu=op.is_gpu()) + map_chunks = cls.partition_local_data_op(op, chunks, pivot, in_df, index_table) + + proxy_chunk = DataFrameShuffleProxy(output_types=[OutputType.dataframe]).new_chunk( + map_chunks, shape=() + ) + + partition_sort_chunks = cls.partition_merge_data_op(op, map_chunks, proxy_chunk, in_df) return partition_sort_chunks diff --git a/mars/dataframe/groupby/tests/test_groupby_execution.py b/mars/dataframe/groupby/tests/test_groupby_execution.py index efc3e4d910..6e20396a16 100644 --- a/mars/dataframe/groupby/tests/test_groupby_execution.py +++ b/mars/dataframe/groupby/tests/test_groupby_execution.py @@ -1371,8 +1371,6 @@ def test_dataframe_groupby_agg_op(setup): ) mdf = md.DataFrame(raw, chunk_size=13) - # r = mdf.groupby(mdf["c2"], sort=False, preserve_order=True).sum(method="shuffle") - # pd.testing.assert_frame_equal(r.execute().fetch(), raw.groupby(raw["c2"], sort=False).sum()) for method in ["tree", "shuffle"]: @@ -1415,9 +1413,6 @@ def test_dataframe_groupby_agg_op(setup): # r = mdf.groupby(mdf["c2"], sort=False, preserve_order=True).sum(method=method) # pd.testing.assert_frame_equal(r.execute().fetch(), raw.groupby(raw["c2"], sort=False).sum()) - # r = mdf.groupby("c2").size(method="tree") - # pd.testing.assert_series_equal(r.execute().fetch(), raw.groupby("c2").size()) - # test inserted kurt method r = mdf.groupby("c2", sort=False, preserve_order=True).kurtosis(method="tree") pd.testing.assert_frame_equal(r.execute().fetch(), raw.groupby("c2", sort=False).kurtosis()) @@ -1437,4 +1432,3 @@ def test_dataframe_groupby_agg_op(setup): raw.groupby(["c1", "c2"], sort=False, as_index=False).agg(["mean", "count"]), ) assert r.op.groupby_params["as_index"] is True - From bb0236defa0acf7881cc66a33fcbd96a700eafb7 Mon Sep 17 00:00:00 2001 From: Saksham Kumar Date: Mon, 9 May 2022 01:42:02 -0400 Subject: [PATCH 26/32] resolved a few review comments --- mars/dataframe/groupby/aggregation.py | 56 +++++++++++++++++++-------- mars/dataframe/groupby/sort.py | 45 ++++++++++----------- 2 files changed, 61 insertions(+), 40 deletions(-) diff --git a/mars/dataframe/groupby/aggregation.py b/mars/dataframe/groupby/aggregation.py index 28a9dc2653..fa0bf56636 100644 --- a/mars/dataframe/groupby/aggregation.py +++ b/mars/dataframe/groupby/aggregation.py @@ -22,11 +22,6 @@ import pandas as pd from scipy.stats import variation -from .sort import ( - DataFramePSRSGroupbySample, - DataFrameGroupbyConcatPivot, - DataFrameGroupbySortShuffle, -) from ... import opcodes as OperandDef from ...config import options from ...core.custom_log import redirect_custom_log @@ -60,6 +55,11 @@ from ..reduction.aggregation import is_funcs_aggregate, normalize_reduction_funcs from ..utils import parse_index, build_concatenated_rows_frame, is_cudf from .core import DataFrameGroupByOperand +from .sort import ( + DataFramePSRSGroupbySample, + DataFrameGroupbyConcatPivot, + DataFrameGroupbySortShuffle, +) cp = lazy_import("cupy", globals=globals(), rename="cp") cudf = lazy_import("cudf", globals=globals()) @@ -299,7 +299,12 @@ def __call__(self, groupby): return self._call_series(groupby, df) @classmethod - def partition_merge_data(cls, op, partition_chunks, proxy_chunk, in_df): + def partition_merge_data( + cls, + op: "DataFrameGroupByAgg", + partition_chunks: List[ChunkType], + proxy_chunk: ChunkType, + ): # stage 4: all *ith* classes are gathered and merged partition_sort_chunks = [] properties = dict(by=op.groupby_params["by"], gpu=op.is_gpu()) @@ -339,7 +344,13 @@ def partition_merge_data(cls, op, partition_chunks, proxy_chunk, in_df): return partition_sort_chunks @classmethod - def partition_local_data(cls, op, sorted_chunks, concat_pivot_chunk, in_df): + def partition_local_data( + cls, + op: "DataFrameGroupByAgg", + sorted_chunks: List[ChunkType], + concat_pivot_chunk: ChunkType, + in_df: TileableType, + ): # properties = dict(by=op.groupby_params["by"], gpu=op.is_gpu()) out_df = op.outputs[0] map_chunks = [] @@ -381,7 +392,13 @@ def partition_local_data(cls, op, sorted_chunks, concat_pivot_chunk, in_df): return map_chunks @classmethod - def _gen_shuffle_chunks_with_pivot(cls, op, in_df, chunks, pivot): + def _gen_shuffle_chunks_with_pivot( + cls, + op: "DataFrameGroupByAgg", + in_df: TileableType, + chunks: List[ChunkType], + pivot: ChunkType, + ): map_chunks = cls.partition_local_data(op, chunks, pivot, in_df) proxy_chunk = DataFrameShuffleProxy( @@ -389,7 +406,7 @@ def _gen_shuffle_chunks_with_pivot(cls, op, in_df, chunks, pivot): ).new_chunk(map_chunks, shape=()) partition_sort_chunks = cls.partition_merge_data( - op, map_chunks, proxy_chunk, in_df + op, map_chunks, proxy_chunk ) return partition_sort_chunks @@ -528,11 +545,10 @@ def _tile_with_shuffle( agg_chunks = cls._gen_map_chunks(op, in_df.chunks, out_df, func_infos) pivot_chunk = None if op.groupby_params["sort"] and len(in_df.chunks) > 1: - out_idx = ((0,) if in_df.ndim == 2 else (),) agg_chunk_len = len(agg_chunks) sample_chunks = cls._sample_chunks(op, agg_chunks) pivot_chunk = cls._gen_pivot_chunk( - op, sample_chunks, out_idx, agg_chunk_len + op, sample_chunks, agg_chunk_len ) return cls._perform_shuffle( @@ -540,7 +556,12 @@ def _tile_with_shuffle( ) @classmethod - def _gen_pivot_chunk(cls, op, sample_chunks, out_idx, agg_chunk_len): + def _gen_pivot_chunk( + cls, + op: "DataFrameGroupByAgg", + sample_chunks: List[ChunkType], + agg_chunk_len: int, + ): properties = dict( by=op.groupby_params["by"], @@ -566,7 +587,11 @@ def _gen_pivot_chunk(cls, op, sample_chunks, out_idx, agg_chunk_len): return concat_pivot_chunk @classmethod - def _sample_chunks(cls, op, agg_chunks): + def _sample_chunks( + cls, + op: "DataFrameGroupByAgg", + agg_chunks: List[ChunkType], + ): chunk_shape = len(agg_chunks) sampled_chunks = [] @@ -619,7 +644,7 @@ def _perform_shuffle( in_df: TileableType, out_df: TileableType, func_infos: ReductionSteps, - pivot_chunk, + pivot_chunk: ChunkType, ): # Shuffle the aggregation chunk. if op.groupby_params["sort"] and pivot_chunk is not None: @@ -822,11 +847,10 @@ def _tile_auto( # otherwise, use shuffle pivot_chunk = None if op.groupby_params["sort"] and len(in_df.chunks) > 1: - out_idx = ((0,) if in_df.ndim == 2 else (),) agg_chunk_len = len(chunks + left_chunks) sample_chunks = cls._sample_chunks(op, chunks + left_chunks) pivot_chunk = cls._gen_pivot_chunk( - op, sample_chunks, out_idx, agg_chunk_len + op, sample_chunks, agg_chunk_len ) logger.debug("Choose shuffle method for groupby operand %s", op) diff --git a/mars/dataframe/groupby/sort.py b/mars/dataframe/groupby/sort.py index 975297ecf5..ffa716c9dd 100644 --- a/mars/dataframe/groupby/sort.py +++ b/mars/dataframe/groupby/sort.py @@ -15,18 +15,19 @@ import numpy as np import pandas as pd -from ..operands import DataFrameOperandMixin -from ..sort.psrs import DataFramePSRSChunkOperand from ... import opcodes as OperandDef from ...core import OutputType from ...core.operand import MapReduceOperand, OperandStage from ...serialization.serializables import ( Int32Field, ListField, + FieldTypes, ) from ...utils import ( lazy_import, ) +from ..operands import DataFrameOperandMixin +from ..sort.psrs import DataFramePSRSChunkOperand cudf = lazy_import("cudf", globals=globals()) @@ -46,7 +47,7 @@ def output_limit(self): return 1 @classmethod - def execute(cls, ctx, op): + def execute(cls, ctx, op: "DataFrameGroupbyConcatPivot"): inputs = [ctx[c.key] for c in op.inputs if len(ctx[c.key]) > 0] xdf = pd if isinstance(inputs[0], (pd.DataFrame, pd.Series)) else cudf @@ -79,7 +80,7 @@ def output_limit(self): return 1 @classmethod - def execute(cls, ctx, op): + def execute(cls, ctx, op: "DataFramePSRSGroupbySample"): a = ctx[op.inputs[0].key][0] xdf = pd if isinstance(a, (pd.DataFrame, pd.Series)) else cudf if isinstance(a, xdf.Series) and op.output_types[0] == OutputType.dataframe: @@ -107,30 +108,26 @@ class DataFrameGroupbySortShuffle(MapReduceOperand, DataFrameOperandMixin): _op_type_ = OperandDef.GROUPBY_SORT_SHUFFLE # for shuffle map - _by = ListField("by") - _n_partition = Int32Field("n_partition") - - def __init__( - self, by=None, n_partition=None, inplace=None, output_types=None, **kw - ): - super().__init__( - _by=by, _n_partition=n_partition, _output_types=output_types, **kw - ) - - @property - def by(self): - return self._by - - @property - def n_partition(self): - return self._n_partition + by = ListField("by") + n_partition = Int32Field("n_partition") + output_types = ListField( + "output_type", FieldTypes.reference(OutputType), default=None + ) + + # @property + # def by(self): + # return self.by + # + # @property + # def n_partition(self): + # return self.n_partition @property def output_limit(self): return 1 @classmethod - def _execute_map(cls, ctx, op): + def _execute_map(cls, ctx, op: "DataFrameGroupbySortShuffle"): df, pivots = [ctx[c.key] for c in op.inputs] out = op.outputs[0] @@ -168,7 +165,7 @@ def _execute_reduce(cls, ctx, op: "DataFrameGroupbySortShuffle"): ctx[op.outputs[0].key] = r + (by,) @classmethod - def estimate_size(cls, ctx, op): + def estimate_size(cls, ctx, op: "DataFrameGroupbySortShuffle"): super().estimate_size(ctx, op) result = ctx[op.outputs[0].key] if op.stage == OperandStage.reduce: @@ -177,7 +174,7 @@ def estimate_size(cls, ctx, op): ctx[op.outputs[0].key] = result @classmethod - def execute(cls, ctx, op): + def execute(cls, ctx, op: "DataFrameGroupbySortShuffle"): if op.stage == OperandStage.map: cls._execute_map(ctx, op) else: From daf431989ce06416f308eadab34d72b2c6601ba6 Mon Sep 17 00:00:00 2001 From: Saksham Kumar Date: Mon, 9 May 2022 01:51:10 -0400 Subject: [PATCH 27/32] code style checks --- mars/dataframe/groupby/aggregation.py | 12 +++--------- 1 file changed, 3 insertions(+), 9 deletions(-) diff --git a/mars/dataframe/groupby/aggregation.py b/mars/dataframe/groupby/aggregation.py index fa0bf56636..3a8f99b2d5 100644 --- a/mars/dataframe/groupby/aggregation.py +++ b/mars/dataframe/groupby/aggregation.py @@ -405,9 +405,7 @@ def _gen_shuffle_chunks_with_pivot( output_types=[OutputType.dataframe] ).new_chunk(map_chunks, shape=()) - partition_sort_chunks = cls.partition_merge_data( - op, map_chunks, proxy_chunk - ) + partition_sort_chunks = cls.partition_merge_data(op, map_chunks, proxy_chunk) return partition_sort_chunks @@ -547,9 +545,7 @@ def _tile_with_shuffle( if op.groupby_params["sort"] and len(in_df.chunks) > 1: agg_chunk_len = len(agg_chunks) sample_chunks = cls._sample_chunks(op, agg_chunks) - pivot_chunk = cls._gen_pivot_chunk( - op, sample_chunks, agg_chunk_len - ) + pivot_chunk = cls._gen_pivot_chunk(op, sample_chunks, agg_chunk_len) return cls._perform_shuffle( op, agg_chunks, in_df, out_df, func_infos, pivot_chunk @@ -849,9 +845,7 @@ def _tile_auto( if op.groupby_params["sort"] and len(in_df.chunks) > 1: agg_chunk_len = len(chunks + left_chunks) sample_chunks = cls._sample_chunks(op, chunks + left_chunks) - pivot_chunk = cls._gen_pivot_chunk( - op, sample_chunks, agg_chunk_len - ) + pivot_chunk = cls._gen_pivot_chunk(op, sample_chunks, agg_chunk_len) logger.debug("Choose shuffle method for groupby operand %s", op) return cls._perform_shuffle( From 0c8ca0b87136cd2e056587082bacf49c17dbb39f Mon Sep 17 00:00:00 2001 From: Saksham Kumar Date: Mon, 9 May 2022 01:58:24 -0400 Subject: [PATCH 28/32] code style checks --- mars/dataframe/groupby/sort.py | 8 -------- 1 file changed, 8 deletions(-) diff --git a/mars/dataframe/groupby/sort.py b/mars/dataframe/groupby/sort.py index ffa716c9dd..a70b61274b 100644 --- a/mars/dataframe/groupby/sort.py +++ b/mars/dataframe/groupby/sort.py @@ -114,14 +114,6 @@ class DataFrameGroupbySortShuffle(MapReduceOperand, DataFrameOperandMixin): "output_type", FieldTypes.reference(OutputType), default=None ) - # @property - # def by(self): - # return self.by - # - # @property - # def n_partition(self): - # return self.n_partition - @property def output_limit(self): return 1 From e98e5853d450ffa49e8fb67260e35b2cad1e9190 Mon Sep 17 00:00:00 2001 From: Saksham Kumar Date: Mon, 9 May 2022 02:14:33 -0400 Subject: [PATCH 29/32] resolved a few review comments --- mars/dataframe/groupby/aggregation.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mars/dataframe/groupby/aggregation.py b/mars/dataframe/groupby/aggregation.py index 3a8f99b2d5..fbe671b277 100644 --- a/mars/dataframe/groupby/aggregation.py +++ b/mars/dataframe/groupby/aggregation.py @@ -643,7 +643,7 @@ def _perform_shuffle( pivot_chunk: ChunkType, ): # Shuffle the aggregation chunk. - if op.groupby_params["sort"] and pivot_chunk is not None: + if pivot_chunk is not None: reduce_chunks = cls._gen_shuffle_chunks_with_pivot( op, in_df, agg_chunks, pivot_chunk ) From 8ac2555c5ea572b0fc9182b9eb858787915f68ae Mon Sep 17 00:00:00 2001 From: Saksham Kumar Date: Mon, 9 May 2022 02:35:13 -0400 Subject: [PATCH 30/32] resolved review comment to direct output type to init --- mars/dataframe/groupby/sort.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/mars/dataframe/groupby/sort.py b/mars/dataframe/groupby/sort.py index a70b61274b..141e204c6a 100644 --- a/mars/dataframe/groupby/sort.py +++ b/mars/dataframe/groupby/sort.py @@ -110,9 +110,9 @@ class DataFrameGroupbySortShuffle(MapReduceOperand, DataFrameOperandMixin): # for shuffle map by = ListField("by") n_partition = Int32Field("n_partition") - output_types = ListField( - "output_type", FieldTypes.reference(OutputType), default=None - ) + + def __init__(self, *args, output_types=None, **kw): + super().__init__(self, *args, _output_types=output_types, **kw) @property def output_limit(self): From b09f47c243282b4f1532ec4f8ca8d50d99d75a5b Mon Sep 17 00:00:00 2001 From: Saksham Kumar Date: Mon, 9 May 2022 02:39:10 -0400 Subject: [PATCH 31/32] removed unused sort --- mars/dataframe/groupby/sort.py | 1 - 1 file changed, 1 deletion(-) diff --git a/mars/dataframe/groupby/sort.py b/mars/dataframe/groupby/sort.py index 141e204c6a..545e49248d 100644 --- a/mars/dataframe/groupby/sort.py +++ b/mars/dataframe/groupby/sort.py @@ -21,7 +21,6 @@ from ...serialization.serializables import ( Int32Field, ListField, - FieldTypes, ) from ...utils import ( lazy_import, From ea997b28b261b484fc1ae7b47b0c363419512439 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=BB=A7=E7=9B=9B?= Date: Mon, 9 May 2022 16:14:23 +0800 Subject: [PATCH 32/32] Fix ut --- mars/dataframe/groupby/sort.py | 4 ++-- mars/serialization/serializables/core.py | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/mars/dataframe/groupby/sort.py b/mars/dataframe/groupby/sort.py index 545e49248d..d0d068e1f8 100644 --- a/mars/dataframe/groupby/sort.py +++ b/mars/dataframe/groupby/sort.py @@ -110,8 +110,8 @@ class DataFrameGroupbySortShuffle(MapReduceOperand, DataFrameOperandMixin): by = ListField("by") n_partition = Int32Field("n_partition") - def __init__(self, *args, output_types=None, **kw): - super().__init__(self, *args, _output_types=output_types, **kw) + def __init__(self, output_types=None, **kw): + super().__init__(_output_types=output_types, **kw) @property def output_limit(self): diff --git a/mars/serialization/serializables/core.py b/mars/serialization/serializables/core.py index cee72105db..e11e1245cf 100644 --- a/mars/serialization/serializables/core.py +++ b/mars/serialization/serializables/core.py @@ -124,7 +124,7 @@ class Serializable(metaclass=SerializableMeta): def __init__(self, *args, **kwargs): if args: # pragma: no cover - values = dict(zip(self.__slots__, args)) + values = dict(zip(self._FIELDS, args)) values.update(kwargs) else: values = kwargs