Пример #1
0
def _get_read_tasks(
    ds: Datasource,
    ctx: DatasetContext,
    cur_pg: Optional[PlacementGroup],
    parallelism: int,
    kwargs: dict,
) -> (int, int, List[ReadTask]):
    """Generates read tasks.

    Args:
        ds: Datasource to read from.
        ctx: Dataset config to use.
        cur_pg: The current placement group, if any.
        parallelism: The user-requested parallelism, or -1 for autodetection.
        kwargs: Additional kwargs to pass to the reader.

    Returns:
        Request parallelism from the datasource, the min safe parallelism to avoid
        OOM, and the list of read tasks generated.
    """
    kwargs = _unwrap_arrow_serialization_workaround(kwargs)
    DatasetContext._set_current(ctx)
    reader = ds.create_reader(**kwargs)
    requested_parallelism, min_safe_parallelism = _autodetect_parallelism(
        parallelism, cur_pg, DatasetContext.get_current(), reader)
    return (
        requested_parallelism,
        min_safe_parallelism,
        reader.get_read_tasks(requested_parallelism),
    )
Пример #2
0
def pipeline_stage(fn: Callable[[], Dataset[T]],
                   context: DatasetContext) -> Dataset[T]:
    DatasetContext._set_current(context)
    try:
        prev = set_progress_bars(False)
        return fn()
    finally:
        set_progress_bars(prev)
Пример #3
0
 def __init__(self, pipeline: "DatasetPipeline[T]", n: int,
              splitter: Callable[[Dataset], "DatasetPipeline[T]"],
              context: DatasetContext):
     DatasetContext._set_current(context)
     self.executor = PipelineExecutor(pipeline)
     self.n = n
     self.splitter = splitter
     self.cur_splits = [None] * self.n
Пример #4
0
 def run(self, fn: Callable[[], Dataset[T]],
         context: DatasetContext) -> Dataset[T]:
     DatasetContext._set_current(context)
     try:
         prev = set_progress_bars(False)
         # Force eager evaluation of all blocks in the pipeline stage. This
         # prevents resource deadlocks due to overlapping stage execution
         # (e.g., task -> actor stage).
         return fn().force_reads()
     finally:
         set_progress_bars(prev)
Пример #5
0
def test_map_pipeline(ray_start_regular_shared):
    context = DatasetContext.get_current()
    context.foo = 8
    pipe = ray.data.range(2).repeat(2)
    pipe = pipe.map(lambda x: DatasetContext.get_current().foo)
    [a, b] = pipe.split(2)

    @ray.remote
    def fetch(shard):
        return shard.take_all()

    assert ray.get([fetch.remote(a), fetch.remote(b)]) == [[8, 8], [8, 8]]
Пример #6
0
 def __init__(
     self,
     pipeline: "DatasetPipeline[T]",
     n: int,
     splitter: Callable[[Dataset], List["Dataset[T]"]],
     context: DatasetContext,
 ):
     DatasetContext._set_current(context)
     pipeline._optimize_stages()
     self.executor = PipelineExecutor(pipeline)
     self.n = n
     self.splitter = splitter
     self.cur_splits = [None] * self.n
Пример #7
0
    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
Пример #8
0
def test_dataset_stats_shuffle(ray_start_regular_shared):
    context = DatasetContext.get_current()
    context.optimize_fuse_stages = True
    ds = ray.data.range(1000, parallelism=10)
    ds = ds.random_shuffle().repartition(1, shuffle=True)
    stats = canonicalize(ds.stats())
    assert (
        stats == """Stage N read->random_shuffle_map: N/N blocks executed in T
* Remote wall time: T min, T max, T mean, T total
* Remote cpu time: T min, T max, T mean, T total
* Output num rows: N min, N max, N mean, N total
* Output size bytes: N min, N max, N mean, N total
* Tasks per node: N min, N max, N mean; N nodes used

Stage N random_shuffle_reduce: N/N blocks executed in T
* Remote wall time: T min, T max, T mean, T total
* Remote cpu time: T min, T max, T mean, T total
* Output num rows: N min, N max, N mean, N total
* Output size bytes: N min, N max, N mean, N total
* Tasks per node: N min, N max, N mean; N nodes used

Stage N repartition_map: N/N blocks executed in T
* Remote wall time: T min, T max, T mean, T total
* Remote cpu time: T min, T max, T mean, T total
* Output num rows: N min, N max, N mean, N total
* Output size bytes: N min, N max, N mean, N total
* Tasks per node: N min, N max, N mean; N nodes used

Stage N repartition_reduce: N/N blocks executed in T
* Remote wall time: T min, T max, T mean, T total
* Remote cpu time: T min, T max, T mean, T total
* Output num rows: N min, N max, N mean, N total
* Output size bytes: N min, N max, N mean, N total
* Tasks per node: N min, N max, N mean; N nodes used
""")
Пример #9
0
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()

    context = DatasetContext.get_current()
    if context.use_push_based_shuffle:
        sort_op_cls = PushBasedSortOp
    else:
        sort_op_cls = SimpleSortOp
    sort_op = sort_op_cls(
        map_args=[boundaries, key, descending], reduce_args=[key, descending]
    )
    return sort_op.execute(
        blocks,
        num_reducers,
        clear_input_blocks,
    )
