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

[AIR] Fix DatasetIterator backwards compability #32526

Merged
merged 7 commits into from
Feb 15, 2023
Merged
Show file tree
Hide file tree
Changes from 4 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 4 additions & 0 deletions python/ray/data/_internal/bulk_dataset_iterator.py
Original file line number Diff line number Diff line change
Expand Up @@ -87,6 +87,10 @@ def to_tf(
def stats(self) -> str:
return self._base_dataset.stats()

@property
def _base_dataset_or_pipeline(self) -> "Dataset":
return self._base_dataset

def _to_train_iterator(self) -> "TrainDatasetIterator":
from ray.train._internal.dataset_iterator import TrainDatasetIterator

Expand Down
4 changes: 4 additions & 0 deletions python/ray/data/_internal/pipelined_dataset_iterator.py
Original file line number Diff line number Diff line change
Expand Up @@ -105,6 +105,10 @@ def to_tf(
def stats(self) -> str:
return self._base_dataset_pipeline.stats()

@property
def _base_dataset_or_pipeline(self) -> "DatasetPipeline":
return self._base_dataset_pipeline

def _to_train_iterator(self) -> "TrainDatasetIterator":
from ray.train._internal.dataset_iterator import TrainDatasetIterator

Expand Down
8 changes: 8 additions & 0 deletions python/ray/data/dataset_iterator.py
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,8 @@
import tensorflow as tf
import torch
from ray.data._internal.torch_iterable_dataset import TorchTensorBatchType
from ray.data.dataset import Dataset
from ray.data.dataset_pipeline import DatasetPipeline
from ray.train._internal.dataset_iterator import TrainDatasetIterator


Expand Down Expand Up @@ -239,6 +241,12 @@ def stats(self) -> str:
"""Returns a string containing execution timing information."""
raise NotImplementedError

@property
def _base_dataset_or_pipeline(self) -> Union["Dataset", "DatasetPipeline"]:
"""The :class:`~ray.data.dataset.Dataset` or
:class:`~ray.data.dataset.DatasetPipeline` that this object iterates over."""
raise NotImplementedError

def iter_epochs(self, max_epoch: int = -1) -> None:
raise DeprecationWarning(
"If you are using AIR, note that session.get_dataset_shard() "
Expand Down
10 changes: 8 additions & 2 deletions python/ray/train/_internal/dataset_iterator.py
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
from typing import Iterator, Optional, TYPE_CHECKING
from typing import Iterator, Optional, Union, TYPE_CHECKING
import warnings

from ray.data.block import DataBatch
Expand All @@ -8,6 +8,8 @@
if TYPE_CHECKING:
import tensorflow as tf
from ray.data._internal.torch_iterable_dataset import TorchTensorBatchType
from ray.data.dataset import Dataset
from ray.data.dataset_pipeline import DatasetPipeline


class TrainDatasetIterator(DatasetIterator):
Expand Down Expand Up @@ -47,6 +49,10 @@ def to_tf(self, *args, **kwargs) -> "tf.data.Dataset":
def stats(self) -> str:
return self._dataset_iterator.stats()

@property
def _base_dataset_or_pipeline(self) -> Union["Dataset", "DatasetPipeline"]:
return self._dataset_iterator._base_dataset_or_pipeline

def __getattr__(self, name):
if name == "_dataset_iterator":
raise AttributeError
Expand All @@ -64,4 +70,4 @@ def __getattr__(self, name):
"for full DatasetIterator docs."
)

return getattr(self._dataset_iterator._base_dataset, name)
return getattr(self._base_dataset_or_pipeline, name)
16 changes: 16 additions & 0 deletions python/ray/train/tests/test_dataset_iterator.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,16 @@
import ray
from ray.data._internal.bulk_dataset_iterator import BulkDatasetIterator
from ray.data._internal.pipelined_dataset_iterator import PipelinedDatasetIterator
from ray.train._internal.dataset_iterator import TrainDatasetIterator


def test_backwards_compatibility():
# `DatasetIterator` doesn't expose a `count` method, but you should still be able to
# access it for backwards compatibility.
dataset = ray.data.range(1)
iterator = TrainDatasetIterator(BulkDatasetIterator(dataset))
assert iterator.count() == 1

pipeline = dataset.repeat(1)
iterator = TrainDatasetIterator(PipelinedDatasetIterator(pipeline))
assert iterator.count() == 1