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.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, ) 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 aggregate(self, *aggs: AggregateFn) -> Dataset[U]: """Implements an 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 an Arrow dataset then the output is an Arrow dataset of ``n + 1`` columns where the 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. """ stats = self._dataset._stats.child_builder("aggregate") 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 self._dataset.num_blocks() == 0: return self._dataset blocks = self._dataset._blocks.get_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 + 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): 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 Dataset(BlockList(blocks, metadata), self._dataset._epoch, stats.build_multistage(stage_info))
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, ) -> 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) 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, ).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, ).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 = list(input_blocks.iter_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))
def apply(self, fn: Any, remote_args: dict, blocks: Iterable[Block]) -> Iterable[ObjectRef[Block]]: map_bar = ProgressBar("Map Progress", total=len(blocks)) class BlockWorker: def ready(self): return "ok" @ray.method(num_returns=2) def process_block(self, block: Block, meta: BlockMetadata) -> (Block, BlockMetadata): new_block = fn(block) accessor = BlockAccessor.for_block(new_block) new_metadata = BlockMetadata(num_rows=accessor.num_rows(), size_bytes=accessor.size_bytes(), schema=accessor.schema(), input_files=meta.input_files) return new_block, new_metadata if not remote_args: remote_args["num_cpus"] = 1 BlockWorker = ray.remote(**remote_args)(BlockWorker) self.workers = [BlockWorker.remote()] metadata_mapping = {} tasks = {w.ready.remote(): w for w in self.workers} ready_workers = set() blocks_in = [(b, m) for (b, m) in zip(blocks, blocks.get_metadata())] blocks_out = [] while len(blocks_out) < len(blocks): ready, _ = ray.wait(list(tasks), timeout=0.01, num_returns=1, fetch_local=False) if not ready: if len(ready_workers) / len(self.workers) > 0.8: w = BlockWorker.remote() self.workers.append(w) tasks[w.ready.remote()] = w map_bar.set_description( "Map Progress ({} actors {} pending)".format( len(ready_workers), len(self.workers) - len(ready_workers))) continue [obj_id] = ready worker = tasks[obj_id] del tasks[obj_id] # Process task result. if worker in ready_workers: blocks_out.append(obj_id) map_bar.update(1) else: ready_workers.add(worker) # Schedule a new task. if blocks_in: block_ref, meta_ref = worker.process_block.remote( *blocks_in.pop()) metadata_mapping[block_ref] = meta_ref tasks[block_ref] = worker new_metadata = ray.get([metadata_mapping[b] for b in blocks_out]) map_bar.close() return BlockList(blocks_out, new_metadata)
def _apply( self, fn: Any, remote_args: dict, block_list: BlockList, clear_input_blocks: bool, ) -> 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 = [] map_bar = ProgressBar("Map Progress", 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} metadata_mapping = {} ready_workers = set() while len(results) < orig_num_blocks: ready, _ = ray.wait( list(tasks), 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[obj_id] del tasks[obj_id] # Process task result. if worker in ready_workers: results.append(obj_id) 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. if blocks_in: 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 map_bar.close() new_blocks, new_metadata = [], [] 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["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 fast_repartition(blocks, num_blocks): from ray.data.dataset import Dataset wrapped_ds = Dataset(ExecutionPlan(blocks, DatasetStats(stages={}, parent=None)), 0, lazy=False) # Compute the (n-1) indices needed for an equal split of the data. count = wrapped_ds.count() dataset_format = wrapped_ds._dataset_format() indices = [] cur_idx = 0 for _ in range(num_blocks - 1): cur_idx += count / num_blocks indices.append(int(cur_idx)) assert len(indices) < num_blocks, (indices, num_blocks) if indices: splits = wrapped_ds.split_at_indices(indices) else: splits = [wrapped_ds] # TODO(ekl) include stats for the split tasks. We may also want to # consider combining the split and coalesce tasks as an optimization. # Coalesce each split into a single block. reduce_task = cached_remote_fn( ShufflePartitionOp.reduce).options(num_returns=2) reduce_bar = ProgressBar("Repartition", position=0, total=len(splits)) reduce_out = [ reduce_task.remote(False, None, *s.get_internal_block_refs()) for s in splits if s.num_blocks() > 0 ] # Early-release memory. del splits, blocks, wrapped_ds new_blocks, new_metadata = zip(*reduce_out) new_blocks, new_metadata = list(new_blocks), list(new_metadata) new_metadata = reduce_bar.fetch_until_complete(new_metadata) reduce_bar.close() # Handle empty blocks. if len(new_blocks) < num_blocks: from ray.data.impl.arrow_block import ArrowBlockBuilder from ray.data.impl.pandas_block import PandasBlockBuilder from ray.data.impl.simple_block import SimpleBlockBuilder num_empties = num_blocks - len(new_blocks) if dataset_format == "arrow": builder = ArrowBlockBuilder() elif dataset_format == "pandas": builder = PandasBlockBuilder() else: builder = SimpleBlockBuilder() empty_block = builder.build() empty_meta = BlockAccessor.for_block(empty_block).get_metadata( input_files=None, exec_stats=None) # No stats for empty block. empty_blocks, empty_metadata = zip(*[(ray.put(empty_block), empty_meta) for _ in range(num_empties)]) new_blocks += empty_blocks new_metadata += empty_metadata return BlockList(new_blocks, new_metadata), {}
def run_actor_workload(total_num_cpus, smoke): """Run actor-based workload. The test checks if actor restart -1 and task_retries -1 works as expected. It basically requires many actors to report the seqno to the centralized DB actor while there are failures. If at least once is guaranteed upon failures, this test shouldn't fail. """ @ray.remote(num_cpus=0) class DBActor: def __init__(self): self.letter_dict = set() def add(self, letter): self.letter_dict.add(letter) def get(self): return self.letter_dict @ray.remote(num_cpus=1, max_restarts=-1, max_task_retries=-1) class ReportActor: def __init__(self, db_actor): self.db_actor = db_actor def add(self, letter): ray.get(self.db_actor.add.remote(letter)) NUM_CPUS = int(total_num_cpus) multiplier = 2 # For smoke mode, run less number of tasks if smoke: multiplier = 1 TOTAL_TASKS = int(300 * multiplier) current_node_ip = ray.worker.global_worker.node_ip_address db_actors = [ DBActor.options(resources={f"node:{current_node_ip}": 0.001}).remote() for _ in range(NUM_CPUS) ] pb = ProgressBar("Chaos test", TOTAL_TASKS * NUM_CPUS) actors = [] for db_actor in db_actors: actors.append(ReportActor.remote(db_actor)) results = [] highest_reported_num = 0 for a in actors: for _ in range(TOTAL_TASKS): results.append(a.add.remote(str(highest_reported_num))) highest_reported_num += 1 pb.fetch_until_complete(results) pb.close() for actor in actors: ray.kill(actor) # Consistency check wait_for_condition( lambda: ( ray.cluster_resources().get("CPU", 0) == ray.available_resources().get("CPU", 0) ), timeout=60, ) letter_set = set() for db_actor in db_actors: letter_set.update(ray.get(db_actor.get.remote())) # Make sure the DB actor didn't lose any report. # If this assert fails, that means at least once actor task semantic # wasn't guaranteed. for i in range(highest_reported_num): assert str(i) in letter_set, i
def simple_shuffle( input_blocks: BlockList[T], 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) -> BlockList[T]: # Check for spread resource labels in environment variable, and use # the given labels for round-robin resource-based scheduling. shuffle_spread_custom_resource_labels = os.getenv( "RAY_DATASETS_SHUFFLE_SPREAD_CUSTOM_RESOURCE_LABELS", None) if shuffle_spread_custom_resource_labels is not None: shuffle_spread_custom_resource_labels = ( shuffle_spread_custom_resource_labels.split(",")) round_robin_resource_provider = itertools.cycle( map(lambda resource: {resource: 0.001}, shuffle_spread_custom_resource_labels)) else: # If no round-robin resource provider given, yield an empty # dictionary. round_robin_resource_provider = itertools.repeat({}) # Create separate resource iterators for the map and reduce stages. map_resource_iter, reduce_resource_iter = itertools.tee( round_robin_resource_provider, 2) 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) 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))
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