Пример #10
0
    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))
Пример #11
0
Файл: plan.py Проект: vakker/ray
 def _optimize(self) -> None:
     """Apply stage fusion optimizations, updating this plan."""
     context = DatasetContext.get_current()
     if context.optimize_fuse_stages:
         if context.optimize_fuse_read_stages:
             self._rewrite_read_stages()
         self._fuse_one_to_one_stages()
 def prepare_read(self, parallelism: int):
     value = DatasetContext.get_current().foo
     meta = BlockMetadata(num_rows=1,
                          size_bytes=8,
                          schema=None,
                          input_files=None)
     return [ReadTask(lambda: [[value]], meta)]
Пример #13
0
def test_window(ray_start_regular_shared):
    context = DatasetContext.get_current()
    context.optimize_fuse_stages = True
    ds = ray.data.range(10)
    pipe = ds.window(blocks_per_window=1)
    assert str(pipe) == "DatasetPipeline(num_windows=10, num_stages=2)"
    pipe = pipe.rewindow(blocks_per_window=3)
    assert str(pipe) == "DatasetPipeline(num_windows=None, num_stages=1)"
    datasets = list(pipe.iter_datasets())
    assert len(datasets) == 4
    assert datasets[0].take() == [0, 1, 2]
    assert datasets[1].take() == [3, 4, 5]
    assert datasets[2].take() == [6, 7, 8]
    assert datasets[3].take() == [9]

    ds = ray.data.range(10)
    pipe = ds.window(blocks_per_window=5)
    assert str(pipe) == "DatasetPipeline(num_windows=2, num_stages=2)"
    pipe = pipe.rewindow(blocks_per_window=3)
    assert str(pipe) == "DatasetPipeline(num_windows=None, num_stages=1)"
    datasets = list(pipe.iter_datasets())
    assert len(datasets) == 4
    assert datasets[0].take() == [0, 1, 2]
    assert datasets[1].take() == [3, 4, 5]
    assert datasets[2].take() == [6, 7, 8]
    assert datasets[3].take() == [9]
Пример #14
0
def _map_block_split(
    block: Block,
    block_fn: BlockTransform,
    input_files: List[str],
    fn: Optional[UDF],
    *fn_args,
    **fn_kwargs,
) -> BlockPartition:
    output = []
    stats = BlockExecStats.builder()
    if fn is not None:
        fn_args = (fn,) + fn_args
    for new_block in block_fn(block, *fn_args, **fn_kwargs):
        accessor = BlockAccessor.for_block(new_block)
        new_meta = BlockMetadata(
            num_rows=accessor.num_rows(),
            size_bytes=accessor.size_bytes(),
            schema=accessor.schema(),
            input_files=input_files,
            exec_stats=stats.build(),
        )
        owner = DatasetContext.get_current().block_owner
        output.append((ray.put(new_block, _owner=owner), new_meta))
        stats = BlockExecStats.builder()
    return output
Пример #15
0
        def do_agg(blocks, clear_input_blocks: bool, *_):
            # 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,
                )
            ctx = DatasetContext.get_current()
            if ctx.use_push_based_shuffle:
                shuffle_op_cls = PushBasedGroupbyOp
            else:
                shuffle_op_cls = SimpleShuffleGroupbyOp
            shuffle_op = shuffle_op_cls(map_args=[boundaries, self._key, aggs],
                                        reduce_args=[self._key, aggs])
            return shuffle_op.execute(
                blocks,
                num_reducers,
                clear_input_blocks,
            )
