def do_agg(blocks, clear_input_blocks: bool, *_): # TODO: implement clear_input_blocks stage_info = {} if len(aggs) == 0: raise ValueError("Aggregate requires at least one aggregation") for agg in aggs: agg._validate(self._dataset) # Handle empty dataset. if blocks.initial_num_blocks() == 0: return blocks, stage_info num_mappers = blocks.initial_num_blocks() num_reducers = num_mappers if self._key is None: num_reducers = 1 boundaries = [] else: boundaries = sort.sample_boundaries( blocks.get_blocks(), [(self._key, "ascending")] if isinstance(self._key, str) else self._key, num_reducers, ) shuffle_op = GroupbyOp(map_args=[boundaries, self._key, aggs], reduce_args=[self._key, aggs]) return shuffle_op.execute( blocks, num_reducers, clear_input_blocks, )
def do_agg(blocks, clear_input_blocks: bool, block_udf): # TODO: implement clear_input_blocks stage_info = {} if len(aggs) == 0: raise ValueError("Aggregate requires at least one aggregation") for agg in aggs: agg._validate(self._dataset) # Handle empty dataset. if blocks.initial_num_blocks() == 0: return blocks, stage_info num_mappers = blocks.initial_num_blocks() num_reducers = num_mappers if self._key is None: num_reducers = 1 boundaries = [] else: boundaries = sort.sample_boundaries( blocks.get_blocks(), [(self._key, "ascending")] if isinstance(self._key, str) else self._key, num_reducers, ) partition_and_combine_block = cached_remote_fn( _partition_and_combine_block).options( num_returns=num_reducers + 1) aggregate_combined_blocks = cached_remote_fn( _aggregate_combined_blocks, num_returns=2) map_results = np.empty((num_mappers, num_reducers), dtype=object) map_meta = [] for i, block in enumerate(blocks.get_blocks()): results = partition_and_combine_block.remote( block, boundaries, self._key, aggs) map_results[i, :] = results[:-1] map_meta.append(results[-1]) map_bar = ProgressBar("GroupBy Map", len(map_results)) map_bar.block_until_complete(map_meta) stage_info["map"] = ray.get(map_meta) map_bar.close() blocks = [] metadata = [] for j in range(num_reducers): block, meta = aggregate_combined_blocks.remote( num_reducers, self._key, aggs, *map_results[:, j].tolist()) blocks.append(block) metadata.append(meta) reduce_bar = ProgressBar("GroupBy Reduce", len(blocks)) reduce_bar.block_until_complete(blocks) reduce_bar.close() metadata = ray.get(metadata) stage_info["reduce"] = metadata return BlockList(blocks, metadata), stage_info
def aggregate(self, *aggs: AggregateFn) -> Dataset[U]: """Implements the accumulator-based aggregation. This is a blocking operation. Examples: >>> grouped_ds.aggregate(AggregateFn( ... init=lambda k: [], ... accumulate=lambda a, r: a + [r], ... merge=lambda a1, a2: a1 + a2, ... finalize=lambda a: a ... )) Args: aggs: Aggregations to do. Returns: If the input dataset is simple dataset then the output is a simple dataset of (k, v_1, ..., v_n) tuples where k is the groupby key and v_i is the result of the ith given aggregation. If the input dataset is Arrow dataset then the output is an Arrow dataset of n + 1 columns where first column is the groupby key and the second through n + 1 columns are the results of the aggregations. If groupby key is None then the key part of return is omitted. """ if len(aggs) == 0: raise ValueError("Aggregate requires at least one aggregation") # Handle empty dataset. if self._dataset.num_blocks() == 0: return self._dataset blocks = list(self._dataset._blocks.iter_blocks()) num_mappers = len(blocks) num_reducers = num_mappers if self._key is None: num_reducers = 1 boundaries = [] else: boundaries = sort.sample_boundaries( blocks, [(self._key, "ascending")] if isinstance(self._key, str) else self._key, num_reducers) partition_and_combine_block = cached_remote_fn( _partition_and_combine_block).options(num_returns=num_reducers) aggregate_combined_blocks = cached_remote_fn( _aggregate_combined_blocks, num_returns=2) map_results = np.empty((num_mappers, num_reducers), dtype=object) for i, block in enumerate(blocks): map_results[i, :] = partition_and_combine_block.remote( block, boundaries, self._key, aggs) map_bar = ProgressBar("GroupBy Map", len(map_results)) map_bar.block_until_complete([ret[0] for ret in map_results]) map_bar.close() blocks = [] metadata = [] for j in range(num_reducers): block, meta = aggregate_combined_blocks.remote( num_reducers, self._key, aggs, *map_results[:, j].tolist()) blocks.append(block) metadata.append(meta) reduce_bar = ProgressBar("GroupBy Reduce", len(blocks)) reduce_bar.block_until_complete(blocks) reduce_bar.close() metadata = ray.get(metadata) return Dataset(BlockList(blocks, metadata), self._dataset._epoch)