Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[Datasets] Iterating through DatasetPipeline fails with ZeroDivisionError #31505

Closed
amogkam opened this issue Jan 6, 2023 · 2 comments · Fixed by #31513
Closed

[Datasets] Iterating through DatasetPipeline fails with ZeroDivisionError #31505

amogkam opened this issue Jan 6, 2023 · 2 comments · Fixed by #31513
Assignees
Labels
bug Something that is supposed to be working; but isn't data Ray Data-related issues P1 Issue that should be fixed within a few weeks triage Needs triage (eg: priority, bug/not-bug, and owning component)

Comments

@amogkam
Copy link
Contributor

amogkam commented Jan 6, 2023

What happened + What you expected to happen

I expect the below code snippet to not fail.

Instead it fails with the following error:

self = <ray.data.dataset_pipeline.DatasetPipeline.repeat.<locals>.RepeatIterator object at 0x16a801a30>

    def __next__(self) -> Dataset[T]:
        # Still going through the original pipeline.
        if self._original_iter:
            try:
                make_ds = next(self._original_iter)
                self._results.append(make_ds)

                def gen():
                    res = make_ds()
                    res._set_epoch(0)
                    return res

                return gen
            except StopIteration:
                self._original_iter = None
                # Calculate the cursor limit.
                if times:
                    self._max_i = len(self._results) * (times - 1)
                else:
                    self._max_i = float("inf")
        # Going through a repeat of the pipeline.
        if self._i < self._max_i:
>           make_ds = self._results[self._i % len(self._results)]
E           ZeroDivisionError: integer division or modulo by zero

Versions / Dependencies

master

Reproduction script

pipe = ray.data.range(6, parallelism=6).window(blocks_per_window=2).repeat()
assert pipe.schema() == int
pipe = pipe.map_batches(lambda x: x)
next(pipe.iter_epochs())

Issue Severity

High: It blocks me from completing my task.

@amogkam amogkam added bug Something that is supposed to be working; but isn't triage Needs triage (eg: priority, bug/not-bug, and owning component) labels Jan 6, 2023
@amogkam amogkam added the data Ray Data-related issues label Jan 6, 2023
@jianoaix jianoaix added the P1 Issue that should be fixed within a few weeks label Jan 7, 2023
@jianoaix jianoaix changed the title [Data] Iterating through DatasetPipeline fails with ZeroDivisionError [Datasets] Iterating through DatasetPipeline fails with ZeroDivisionError Jan 7, 2023
@jianoaix
Copy link
Contributor

jianoaix commented Jan 7, 2023

A mitigation is switching the order of pipe.schema() and pipe.map_batches():

pipe = ray.data.range(6, parallelism=6).window(blocks_per_window=2).repeat()
pipe = pipe.map_batches(lambda x: x)
assert pipe.schema() == int
next(pipe.iter_epochs())

The issue seems to be the peek() (used in pipe.schema()) advanced the base_iterable of pipeline, which was then used to create another pipeline (with pipe.map_batches()).

@amogkam
Copy link
Contributor Author

amogkam commented Jan 7, 2023

In this case, we're using the schema to determine what batch format to use for the map_batches call so I won't be able to switch the order.

amogkam added a commit that referenced this issue Jan 18, 2023
Signed-off-by: amogkam [email protected]

Closes #31505.

When peeking a DatasetPipeline via .schema() for example, the first dataset in the base iterator is consumed. Then when chaining new operations on the pipeline, such as a map_batches, the dataset that was peeked is lost.

In this PR, we change the implementation of peek to not consume the base iterable, but rather create a new iterable consisting of just the first dataset.
andreapiso pushed a commit to andreapiso/ray that referenced this issue Jan 22, 2023
…roject#31513)

Signed-off-by: amogkam [email protected]

Closes ray-project#31505.

When peeking a DatasetPipeline via .schema() for example, the first dataset in the base iterator is consumed. Then when chaining new operations on the pipeline, such as a map_batches, the dataset that was peeked is lost.

In this PR, we change the implementation of peek to not consume the base iterable, but rather create a new iterable consisting of just the first dataset.

Signed-off-by: Andrea Pisoni <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Something that is supposed to be working; but isn't data Ray Data-related issues P1 Issue that should be fixed within a few weeks triage Needs triage (eg: priority, bug/not-bug, and owning component)
Projects
None yet
Development

Successfully merging a pull request may close this issue.

2 participants