def sort_impl( blocks: BlockList, clear_input_blocks: bool, key: SortKeyT, descending: bool = False ) -> Tuple[BlockList, dict]: stage_info = {} blocks_list = blocks.get_blocks() if len(blocks_list) == 0: return BlockList([], []), stage_info if isinstance(key, str): key = [(key, "descending" if descending else "ascending")] if isinstance(key, list): descending = key[0][1] == "descending" num_mappers = len(blocks_list) # Use same number of output partitions. num_reducers = num_mappers # TODO(swang): sample_boundaries could be fused with a previous stage. boundaries = sample_boundaries(blocks_list, key, num_reducers) if descending: boundaries.reverse() shuffle_op = SortOp( map_args=[boundaries, key, descending], reduce_args=[key, descending] ) return shuffle_op.execute( blocks, num_reducers, clear_input_blocks, )
def sort_impl(blocks: BlockList, key: SortKeyT, descending: bool = False) -> Tuple[BlockList, dict]: stage_info = {} blocks = blocks.get_blocks() if len(blocks) == 0: return BlockList([], []), stage_info if isinstance(key, str): key = [(key, "descending" if descending else "ascending")] if isinstance(key, list): descending = key[0][1] == "descending" num_mappers = len(blocks) num_reducers = num_mappers boundaries = sample_boundaries(blocks, key, num_reducers) if descending: boundaries.reverse() sort_block = cached_remote_fn(_sort_block).options( num_returns=num_reducers + 1) merge_sorted_blocks = cached_remote_fn(_merge_sorted_blocks, num_returns=2) map_results = np.empty((num_mappers, num_reducers), dtype=object) map_meta = [] for i, block in enumerate(blocks): result = sort_block.remote(block, boundaries, key, descending) map_results[i, :] = result[:-1] map_meta.append(result[-1]) # Early release memory. del blocks map_bar = ProgressBar("Sort Map", len(map_results)) map_bar.block_until_complete(map_meta) map_bar.close() stage_info["map"] = ray.get(map_meta) reduce_results = [] for j in range(num_reducers): ret = merge_sorted_blocks.remote(key, descending, *map_results[:, j].tolist()) reduce_results.append(ret) # Early release memory. del map_results merge_bar = ProgressBar("Sort Merge", len(reduce_results)) merge_bar.block_until_complete([ret[0] for ret in reduce_results]) merge_bar.close() blocks = [b for b, _ in reduce_results] metadata = ray.get([m for _, m in reduce_results]) stage_info["merge"] = metadata return BlockList(blocks, metadata), stage_info
def execute( self, input_blocks: BlockList, output_num_blocks: int, clear_input_blocks: bool, *, map_ray_remote_args: Optional[Dict[str, Any]] = None, reduce_ray_remote_args: Optional[Dict[str, Any]] = None, merge_factor: int = 2, ) -> Tuple[BlockList, Dict[str, List[BlockMetadata]]]: logger.info("Using experimental push-based shuffle.") # TODO(swang): For jobs whose reduce work is heavier than the map work, # we should support fractional merge factors. # TODO(swang): For large jobs, we should try to choose the merge factor # automatically, e.g., by running one test round of map and merge tasks # and comparing their run times. # TODO(swang): Add option to automatically reduce write amplification # during map-merge stage, by limiting how many partitions can be # processed concurrently. input_blocks_list = input_blocks.get_blocks() # Preemptively clear the blocks list since we will incrementally delete # the last remaining references as we submit the dependent map tasks # during the map-merge stage. if clear_input_blocks: input_blocks.clear() if map_ray_remote_args is None: map_ray_remote_args = {} if reduce_ray_remote_args is None: reduce_ray_remote_args = {} # The placement strategy for reduce tasks is overwritten to colocate # them with their inputs from the merge stage, so remove any # pre-specified scheduling strategy here. reduce_ray_remote_args = reduce_ray_remote_args.copy() reduce_ray_remote_args.pop("scheduling_strategy", None) map_fn = self._map_partition merge_fn = self._merge def map_partition(*args, **kwargs): return map_fn(self.map, *args, **kwargs) def merge(*args, **kwargs): return merge_fn(self.reduce, *args, **kwargs) shuffle_map = cached_remote_fn(map_partition) shuffle_merge = cached_remote_fn(merge) def submit_map_task(arg): mapper_idx, block = arg # NOTE(swang): Results are shuffled between map and merge tasks, so # there is no advantage to colocating specific map and merge tasks. # Therefore, we do not specify a node affinity policy for map tasks # in case the caller or Ray has a better scheduling strategy, e.g., # based on data locality. map_result = shuffle_map.options( **map_ray_remote_args, num_returns=1 + schedule.num_merge_tasks_per_round, ).remote( mapper_idx, block, output_num_blocks, schedule, *self._map_args, ) metadata_ref = map_result.pop(0) return metadata_ref, map_result def submit_merge_task(arg): merge_idx, map_results = arg num_merge_returns = schedule.get_num_reducers_per_merge_idx(merge_idx) merge_result = shuffle_merge.options( num_returns=1 + num_merge_returns, **schedule.get_merge_task_options(merge_idx), ).remote( *map_results, reduce_args=self._reduce_args, ) metadata_ref = merge_result.pop(0) return metadata_ref, merge_result # Compute all constants used for task scheduling. num_cpus_per_node_map = _get_num_cpus_per_node_map() schedule = self._compute_shuffle_schedule( num_cpus_per_node_map, len(input_blocks_list), merge_factor, output_num_blocks, ) # ObjectRef results from the last round of tasks. Used to add # backpressure during pipelining of map and merge tasks. last_map_metadata_results = [] last_merge_metadata_results = [] # Final outputs from the map-merge stage. # This is a map from merge task index to a nested list of merge results # (ObjectRefs). Each merge task index corresponds to a partition of P # final reduce tasks. all_merge_results = [[] for _ in range(schedule.num_merge_tasks_per_round)] shuffle_map_metadata = [] shuffle_merge_metadata = [] map_bar = ProgressBar("Shuffle Map", position=0, total=len(input_blocks_list)) # Execute the map-merge stage. This submits tasks in rounds of M map # tasks and N merge tasks each. Task execution between map and merge is # pipelined, so that while executing merge for one round of inputs, we # also execute the map tasks for the following round. input_blocks_list = list(enumerate(input_blocks_list)) while input_blocks_list: # Execute one round of the map stage. # Pop from the inputs so that we can clear the memory ASAP. round_input_blocks = [] try: for _ in range(schedule.num_map_tasks_per_round): round_input_blocks.append(input_blocks_list.pop(0)) except IndexError: pass ( prev_map_metadata, last_map_metadata_results, map_results, ) = _execute_pipelined_stage( submit_map_task, last_map_metadata_results, round_input_blocks, progress_bar=map_bar, ) shuffle_map_metadata += prev_map_metadata # Shuffle the map results for the merge tasks. merge_args = [ (merge_idx, [map_result.pop(0) for map_result in map_results]) for merge_idx in range(schedule.num_merge_tasks_per_round) ] assert all([not map_result for map_result in map_results]) # Execute one round of the merge stage. ( prev_merge_metadata, last_merge_metadata_results, merge_results, ) = _execute_pipelined_stage( submit_merge_task, last_merge_metadata_results, merge_args, ) shuffle_merge_metadata += prev_merge_metadata for merge_idx, merge_result in enumerate(merge_results): all_merge_results[merge_idx].append(merge_result) del merge_results # Wait for last map and merge tasks to finish. prev_map_metadata, _, _ = _execute_pipelined_stage( None, last_map_metadata_results, [], progress_bar=map_bar ) shuffle_map_metadata += prev_map_metadata map_bar.close() prev_merge_metadata, _, _ = _execute_pipelined_stage( None, last_merge_metadata_results, [] ) shuffle_merge_metadata += prev_merge_metadata # Execute and wait for the reduce stage. new_metadata, new_blocks = self._execute_reduce_stage( output_num_blocks, schedule, reduce_ray_remote_args, all_merge_results ) stats = { "map": shuffle_map_metadata, "merge": shuffle_merge_metadata, "reduce": new_metadata, } return BlockList(list(new_blocks), list(new_metadata)), stats
def execute( self, input_blocks: BlockList, output_num_blocks: int, clear_input_blocks: bool, *, map_ray_remote_args: Optional[Dict[str, Any]] = None, reduce_ray_remote_args: Optional[Dict[str, Any]] = None ) -> Tuple[BlockList, Dict[str, List[BlockMetadata]]]: input_blocks_list = input_blocks.get_blocks() input_num_blocks = len(input_blocks_list) if map_ray_remote_args is None: map_ray_remote_args = {} if reduce_ray_remote_args is None: reduce_ray_remote_args = {} if "scheduling_strategy" not in reduce_ray_remote_args: reduce_ray_remote_args = reduce_ray_remote_args.copy() reduce_ray_remote_args["scheduling_strategy"] = "SPREAD" shuffle_map = cached_remote_fn(self.map) shuffle_reduce = cached_remote_fn(self.reduce) map_bar = ProgressBar("Shuffle Map", total=input_num_blocks) shuffle_map_out = [ shuffle_map.options( **map_ray_remote_args, num_returns=1 + output_num_blocks, ).remote(i, block, output_num_blocks, *self._map_args) for i, block in enumerate(input_blocks_list) ] # The first item returned is the BlockMetadata. shuffle_map_metadata = [] for i, refs in enumerate(shuffle_map_out): shuffle_map_metadata.append(refs[0]) shuffle_map_out[i] = refs[1:] # Eagerly delete the input block references in order to eagerly release # the blocks' memory. del input_blocks_list if clear_input_blocks: input_blocks.clear() shuffle_map_metadata = map_bar.fetch_until_complete( shuffle_map_metadata) map_bar.close() reduce_bar = ProgressBar("Shuffle Reduce", total=output_num_blocks) shuffle_reduce_out = [ shuffle_reduce.options( **reduce_ray_remote_args, num_returns=2, ).remote( *self._reduce_args, *[shuffle_map_out[i][j] for i in range(input_num_blocks)], ) for j in range(output_num_blocks) ] # Eagerly delete the map block references in order to eagerly release # the blocks' memory. del shuffle_map_out new_blocks, new_metadata = zip(*shuffle_reduce_out) new_metadata = reduce_bar.fetch_until_complete(list(new_metadata)) reduce_bar.close() stats = { "map": shuffle_map_metadata, "reduce": new_metadata, } return BlockList(list(new_blocks), list(new_metadata)), stats
def simple_shuffle( input_blocks: BlockList, block_udf: Optional[Callable[[Block], Iterable[Block]]], output_num_blocks: int, *, random_shuffle: bool = False, random_seed: Optional[int] = None, map_ray_remote_args: Optional[Dict[str, Any]] = None, reduce_ray_remote_args: Optional[Dict[str, Any]] = None, _spread_resource_prefix: Optional[str] = None ) -> Tuple[BlockList, Dict[str, List[BlockMetadata]]]: input_blocks = input_blocks.get_blocks() if map_ray_remote_args is None: map_ray_remote_args = {} if reduce_ray_remote_args is None: reduce_ray_remote_args = {} if "scheduling_strategy" not in reduce_ray_remote_args: reduce_ray_remote_args = reduce_ray_remote_args.copy() reduce_ray_remote_args["scheduling_strategy"] = "SPREAD" input_num_blocks = len(input_blocks) if _spread_resource_prefix is not None: # Use given spread resource prefix for round-robin resource-based # scheduling. nodes = ray.nodes() map_resource_iter = _get_spread_resources_iter( nodes, _spread_resource_prefix, map_ray_remote_args ) reduce_resource_iter = _get_spread_resources_iter( nodes, _spread_resource_prefix, reduce_ray_remote_args ) else: # If no spread resource prefix given, yield an empty dictionary. map_resource_iter, reduce_resource_iter = itertools.tee(itertools.repeat({}), 2) shuffle_map = cached_remote_fn(_shuffle_map) shuffle_reduce = cached_remote_fn(_shuffle_reduce) map_bar = ProgressBar("Shuffle Map", position=0, total=input_num_blocks) shuffle_map_out = [ shuffle_map.options( **map_ray_remote_args, num_returns=1 + output_num_blocks, resources=next(map_resource_iter) ).remote(block, block_udf, i, output_num_blocks, random_shuffle, random_seed) for i, block in enumerate(input_blocks) ] # The first item returned is the BlockMetadata. shuffle_map_metadata = [] for i, refs in enumerate(shuffle_map_out): shuffle_map_metadata.append(refs[0]) shuffle_map_out[i] = refs[1:] # Eagerly delete the input block references in order to eagerly release # the blocks' memory. del input_blocks shuffle_map_metadata = map_bar.fetch_until_complete(shuffle_map_metadata) map_bar.close() # Randomize the reduce order of the blocks. if random_shuffle: random = np.random.RandomState(random_seed) random.shuffle(shuffle_map_out) reduce_bar = ProgressBar("Shuffle Reduce", position=0, total=output_num_blocks) shuffle_reduce_out = [ shuffle_reduce.options( **reduce_ray_remote_args, num_returns=2, resources=next(reduce_resource_iter) ).remote(*[shuffle_map_out[i][j] for i in range(input_num_blocks)]) for j in range(output_num_blocks) ] # Eagerly delete the map block references in order to eagerly release # the blocks' memory. del shuffle_map_out new_blocks, new_metadata = zip(*shuffle_reduce_out) reduce_bar.block_until_complete(list(new_blocks)) new_metadata = ray.get(list(new_metadata)) reduce_bar.close() stats = { "map": shuffle_map_metadata, "reduce": new_metadata, } return BlockList(list(new_blocks), list(new_metadata)), stats
def simple_shuffle(input_blocks: BlockList, output_num_blocks: int, *, random_shuffle: bool = False, random_seed: Optional[int] = None, map_ray_remote_args: Optional[Dict[str, Any]] = None, reduce_ray_remote_args: Optional[Dict[str, Any]] = None, _spread_resource_prefix: Optional[str] = None) -> BlockList: input_blocks = input_blocks.get_blocks() if map_ray_remote_args is None: map_ray_remote_args = {} if reduce_ray_remote_args is None: reduce_ray_remote_args = {} input_num_blocks = len(input_blocks) if _spread_resource_prefix is not None: # Use given spread resource prefix for round-robin resource-based # scheduling. nodes = ray.nodes() map_resource_iter = _get_spread_resources_iter( nodes, _spread_resource_prefix, map_ray_remote_args) reduce_resource_iter = _get_spread_resources_iter( nodes, _spread_resource_prefix, reduce_ray_remote_args) else: # If no spread resource prefix given, yield an empty dictionary. map_resource_iter, reduce_resource_iter = itertools.tee( itertools.repeat({}), 2) shuffle_map = cached_remote_fn(_shuffle_map) shuffle_reduce = cached_remote_fn(_shuffle_reduce) map_bar = ProgressBar("Shuffle Map", position=0, total=input_num_blocks) shuffle_map_out = [ shuffle_map.options(**map_ray_remote_args, num_returns=output_num_blocks, resources=next(map_resource_iter)).remote( block, i, output_num_blocks, random_shuffle, random_seed) for i, block in enumerate(input_blocks) ] # Eagerly delete the input block references in order to eagerly release # the blocks' memory. del input_blocks if output_num_blocks == 1: # Handle the num_returns=1 edge case which doesn't return a list. shuffle_map_out = [[x] for x in shuffle_map_out] map_bar.block_until_complete([x[0] for x in shuffle_map_out]) map_bar.close() # Randomize the reduce order of the blocks. if random_shuffle: random = np.random.RandomState(random_seed) random.shuffle(shuffle_map_out) reduce_bar = ProgressBar("Shuffle Reduce", position=0, total=output_num_blocks) shuffle_reduce_out = [ shuffle_reduce.options(**reduce_ray_remote_args, num_returns=2, resources=next(reduce_resource_iter)). remote(*[shuffle_map_out[i][j] for i in range(input_num_blocks)]) for j in range(output_num_blocks) ] # Eagerly delete the map block references in order to eagerly release # the blocks' memory. del shuffle_map_out new_blocks, new_metadata = zip(*shuffle_reduce_out) reduce_bar.block_until_complete(list(new_blocks)) new_metadata = ray.get(list(new_metadata)) reduce_bar.close() return BlockList(list(new_blocks), list(new_metadata))