Пример #16
0
def test_optimize_lazy_reuse_base_data(ray_start_regular_shared, local_path,
                                       enable_dynamic_splitting, with_shuffle):
    context = DatasetContext.get_current()
    context.block_splitting_enabled = enable_dynamic_splitting

    num_blocks = 4
    dfs = [
        pd.DataFrame({"one": list(range(i, i + 4))}) for i in range(num_blocks)
    ]
    paths = [
        os.path.join(local_path, f"test{i}.csv") for i in range(num_blocks)
    ]
    for df, path in zip(dfs, paths):
        df.to_csv(path, index=False)
    counter = Counter.remote()
    source = MySource(counter)
    ds = ray.data.read_datasource(source, parallelism=4, paths=paths)
    num_reads = ray.get(counter.get.remote())
    assert num_reads == 1, num_reads
    ds = ds.experimental_lazy()
    ds = ds.map(lambda x: x)
    if with_shuffle:
        ds = ds.random_shuffle()
    ds.take()
    num_reads = ray.get(counter.get.remote())
    assert num_reads == num_blocks, num_reads
Пример #17
0
    def remote_read(i: int, task: ReadTask) -> MaybeBlockPartition:
        DatasetContext._set_current(context)
        stats = BlockExecStats.builder()

        # Execute the read task.
        block = task()

        if context.block_splitting_enabled:
            metadata = task.get_metadata()
            metadata.exec_stats = stats.build()
        else:
            metadata = BlockAccessor.for_block(block).get_metadata(
                input_files=task.get_metadata().input_files,
                exec_stats=stats.build())
        stats_actor.record_task.remote(stats_uuid, i, metadata)
        return block
Пример #18
0
def test_optimize_reread_base_data(ray_start_regular_shared, local_path):
    context = DatasetContext.get_current()
    context.optimize_fuse_stages = True
    context.optimize_fuse_read_stages = True
    context.optimize_fuse_shuffle_stages = True

    # Re-read on.
    N = 4
    df1 = pd.DataFrame({"one": [1, 2, 3], "two": ["a", "b", "c"]})
    path1 = os.path.join(local_path, "test1.csv")
    df1.to_csv(path1, index=False, storage_options={})
    counter = Counter.remote()
    source = MySource(counter)
    ds1 = ray.data.read_datasource(source, parallelism=1, paths=path1)
    pipe = ds1.repeat(N)
    pipe.take()
    num_reads = ray.get(counter.get.remote())
    assert num_reads == N + 1, num_reads

    # Re-read off.
    context.optimize_fuse_read_stages = False
    ray.get(counter.reset.remote())
    ds1 = ray.data.read_datasource(source, parallelism=1, paths=path1)
    pipe = ds1.repeat(N)
    pipe.take()
    num_reads = ray.get(counter.get.remote())
    assert num_reads == 1, num_reads
Пример #19
0
    def __call__(self) -> MaybeBlockPartition:
        context = DatasetContext.get_current()
        result = self._read_fn()
        if not hasattr(result, "__iter__"):
            DeprecationWarning(
                "Read function must return Iterable[Block], got {}. "
                "Probably you need to return `[block]` instead of "
                "`block`.".format(result))

        if context.block_splitting_enabled:
            partition: BlockPartition = []
            for block in result:
                metadata = BlockAccessor.for_block(block).get_metadata(
                    input_files=self._metadata.input_files,
                    exec_stats=None)  # No exec stats for the block splits.
                assert context.block_owner
                partition.append(
                    (ray.put(block, _owner=context.block_owner), metadata))
            if len(partition) == 0:
                raise ValueError("Read task must return non-empty list.")
            return partition
        else:
            builder = DelegatingBlockBuilder()
            for block in result:
                builder.add_block(block)
            return builder.build()
Пример #20
0
    def __init__(self):
        ctx = DatasetContext.get_current()

        # Setup a dummy actor to send the data. In a real datasource, write
        # tasks would send data to an external system instead of a Ray actor.
        @ray.remote(scheduling_strategy=ctx.scheduling_strategy)
        class DataSink:
            def __init__(self):
                self.rows_written = 0
                self.enabled = True

            def write(self, block: Block) -> str:
                block = BlockAccessor.for_block(block)
                if not self.enabled:
                    raise ValueError("disabled")
                self.rows_written += block.num_rows()
                return "ok"

            def get_rows_written(self):
                return self.rows_written

            def set_enabled(self, enabled):
                self.enabled = enabled

        self.data_sink = DataSink.remote()
        self.num_ok = 0
        self.num_failed = 0
