def main(): """The test simulates the workload with many threaded actors. Test is doing 4 things for 1 hour. - It first creates actors as many as num_cpus with max_concurrency=10 - Each actor computes pi and put the result to the queue. - Driver keeps getting result & metadata from the actor. - Every X seconds, it kills all actors and restarts them. """ ray.init(address="auto") args, unknown = parse_script_args() num_cpus = ray.cluster_resources()["CPU"] num_nodes = sum(1 for n in ray.nodes() if n["Alive"]) print(f"Total number of actors: {num_cpus}, nodes: {num_nodes}") monitor_actor = monitor_memory_usage() start = time.time() while time.time() - start < args.test_runtime: # Step 1: Create actors and start computation loop. print("Create actors.") actors = start_actors(num_cpus, num_nodes) # Step 2: Get the pi result from actors. compute_start = time.time() print("Start computation.") while time.time() - compute_start < args.kill_interval_s: # Get the metadata. ray.get([actor.get_metadata.remote() for actor in actors]) # Get the result. pb = ProgressBar("Computing Pi", num_cpus) results = [actor.get_pi.remote() for actor in actors] pb.fetch_until_complete(results) pb.close() # Step 3: Kill actors. print("Kill all actors.") for actor in actors: ray.kill(actor) # Report the result. print("PASSED.") used_gb, usage = ray.get(monitor_actor.get_peak_memory_info.remote()) print("Memory usage with failures.") print(f"Peak memory usage: {round(used_gb, 2)}GB") print(f"Peak memory usage per processes:\n {usage}") # Report the result. ray.get(monitor_actor.stop_run.remote()) result = {"success": 0} with open(os.environ["TEST_OUTPUT_JSON"], "w") as f: f.write(json.dumps(result))
def test_chaos_actor_retry(set_kill_interval): # Chaos testing. @ray.remote(num_cpus=0.25, max_restarts=-1, max_task_retries=-1) class Actor: def __init__(self): self.letter_dict = set() def add(self, letter): self.letter_dict.add(letter) NUM_CPUS = 16 TOTAL_TASKS = 300 pb = ProgressBar("Chaos test sanity check", TOTAL_TASKS * NUM_CPUS) actors = [Actor.remote() for _ in range(NUM_CPUS)] results = [] for a in actors: results.extend([a.add.remote(str(i)) for i in range(TOTAL_TASKS)]) start = time.time() pb.fetch_until_complete(results) runtime_with_failure = time.time() - start print(f"Runtime when there are many failures: {runtime_with_failure}") pb.close()
def _fetch_metadata_remotely( pieces: List["pyarrow._dataset.ParquetFileFragment"], ) -> List[ObjectRef["pyarrow.parquet.FileMetaData"]]: remote_fetch_metadata = cached_remote_fn( _fetch_metadata_serialization_wrapper) metas = [] parallelism = min(len(pieces) // PIECES_PER_META_FETCH, 100) meta_fetch_bar = ProgressBar("Metadata Fetch Progress", total=parallelism) for pcs in np.array_split(pieces, parallelism): if len(pcs) == 0: continue metas.append( remote_fetch_metadata.remote([_SerializedPiece(p) for p in pcs])) metas = meta_fetch_bar.fetch_until_complete(metas) return list(itertools.chain.from_iterable(metas))
def _execute_reduce_stage( self, output_num_blocks: int, schedule: _PushBasedShuffleTaskSchedule, reduce_ray_remote_args: Dict[str, Any], all_merge_results: List[List[ObjectRef]], ): shuffle_reduce = cached_remote_fn(self.reduce) # Execute the final reduce stage. shuffle_reduce_out = [] for reducer_idx in range(output_num_blocks): merge_idx = schedule.get_merge_idx_for_reducer_idx(reducer_idx) # Submit one partition of reduce tasks, one for each of the P # outputs produced by the corresponding merge task. # We also add the merge task arguments so that the reduce task # is colocated with its inputs. shuffle_reduce_out.append( shuffle_reduce.options( **reduce_ray_remote_args, **schedule.get_merge_task_options(merge_idx), num_returns=2, ).remote( *self._reduce_args, *[ merge_results.pop(0) for merge_results in all_merge_results[merge_idx] ], ) ) for merge_idx, merge_results in enumerate(all_merge_results): assert all(len(merge_result) == 0 for merge_result in merge_results), ( "Reduce stage did not process outputs from merge tasks at index: " f"{merge_idx}" ) assert ( len(shuffle_reduce_out) == output_num_blocks ), f"Expected {output_num_blocks} outputs, produced {len(shuffle_reduce_out)}" reduce_bar = ProgressBar("Shuffle Reduce", total=output_num_blocks) reduce_blocks, reduce_metadata = zip(*shuffle_reduce_out) reduce_metadata = reduce_bar.fetch_until_complete(list(reduce_metadata)) reduce_bar.close() return reduce_metadata, reduce_blocks
def _get_blocks_with_metadata( self, ) -> Tuple[List[ObjectRef[Block]], List[BlockMetadata]]: """Get all underlying block futures and concrete metadata. This will block on the completion of the underlying read tasks and will fetch all block metadata outputted by those tasks. """ context = DatasetContext.get_current() block_refs, meta_refs = [], [] for block_ref, meta_ref in self._iter_block_partition_refs(): block_refs.append(block_ref) meta_refs.append(meta_ref) if context.block_splitting_enabled: # If block splitting is enabled, fetch the partitions. parts = ray.get(block_refs) block_refs, metadata = [], [] for part in parts: for block_ref, meta in part: block_refs.append(block_ref) metadata.append(meta) self._cached_metadata = metadata return block_refs, metadata if all(meta is not None for meta in self._cached_metadata): # Short-circuit on cached metadata. return block_refs, self._cached_metadata if not meta_refs: # Short-circuit on empty set of block partitions. assert not block_refs, block_refs return [], [] read_progress_bar = ProgressBar("Read progress", total=len(meta_refs)) # Fetch the metadata in bulk. # Handle duplicates (e.g. due to unioning the same dataset). unique_meta_refs = set(meta_refs) metadata = read_progress_bar.fetch_until_complete( list(unique_meta_refs)) ref_to_data = { meta_ref: data for meta_ref, data in zip(unique_meta_refs, metadata) } metadata = [ref_to_data[meta_ref] for meta_ref in meta_refs] self._cached_metadata = metadata return block_refs, metadata
def _fetch_metadata_remotely( pieces: List["pyarrow._dataset.ParquetFileFragment"], ) -> List[ObjectRef["pyarrow.parquet.FileMetaData"]]: from ray import cloudpickle remote_fetch_metadata = cached_remote_fn( _fetch_metadata_serialization_wrapper) metas = [] parallelism = min(len(pieces) // PIECES_PER_META_FETCH, 100) meta_fetch_bar = ProgressBar("Metadata Fetch Progress", total=parallelism) try: _register_parquet_file_fragment_serialization() for pcs in np.array_split(pieces, parallelism): if len(pcs) == 0: continue metas.append(remote_fetch_metadata.remote(cloudpickle.dumps(pcs))) finally: _deregister_parquet_file_fragment_serialization() metas = meta_fetch_bar.fetch_until_complete(metas) return list(itertools.chain.from_iterable(metas))
def sample_boundaries(blocks: List[ObjectRef[Block]], key: SortKeyT, num_reducers: int) -> List[T]: """ Return (num_reducers - 1) items in ascending order from the blocks that partition the domain into ranges with approximately equally many elements. """ # TODO(Clark): Support multiple boundary sampling keys. if isinstance(key, list) and len(key) > 1: raise ValueError("Multiple boundary sampling keys not supported.") n_samples = int(num_reducers * 10 / len(blocks)) sample_block = cached_remote_fn(_sample_block) sample_results = [ sample_block.remote(block, n_samples, key) for block in blocks ] sample_bar = ProgressBar("Sort Sample", len(sample_results)) samples = sample_bar.fetch_until_complete(sample_results) sample_bar.close() del sample_results samples = [s for s in samples if len(s) > 0] # The dataset is empty if len(samples) == 0: return [None] * (num_reducers - 1) builder = DelegatingBlockBuilder() for sample in samples: builder.add_block(sample) samples = builder.build() column = key[0][0] if isinstance(key, list) else None sample_items = BlockAccessor.for_block(samples).to_numpy(column) sample_items = np.sort(sample_items) ret = [ np.quantile(sample_items, q, interpolation="nearest") for q in np.linspace(0, 1, num_reducers) ] return ret[1:]
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: context = DatasetContext.get_current() # Handle empty datasets. if block_list.initial_num_blocks() == 0: return block_list blocks = block_list.get_blocks_with_metadata() if name is None: name = "map" name = name.title() map_bar = ProgressBar(name, 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 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._internal.arrow_block import ArrowBlockBuilder from ray.data._internal.pandas_block import PandasBlockBuilder from ray.data._internal.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