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 remote_args["scheduling_strategy"] = context.scheduling_strategy 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 _apply( self, block_fn: BlockTransform, remote_args: dict, block_list: BlockList, clear_input_blocks: bool, name: Optional[str] = None, fn: Optional[UDF] = None, fn_args: Optional[Iterable[Any]] = None, fn_kwargs: Optional[Dict[str, Any]] = None, fn_constructor_args: Optional[Iterable[Any]] = None, fn_constructor_kwargs: Optional[Dict[str, Any]] = None, ) -> BlockList: """Note: this is not part of the Dataset public API.""" if fn_args is None: fn_args = tuple() if fn_kwargs is None: fn_kwargs = {} if fn_constructor_args is None: fn_constructor_args = tuple() if fn_constructor_kwargs is None: fn_constructor_kwargs = {} 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 __init__( self, *fn_constructor_args: Any, **fn_constructor_kwargs: Any, ): if not isinstance(fn, CallableClass): if fn_constructor_args or fn_constructor_kwargs: raise ValueError( "fn_constructor_{kw}args only valid for CallableClass " f"UDFs, but got: {fn}" ) self.fn = fn else: self.fn = fn(*fn_constructor_args, **fn_constructor_kwargs) def ready(self): return "ok" def map_block_split( self, block: Block, input_files: List[str], *fn_args, **fn_kwargs, ) -> BlockPartition: return _map_block_split( block, block_fn, input_files, self.fn, *fn_args, **fn_kwargs ) @ray.method(num_returns=2) def map_block_nosplit( self, block: Block, input_files: List[str], *fn_args, **fn_kwargs, ) -> Tuple[Block, BlockMetadata]: return _map_block_nosplit( block, block_fn, input_files, self.fn, *fn_args, **fn_kwargs ) if "num_cpus" not in remote_args: remote_args["num_cpus"] = 1 if "scheduling_strategy" not in remote_args: ctx = DatasetContext.get_current() if ctx.scheduling_strategy == DEFAULT_SCHEDULING_STRATEGY: remote_args["scheduling_strategy"] = "SPREAD" else: remote_args["scheduling_strategy"] = ctx.scheduling_strategy BlockWorker = ray.remote(**remote_args)(BlockWorker) workers = [ BlockWorker.remote(*fn_constructor_args, **fn_constructor_kwargs) 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() try: 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) > self.ready_to_total_workers_ratio ): w = BlockWorker.remote( *fn_constructor_args, **fn_constructor_kwargs ) 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, *fn_args, **fn_kwargs, ) else: ref, meta_ref = worker.map_block_nosplit.remote( block, meta.input_files, *fn_args, **fn_kwargs, ) metadata_mapping[ref] = meta_ref tasks[ref] = worker block_indices[ref] = len(blocks_in) tasks_in_flight[worker] += 1 map_bar.close() self.num_workers += len(workers) 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) except Exception as e: try: for worker in workers: ray.kill(worker) except Exception as err: logger.exception(f"Error killing workers: {err}") finally: raise e