Пример #21
0
def test_dataset_stats_basic(ray_start_regular_shared):
    context = DatasetContext.get_current()
    context.optimize_fuse_stages = True
    ds = ray.data.range(1000, parallelism=10)
    ds = ds.map_batches(lambda x: x)
    ds = ds.map(lambda x: x)
    for batch in ds.iter_batches():
        pass
    stats = canonicalize(ds.stats())
    assert (stats == """Stage N read->map_batches: N/N blocks executed in T
* Remote wall time: T min, T max, T mean, T total
* Remote cpu time: T min, T max, T mean, T total
* Output num rows: N min, N max, N mean, N total
* Output size bytes: N min, N max, N mean, N total
* Tasks per node: N min, N max, N mean; N nodes used

Stage N map: N/N blocks executed in T
* Remote wall time: T min, T max, T mean, T total
* Remote cpu time: T min, T max, T mean, T total
* Output num rows: N min, N max, N mean, N total
* Output size bytes: N min, N max, N mean, N total
* Tasks per node: N min, N max, N mean; N nodes used

Dataset iterator time breakdown:
* In ray.wait(): T
* In ray.get(): T
* In format_batch(): T
* In user code: T
* Total time: T
""")
Пример #22
0
 def do_shuffle(block_list, clear_input_blocks: bool, block_udf, remote_args):
     num_blocks = block_list.executed_num_blocks()  # Blocking.
     if num_blocks == 0:
         return block_list, {}
     if clear_input_blocks:
         blocks = block_list.copy()
         block_list.clear()
     else:
         blocks = block_list
     context = DatasetContext.get_current()
     if context.use_push_based_shuffle:
         if output_num_blocks is not None:
             raise NotImplementedError(
                 "Push-based shuffle doesn't support setting num_blocks yet."
             )
         shuffle_op_cls = PushBasedShufflePartitionOp
     else:
         shuffle_op_cls = SimpleShufflePartitionOp
     random_shuffle_op = shuffle_op_cls(
         block_udf, random_shuffle=True, random_seed=seed
     )
     return random_shuffle_op.execute(
         blocks,
         output_num_blocks or num_blocks,
         clear_input_blocks,
         map_ray_remote_args=remote_args,
         reduce_ray_remote_args=remote_args,
     )
Пример #23
0
    def iter_blocks_with_metadata(
            self) -> Iterator[Tuple[ObjectRef[Block], BlockMetadata]]:
        context = DatasetContext.get_current()
        self._check_if_cleared()
        outer = self

        class Iter:
            def __init__(self):
                self._base_iter = outer._iter_block_partitions()
                self._buffer = []

            def __iter__(self):
                return self

            def __next__(self):
                while not self._buffer:
                    if context.block_splitting_enabled:
                        part_ref, _ = next(self._base_iter)
                        partition = ray.get(part_ref)
                    else:
                        block, metadata = next(self._base_iter)
                        partition = [(block, metadata)]
                    for ref, metadata in partition:
                        self._buffer.append((ref, metadata))
                return self._buffer.pop(0)

        return Iter()
Пример #24
0
def objective(*args):
    # Tell Datasets to use the current placement group for all Datasets tasks.
    ctx = DatasetContext.get_current()
    ctx.scheduling_strategy = PlacementGroupSchedulingStrategy(
        ray.util.get_current_placement_group())
    # This Dataset workload will use that placement group for all read and map tasks.
    ray.data.range(10).show()
Пример #25
0
def test_basic_pipeline(ray_start_regular_shared):
    context = DatasetContext.get_current()
    context.optimize_fuse_stages = True
    ds = ray.data.range(10, parallelism=10)

    pipe = ds.window(blocks_per_window=1)
    assert str(pipe) == "DatasetPipeline(num_windows=10, num_stages=2)"
    assert pipe.count() == 10
    pipe = ds.window(blocks_per_window=1)
    pipe.show()

    pipe = ds.window(blocks_per_window=1).map(lambda x: x).map(lambda x: x)
    assert str(pipe) == "DatasetPipeline(num_windows=10, num_stages=4)"
    assert pipe.take() == list(range(10))

    pipe = (ds.window(
        blocks_per_window=1).map(lambda x: x).flat_map(lambda x: [x, x + 1]))
    assert str(pipe) == "DatasetPipeline(num_windows=10, num_stages=4)"
    assert pipe.count() == 20

    pipe = ds.window(blocks_per_window=1).filter(lambda x: x % 2 == 0)
    assert str(pipe) == "DatasetPipeline(num_windows=10, num_stages=3)"
    assert pipe.count() == 5

    pipe = ds.window(blocks_per_window=999)
    assert str(pipe) == "DatasetPipeline(num_windows=1, num_stages=2)"
    assert pipe.count() == 10

    pipe = ds.repeat(10)
    assert str(pipe) == "DatasetPipeline(num_windows=10, num_stages=2)"
    assert pipe.count() == 100
    pipe = ds.repeat(10)
    assert pipe.sum() == 450
    pipe = ds.repeat(10)
    assert len(pipe.take_all()) == 100
