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 _apply( self, fn: Any, remote_args: dict, block_list: BlockList, clear_input_blocks: bool, ) -> BlockList: context = DatasetContext.get_current() # Handle empty datasets. if block_list.initial_num_blocks() == 0: return block_list blocks = block_list.get_blocks_with_metadata() map_bar = ProgressBar("Map Progress", total=len(blocks)) if context.block_splitting_enabled: map_block = cached_remote_fn(_map_block_split).options(**remote_args) refs = [map_block.remote(b, fn, m.input_files) for b, m in blocks] else: map_block = cached_remote_fn(_map_block_nosplit).options( **dict(remote_args, num_returns=2) ) all_refs = [map_block.remote(b, fn, m.input_files) for b, m in blocks] data_refs = [r[0] for r in all_refs] refs = [r[1] for r in all_refs] # Release input block references. if clear_input_blocks: del blocks block_list.clear() # Common wait for non-data refs. try: results = map_bar.fetch_until_complete(refs) except (ray.exceptions.RayTaskError, KeyboardInterrupt) as e: # One or more mapper tasks failed, or we received a SIGINT signal # while waiting; either way, we cancel all map tasks. for ref in refs: ray.cancel(ref) # Wait until all tasks have failed or been cancelled. for ref in refs: try: ray.get(ref) except (ray.exceptions.RayTaskError, ray.exceptions.TaskCancelledError): pass # Reraise the original task failure exception. raise e from None new_blocks, new_metadata = [], [] if context.block_splitting_enabled: for result in results: for block, metadata in result: new_blocks.append(block) new_metadata.append(metadata) else: for block, metadata in zip(data_refs, results): new_blocks.append(block) new_metadata.append(metadata) return BlockList(list(new_blocks), list(new_metadata))
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 from_pandas_refs( dfs: Union[ObjectRef["pandas.DataFrame"], List[ObjectRef["pandas.DataFrame"]]] ) -> Dataset[ArrowRow]: """Create a dataset from a list of Ray object references to Pandas dataframes. Args: dfs: A Ray object references to pandas dataframe, or a list of Ray object references to pandas dataframes. Returns: Dataset holding Arrow records read from the dataframes. """ if isinstance(dfs, ray.ObjectRef): dfs = [dfs] elif isinstance(dfs, list): for df in dfs: if not isinstance(df, ray.ObjectRef): raise ValueError( "Expected list of Ray object refs, " f"got list containing {type(df)}" ) else: raise ValueError( "Expected Ray object ref or list of Ray object refs, " f"got {type(df)}" ) context = DatasetContext.get_current() if context.enable_pandas_block: get_metadata = cached_remote_fn(_get_metadata) metadata = [get_metadata.remote(df) for df in dfs] return Dataset( ExecutionPlan(BlockList(dfs, ray.get(metadata)), DatasetStats.TODO()), 0, False, ) df_to_block = cached_remote_fn(_df_to_block, num_returns=2) res = [df_to_block.remote(df) for df in dfs] blocks, metadata = zip(*res) return Dataset( ExecutionPlan( BlockList(blocks, ray.get(list(metadata))), DatasetStats(stages={"from_pandas_refs": metadata}, parent=None), ), 0, False, )
def _optimize_stages(self): """Optimize this pipeline, fusing stages together as possible.""" context = DatasetContext.get_current() if not context.optimize_fuse_stages: self._optimized_stages = self._stages return # This dummy dataset will be used to get a set of optimized stages. dummy_ds = Dataset( ExecutionPlan(BlockList([], []), DatasetStats(stages={}, parent=None)), 0, True, ) # Apply all pipeline operations to the dummy dataset. for stage in self._stages: dummy_ds = stage(dummy_ds) # Get the optimized stages. _, _, stages = dummy_ds._plan._optimize() # Apply these optimized stages to the datasets underlying the pipeline. # These optimized stages will be executed by the PipelineExecutor. optimized_stages = [] for stage in stages: optimized_stages.append(lambda ds, stage=stage: Dataset( ds._plan.with_stage(stage), ds._epoch, True)) self._optimized_stages = optimized_stages
def _test_equal_split_balanced(block_sizes, num_splits): blocks = [] metadata = [] total_rows = 0 for block_size in block_sizes: block = list(range(total_rows, total_rows + block_size)) blocks.append(ray.put(block)) metadata.append( BlockAccessor.for_block(block).get_metadata(None, None)) total_rows += block_size block_list = BlockList(blocks, metadata) ds = Dataset( ExecutionPlan(block_list, DatasetStats.TODO()), 0, False, ) splits = ds.split(num_splits, equal=True) split_counts = [split.count() for split in splits] assert len(split_counts) == num_splits expected_block_size = total_rows // num_splits # Check that all splits are the expected size. assert all([count == expected_block_size for count in split_counts]) expected_total_rows = sum(split_counts) # Check that the expected number of rows were dropped. assert total_rows - expected_total_rows == total_rows % num_splits # Check that all rows are unique (content check). split_rows = [row for split in splits for row in split.take(total_rows)] assert len(set(split_rows)) == len(split_rows)
def from_arrow_refs( tables: Union[ ObjectRef[Union["pyarrow.Table", bytes]], List[ObjectRef[Union["pyarrow.Table", bytes]]], ] ) -> Dataset[ArrowRow]: """Create a dataset from a set of Arrow tables. Args: tables: A Ray object reference to Arrow table, or list of Ray object references to Arrow tables, or its streaming format in bytes. Returns: Dataset holding Arrow records from the tables. """ if isinstance(tables, ray.ObjectRef): tables = [tables] get_metadata = cached_remote_fn(_get_metadata) metadata = [get_metadata.remote(t) for t in tables] return Dataset( ExecutionPlan( BlockList(tables, ray.get(metadata)), DatasetStats(stages={"from_arrow_refs": metadata}, parent=None), ), 0, False, )
def _rewrite_read_stage( in_blocks: LazyBlockList, ) -> Tuple[BlockList, DatasetStats, Stage]: """Rewrite the read stage to a OneToOne stage over read tasks as input. For example, suppose the plan was [Read -> MapBatches(Fn)]. These stages cannot be fused, since read stages are handled specially. After rewriting to [GetReadTasks -> MapBatches(DoRead) -> MapBatches(Fn)], now we can fuse the latter two MapBatches stages into a single OneToOne stage: [GetReadTasks -> MapBatches(DoRead -> Fn)]. Args: blocks: Lazy block list representing read stage. Returns: Non-lazy block list containing read tasks for not-yet-read block partitions, new stats for the block list, and the new one-to-one read stage. """ # Generate the "GetReadTasks" stage blocks. remote_args = in_blocks._remote_args blocks, metadata = [], [] for read_task in in_blocks._tasks: blocks.append(ray.put(read_task._read_fn)) metadata.append(read_task.get_metadata()) block_list = BlockList(blocks, metadata) def block_fn(read_fn: Callable[[], Iterator[Block]]) -> Iterator[Block]: for block in read_fn(): yield block stage = OneToOneStage("read", block_fn, "tasks", remote_args) stats = DatasetStats(stages={}, parent=None) return block_list, stats, stage
def from_vineyard(object_id): vineyard_to_block = cached_remote_fn(_vineyard_to_block, num_cpus=0.1, num_returns=2) get_vineyard_instance_id = cached_remote_fn(_get_vineyard_instance_id, num_cpus=0.1) get_remote_chunks_map = cached_remote_fn(_get_remote_chunks_map, num_cpus=0.1) chunks = ray.get(get_remote_chunks_map.remote(object_id)) with spread_to_all_nodes(get_vineyard_instance_id) as (nodes, pg): instances = dict() # instance_id -> placement group index for index in range(nodes): instance = ray.get( get_vineyard_instance_id.options( placement_group=pg, placement_group_bundle_index=index).remote()) instances[instance] = index blocks, metadatas = [], [] for object_id, location in chunks.items(): block, metadata = vineyard_to_block.options( placement_group=pg, placement_group_bundle_index=instances[location]).remote( vineyard.ObjectID(object_id)) blocks.append(block) metadatas.append(metadata) return Dataset(BlockList(blocks, ray.get(metadatas)))
def _rewrite_read_stage(self) -> Tuple[BlockList, "Stage"]: """Rewrite the read stage to a OneToOne stage over read tasks as input. For example, suppose the plan was [Read -> MapBatches(Fn)]. These stages cannot be fused, since read stages are handled specially. After rewriting to [GetReadTasks -> MapBatches(DoRead) -> MapBatches(Fn)], now we can fuse the latter two MapBatches stages into a single OneToOne stage: [GetReadTasks -> MapBatches(DoRead -> Fn)]. """ # Generate the "GetReadTasks" stage blocks. blocks = [] metadata = [] for i, read_task in enumerate(self._in_blocks._read_tasks): blocks.append(ray.put([read_task])) metadata.append(self._in_blocks._metadata[i]) block_list = BlockList(blocks, metadata) def block_fn(block: Block) -> Iterable[Block]: [read_task] = block for tmp1 in read_task._read_fn(): yield tmp1 # TODO(ekl): add num_cpus properly here and make the read default num_cpus=1. return block_list, OneToOneStage("read", block_fn, None, {})
def build(self, final_blocks: BlockList) -> "DatasetStats": stats = DatasetStats( stages={self.stage_name: final_blocks.get_metadata()}, parent=self.parent, ) stats.time_total_s = time.perf_counter() - self.start_time return stats
def from_items(items: List[Any], *, parallelism: int = 200) -> Dataset[Any]: """Create a dataset from a list of local Python objects. Examples: >>> ray.data.from_items([1, 2, 3, 4, 5]) Args: items: List of local Python objects. parallelism: The amount of parallelism to use for the dataset. Returns: Dataset holding the items. """ block_size = max(1, len(items) // parallelism) blocks: List[ObjectRef[Block]] = [] metadata: List[BlockMetadata] = [] i = 0 while i < len(items): builder = DelegatingArrowBlockBuilder() for item in items[i:i + block_size]: builder.add(item) block = builder.build() blocks.append(ray.put(block)) metadata.append( BlockAccessor.for_block(block).get_metadata(input_files=None)) i += block_size return Dataset(BlockList(blocks, metadata))
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 sort_impl(blocks: BlockList, key: SortKeyT, descending: bool = False) -> BlockList: blocks = list(blocks.iter_blocks()) if len(blocks) == 0: return BlockList([], []) 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) merge_sorted_blocks = cached_remote_fn(_merge_sorted_blocks, num_returns=2) map_results = np.empty((num_mappers, num_reducers), dtype=object) for i, block in enumerate(blocks): map_results[i, :] = sort_block.remote(block, boundaries, key, descending) map_bar = ProgressBar("Sort Map", len(map_results)) map_bar.block_until_complete([ret[0] for ret in map_results]) map_bar.close() reduce_results = [] for j in range(num_reducers): ret = merge_sorted_blocks.remote(key, descending, *map_results[:, j].tolist()) reduce_results.append(ret) 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]) return BlockList(blocks, metadata)
def from_arrow(tables: List[ObjectRef["pyarrow.Table"]]) -> Dataset[ArrowRow]: """Create a dataset from a set of Arrow tables. Args: dfs: A list of Ray object references to Arrow tables. Returns: Dataset holding Arrow records from the tables. """ get_metadata = cached_remote_fn(_get_metadata) metadata = [get_metadata.remote(t) for t in tables] return Dataset(BlockList(tables, ray.get(metadata)))
def apply(self, fn: Any, remote_args: dict, blocks: BlockList) -> BlockList: # Handle empty datasets. if blocks.initial_num_blocks() == 0: return blocks blocks = list(blocks.iter_blocks_with_metadata()) map_bar = ProgressBar("Map Progress", total=len(blocks)) map_block = cached_remote_fn(_map_block) refs = [ map_block.options(**remote_args).remote(b, fn, m.input_files) for b, m in blocks ] try: results = map_bar.fetch_until_complete(refs) except (ray.exceptions.RayTaskError, KeyboardInterrupt) as e: # One or more mapper tasks failed, or we received a SIGINT signal # while waiting; either way, we cancel all map tasks. for ref in refs: ray.cancel(ref) # Wait until all tasks have failed or been cancelled. for ref in refs: try: ray.get(ref) except (ray.exceptions.RayTaskError, ray.exceptions.TaskCancelledError): pass # Reraise the original task failure exception. raise e from None new_blocks, new_metadata = [], [] for result in results: for block, metadata in result: new_blocks.append(block) new_metadata.append(metadata) return BlockList(list(new_blocks), list(new_metadata))
def from_numpy(ndarrays: List[ObjectRef[np.ndarray]]) -> Dataset[np.ndarray]: """Create a dataset from a set of NumPy ndarrays. Args: ndarrays: A list of Ray object references to NumPy ndarrays. Returns: Dataset holding the given ndarrays. """ ndarray_to_block = cached_remote_fn(_ndarray_to_block, num_returns=2) res = [ndarray_to_block.remote(ndarray) for ndarray in ndarrays] blocks, metadata = zip(*res) return Dataset(BlockList(blocks, ray.get(list(metadata))))
def from_pandas(dfs: List[ObjectRef["pandas.DataFrame"]]) -> Dataset[ArrowRow]: """Create a dataset from a set of Pandas dataframes. Args: dfs: A list of Ray object references to pandas dataframes. Returns: Dataset holding Arrow records read from the dataframes. """ df_to_block = cached_remote_fn(_df_to_block, num_returns=2) res = [df_to_block.remote(df) for df in dfs] blocks, metadata = zip(*res) return Dataset(BlockList(blocks, ray.get(list(metadata))))
def apply(self, fn: Any, remote_args: dict, blocks: BlockList[Any]) -> BlockList[Any]: map_bar = ProgressBar("Map Progress", total=len(blocks)) kwargs = remote_args.copy() kwargs["num_returns"] = 2 map_block = cached_remote_fn(_map_block) refs = [ map_block.options(**kwargs).remote(b, m, fn) for b, m in zip(blocks, blocks.get_metadata()) ] new_blocks, new_metadata = zip(*refs) map_bar.block_until_complete(list(new_blocks)) new_metadata = ray.get(list(new_metadata)) return BlockList(list(new_blocks), list(new_metadata))
def from_items(items: List[Any], *, parallelism: int = 200) -> Dataset[Any]: """Create a dataset from a list of local Python objects. Examples: >>> import ray >>> ray.data.from_items([1, 2, 3, 4, 5]) # doctest: +SKIP Args: items: List of local Python objects. parallelism: The amount of parallelism to use for the dataset. Parallelism may be limited by the number of items. Returns: Dataset holding the items. """ block_size = max(1, len(items) // parallelism) blocks: List[ObjectRef[Block]] = [] metadata: List[BlockMetadata] = [] i = 0 while i < len(items): stats = BlockExecStats.builder() builder = DelegatingBlockBuilder() for item in items[i : i + block_size]: builder.add(item) block = builder.build() blocks.append(ray.put(block)) metadata.append( BlockAccessor.for_block(block).get_metadata( input_files=None, exec_stats=stats.build() ) ) i += block_size return Dataset( ExecutionPlan( BlockList(blocks, metadata), DatasetStats(stages={"from_items": metadata}, parent=None), ), 0, False, )
def simple_shuffle(input_blocks: BlockList[T], output_num_blocks: int, *, random_shuffle: bool = False, random_seed: Optional[int] = None) -> BlockList[T]: input_num_blocks = len(input_blocks) shuffle_map = cached_remote_fn(_shuffle_map).options( num_returns=output_num_blocks) shuffle_reduce = cached_remote_fn(_shuffle_reduce, num_returns=2) map_bar = ProgressBar("Shuffle Map", position=0, total=input_num_blocks) shuffle_map_out = [ shuffle_map.remote(block, i, output_num_blocks, random_shuffle, random_seed) for i, block in enumerate(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.remote( *[shuffle_map_out[i][j] for i in range(input_num_blocks)]) for j in range(output_num_blocks) ] 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))
def from_numpy(ndarrays: List[ObjectRef[np.ndarray]]) -> Dataset[ArrowRow]: """Create a dataset from a set of NumPy ndarrays. Args: ndarrays: A list of Ray object references to NumPy ndarrays. Returns: Dataset holding the given ndarrays. """ ndarray_to_block = cached_remote_fn(_ndarray_to_block, num_returns=2) res = [ndarray_to_block.remote(ndarray) for ndarray in ndarrays] blocks, metadata = zip(*res) return Dataset( ExecutionPlan( BlockList(blocks, ray.get(list(metadata))), DatasetStats(stages={"from_numpy": metadata}, parent=None), ), 0, False, )
def _optimize_stages(self): """Optimize this pipeline, fusing stages together as possible.""" context = DatasetContext.get_current() if not context.optimize_fuse_stages: self._optimized_stages = self._stages return dummy_ds = Dataset( ExecutionPlan(BlockList([], []), DatasetStats(stages={}, parent=None)), 0, True, ) for stage in self._stages: dummy_ds = stage(dummy_ds) dummy_ds._plan._optimize() optimized_stages = [] for stage in dummy_ds._plan._stages: optimized_stages.append(lambda ds, stage=stage: Dataset( ds._plan.with_stage(stage), ds._epoch, True)) self._optimized_stages = optimized_stages
def from_numpy_refs( ndarrays: Union[ObjectRef[np.ndarray], List[ObjectRef[np.ndarray]]], ) -> Dataset[ArrowRow]: """Create a dataset from a list of NumPy ndarray futures. Args: ndarrays: A Ray object reference to a NumPy ndarray or a list of Ray object references to NumPy ndarrays. Returns: Dataset holding the given ndarrays. """ if isinstance(ndarrays, ray.ObjectRef): ndarrays = [ndarrays] elif isinstance(ndarrays, list): for ndarray in ndarrays: if not isinstance(ndarray, ray.ObjectRef): raise ValueError( "Expected list of Ray object refs, " f"got list containing {type(ndarray)}" ) else: raise ValueError( f"Expected Ray object ref or list of Ray object refs, got {type(ndarray)}" ) ndarray_to_block = cached_remote_fn(_ndarray_to_block, num_returns=2) res = [ndarray_to_block.remote(ndarray) for ndarray in ndarrays] blocks, metadata = zip(*res) return Dataset( ExecutionPlan( BlockList(blocks, ray.get(list(metadata))), DatasetStats(stages={"from_numpy": metadata}, parent=None), ), 0, False, )
def from_pandas_refs( dfs: Union[ObjectRef["pandas.DataFrame"], List[ObjectRef["pandas.DataFrame"]]] ) -> Dataset[ArrowRow]: """Create a dataset from a list of Ray object references to Pandas dataframes. Args: dfs: A Ray object references to pandas dataframe, or a list of Ray object references to pandas dataframes. Returns: Dataset holding Arrow records read from the dataframes. """ if isinstance(dfs, ray.ObjectRef): dfs = [dfs] df_to_block = cached_remote_fn(_df_to_block, num_returns=2) res = [df_to_block.remote(df) for df in dfs] blocks, metadata = zip(*res) return Dataset(BlockList(blocks, ray.get(list(metadata))), 0, DatasetStats.TODO())
def apply(self, fn: Any, remote_args: dict, blocks: BlockList[Any]) -> BlockList[Any]: # Handle empty datasets. if len(blocks) == 0: return blocks map_bar = ProgressBar("Map Progress", total=len(blocks)) kwargs = remote_args.copy() kwargs["num_returns"] = 2 map_block = cached_remote_fn(_map_block) refs = [ map_block.options(**kwargs).remote(b, m, fn) for b, m in zip(blocks, blocks.get_metadata()) ] new_blocks, new_metadata = zip(*refs) new_metadata = list(new_metadata) try: new_metadata = map_bar.fetch_until_complete(new_metadata) except (ray.exceptions.RayTaskError, KeyboardInterrupt) as e: # One or more mapper tasks failed, or we received a SIGINT signal # while waiting; either way, we cancel all map tasks. for ref in new_metadata: ray.cancel(ref) # Wait until all tasks have failed or been cancelled. for ref in new_metadata: try: ray.get(ref) except (ray.exceptions.RayTaskError, ray.exceptions.TaskCancelledError): pass # Reraise the original task failure exception. raise e from None return BlockList(list(new_blocks), list(new_metadata))
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 _apply( self, fn: Any, remote_args: dict, block_list: BlockList, clear_input_blocks: bool, name: Optional[str] = None, ) -> BlockList: """Note: this is not part of the Dataset public API.""" context = DatasetContext.get_current() blocks_in = block_list.get_blocks_with_metadata() # Early release block references. if clear_input_blocks: block_list.clear() orig_num_blocks = len(blocks_in) results = [] if name is None: name = "map" name = name.title() map_bar = ProgressBar(name, total=orig_num_blocks) class BlockWorker: def ready(self): return "ok" def map_block_split(self, block: Block, input_files: List[str]) -> BlockPartition: return _map_block_split(block, fn, input_files) @ray.method(num_returns=2) def map_block_nosplit( self, block: Block, input_files: List[str]) -> Tuple[Block, BlockMetadata]: return _map_block_nosplit(block, fn, input_files) if not remote_args: remote_args["num_cpus"] = 1 BlockWorker = ray.remote(**remote_args)(BlockWorker) workers = [BlockWorker.remote() for _ in range(self.min_size)] tasks = {w.ready.remote(): w for w in workers} tasks_in_flight = collections.defaultdict(int) metadata_mapping = {} block_indices = {} ready_workers = set() while len(results) < orig_num_blocks: ready, _ = ray.wait(list(tasks.keys()), timeout=0.01, num_returns=1, fetch_local=False) if not ready: if (len(workers) < self.max_size and len(ready_workers) / len(workers) > 0.8): w = BlockWorker.remote() workers.append(w) tasks[w.ready.remote()] = w map_bar.set_description( "Map Progress ({} actors {} pending)".format( len(ready_workers), len(workers) - len(ready_workers))) continue [obj_id] = ready worker = tasks.pop(obj_id) # Process task result. if worker in ready_workers: results.append(obj_id) tasks_in_flight[worker] -= 1 map_bar.update(1) else: ready_workers.add(worker) map_bar.set_description( "Map Progress ({} actors {} pending)".format( len(ready_workers), len(workers) - len(ready_workers))) # Schedule a new task. while (blocks_in and tasks_in_flight[worker] < self.max_tasks_in_flight_per_actor): block, meta = blocks_in.pop() if context.block_splitting_enabled: ref = worker.map_block_split.remote( block, meta.input_files) else: ref, meta_ref = worker.map_block_nosplit.remote( block, meta.input_files) metadata_mapping[ref] = meta_ref tasks[ref] = worker block_indices[ref] = len(blocks_in) tasks_in_flight[worker] += 1 map_bar.close() new_blocks, new_metadata = [], [] # Put blocks in input order. results.sort(key=block_indices.get) if context.block_splitting_enabled: for result in ray.get(results): for block, metadata in result: new_blocks.append(block) new_metadata.append(metadata) else: for block in results: new_blocks.append(block) new_metadata.append(metadata_mapping[block]) new_metadata = ray.get(new_metadata) return BlockList(new_blocks, new_metadata)
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