Пример #26
0
    def __init__(self, pipeline: "DatasetPipeline[T]"):
        self._pipeline: "DatasetPipeline[T]" = pipeline
        self._stages: List[ObjectRef[Dataset[Any]]] = [None] * (
            len(self._pipeline._optimized_stages) + 1)
        # Pin the child actors to the local node so that they fate-share with
        # the PipelineExecutor during a node failure.
        local_node_resource = "node:{}".format(ray.util.get_node_ip_address())
        self._stage_runners = [
            _StageRunner.options(resources={
                local_node_resource: 0.0001
            },
                                 placement_group=None).remote()
            for _ in self._stages
        ]
        self._iter = iter(self._pipeline._base_iterable)
        self._stages[0] = self._stage_runners[0].run.remote(
            next(self._iter), DatasetContext.get_current())

        if self._pipeline._length and self._pipeline._length != float("inf"):
            length = self._pipeline._length
        else:
            length = 1

        if self._pipeline._progress_bars:
            self._bars = [
                ProgressBar("Stage {}".format(i), length, position=i)
                for i in range(len(self._stages))
            ]
        else:
            self._bars = None
Пример #27
0
    def _split(self, n: int, splitter: Callable[[Dataset],
                                                "DatasetPipeline[T]"]):

        resources = {}
        if not ray.util.client.ray.is_connected():
            # Pin the coordinator (and any child actors) to the local node to avoid
            # errors during node failures. If the local node dies, then the driver
            # will fate-share with the coordinator anyway.
            resources["node:{}".format(
                ray.util.get_node_ip_address())] = 0.0001

        coordinator = PipelineSplitExecutorCoordinator.options(
            resources=resources,
            placement_group=None,
        ).remote(self, n, splitter, DatasetContext.get_current())
        if self._executed[0]:
            raise RuntimeError("Pipeline cannot be read multiple times.")
        self._executed[0] = True

        class SplitIterator:
            def __init__(self, split_index, coordinator):
                self.split_index = split_index
                self.coordinator = coordinator
                self.warn_threshold = 100
                self.wait_delay_s = 0.1

            def __iter__(self):
                return self

            def __next__(self):
                ds = None
                tries = 0
                while ds is None:
                    ds = ray.get(
                        self.coordinator.next_dataset_if_ready.remote(
                            self.split_index))
                    # Wait for other shards to catch up reading.
                    if not ds:
                        time.sleep(self.wait_delay_s)
                        tries += 1
                    if tries > self.warn_threshold:
                        print("Warning: reader on shard {} of the pipeline "
                              "has been blocked more than {}s waiting for "
                              "other readers to catch up. All pipeline shards "
                              "must be read from concurrently.".format(
                                  self.split_index,
                                  self.wait_delay_s * self.warn_threshold,
                              ))
                        self.warn_threshold *= 2
                return lambda: ds

        return [
            # Disable progress bars for the split readers since they would
            # overwhelm the console.
            DatasetPipeline(
                SplitIterator(idx, coordinator),
                length=self._length,
                progress_bars=False,
            ) for idx in range(n)
        ]
Пример #28
0
    def __next__(self):
        output = None
        start = time.perf_counter()

        while output is None:
            if all(s is None for s in self._stages):
                raise StopIteration

            # Wait for any completed stages.
            pending = [s for s in self._stages if s is not None]
            ready, _ = ray.wait(pending, timeout=0.1, num_returns=len(pending))

            # Bubble elements down the pipeline as they become ready.
            for i in range(len(self._stages))[::-1]:
                is_last = i + 1 >= len(self._stages)
                next_slot_free = is_last or self._stages[i + 1] is None
                if not next_slot_free:
                    continue

                slot_ready = self._stages[i] in ready
                if not slot_ready:
                    continue

                # Bubble.
                result = ray.get(self._stages[i])
                if self._bars:
                    self._bars[i].update(1)
                self._stages[i] = None
                if is_last:
                    output = result
                else:
                    fn = self._pipeline._stages[i]
                    self._stages[i + 1] = self._stage_runners[i].run.remote(
                        lambda: fn(result), DatasetContext.get_current())

            # Pull a new element for the initial slot if possible.
            if self._stages[0] is None:
                try:
                    self._stages[0] = self._stage_runners[0].run.remote(
                        next(self._iter), DatasetContext.get_current())
                except StopIteration:
                    pass

        self._pipeline._stats.wait_time_s.append(time.perf_counter() - start)
        self._pipeline._stats.add(output._stats)
        return output
Пример #29
0
def test_memory_release_pipeline(shutdown_only, lazy_input):
    context = DatasetContext.get_current()
    # Disable stage fusion so we can keep reads and maps from being fused together,
    # since we're trying to test multi-stage memory releasing here.
    context.optimize_fuse_stages = False
    # This object store allocation can hold at most 1 copy of the transformed dataset.
    if lazy_input:
        object_store_memory = 3000e6
    else:
        object_store_memory = 3000e6

    n = 10
    info = ray.init(num_cpus=n, object_store_memory=object_store_memory)
    if lazy_input:
        ds = ray.data.read_datasource(
            OnesSource(),
            parallelism=n,
            n_per_block=100 * 1024 * 1024,
        )
    else:
        ds = ray.data.from_items(list(range(n)), parallelism=n)

    # Create a single-window pipeline.
    pipe = ds.window(blocks_per_window=n)

    # Round 1.
    def gen(x):
        import time

        # TODO(Clark): Remove this sleep once we have fixed memory pressure handling.
        time.sleep(2)
        if isinstance(x, np.ndarray):
            return x
        else:
            return np.ones(100 * 1024 * 1024, dtype=np.uint8)

    pipe = pipe.map(gen)

    def inc(x):
        import time

        # TODO(Clark): Remove this sleep once we have fixed memory pressure handling.
        time.sleep(2)
        return x + 1

    num_rounds = 10
    for _ in range(num_rounds):
        pipe = pipe.map(inc)

    for block in pipe.iter_batches(batch_size=None):
        for arr in block:
            np.testing.assert_equal(
                arr,
                np.ones(100 * 1024 * 1024, dtype=np.uint8) + num_rounds,
            )
    meminfo = memory_summary(info["address"], stats_only=True)
    assert "Spilled" not in meminfo, meminfo
Пример #30
0
def test_dataset_pipeline_stats_basic(ray_start_regular_shared):
    context = DatasetContext.get_current()
    context.optimize_fuse_stages = True
    ds = ray.data.range(1000, parallelism=10)
    ds = ds.map_batches(lambda x: x)
    pipe = ds.repeat(5)
    pipe = pipe.map(lambda x: x)
    for batch in pipe.iter_batches():
        pass
    stats = canonicalize(pipe.stats())
    assert (stats == """== Pipeline Window N ==
Stage N read->map_batches: N/N blocks executed in T
* Remote wall time: T min, T max, T mean, T total
* Remote cpu time: T min, T max, T mean, T total
* Output num rows: N min, N max, N mean, N total
* Output size bytes: N min, N max, N mean, N total
* Tasks per node: N min, N max, N mean; N nodes used

Stage N map: N/N blocks executed in T
* Remote wall time: T min, T max, T mean, T total
* Remote cpu time: T min, T max, T mean, T total
* Output num rows: N min, N max, N mean, N total
* Output size bytes: N min, N max, N mean, N total
* Tasks per node: N min, N max, N mean; N nodes used

== Pipeline Window N ==
Stage N read->map_batches: [execution cached]

Stage N map: N/N blocks executed in T
* Remote wall time: T min, T max, T mean, T total
* Remote cpu time: T min, T max, T mean, T total
* Output num rows: N min, N max, N mean, N total
* Output size bytes: N min, N max, N mean, N total
* Tasks per node: N min, N max, N mean; N nodes used

== Pipeline Window N ==
Stage N read->map_batches: [execution cached]

Stage N map: N/N blocks executed in T
* Remote wall time: T min, T max, T mean, T total
* Remote cpu time: T min, T max, T mean, T total
* Output num rows: N min, N max, N mean, N total
* Output size bytes: N min, N max, N mean, N total
* Tasks per node: N min, N max, N mean; N nodes used

##### Overall Pipeline Time Breakdown #####
* Time stalled waiting for next dataset: T min, T max, T mean, T total

DatasetPipeline iterator time breakdown:
* Waiting for next dataset: T
* In ray.wait(): T
* In ray.get(): T
* In format_batch(): T
* In user code: T
* Total time: T
""")