From 6d69d79cfef1741dc11b0d9662ed120a90dd99a4 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Fri, 14 Apr 2023 12:13:52 -0700 Subject: [PATCH] [data] [streaming] [part 3/n] Rename Dataset => Datastream in internal files (#34340) --- python/ray/data/__init__.py | 2 +- python/ray/data/_internal/arrow_block.py | 2 +- .../block_batching/block_batching.py | 14 +- .../_internal/block_batching/iter_batches.py | 10 +- .../ray/data/_internal/block_batching/util.py | 16 +- python/ray/data/_internal/block_list.py | 4 +- python/ray/data/_internal/compute.py | 10 +- .../dataset_iterator/dataset_iterator_impl.py | 26 +- .../pipelined_dataset_iterator.py | 18 +- .../stream_split_dataset_iterator.py | 36 +- python/ray/data/_internal/dataset_logger.py | 30 +- .../execution/autoscaling_requester.py | 2 +- .../data/_internal/execution/bulk_executor.py | 14 +- .../data/_internal/execution/interfaces.py | 16 +- .../data/_internal/execution/legacy_compat.py | 32 +- .../operators/actor_pool_map_operator.py | 6 +- .../execution/operators/input_data_buffer.py | 2 +- .../execution/operators/map_operator.py | 2 +- .../execution/operators/output_splitter.py | 2 +- .../operators/task_pool_map_operator.py | 2 +- .../execution/operators/zip_operator.py | 2 +- .../_internal/execution/streaming_executor.py | 20 +- python/ray/data/_internal/fast_repartition.py | 14 +- python/ray/data/_internal/lazy_block_list.py | 22 +- .../ray/data/_internal/logical/interfaces.py | 2 +- .../logical/operators/all_to_all_operator.py | 2 +- .../logical/operators/map_operator.py | 4 +- python/ray/data/_internal/memory_tracing.py | 2 +- python/ray/data/_internal/null_aggregate.py | 2 +- python/ray/data/_internal/pandas_block.py | 6 +- .../ray/data/_internal/pipeline_executor.py | 16 +- python/ray/data/_internal/plan.py | 110 +-- .../planner/exchange/sort_task_spec.py | 2 +- python/ray/data/_internal/planner/write.py | 2 +- python/ray/data/_internal/progress_bar.py | 4 +- python/ray/data/_internal/remote_fn.py | 2 +- python/ray/data/_internal/sort.py | 4 +- python/ray/data/_internal/stage_impl.py | 28 +- python/ray/data/_internal/stats.py | 114 +-- python/ray/data/_internal/util.py | 20 +- python/ray/data/context.py | 4 +- python/ray/data/dataset.py | 37 +- python/ray/data/dataset_iterator.py | 92 +- python/ray/data/dataset_pipeline.py | 285 +++--- python/ray/data/datasource/datasource.py | 10 +- .../data/datasource/file_based_datasource.py | 34 +- .../ray/data/datasource/file_meta_provider.py | 18 +- .../ray/data/datasource/parquet_datasource.py | 23 +- python/ray/data/datasource/partitioning.py | 16 +- .../data/datasource/webdataset_datasource.py | 6 +- python/ray/data/grouped_dataset.py | 143 +-- python/ray/data/preprocessors/batch_mapper.py | 16 +- python/ray/data/preprocessors/chain.py | 14 +- python/ray/data/preprocessors/concatenator.py | 6 +- python/ray/data/preprocessors/discretizer.py | 6 +- python/ray/data/preprocessors/encoder.py | 39 +- python/ray/data/preprocessors/imputer.py | 12 +- python/ray/data/preprocessors/scaler.py | 28 +- python/ray/data/preprocessors/torch.py | 18 +- python/ray/data/preprocessors/vectorizer.py | 10 +- python/ray/data/read_api.py | 16 +- .../data/tests/block_batching/test_util.py | 6 +- python/ray/data/tests/conftest.py | 15 +- python/ray/data/tests/mock_server.py | 2 +- .../tests/preprocessors/test_preprocessors.py | 12 +- .../data/tests/preprocessors/test_torch.py | 34 +- .../ray/data/tests/test_dataset_all_to_all.py | 26 +- .../data/tests/test_dataset_consumption.py | 12 +- python/ray/data/tests/test_dataset_logger.py | 6 +- python/ray/data/tests/test_dataset_map.py | 4 +- .../ray/data/tests/test_dataset_pipeline.py | 12 +- python/ray/data/tests/test_split.py | 4 +- python/ray/data/tests/test_stats.py | 70 +- .../data/tests/test_streaming_integration.py | 18 +- .../horovod/horovod_cifar_pbt_example.py | 196 +++- python/ray/train/tests/pytorch_pbt_failure.py | 80 +- .../test_myst_doc.py | 77 +- scripts/bazel.py | 266 +++++- scripts/bazel_sharding.py | 428 ++++++++- scripts/build-docker-images.py | 886 +++++++++++++++++- scripts/build-multinode-image.py | 36 +- scripts/check-bazel-team-owner.py | 46 +- scripts/check-test-run.py | 38 +- scripts/check_import_order.py | 86 +- scripts/check_minimal_install.py | 51 +- scripts/clang-tidy-diff.py | 307 +++++- scripts/determine_tests_to_run.py | 367 +++++++- scripts/get_build_info.py | 88 +- scripts/py_dep_analysis.py | 385 +++++++- scripts/py_dep_analysis_test.py | 153 ++- 90 files changed, 4329 insertions(+), 839 deletions(-) mode change 120000 => 100755 python/ray/train/examples/horovod/horovod_cifar_pbt_example.py mode change 120000 => 100644 python/ray/train/tests/pytorch_pbt_failure.py mode change 120000 => 100644 release/air_examples/gptj_deepspeed_finetuning/test_myst_doc.py mode change 120000 => 100755 scripts/bazel.py mode change 120000 => 100644 scripts/bazel_sharding.py mode change 120000 => 100644 scripts/build-docker-images.py mode change 120000 => 100644 scripts/build-multinode-image.py mode change 120000 => 100644 scripts/check-bazel-team-owner.py mode change 120000 => 100644 scripts/check-test-run.py mode change 120000 => 100644 scripts/check_import_order.py mode change 120000 => 100644 scripts/check_minimal_install.py mode change 120000 => 100755 scripts/clang-tidy-diff.py mode change 120000 => 100644 scripts/determine_tests_to_run.py mode change 120000 => 100755 scripts/get_build_info.py mode change 120000 => 100644 scripts/py_dep_analysis.py mode change 120000 => 100644 scripts/py_dep_analysis_test.py diff --git a/python/ray/data/__init__.py b/python/ray/data/__init__.py index 3b72e4084932..8e813ef5a530 100644 --- a/python/ray/data/__init__.py +++ b/python/ray/data/__init__.py @@ -57,8 +57,8 @@ __all__ = [ "ActorPoolStrategy", - "Dataset", "Datastream", + "Dataset", # Backwards compatibility alias. "DataContext", "DatasetContext", # Backwards compatibility alias. "DataIterator", diff --git a/python/ray/data/_internal/arrow_block.py b/python/ray/data/_internal/arrow_block.py index 8fa6130d28aa..0232f6cabd9f 100644 --- a/python/ray/data/_internal/arrow_block.py +++ b/python/ray/data/_internal/arrow_block.py @@ -68,7 +68,7 @@ def get_concat_and_sort_transform(context: DataContext) -> Callable: class ArrowRow(TableRow): """ - Row of a tabular Dataset backed by a Arrow Table block. + Row of a tabular Datastream backed by a Arrow Table block. """ def __getitem__(self, key: str) -> Any: diff --git a/python/ray/data/_internal/block_batching/block_batching.py b/python/ray/data/_internal/block_batching/block_batching.py index 6a42c8959ef0..c6e655aca66e 100644 --- a/python/ray/data/_internal/block_batching/block_batching.py +++ b/python/ray/data/_internal/block_batching/block_batching.py @@ -15,7 +15,7 @@ ActorBlockPrefetcher, ) from ray.data._internal.memory_tracing import trace_deallocation -from ray.data._internal.stats import DatasetPipelineStats, DatasetStats +from ray.data._internal.stats import DatasetPipelineStats, DatastreamStats from ray.data.block import Block, DataBatch from ray.data.context import DataContext from ray.types import ObjectRef @@ -35,7 +35,7 @@ def nullcontext(enter_result=None): def batch_block_refs( block_refs: Iterator[ObjectRef[Block]], *, - stats: Optional[Union[DatasetStats, DatasetPipelineStats]] = None, + stats: Optional[Union[DatastreamStats, DatasetPipelineStats]] = None, prefetch_blocks: int = 0, clear_block_after_read: bool = False, batch_size: Optional[int] = None, @@ -51,8 +51,8 @@ def batch_block_refs( This takes a block iterator and creates batch_size batches, slicing, unioning, shuffling, prefetching, and formatting blocks as needed. - This is used by both Dataset.iter_batches()/DatasetPipeline.iter_batches() - and Dataset.map_batches()/DatasetPipeline.map_batches(). + This is used by both Datastream.iter_batches()/DatasetPipeline.iter_batches() + and Datastream.map_batches()/DatasetPipeline.map_batches(). Args: block_refs: An iterator over block object references. @@ -123,7 +123,7 @@ def batch_block_refs( def batch_blocks( blocks: Iterator[Block], *, - stats: Optional[Union[DatasetStats, DatasetPipelineStats]] = None, + stats: Optional[Union[DatastreamStats, DatasetPipelineStats]] = None, batch_size: Optional[int] = None, batch_format: str = "default", drop_last: bool = False, @@ -173,7 +173,7 @@ def _prefetch_blocks( prefetcher: BlockPrefetcher, num_blocks_to_prefetch: int, eager_free: bool = False, - stats: Optional[Union[DatasetStats, DatasetPipelineStats]] = None, + stats: Optional[Union[DatastreamStats, DatasetPipelineStats]] = None, ) -> Iterator[ObjectRef[Block]]: """Given an iterable of Block Object References, returns an iterator over these object reference while prefetching `num_block_to_prefetch` @@ -183,7 +183,7 @@ def _prefetch_blocks( block_ref_iter: An iterator over block object references. num_blocks_to_prefetch: The number of blocks to prefetch ahead of the current block during the scan. - stats: Dataset stats object used to store block wait time. + stats: Datastream stats object used to store block wait time. """ if num_blocks_to_prefetch == 0: for block_ref in block_ref_iter: diff --git a/python/ray/data/_internal/block_batching/iter_batches.py b/python/ray/data/_internal/block_batching/iter_batches.py index 25f6ae8101e1..9e59583d6abe 100644 --- a/python/ray/data/_internal/block_batching/iter_batches.py +++ b/python/ray/data/_internal/block_batching/iter_batches.py @@ -20,7 +20,7 @@ make_async_gen, ) from ray.data._internal.memory_tracing import trace_deallocation -from ray.data._internal.stats import DatasetStats +from ray.data._internal.stats import DatastreamStats from ray.data.context import DataContext if sys.version_info >= (3, 7): @@ -36,7 +36,7 @@ def nullcontext(enter_result=None): def iter_batches( block_refs: Iterator[Tuple[ObjectRef[Block], BlockMetadata]], *, - stats: Optional[DatasetStats] = None, + stats: Optional[DatastreamStats] = None, clear_block_after_read: bool = False, batch_size: Optional[int] = None, batch_format: Optional[str] = "default", @@ -83,7 +83,7 @@ def iter_batches( Args: block_refs: An iterator over block object references and their corresponding metadata. - stats: DatasetStats object to record timing and other statistics. + stats: DatastreamStats object to record timing and other statistics. clear_block_after_read: Whether to clear the block from object store manually (i.e. without waiting for Python's automatic GC) after it is read. Doing so will reclaim memory faster and hence reduce the @@ -185,7 +185,7 @@ def _async_iter_batches( def _format_in_threadpool( batch_iter: Iterator[Batch], - stats: DatasetStats, + stats: DatastreamStats, batch_format: Optional[str], collate_fn: Optional[Callable[[DataBatch], Any]], num_threadpool_workers: int, @@ -194,7 +194,7 @@ def _format_in_threadpool( Args: logical_batch_iterator: An iterator over logical batches. - stats: DatasetStats object to record timing and other statistics. + stats: DatastreamStats object to record timing and other statistics. batch_format: The format in which to return each batch. Specify "default" to use the current block format (promoting Arrow to pandas automatically), "pandas" to diff --git a/python/ray/data/_internal/block_batching/util.py b/python/ray/data/_internal/block_batching/util.py index ef2da8141ad6..c642f6c2ac63 100644 --- a/python/ray/data/_internal/block_batching/util.py +++ b/python/ray/data/_internal/block_batching/util.py @@ -14,7 +14,7 @@ CollatedBatch, BlockPrefetcher, ) -from ray.data._internal.stats import DatasetPipelineStats, DatasetStats +from ray.data._internal.stats import DatasetPipelineStats, DatastreamStats from ray.util.scheduling_strategies import NodeAffinitySchedulingStrategy T = TypeVar("T") @@ -48,7 +48,7 @@ def _calculate_ref_hits(refs: List[ObjectRef[Any]]) -> Tuple[int, int, int]: def resolve_block_refs( block_ref_iter: Iterator[ObjectRef[Block]], - stats: Optional[Union[DatasetStats, DatasetPipelineStats]] = None, + stats: Optional[Union[DatastreamStats, DatasetPipelineStats]] = None, ) -> Iterator[Block]: """Resolves the block references for each logical batch. @@ -80,7 +80,7 @@ def resolve_block_refs( def blocks_to_batches( block_iter: Iterator[Block], - stats: Optional[Union[DatasetStats, DatasetPipelineStats]] = None, + stats: Optional[Union[DatastreamStats, DatasetPipelineStats]] = None, batch_size: Optional[int] = None, drop_last: bool = False, shuffle_buffer_min_size: Optional[int] = None, @@ -95,7 +95,7 @@ def blocks_to_batches( Args: block_iter: An iterator over blocks. - stats: Dataset stats object used to store block batching time. + stats: Datastream stats object used to store block batching time. batch_size: Record batch size, or None to let the system pick. drop_last: Whether to drop the last batch if it's incomplete. shuffle_buffer_min_size: If non-None, the data will be randomly shuffled @@ -152,7 +152,7 @@ def get_iter_next_batch_s_timer(): def format_batches( block_iter: Iterator[Batch], batch_format: Optional[str], - stats: Optional[Union[DatasetStats, DatasetPipelineStats]] = None, + stats: Optional[Union[DatastreamStats, DatasetPipelineStats]] = None, ) -> Iterator[Batch]: """Given an iterator of blocks, returns an iterator of formatted batches. @@ -175,7 +175,7 @@ def format_batches( def collate( batch_iter: Iterator[Batch], collate_fn: Optional[Callable[[DataBatch], Any]], - stats: Optional[DatasetStats] = None, + stats: Optional[DatastreamStats] = None, ) -> Iterator[CollatedBatch]: """Returns an iterator with the provided collate_fn applied to items of the batch iterator. @@ -278,7 +278,7 @@ def execute_computation(thread_index: int): break -PREFETCHER_ACTOR_NAMESPACE = "ray.dataset" +PREFETCHER_ACTOR_NAMESPACE = "ray.datastream" class WaitBlockPrefetcher(BlockPrefetcher): @@ -300,7 +300,7 @@ def __init__(self): @staticmethod def _get_or_create_actor_prefetcher() -> "ActorHandle": node_id = ray.get_runtime_context().get_node_id() - actor_name = f"dataset-block-prefetcher-{node_id}" + actor_name = f"datastream-block-prefetcher-{node_id}" return _BlockPretcher.options( scheduling_strategy=NodeAffinitySchedulingStrategy(node_id, soft=False), name=actor_name, diff --git a/python/ray/data/_internal/block_list.py b/python/ray/data/_internal/block_list.py index b046c24e4cc7..06dd3cf5cad4 100644 --- a/python/ray/data/_internal/block_list.py +++ b/python/ray/data/_internal/block_list.py @@ -58,8 +58,8 @@ def _check_if_cleared(self) -> None: """Raise an error if this BlockList has been previously cleared.""" if self.is_cleared(): raise ValueError( - "This Dataset's blocks have been moved, which means that you " - "can no longer use this Dataset." + "This Datastream's blocks have been moved, which means that you " + "can no longer use this Datastream." ) def split(self, split_size: int) -> List["BlockList"]: diff --git a/python/ray/data/_internal/compute.py b/python/ray/data/_internal/compute.py index 745556f4b023..6e74fbcf6c3b 100644 --- a/python/ray/data/_internal/compute.py +++ b/python/ray/data/_internal/compute.py @@ -83,7 +83,7 @@ def _apply( context = DataContext.get_current() - # Handle empty datasets. + # Handle empty datastreams. if block_list.initial_num_blocks() == 0: return block_list @@ -182,10 +182,10 @@ def __eq__(self, other: Any) -> bool: @PublicAPI class ActorPoolStrategy(ComputeStrategy): - """Specify the compute strategy for a Dataset transform. + """Specify the compute strategy for a Datastream transform. ActorPoolStrategy specifies that an autoscaling pool of actors should be used - for a given Dataset transform. This is useful for stateful setup of callable + for a given Datastream transform. This is useful for stateful setup of callable classes. For a fixed-sized pool of size ``n``, specify ``compute=ActorPoolStrategy(size=n)``. @@ -209,7 +209,7 @@ def __init__( max_size: Optional[int] = None, max_tasks_in_flight_per_actor: Optional[int] = None, ): - """Construct ActorPoolStrategy for a Dataset transform. + """Construct ActorPoolStrategy for a Datastream transform. Args: size: Specify a fixed size actor pool of this size. It is an error to @@ -276,7 +276,7 @@ def _apply( 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.""" + """Note: this is not part of the Datastream public API.""" assert not DataContext.get_current().new_execution_backend, "Legacy backend off" if fn_args is None: fn_args = tuple() diff --git a/python/ray/data/_internal/dataset_iterator/dataset_iterator_impl.py b/python/ray/data/_internal/dataset_iterator/dataset_iterator_impl.py index 4700eed91028..f89494e6ac62 100644 --- a/python/ray/data/_internal/dataset_iterator/dataset_iterator_impl.py +++ b/python/ray/data/_internal/dataset_iterator/dataset_iterator_impl.py @@ -4,50 +4,52 @@ from ray.data.block import Block, BlockMetadata from ray.data.context import DataContext from ray.data.dataset_iterator import DataIterator -from ray.data._internal.stats import DatasetStats +from ray.data._internal.stats import DatastreamStats if TYPE_CHECKING: import pyarrow - from ray.data import Dataset + from ray.data import Datastream class DataIteratorImpl(DataIterator): def __init__( self, - base_dataset: "Dataset", + base_datastream: "Datastream", ): - self._base_dataset = base_dataset + self._base_datastream = base_datastream self._base_context = DataContext.get_current() def __repr__(self) -> str: - return f"DataIterator({self._base_dataset})" + return f"DataIterator({self._base_datastream})" def _to_block_iterator( self, ) -> Tuple[ - Iterator[Tuple[ObjectRef[Block], BlockMetadata]], Optional[DatasetStats], bool + Iterator[Tuple[ObjectRef[Block], BlockMetadata]], + Optional[DatastreamStats], + bool, ]: - ds = self._base_dataset + ds = self._base_datastream block_iterator, stats, executor = ds._plan.execute_to_iterator() ds._current_executor = executor return block_iterator, stats, False def stats(self) -> str: - return self._base_dataset.stats() + return self._base_datastream.stats() def schema(self) -> Union[type, "pyarrow.lib.Schema"]: - return self._base_dataset.schema() + return self._base_datastream.schema() def __getattr__(self, name): - if name == "_base_dataset": + if name == "_base_datastream": raise AttributeError() - if hasattr(self._base_dataset, name) and not name.startswith("_"): + if hasattr(self._base_datastream, name) and not name.startswith("_"): # Raise error for backwards compatibility. # TODO: remove this method in 2.6. raise DeprecationWarning( "session.get_dataset_shard returns a ray.data.DataIterator " - "instead of a Dataset/DatasetPipeline as of Ray v2.3. " + "instead of a Datastream/DatasetPipeline as of Ray v2.3. " "Use iter_torch_batches(), to_tf(), or iter_batches() to " "iterate over one epoch. See " "https://docs.ray.io/en/latest/data/api/dataset_iterator.html " diff --git a/python/ray/data/_internal/dataset_iterator/pipelined_dataset_iterator.py b/python/ray/data/_internal/dataset_iterator/pipelined_dataset_iterator.py index a8be9ad2fc64..573116ff2c54 100644 --- a/python/ray/data/_internal/dataset_iterator/pipelined_dataset_iterator.py +++ b/python/ray/data/_internal/dataset_iterator/pipelined_dataset_iterator.py @@ -3,7 +3,7 @@ from ray.types import ObjectRef from ray.data.block import Block, BlockMetadata, DataBatch from ray.data.dataset_iterator import DataIterator -from ray.data._internal.stats import DatasetStats +from ray.data._internal.stats import DatastreamStats if TYPE_CHECKING: import pyarrow @@ -21,7 +21,7 @@ def __init__( def __repr__(self) -> str: return f"DataIterator({self._base_dataset_pipeline})" - def _get_next_dataset(self) -> "DatasetPipeline": + def _get_next_datastream(self) -> "DatasetPipeline": if self._epoch_iterator is None: self._epoch_iterator = self._base_dataset_pipeline.iter_epochs() @@ -31,17 +31,19 @@ def _get_next_dataset(self) -> "DatasetPipeline": def _to_block_iterator( self, ) -> Tuple[ - Iterator[Tuple[ObjectRef[Block], BlockMetadata]], Optional[DatasetStats], bool + Iterator[Tuple[ObjectRef[Block], BlockMetadata]], + Optional[DatastreamStats], + bool, ]: - epoch_pipeline = self._get_next_dataset() + epoch_pipeline = self._get_next_datastream() - # Peek the first dataset from the pipeline to see if blocks are owned + # Peek the first datastream from the pipeline to see if blocks are owned # by consumer. If so, the blocks are safe to be eagerly cleared after use # because memories are not shared across different consumers. This will # improve the memory efficiency. - if epoch_pipeline._first_dataset is not None: + if epoch_pipeline._first_datastream is not None: blocks_owned_by_consumer = ( - epoch_pipeline._first_dataset._plan.execute()._owned_by_consumer + epoch_pipeline._first_datastream._plan.execute()._owned_by_consumer ) else: blocks_owned_by_consumer = ( @@ -94,7 +96,7 @@ def __getattr__(self, name): # TODO: remove this method in 2.6. raise DeprecationWarning( "session.get_dataset_shard returns a ray.data.DataIterator " - "instead of a Dataset/DatasetPipeline as of Ray v2.3. " + "instead of a Datastream/DatasetPipeline as of Ray v2.3. " "Use iter_torch_batches(), to_tf(), or iter_batches() to " "iterate over one epoch. See " "https://docs.ray.io/en/latest/data/api/dataset_iterator.html " diff --git a/python/ray/data/_internal/dataset_iterator/stream_split_dataset_iterator.py b/python/ray/data/_internal/dataset_iterator/stream_split_dataset_iterator.py index a4625d0ca37b..4029f6a609c0 100644 --- a/python/ray/data/_internal/dataset_iterator/stream_split_dataset_iterator.py +++ b/python/ray/data/_internal/dataset_iterator/stream_split_dataset_iterator.py @@ -23,14 +23,14 @@ ) from ray.data._internal.execution.operators.output_splitter import OutputSplitter from ray.data._internal.execution.interfaces import NodeIdStr, RefBundle -from ray.data._internal.stats import DatasetStats +from ray.data._internal.stats import DatastreamStats from ray.types import ObjectRef from ray.util.debug import log_once from ray.util.scheduling_strategies import NodeAffinitySchedulingStrategy if TYPE_CHECKING: import pyarrow - from ray.data import Dataset + from ray.data import Datastream logger = logging.getLogger(__name__) @@ -43,14 +43,14 @@ class StreamSplitDataIterator(DataIterator): @staticmethod def create( - base_dataset: "Dataset", + base_datastream: "Datastream", n: int, equal: bool, locality_hints: Optional[List[NodeIdStr]], ) -> List["StreamSplitDataIterator"]: - """Create a split iterator from the given base Dataset and options. + """Create a split iterator from the given base Datastream and options. - See also: `Dataset.streaming_split`. + See also: `Datastream.streaming_split`. """ ctx = DataContext.get_current() @@ -60,24 +60,28 @@ def create( scheduling_strategy=NodeAffinitySchedulingStrategy( ray.get_runtime_context().get_node_id(), soft=False ), - ).remote(ctx, base_dataset, n, equal, locality_hints) + ).remote(ctx, base_datastream, n, equal, locality_hints) - return [StreamSplitDataIterator(base_dataset, coord_actor, i) for i in range(n)] + return [ + StreamSplitDataIterator(base_datastream, coord_actor, i) for i in range(n) + ] def __init__( self, - base_dataset: "Dataset", + base_datastream: "Datastream", coord_actor: ray.actor.ActorHandle, output_split_idx: int, ): - self._base_dataset = base_dataset + self._base_datastream = base_datastream self._coord_actor = coord_actor self._output_split_idx = output_split_idx def _to_block_iterator( self, ) -> Tuple[ - Iterator[Tuple[ObjectRef[Block], BlockMetadata]], Optional[DatasetStats], bool + Iterator[Tuple[ObjectRef[Block], BlockMetadata]], + Optional[DatastreamStats], + bool, ]: def gen_blocks() -> Iterator[Tuple[ObjectRef[Block], BlockMetadata]]: cur_epoch = ray.get( @@ -102,11 +106,11 @@ def gen_blocks() -> Iterator[Tuple[ObjectRef[Block], BlockMetadata]]: def stats(self) -> str: """Implements DataIterator.""" - return self._base_dataset.stats() + return self._base_datastream.stats() def schema(self) -> Union[type, "pyarrow.lib.Schema"]: """Implements DataIterator.""" - return self._base_dataset.schema() + return self._base_datastream.schema() @ray.remote(num_cpus=0) @@ -120,7 +124,7 @@ class SplitCoordinator: def __init__( self, ctx: DataContext, - dataset: "Dataset", + datastream: "Datastream", n: int, equal: bool, locality_hints: Optional[List[NodeIdStr]], @@ -131,7 +135,7 @@ def __init__( logger.info(f"Auto configuring locality_with_output={locality_hints}") DataContext._set_current(ctx) - self._base_dataset = dataset + self._base_datastream = datastream self._n = n self._equal = equal self._locality_hints = locality_hints @@ -151,9 +155,9 @@ def add_split_op(dag): output_iterator = execute_to_legacy_bundle_iterator( executor, - dataset._plan, + datastream._plan, True, - dataset._plan._dataset_uuid, + datastream._plan._datastream_uuid, dag_rewrite=add_split_op, ) yield output_iterator diff --git a/python/ray/data/_internal/dataset_logger.py b/python/ray/data/_internal/dataset_logger.py index d11a3a0c6236..8ed7dde89602 100644 --- a/python/ray/data/_internal/dataset_logger.py +++ b/python/ray/data/_internal/dataset_logger.py @@ -5,16 +5,16 @@ from ray._private.ray_constants import LOGGER_FORMAT, LOGGER_LEVEL -class DatasetLogger: - """Logger for Ray Datasets which writes logs to a separate log file - at `DatasetLogger.DEFAULT_DATASET_LOG_PATH`. Can optionally turn off +class DatastreamLogger: + """Logger for Ray Datastreams which writes logs to a separate log file + at `DatastreamLogger.DEFAULT_DATASET_LOG_PATH`. Can optionally turn off logging to stdout to reduce clutter (but always logs to the aformentioned - Datasets-specific log file). + Datastreams-specific log file). After initialization, always use the `get_logger()` method to correctly set whether to log to stdout. Example usage: ``` - logger = DatasetLogger(__name__) + logger = DatastreamLogger(__name__) logger.get_logger().info("This logs to file and stdout") logger.get_logger(log_to_stdout=False).info("This logs to file only) logger.get_logger().warning("Can call the usual Logger methods") @@ -24,7 +24,7 @@ class DatasetLogger: DEFAULT_DATASET_LOG_PATH = "logs/ray-data.log" def __init__(self, log_name: str): - """Initialize DatasetLogger for a given `log_name`. + """Initialize DatastreamLogger for a given `log_name`. Args: log_name: Name of logger (usually passed into `logging.getLogger(...)`) @@ -40,7 +40,7 @@ def __init__(self, log_name: str): def _initialize_logger(self) -> logging.Logger: """Internal method to initialize the logger and the extra file handler - for writing to the Dataset log file. Not intended (nor should it be necessary) + for writing to the Datastream log file. Not intended (nor necessary) to call explicitly. Assumes that `ray.init()` has already been called prior to calling this method; otherwise raises a `ValueError`.""" @@ -62,20 +62,20 @@ def _initialize_logger(self) -> logging.Logger: # If ray.init() is called and the global node session directory path # is valid, we can create the additional handler to write to the - # Dataset log file. If this is not the case (e.g. when used in Ray + # Datastream log file. If this is not the case (e.g. when used in Ray # Client), then we skip initializing the FileHandler. global_node = ray._private.worker._global_node if global_node is not None: - # Add a FileHandler to write to the specific Ray Datasets log file - # at `DatasetLogger.DEFAULT_DATASET_LOG_PATH`, using the standard + # Add a FileHandler to write to the specific Ray Datastreams log file + # at `DatastreamLogger.DEFAULT_DATASET_LOG_PATH`, using the standard # default logger format used by the root logger session_dir = global_node.get_session_dir_path() - datasets_log_path = os.path.join( + datastreams_log_path = os.path.join( session_dir, - DatasetLogger.DEFAULT_DATASET_LOG_PATH, + DatastreamLogger.DEFAULT_DATASET_LOG_PATH, ) file_log_formatter = logging.Formatter(fmt=LOGGER_FORMAT) - file_log_handler = logging.FileHandler(datasets_log_path) + file_log_handler = logging.FileHandler(datastreams_log_path) file_log_handler.setLevel(LOGGER_LEVEL.upper()) file_log_handler.setFormatter(file_log_formatter) logger.addHandler(file_log_handler) @@ -85,10 +85,10 @@ def get_logger(self, log_to_stdout: bool = True) -> logging.Logger: """ Returns the underlying Logger, with the `propagate` attribute set to the same value as `log_to_stdout`. For example, when - `log_to_stdout = False`, we do not want the `DatasetLogger` to + `log_to_stdout = False`, we do not want the `DatastreamLogger` to propagate up to the base Logger which writes to stdout. - This is a workaround needed due to the DatasetLogger wrapper object + This is a workaround needed due to the DatastreamLogger wrapper object not having access to the log caller's scope in Python <3.8. In the future, with Python 3.8 support, we can use the `stacklevel` arg, which allows the logger to fetch the correct calling file/line and diff --git a/python/ray/data/_internal/execution/autoscaling_requester.py b/python/ray/data/_internal/execution/autoscaling_requester.py index 7d14d6c6920d..92f6a5dd690c 100644 --- a/python/ray/data/_internal/execution/autoscaling_requester.py +++ b/python/ray/data/_internal/execution/autoscaling_requester.py @@ -19,7 +19,7 @@ @ray.remote(num_cpus=0, max_restarts=-1, max_task_retries=-1) class AutoscalingRequester: - """Actor to make resource requests to autoscaler for the datasets. + """Actor to make resource requests to autoscaler for the datastreams. The resource requests are set to timeout after RESOURCE_REQUEST_TIMEOUT seconds. For those live requests, we keep track of the last request made for each execution, diff --git a/python/ray/data/_internal/execution/bulk_executor.py b/python/ray/data/_internal/execution/bulk_executor.py index 4b36c9239da5..dc3ad42f9b58 100644 --- a/python/ray/data/_internal/execution/bulk_executor.py +++ b/python/ray/data/_internal/execution/bulk_executor.py @@ -9,18 +9,18 @@ RefBundle, PhysicalOperator, ) -from ray.data._internal.dataset_logger import DatasetLogger +from ray.data._internal.dataset_logger import DatastreamLogger from ray.data._internal.execution.operators.input_data_buffer import InputDataBuffer from ray.data._internal.progress_bar import ProgressBar -from ray.data._internal.stats import DatasetStats +from ray.data._internal.stats import DatastreamStats -logger = DatasetLogger(__name__) +logger = DatastreamLogger(__name__) class BulkExecutor(Executor): """A bulk (BSP) operator executor. - This implementation emulates the behavior of the legacy Datasets backend. It + This implementation emulates the behavior of the legacy Data backend. It is intended to be replaced by default by StreamingExecutor in the future. """ @@ -28,11 +28,11 @@ def __init__(self, options: ExecutionOptions): # Bulk executor always preserves order. options.preserve_order = True super().__init__(options) - self._stats: Optional[DatasetStats] = DatasetStats(stages={}, parent=None) + self._stats: Optional[DatastreamStats] = DatastreamStats(stages={}, parent=None) self._executed = False def execute( - self, dag: PhysicalOperator, initial_stats: Optional[DatasetStats] = None + self, dag: PhysicalOperator, initial_stats: Optional[DatastreamStats] = None ) -> Iterator[RefBundle]: """Synchronously executes the DAG via bottom-up recursive traversal.""" @@ -84,7 +84,7 @@ def execute_recursive(op: PhysicalOperator) -> List[RefBundle]: return OutputIterator(execute_recursive(dag)) - def get_stats(self) -> DatasetStats: + def get_stats(self) -> DatastreamStats: return self._stats diff --git a/python/ray/data/_internal/execution/interfaces.py b/python/ray/data/_internal/execution/interfaces.py index 2694f8343507..66278aa5108d 100644 --- a/python/ray/data/_internal/execution/interfaces.py +++ b/python/ray/data/_internal/execution/interfaces.py @@ -8,7 +8,7 @@ from ray.data._internal.logical.interfaces import Operator from ray.data._internal.memory_tracing import trace_deallocation from ray.data._internal.progress_bar import ProgressBar -from ray.data._internal.stats import DatasetStats, StatsDict +from ray.data._internal.stats import DatastreamStats, StatsDict from ray.data.block import Block, BlockMetadata from ray.data.context import DataContext from ray.types import ObjectRef @@ -250,7 +250,7 @@ class PhysicalOperator(Operator): output stream of RefBundles. Physical operators are stateful and non-serializable; they live on the driver side - of the Dataset only. + of the Datastream only. Here's a simple example of implementing a basic "Map" operator: @@ -295,7 +295,7 @@ def completed(self) -> bool: ) def get_stats(self) -> StatsDict: - """Return recorded execution stats for use with DatasetStats.""" + """Return recorded execution stats for use with DatastreamStats.""" raise NotImplementedError def get_metrics(self) -> Dict[str, int]: @@ -449,7 +449,7 @@ def incremental_resource_usage(self) -> ExecutionResources: class OutputIterator(Iterator[RefBundle]): """Iterator used to access the output of an Executor execution. - This is a blocking iterator. Datasets guarantees that all its iterators are + This is a blocking iterator. Datastreams guarantees that all its iterators are thread-safe (i.e., multiple threads can block on them at the same time). """ @@ -464,7 +464,7 @@ def get_next(self, output_split_idx: Optional[int] = None) -> RefBundle: Args: output_split_idx: The output split index to get results for. This arg is - only allowed for iterators created by `Dataset.streaming_split()`. + only allowed for iterators created by `Datastream.streaming_split()`. Raises: StopIteration if there are no more outputs to return. @@ -490,13 +490,13 @@ def __init__(self, options: ExecutionOptions): self._options = options def execute( - self, dag: PhysicalOperator, initial_stats: Optional[DatasetStats] = None + self, dag: PhysicalOperator, initial_stats: Optional[DatastreamStats] = None ) -> OutputIterator: """Start execution. Args: dag: The operator graph to execute. - initial_stats: The DatasetStats to prepend to the stats returned by the + initial_stats: The DatastreamStats to prepend to the stats returned by the executor. These stats represent actions done to compute inputs. """ raise NotImplementedError @@ -508,7 +508,7 @@ def shutdown(self): """ pass - def get_stats(self) -> DatasetStats: + def get_stats(self) -> DatastreamStats: """Return stats for the execution so far. This is generally called after `execute` has completed, but may be called diff --git a/python/ray/data/_internal/execution/legacy_compat.py b/python/ray/data/_internal/execution/legacy_compat.py index 10c8e70f8ddf..a343c5384a3b 100644 --- a/python/ray/data/_internal/execution/legacy_compat.py +++ b/python/ray/data/_internal/execution/legacy_compat.py @@ -13,7 +13,7 @@ from ray.types import ObjectRef from ray.data.block import Block, BlockMetadata, List from ray.data.datasource import ReadTask -from ray.data._internal.stats import StatsDict, DatasetStats +from ray.data._internal.stats import StatsDict, DatastreamStats from ray.data._internal.stage_impl import RandomizeBlocksStage from ray.data._internal.block_list import BlockList from ray.data._internal.lazy_block_list import LazyBlockList @@ -41,11 +41,11 @@ def execute_to_legacy_block_iterator( executor: Executor, plan: ExecutionPlan, allow_clear_input_blocks: bool, - dataset_uuid: str, + datastream_uuid: str, ) -> Iterator[Tuple[ObjectRef[Block], BlockMetadata]]: """Same as execute_to_legacy_bundle_iterator but returning blocks and metadata.""" bundle_iter = execute_to_legacy_bundle_iterator( - executor, plan, allow_clear_input_blocks, dataset_uuid + executor, plan, allow_clear_input_blocks, datastream_uuid ) for bundle in bundle_iter: for block, metadata in bundle.blocks: @@ -56,7 +56,7 @@ def execute_to_legacy_bundle_iterator( executor: Executor, plan: ExecutionPlan, allow_clear_input_blocks: bool, - dataset_uuid: str, + datastream_uuid: str, dag_rewrite=None, ) -> Iterator[RefBundle]: """Execute a plan with the new executor and return a bundle iterator. @@ -65,10 +65,10 @@ def execute_to_legacy_bundle_iterator( executor: The executor to use. plan: The legacy plan to execute. allow_clear_input_blocks: Whether the executor may consider clearing blocks. - dataset_uuid: UUID of the dataset for this execution. + datastream_uuid: UUID of the datastream for this execution. dag_rewrite: Callback that can be used to mutate the DAG prior to execution. This is currently used as a legacy hack to inject the OutputSplit operator - for `Dataset.streaming_split()`. + for `Datastream.streaming_split()`. Returns: The output as a bundle iterator. @@ -90,7 +90,7 @@ def execute_to_legacy_block_list( executor: Executor, plan: ExecutionPlan, allow_clear_input_blocks: bool, - dataset_uuid: str, + datastream_uuid: str, preserve_order: bool, ) -> BlockList: """Execute a plan with the new executor and translate it into a legacy block list. @@ -99,7 +99,7 @@ def execute_to_legacy_block_list( executor: The executor to use. plan: The legacy plan to execute. allow_clear_input_blocks: Whether the executor may consider clearing blocks. - dataset_uuid: UUID of the dataset for this execution. + datastream_uuid: UUID of the datastream for this execution. preserve_order: Whether to preserve order in execution. Returns: @@ -114,7 +114,7 @@ def execute_to_legacy_block_list( bundles = executor.execute(dag, initial_stats=stats) block_list = _bundles_to_block_list(bundles) # Set the stats UUID after execution finishes. - _set_stats_uuid_recursive(executor.get_stats(), dataset_uuid) + _set_stats_uuid_recursive(executor.get_stats(), datastream_uuid) return block_list @@ -123,7 +123,7 @@ def _get_execution_dag( plan: ExecutionPlan, allow_clear_input_blocks: bool, preserve_order: bool, -) -> Tuple[PhysicalOperator, DatasetStats]: +) -> Tuple[PhysicalOperator, DatastreamStats]: """Get the physical operators DAG from a plan.""" # Record usage of logical operators if available. if hasattr(plan, "_logical_plan") and plan._logical_plan is not None: @@ -145,7 +145,7 @@ def _get_execution_dag( return dag, stats -def _get_initial_stats_from_plan(plan: ExecutionPlan) -> DatasetStats: +def _get_initial_stats_from_plan(plan: ExecutionPlan) -> DatastreamStats: assert DataContext.get_current().optimizer_enabled if plan._snapshot_blocks is not None and not plan._snapshot_blocks.is_cleared(): return plan._snapshot_stats @@ -154,7 +154,7 @@ def _get_initial_stats_from_plan(plan: ExecutionPlan) -> DatasetStats: def _to_operator_dag( plan: ExecutionPlan, allow_clear_input_blocks: bool -) -> Tuple[PhysicalOperator, DatasetStats]: +) -> Tuple[PhysicalOperator, DatastreamStats]: """Translate a plan into an operator DAG for the new execution backend.""" blocks, stats, stages = plan._optimize() @@ -363,8 +363,8 @@ def _block_list_to_bundles(blocks: BlockList, owns_blocks: bool) -> List[RefBund return output -def _set_stats_uuid_recursive(stats: DatasetStats, dataset_uuid: str) -> None: - if not stats.dataset_uuid: - stats.dataset_uuid = dataset_uuid +def _set_stats_uuid_recursive(stats: DatastreamStats, datastream_uuid: str) -> None: + if not stats.datastream_uuid: + stats.datastream_uuid = datastream_uuid for parent in stats.parents or []: - _set_stats_uuid_recursive(parent, dataset_uuid) + _set_stats_uuid_recursive(parent, datastream_uuid) diff --git a/python/ray/data/_internal/execution/operators/actor_pool_map_operator.py b/python/ray/data/_internal/execution/operators/actor_pool_map_operator.py index 9876f9720d30..6ddb7584c42d 100644 --- a/python/ray/data/_internal/execution/operators/actor_pool_map_operator.py +++ b/python/ray/data/_internal/execution/operators/actor_pool_map_operator.py @@ -6,7 +6,7 @@ from ray.data.block import Block, BlockMetadata, _CallableClassProtocol from ray.data.context import DataContext, DEFAULT_SCHEDULING_STRATEGY from ray.data._internal.compute import ActorPoolStrategy -from ray.data._internal.dataset_logger import DatasetLogger +from ray.data._internal.dataset_logger import DatastreamLogger from ray.data._internal.execution.interfaces import ( RefBundle, ExecutionResources, @@ -24,7 +24,7 @@ from ray.types import ObjectRef from ray._raylet import ObjectRefGenerator -logger = DatasetLogger(__name__) +logger = DatastreamLogger(__name__) # Higher values here are better for prefetching and locality. It's ok for this to be # fairly high since streaming backpressure prevents us from overloading actors. @@ -56,7 +56,7 @@ def __init__( min_rows_per_bundle: The number of rows to gather per batch passed to the transform_fn, or None to use the block size. Setting the batch size is important for the performance of GPU-accelerated transform functions. - The actual rows passed may be less if the dataset is small. + The actual rows passed may be less if the datastream is small. ray_remote_args: Customize the ray remote args for this op's tasks. """ super().__init__( diff --git a/python/ray/data/_internal/execution/operators/input_data_buffer.py b/python/ray/data/_internal/execution/operators/input_data_buffer.py index b81f1031deae..fb0d686333db 100644 --- a/python/ray/data/_internal/execution/operators/input_data_buffer.py +++ b/python/ray/data/_internal/execution/operators/input_data_buffer.py @@ -11,7 +11,7 @@ class InputDataBuffer(PhysicalOperator): """Defines the input data for the operator DAG. - For example, this may hold cached blocks from a previous Dataset execution, or + For example, this may hold cached blocks from a previous Datastream execution, or the arguments for read tasks. """ diff --git a/python/ray/data/_internal/execution/operators/map_operator.py b/python/ray/data/_internal/execution/operators/map_operator.py index d76b6a32af93..5c580b5c7eac 100644 --- a/python/ray/data/_internal/execution/operators/map_operator.py +++ b/python/ray/data/_internal/execution/operators/map_operator.py @@ -95,7 +95,7 @@ def create( min_rows_per_bundle: The number of rows to gather per batch passed to the transform_fn, or None to use the block size. Setting the batch size is important for the performance of GPU-accelerated transform functions. - The actual rows passed may be less if the dataset is small. + The actual rows passed may be less if the datastream is small. ray_remote_args: Customize the ray remote args for this op's tasks. """ if compute_strategy is None: diff --git a/python/ray/data/_internal/execution/operators/output_splitter.py b/python/ray/data/_internal/execution/operators/output_splitter.py index 58a55dd68c6f..fe649b7fafb2 100644 --- a/python/ray/data/_internal/execution/operators/output_splitter.py +++ b/python/ray/data/_internal/execution/operators/output_splitter.py @@ -21,7 +21,7 @@ class OutputSplitter(PhysicalOperator): The output bundles of this operator will have a `bundle.output_split_idx` attr set to an integer from [0..n-1]. This operator tries to divide the rows evenly across output splits. If the `equal` option is set, the operator will furthermore - guarantee an exact split of rows across outputs, truncating the Dataset as needed. + guarantee an exact split of rows across outputs, truncating the Datastream. Implementation wise, this operator keeps an internal buffer of bundles. The buffer has a minimum size calculated to enable a good locality hit rate, as well as ensure diff --git a/python/ray/data/_internal/execution/operators/task_pool_map_operator.py b/python/ray/data/_internal/execution/operators/task_pool_map_operator.py index 89d51d7857ac..d72f1afe6df0 100644 --- a/python/ray/data/_internal/execution/operators/task_pool_map_operator.py +++ b/python/ray/data/_internal/execution/operators/task_pool_map_operator.py @@ -38,7 +38,7 @@ def __init__( min_rows_per_bundle: The number of rows to gather per batch passed to the transform_fn, or None to use the block size. Setting the batch size is important for the performance of GPU-accelerated transform functions. - The actual rows passed may be less if the dataset is small. + The actual rows passed may be less if the datastream is small. ray_remote_args: Customize the ray remote args for this op's tasks. """ super().__init__( diff --git a/python/ray/data/_internal/execution/operators/zip_operator.py b/python/ray/data/_internal/execution/operators/zip_operator.py index 3238948ff3e4..db2287042810 100644 --- a/python/ray/data/_internal/execution/operators/zip_operator.py +++ b/python/ray/data/_internal/execution/operators/zip_operator.py @@ -121,7 +121,7 @@ def _zip( total_right_rows = sum(right_block_rows) if total_left_rows != total_right_rows: raise ValueError( - "Cannot zip datasets of different number of rows: " + "Cannot zip datastreams of different number of rows: " f"{total_left_rows}, {total_right_rows}" ) diff --git a/python/ray/data/_internal/execution/streaming_executor.py b/python/ray/data/_internal/execution/streaming_executor.py index dc9ba8f69c9d..8dea5aa6c68c 100644 --- a/python/ray/data/_internal/execution/streaming_executor.py +++ b/python/ray/data/_internal/execution/streaming_executor.py @@ -6,7 +6,7 @@ import ray from ray.data.context import DataContext -from ray.data._internal.dataset_logger import DatasetLogger +from ray.data._internal.dataset_logger import DatastreamLogger from ray.data._internal.execution.interfaces import ( Executor, ExecutionOptions, @@ -30,9 +30,9 @@ get_or_create_autoscaling_requester_actor, ) from ray.data._internal.progress_bar import ProgressBar -from ray.data._internal.stats import DatasetStats +from ray.data._internal.stats import DatastreamStats -logger = DatasetLogger(__name__) +logger = DatastreamLogger(__name__) # Set this environment variable for detailed scheduler debugging logs. DEBUG_TRACE_SCHEDULING = "RAY_DATASET_TRACE_SCHEDULING" in os.environ @@ -43,17 +43,17 @@ class StreamingExecutor(Executor, threading.Thread): - """A streaming Dataset executor. + """A streaming Datastream executor. - This implementation executes Dataset DAGs in a fully streamed way. It runs + This implementation executes Datastream DAGs in a fully streamed way. It runs by setting up the operator topology, and then routing blocks through operators in a way that maximizes throughput under resource constraints. """ def __init__(self, options: ExecutionOptions): self._start_time: Optional[float] = None - self._initial_stats: Optional[DatasetStats] = None - self._final_stats: Optional[DatasetStats] = None + self._initial_stats: Optional[DatastreamStats] = None + self._final_stats: Optional[DatastreamStats] = None self._global_info: Optional[ProgressBar] = None self._execution_id = uuid.uuid4().hex @@ -73,7 +73,7 @@ def __init__(self, options: ExecutionOptions): threading.Thread.__init__(self) def execute( - self, dag: PhysicalOperator, initial_stats: Optional[DatasetStats] = None + self, dag: PhysicalOperator, initial_stats: Optional[DatastreamStats] = None ) -> Iterator[RefBundle]: """Executes the DAG using a streaming execution strategy. @@ -177,9 +177,9 @@ def get_stats(self): else: return self._generate_stats() - def _generate_stats(self) -> DatasetStats: + def _generate_stats(self) -> DatastreamStats: """Create a new stats object reflecting execution status so far.""" - stats = self._initial_stats or DatasetStats(stages={}, parent=None) + stats = self._initial_stats or DatastreamStats(stages={}, parent=None) for op in self._topology: if isinstance(op, InputDataBuffer): continue diff --git a/python/ray/data/_internal/fast_repartition.py b/python/ray/data/_internal/fast_repartition.py index 15874a34f139..1825c7bdbd5c 100644 --- a/python/ray/data/_internal/fast_repartition.py +++ b/python/ray/data/_internal/fast_repartition.py @@ -8,16 +8,16 @@ from ray.data._internal.progress_bar import ProgressBar from ray.data._internal.remote_fn import cached_remote_fn from ray.data._internal.shuffle_and_partition import _ShufflePartitionOp -from ray.data._internal.stats import DatasetStats +from ray.data._internal.stats import DatastreamStats def fast_repartition(blocks, num_blocks, ctx: Optional[TaskContext] = None): - from ray.data.dataset import Dataset + from ray.data.dataset import Datastream - wrapped_ds = Dataset( + wrapped_ds = Datastream( ExecutionPlan( blocks, - DatasetStats(stages={}, parent=None), + DatastreamStats(stages={}, parent=None), run_by_consumer=blocks._owned_by_consumer, ), 0, @@ -59,7 +59,7 @@ def fast_repartition(blocks, num_blocks, ctx: Optional[TaskContext] = None): owned_by_consumer = blocks._owned_by_consumer # Schema is safe to fetch here since we have already called - # get_internal_block_refs and executed the dataset. + # get_internal_block_refs and executed the datastream. schema = wrapped_ds.schema(fetch_if_missing=True) # Early-release memory. del splits, blocks, wrapped_ds @@ -84,8 +84,8 @@ def fast_repartition(blocks, num_blocks, ctx: Optional[TaskContext] = None): if schema is None: raise ValueError( - "Dataset is empty or cleared, can't determine the format of " - "the dataset." + "Datastream is empty or cleared, can't determine the format of " + "the datastream." ) elif isinstance(schema, type): builder = SimpleBlockBuilder() diff --git a/python/ray/data/_internal/lazy_block_list.py b/python/ray/data/_internal/lazy_block_list.py index bd3513dd3d90..23331ddd1773 100644 --- a/python/ray/data/_internal/lazy_block_list.py +++ b/python/ray/data/_internal/lazy_block_list.py @@ -7,7 +7,7 @@ from ray.data._internal.progress_bar import ProgressBar from ray.data._internal.remote_fn import cached_remote_fn from ray.data._internal.memory_tracing import trace_allocation -from ray.data._internal.stats import DatasetStats, _get_or_create_stats_actor +from ray.data._internal.stats import DatastreamStats, _get_or_create_stats_actor from ray.data._internal.util import _split_list from ray.data.block import ( Block, @@ -26,7 +26,7 @@ class LazyBlockList(BlockList): """A BlockList that submits tasks lazily on-demand. This BlockList is used for implementing read operations (e.g., to avoid - needing to read all files of a Dataset when the user is just wanting to + needing to read all files of a Datastream when the user is just wanting to .take() the first few rows or view the schema). """ @@ -58,7 +58,7 @@ def __init__( in cached_metadata represents the list of output blocks metadata per the read task. One task can produce multiple output blocks. ray_remote_args: Ray remote arguments for the read tasks. - stats_uuid: UUID for the dataset stats, used to group and fetch read task + stats_uuid: UUID for the datastream stats, used to group and fetch read task stats. If not provided, a new UUID will be created. """ self._tasks = tasks @@ -119,10 +119,10 @@ def get_metadata(self, fetch_if_missing: bool = False) -> List[BlockMetadata]: _, metadata = self._get_blocks_with_metadata() return metadata - def stats(self) -> DatasetStats: - """Create DatasetStats for this LazyBlockList.""" - return DatasetStats( - # Make a copy of metadata, as the DatasetStats may mutate it in-place. + def stats(self) -> DatastreamStats: + """Create DatastreamStats for this LazyBlockList.""" + return DatastreamStats( + # Make a copy of metadata, as the DatastreamStats may mutate it in-place. stages={"Read": self.get_metadata(fetch_if_missing=False).copy()}, parent=None, needs_stats_actor=True, @@ -315,7 +315,7 @@ def _get_blocks_with_metadata( if context.block_splitting_enabled: # If block splitting is enabled, fetch the partitions through generator. read_progress_bar = ProgressBar("Read progress", total=len(block_refs)) - # Handle duplicates (e.g. due to unioning the same dataset). + # Handle duplicates (e.g. due to unioning the same datastream). unique_refs = list(set(block_refs)) generators = read_progress_bar.fetch_until_complete(unique_refs) @@ -341,7 +341,7 @@ def _get_blocks_with_metadata( 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). + # Handle duplicates (e.g. due to unioning the same datastream). unique_meta_refs = set(meta_refs) metadata = read_progress_bar.fetch_until_complete(list(unique_meta_refs)) ref_to_data = { @@ -359,7 +359,7 @@ def compute_first_block(self): """Kick off computation for the first block in the list. This is useful if looking to support rapid lightweight interaction with a small - amount of the dataset. + amount of the datastream. """ if self._tasks: self._get_or_compute(0) @@ -385,7 +385,7 @@ def ensure_metadata_for_first_block(self) -> Optional[BlockMetadata]: try: block_partition_ref, metadata_ref = next(self._iter_block_partition_refs()) except (StopIteration, ValueError): - # Dataset is empty (no blocks) or was manually cleared. + # Datastream is empty (no blocks) or was manually cleared. pass else: # This blocks until the underlying read task is finished. diff --git a/python/ray/data/_internal/logical/interfaces.py b/python/ray/data/_internal/logical/interfaces.py index 7852369a1c83..0bd800a78681 100644 --- a/python/ray/data/_internal/logical/interfaces.py +++ b/python/ray/data/_internal/logical/interfaces.py @@ -7,7 +7,7 @@ class Operator: """Abstract class for operators. - Operators live on the driver side of the Dataset only. + Operators live on the driver side of the Datastream only. """ def __init__(self, name: str, input_dependencies: List["Operator"]): diff --git a/python/ray/data/_internal/logical/operators/all_to_all_operator.py b/python/ray/data/_internal/logical/operators/all_to_all_operator.py index 351a9c340efc..9dacd39ad5ec 100644 --- a/python/ray/data/_internal/logical/operators/all_to_all_operator.py +++ b/python/ray/data/_internal/logical/operators/all_to_all_operator.py @@ -20,7 +20,7 @@ def __init__( """ Args: name: Name for this operator. This is the name that will appear when - inspecting the logical plan of a Dataset. + inspecting the logical plan of a Datastream. input_op: The operator preceding this operator in the plan DAG. The outputs of `input_op` will be the inputs to this operator. num_outputs: The number of expected output bundles outputted by this diff --git a/python/ray/data/_internal/logical/operators/map_operator.py b/python/ray/data/_internal/logical/operators/map_operator.py index 0f41501ab72c..133e2ff59d1a 100644 --- a/python/ray/data/_internal/logical/operators/map_operator.py +++ b/python/ray/data/_internal/logical/operators/map_operator.py @@ -23,7 +23,7 @@ def __init__( """ Args: name: Name for this operator. This is the name that will appear when - inspecting the logical plan of a Dataset. + inspecting the logical plan of a Datastream. input_op: The operator preceding this operator in the plan DAG. The outputs of `input_op` will be the inputs to this operator. ray_remote_args: Args to provide to ray.remote. @@ -53,7 +53,7 @@ def __init__( """ Args: name: Name for this operator. This is the name that will appear when - inspecting the logical plan of a Dataset. + inspecting the logical plan of a Datastream. input_op: The operator preceding this operator in the plan DAG. The outputs of `input_op` will be the inputs to this operator. fn: User-defined function to be called. diff --git a/python/ray/data/_internal/memory_tracing.py b/python/ray/data/_internal/memory_tracing.py index b115fcb73290..27444fec697f 100644 --- a/python/ray/data/_internal/memory_tracing.py +++ b/python/ray/data/_internal/memory_tracing.py @@ -1,4 +1,4 @@ -"""Utility for debugging object store memory eager deletion in Datasets. +"""Utility for debugging object store memory eager deletion in Datastreams. NOTE: the performance overhead of tracing object allocation is fairly substantial. This is meant to use in unit test for debugging. Please do not enable in production, diff --git a/python/ray/data/_internal/null_aggregate.py b/python/ray/data/_internal/null_aggregate.py index 2a881b0260e7..dc1f969ea2c5 100644 --- a/python/ray/data/_internal/null_aggregate.py +++ b/python/ray/data/_internal/null_aggregate.py @@ -15,7 +15,7 @@ # aggregation of non-null values. # 2. Mix of values and nulls - ignore_nulls=False: Return None. # 3. All nulls: Return None. -# 4. Empty dataset: Return None. +# 4. Empty datastream: Return None. # # This is accomplished by checking rows for null values and by propagating nulls # if found AND if we're not ignoring them. If not ignoring nulls, in order to delineate diff --git a/python/ray/data/_internal/pandas_block.py b/python/ray/data/_internal/pandas_block.py index 32808bc476ef..d6c1bc1b807c 100644 --- a/python/ray/data/_internal/pandas_block.py +++ b/python/ray/data/_internal/pandas_block.py @@ -54,7 +54,7 @@ def lazy_import_pandas(): class PandasRow(TableRow): """ - Row of a tabular Dataset backed by a Pandas DataFrame block. + Row of a tabular Datastream backed by a Pandas DataFrame block. """ def __getitem__(self, key: str) -> Any: @@ -186,11 +186,11 @@ def schema(self) -> PandasBlockSchema: names=dtypes.index.tolist(), types=dtypes.values.tolist() ) # Column names with non-str types of a pandas DataFrame is not - # supported by Ray Dataset. + # supported by Ray Datastream. if any(not isinstance(name, str) for name in schema.names): raise ValueError( "A Pandas DataFrame with column names of non-str types" - " is not supported by Ray Dataset. Column names of this" + " is not supported by Ray Datastream. Column names of this" f" DataFrame: {schema.names!r}." ) return schema diff --git a/python/ray/data/_internal/pipeline_executor.py b/python/ray/data/_internal/pipeline_executor.py index 7d0ff5bcf496..8f59dbb4d685 100644 --- a/python/ray/data/_internal/pipeline_executor.py +++ b/python/ray/data/_internal/pipeline_executor.py @@ -6,7 +6,7 @@ import ray from ray.data.block import T from ray.data.context import DataContext -from ray.data.dataset import Dataset +from ray.data.dataset import Datastream from ray.data._internal.progress_bar import ProgressBar from ray.data._internal import progress_bar @@ -16,7 +16,7 @@ from ray.data.dataset_pipeline import DatasetPipeline -def pipeline_stage(fn: Callable[[], Dataset[T]]) -> Dataset[T]: +def pipeline_stage(fn: Callable[[], Datastream[T]]) -> Datastream[T]: # Force eager evaluation of all blocks in the pipeline stage. This # prevents resource deadlocks due to overlapping stage execution (e.g., # task -> actor stage). @@ -26,7 +26,7 @@ def pipeline_stage(fn: Callable[[], Dataset[T]]) -> Dataset[T]: class PipelineExecutor: def __init__(self, pipeline: "DatasetPipeline[T]"): self._pipeline: "DatasetPipeline[T]" = pipeline - self._stages: List[concurrent.futures.Future[Dataset[Any]]] = [None] * ( + self._stages: List[concurrent.futures.Future[Datastream[Any]]] = [None] * ( len(self._pipeline._optimized_stages) + 1 ) self._iter = iter(self._pipeline._base_iterable) @@ -162,7 +162,7 @@ def __init__( self, pipeline: "DatasetPipeline[T]", n: int, - splitter: Callable[[Dataset], List["Dataset[T]"]], + splitter: Callable[[Datastream], List["Datastream[T]"]], context: DataContext, ): DataContext._set_current(context) @@ -172,17 +172,17 @@ def __init__( self.splitter = splitter self.cur_splits = [None] * self.n - def next_dataset_if_ready(self, split_index: int) -> Optional[Dataset[T]]: + def next_datastream_if_ready(self, split_index: int) -> Optional[Datastream[T]]: # TODO(swang): This will hang if one of the consumers fails and is # re-executed from the beginning. To make this fault-tolerant, we need - # to make next_dataset_if_ready idempotent. - # Pull the next dataset once all splits are fully consumed. + # to make next_datastream_if_ready idempotent. + # Pull the next datastream once all splits are fully consumed. if all(s is None for s in self.cur_splits): ds = next(self.executor) self.cur_splits = self.splitter(ds) assert len(self.cur_splits) == self.n, (self.cur_splits, self.n) - # Return the dataset at the split index once per split. + # Return the datastream at the split index once per split. ret = self.cur_splits[split_index] self.cur_splits[split_index] = None return ret diff --git a/python/ray/data/_internal/plan.py b/python/ray/data/_internal/plan.py index 5e5960948fb3..40474b68f891 100644 --- a/python/ray/data/_internal/plan.py +++ b/python/ray/data/_internal/plan.py @@ -30,10 +30,10 @@ get_compute, is_task_compute, ) -from ray.data._internal.dataset_logger import DatasetLogger +from ray.data._internal.dataset_logger import DatastreamLogger from ray.data._internal.execution.interfaces import TaskContext from ray.data._internal.lazy_block_list import LazyBlockList -from ray.data._internal.stats import DatasetStats, DatasetStatsSummary +from ray.data._internal.stats import DatastreamStats, DatastreamStatsSummary from ray.data.block import Block from ray.data.context import DataContext from ray.util.debug import log_once @@ -47,11 +47,11 @@ INHERITABLE_REMOTE_ARGS = ["scheduling_strategy"] -logger = DatasetLogger(__name__) +logger = DatastreamLogger(__name__) class Stage: - """Represents a Dataset transform stage (e.g., map or shuffle).""" + """Represents a Datastream transform stage (e.g., map or shuffle).""" def __init__(self, name: str, num_blocks: Optional[int]): self.name = name @@ -79,7 +79,7 @@ def __str__(self): class ExecutionPlan: - """A lazy execution plan for a Dataset.""" + """A lazy execution plan for a Datastream.""" # Implementation Notes: # @@ -100,8 +100,8 @@ class ExecutionPlan: def __init__( self, in_blocks: BlockList, - stats: DatasetStats, - dataset_uuid=None, + stats: DatastreamStats, + datastream_uuid=None, *, run_by_consumer: bool, ): @@ -110,7 +110,7 @@ def __init__( Args: in_blocks: Base list of blocks. stats: Stats for the base blocks. - dataset_uuid: Dataset's UUID. + datastream_uuid: Datastream's UUID. run_by_consumer: Whether this plan is invoked to run by the consumption APIs (e.g. .iter_batches()). """ @@ -125,16 +125,16 @@ def __init__( # Cache of optimized stages. self._last_optimized_stages = None - self._dataset_uuid = dataset_uuid or uuid.uuid4().hex - if not stats.dataset_uuid: - stats.dataset_uuid = self._dataset_uuid + self._datastream_uuid = datastream_uuid or uuid.uuid4().hex + if not stats.datastream_uuid: + stats.datastream_uuid = self._datastream_uuid self._run_by_consumer = run_by_consumer def __repr__(self) -> str: return ( f"ExecutionPlan(" - f"dataset_uuid={self._dataset_uuid}, " + f"datastream_uuid={self._datastream_uuid}, " f"run_by_consumer={self._run_by_consumer}, " f"in_blocks={self._in_blocks}, " f"stages_before_snapshot={self._stages_before_snapshot}, " @@ -148,7 +148,7 @@ def get_plan_as_string(self, classname: str) -> str: Returns: The string representation of this execution plan. """ - # NOTE: this is used for Dataset.__repr__ to give a user-facing string + # NOTE: this is used for Datastream.__repr__ to give a user-facing string # representation. Ideally ExecutionPlan.__repr__ should be replaced with this # method as well. @@ -156,7 +156,7 @@ def get_plan_as_string(self, classname: str) -> str: # cheap. plan_str = "" num_stages = 0 - dataset_blocks = None + datastream_blocks = None if self._stages_after_snapshot: # Get string representation of each stage in reverse order. for stage in self._stages_after_snapshot[::-1]: @@ -181,17 +181,17 @@ def get_plan_as_string(self, classname: str) -> str: schema = self._get_unified_blocks_schema( self._snapshot_blocks, fetch_if_missing=False ) - dataset_blocks = self._snapshot_blocks + datastream_blocks = self._snapshot_blocks else: assert self._in_blocks is not None schema = self._get_unified_blocks_schema( self._in_blocks, fetch_if_missing=False ) - dataset_blocks = self._in_blocks + datastream_blocks = self._in_blocks else: # Get schema of output blocks. schema = self.schema(fetch_if_missing=False) - dataset_blocks = self._snapshot_blocks + datastream_blocks = self._snapshot_blocks if schema is None: schema_str = "Unknown schema" @@ -205,14 +205,14 @@ def get_plan_as_string(self, classname: str) -> str: schema_str.append(f"{n}: {t}") schema_str = ", ".join(schema_str) schema_str = "{" + schema_str + "}" - count = self._get_num_rows_from_blocks_metadata(dataset_blocks) + count = self._get_num_rows_from_blocks_metadata(datastream_blocks) if count is None: count = "?" - if dataset_blocks is None: + if datastream_blocks is None: num_blocks = "?" else: - num_blocks = dataset_blocks.initial_num_blocks() - dataset_str = "{}(num_blocks={}, num_rows={}, schema={})".format( + num_blocks = datastream_blocks.initial_num_blocks() + datastream_str = "{}(num_blocks={}, num_rows={}, schema={})".format( classname, num_blocks, count, schema_str ) @@ -221,9 +221,9 @@ def get_plan_as_string(self, classname: str) -> str: MIN_FIELD_LENGTH = 10 INDENT_STR = " " * 3 trailing_space = " " * (max(num_stages, 0) * 3) - if len(dataset_str) > SCHEMA_LINE_CHAR_LIMIT: + if len(datastream_str) > SCHEMA_LINE_CHAR_LIMIT: # If the resulting string representation exceeds the line char limit, - # first try breaking up each `Dataset` parameter into its own line + # first try breaking up each `Datastream` parameter into its own line # and check if each line fits within the line limit. We check the # `schema` param's length, since this is likely the longest string. schema_str_on_new_line = f"{trailing_space}{INDENT_STR}schema={schema_str}" @@ -253,7 +253,7 @@ def get_plan_as_string(self, classname: str) -> str: schema_str = ( "{\n" + schema_str + f"\n{trailing_space}{INDENT_STR}" + "}" ) - dataset_str = ( + datastream_str = ( f"{classname}(" f"\n{trailing_space}{INDENT_STR}num_blocks={num_blocks}," f"\n{trailing_space}{INDENT_STR}num_rows={count}," @@ -262,10 +262,10 @@ def get_plan_as_string(self, classname: str) -> str: ) if num_stages == 0: - plan_str = dataset_str + plan_str = datastream_str else: trailing_space = " " * ((num_stages - 1) * 3) - plan_str += f"{trailing_space}+- {dataset_str}" + plan_str += f"{trailing_space}+- {datastream_str}" return plan_str def with_stage(self, stage: "Stage") -> "ExecutionPlan": @@ -320,16 +320,16 @@ def deep_copy(self, preserve_uuid: bool = False) -> "ExecutionPlan": Returns: A deep copy of this execution plan. """ - dataset_uuid = None + datastream_uuid = None if preserve_uuid: - dataset_uuid = self._dataset_uuid + datastream_uuid = self._datastream_uuid in_blocks = self._in_blocks if isinstance(in_blocks, BlockList): in_blocks = in_blocks.copy() plan_copy = ExecutionPlan( in_blocks, copy.copy(self._in_stats), - dataset_uuid=dataset_uuid, + datastream_uuid=datastream_uuid, run_by_consumer=self._run_by_consumer, ) if self._snapshot_blocks: @@ -365,7 +365,7 @@ def schema( fetch_if_missing: Whether to execute the plan to fetch the schema. Returns: - The schema of the output dataset. + The schema of the output datastream. """ from ray.data._internal.stage_impl import RandomizeBlocksStage @@ -393,8 +393,8 @@ def schema( return None elif self._in_blocks is not None and self._snapshot_blocks is None: # If the plan only has input blocks, we execute it, so snapshot has output. - # This applies to newly created dataset. For example, initial dataset from - # read, and output datasets of Dataset.split(). + # This applies to newly created datastream. For example, initial datastream + # from read, and output datastreams of Datastream.split(). self.execute() # Snapshot is now guaranteed to be the output of the final stage or None. blocks = self._snapshot_blocks @@ -459,14 +459,14 @@ def meta_count(self) -> Optional[int]: This method will never trigger any computation. Returns: - The number of records of the result Dataset, or None. + The number of records of the result Datastream, or None. """ if self._stages_after_snapshot: return None elif self._in_blocks is not None and self._snapshot_blocks is None: # If the plan only has input blocks, we execute it, so snapshot has output. - # This applies to newly created dataset. For example, initial dataset from - # read, and output datasets of Dataset.split(). + # This applies to newly created datastream. For example, initial datastream + # from read, and output datastreams of Datastream.split(). self.execute() # Snapshot is now guaranteed to be the final block or None. return self._get_num_rows_from_blocks_metadata(self._snapshot_blocks) @@ -484,7 +484,7 @@ def execute_to_iterator( force_read: bool = False, ) -> Tuple[ Iterator[Tuple[ObjectRef[Block], BlockMetadata]], - DatasetStats, + DatastreamStats, Optional["Executor"], ]: """Execute this plan, returning an iterator. @@ -521,7 +521,7 @@ def execute_to_iterator( executor, self, allow_clear_input_blocks=allow_clear_input_blocks, - dataset_uuid=self._dataset_uuid, + datastream_uuid=self._datastream_uuid, ) # Since the generator doesn't run any code until we try to fetch the first # value, force execution of one bundle before we call get_stats(). @@ -548,14 +548,14 @@ def execute( preserve_order: Whether to preserve order in execution. Returns: - The blocks of the output dataset. + The blocks of the output datastream. """ context = DataContext.get_current() if not ray.available_resources().get("CPU"): if log_once("cpu_warning"): logger.get_logger().warning( "Warning: The Ray cluster currently does not have " - "any available CPUs. The Dataset job will hang unless more CPUs " + "any available CPUs. The Datastream job will hang unless more CPUs " "are freed up. A common reason is that cluster resources are " "used by Actors or Tune trials; see the following link " "for more details: " @@ -581,13 +581,13 @@ def execute( executor, self, allow_clear_input_blocks=allow_clear_input_blocks, - dataset_uuid=self._dataset_uuid, + datastream_uuid=self._datastream_uuid, preserve_order=preserve_order, ) # TODO(ekl) we shouldn't need to set this in the future once we move # to a fully lazy execution model, unless .materialize() is used. Th # reason we need it right now is since the user may iterate over a - # Dataset multiple times after fully executing it once. + # Datastream multiple times after fully executing it once. if not self._run_by_consumer: blocks._owned_by_consumer = False stats = executor.get_stats() @@ -616,7 +616,7 @@ def execute( stats = stats_builder.build_multistage(stage_info) else: stats = stats_builder.build(blocks) - stats.dataset_uuid = self._dataset_uuid + stats.datastream_uuid = self._datastream_uuid stats_summary_string = stats.to_summary().to_string( include_parent=False, ) @@ -627,7 +627,7 @@ def execute( # Set the snapshot to the output of the final stage. self._snapshot_blocks = blocks self._snapshot_stats = stats - self._snapshot_stats.dataset_uuid = self._dataset_uuid + self._snapshot_stats.datastream_uuid = self._datastream_uuid self._stages_before_snapshot += self._stages_after_snapshot self._stages_after_snapshot = [] if _is_lazy(self._snapshot_blocks) and force_read: @@ -652,16 +652,16 @@ def _clear_snapshot(self) -> None: ) self._stages_before_snapshot = [] - def stats(self) -> DatasetStats: + def stats(self) -> DatastreamStats: """Return stats for this plan. If the plan isn't executed, an empty stats object will be returned. """ if not self._snapshot_stats: - return DatasetStats(stages={}, parent=None) + return DatastreamStats(stages={}, parent=None) return self._snapshot_stats - def stats_summary(self) -> DatasetStatsSummary: + def stats_summary(self) -> DatastreamStatsSummary: return self.stats().to_summary() def _should_clear_input_blocks( @@ -686,7 +686,7 @@ def _should_clear_input_blocks( # execution plan, so we don't clear these. return False - def _optimize(self) -> Tuple[BlockList, DatasetStats, List[Stage]]: + def _optimize(self) -> Tuple[BlockList, DatastreamStats, List[Stage]]: """Apply stage fusion optimizations, returning an updated source block list and associated stats, and a set of optimized stages. """ @@ -699,7 +699,7 @@ def _optimize(self) -> Tuple[BlockList, DatasetStats, List[Stage]]: # If using a lazy datasource, rewrite read stage into one-to-one stage # so it can be fused into downstream stages. blocks, stats, stages = _rewrite_read_stages( - blocks, stats, stages, self._dataset_uuid + blocks, stats, stages, self._datastream_uuid ) stages = _fuse_one_to_one_stages(stages) self._last_optimized_stages = stages @@ -707,7 +707,7 @@ def _optimize(self) -> Tuple[BlockList, DatasetStats, List[Stage]]: def _get_source_blocks_and_stages( self, - ) -> Tuple[BlockList, DatasetStats, List[Stage]]: + ) -> Tuple[BlockList, DatastreamStats, List[Stage]]: """Get the source blocks, corresponding stats, and the stages for plan execution. @@ -1146,20 +1146,20 @@ def __call__( def _rewrite_read_stages( blocks: BlockList, - stats: DatasetStats, + stats: DatastreamStats, stages: List[Stage], - dataset_uuid: str, -) -> Tuple[BlockList, DatasetStats, List[Stage]]: + datastream_uuid: str, +) -> Tuple[BlockList, DatastreamStats, List[Stage]]: """Rewrites read stages into one-to-one stages, if needed.""" if _is_lazy(blocks) and stages: blocks, stats, stages = _rewrite_read_stage(blocks, stages) - stats.dataset_uuid = dataset_uuid + stats.datastream_uuid = datastream_uuid return blocks, stats, stages def _rewrite_read_stage( in_blocks: LazyBlockList, stages: List[Stage] -) -> Tuple[BlockList, DatasetStats, List[Stage]]: +) -> Tuple[BlockList, DatastreamStats, List[Stage]]: """Rewrite the read stage to a OneToOne stage over read tasks as input. For example, suppose the plan was [Read -> MapBatches(Fn)]. These stages cannot @@ -1214,7 +1214,7 @@ def block_fn( "tasks", remote_args, ) - stats = DatasetStats(stages={}, parent=None) + stats = DatastreamStats(stages={}, parent=None) stages.insert(0, stage) return block_list, stats, stages diff --git a/python/ray/data/_internal/planner/exchange/sort_task_spec.py b/python/ray/data/_internal/planner/exchange/sort_task_spec.py index 4ba8b0e8e430..4fa17cec8588 100644 --- a/python/ray/data/_internal/planner/exchange/sort_task_spec.py +++ b/python/ray/data/_internal/planner/exchange/sort_task_spec.py @@ -104,7 +104,7 @@ def sample_boundaries( sample_bar.close() del sample_results samples = [s for s in samples if len(s) > 0] - # The dataset is empty + # The datastream is empty if len(samples) == 0: return [None] * (num_reducers - 1) builder = DelegatingBlockBuilder() diff --git a/python/ray/data/_internal/planner/write.py b/python/ray/data/_internal/planner/write.py index b9a4e5251189..0ba317475b8a 100644 --- a/python/ray/data/_internal/planner/write.py +++ b/python/ray/data/_internal/planner/write.py @@ -8,7 +8,7 @@ def generate_write_fn( datasource: Datasource, **write_args ) -> Callable[[Iterator[Block], TaskContext], Iterator[Block]]: - # If the write op succeeds, the resulting Dataset is a list of + # If the write op succeeds, the resulting Datastream is a list of # WriteResult (one element per write task). Otherwise, an error will # be raised. The Datasource can handle execution outcomes with the # on_write_complete() and on_write_failed(). diff --git a/python/ray/data/_internal/progress_bar.py b/python/ray/data/_internal/progress_bar.py index 7bb04ef9e7a2..5cc9c6562544 100644 --- a/python/ray/data/_internal/progress_bar.py +++ b/python/ray/data/_internal/progress_bar.py @@ -60,7 +60,9 @@ def __init__( else: global needs_warning if needs_warning: - print("[dataset]: Run `pip install tqdm` to enable progress reporting.") + print( + "[datastream]: Run `pip install tqdm` to enable progress reporting." + ) needs_warning = False self._bar = None diff --git a/python/ray/data/_internal/remote_fn.py b/python/ray/data/_internal/remote_fn.py index 077008e0c5aa..4a6d93fb0938 100644 --- a/python/ray/data/_internal/remote_fn.py +++ b/python/ray/data/_internal/remote_fn.py @@ -10,7 +10,7 @@ def cached_remote_fn(fn: Any, **ray_remote_args) -> Any: """Lazily defines a ray.remote function. - This is used in Datasets to avoid circular import issues with ray.remote. + This is used in Datastreams to avoid circular import issues with ray.remote. (ray imports ray.data in order to allow ``ray.data.read_foo()`` to work, which means ray.remote cannot be used top-level in ray.data). diff --git a/python/ray/data/_internal/sort.py b/python/ray/data/_internal/sort.py index ac2f32409b6a..f31ea9a6bb7c 100644 --- a/python/ray/data/_internal/sort.py +++ b/python/ray/data/_internal/sort.py @@ -14,7 +14,7 @@ Merging: a merge task would receive a block from every worker that consists of items in a certain range. It then merges the sorted blocks into one sorted -block and becomes part of the new, sorted dataset. +block and becomes part of the new, sorted datastream. """ from typing import Any, Callable, List, Optional, Tuple, TypeVar, Union @@ -113,7 +113,7 @@ def sample_boundaries( sample_bar.close() del sample_results samples = [s for s in samples if len(s) > 0] - # The dataset is empty + # The datastream is empty if len(samples) == 0: return [None] * (num_reducers - 1) builder = DelegatingBlockBuilder() diff --git a/python/ray/data/_internal/stage_impl.py b/python/ray/data/_internal/stage_impl.py index 6053f932b7bd..01ee6345e177 100644 --- a/python/ray/data/_internal/stage_impl.py +++ b/python/ray/data/_internal/stage_impl.py @@ -26,11 +26,11 @@ ) if TYPE_CHECKING: - from ray.data import Dataset + from ray.data import Datastream class RepartitionStage(AllToAllStage): - """Implementation of `Dataset.repartition()`.""" + """Implementation of `Datastream.repartition()`.""" def __init__(self, num_blocks: int, shuffle: bool): if shuffle: @@ -94,7 +94,7 @@ def do_fast_repartition( class RandomizeBlocksStage(AllToAllStage): - """Implementation of `Dataset.randomize_blocks()`.""" + """Implementation of `Datastream.randomize_blocks()`.""" def __init__(self, seed: Optional[int]): self._seed = seed @@ -110,7 +110,7 @@ def do_randomize(self, block_list, *_): class RandomShuffleStage(AllToAllStage): - """Implementation of `Dataset.random_shuffle()`.""" + """Implementation of `Datastream.random_shuffle()`.""" def __init__( self, @@ -165,11 +165,11 @@ def do_shuffle( class ZipStage(AllToAllStage): - """Implementation of `Dataset.zip()`.""" + """Implementation of `Datastream.zip()`.""" - def __init__(self, other: "Dataset"): + def __init__(self, other: "Datastream"): def do_zip_all(block_list: BlockList, clear_input_blocks: bool, *_): - # Repartition other to align with the base dataset, and then zip together + # Repartition other to align with the base datastream, and then zip together # the blocks in parallel. # TODO(Clark): Port this to a streaming zip, e.g. push block pairs through # an actor that buffers and zips. @@ -188,8 +188,8 @@ def do_zip_all(block_list: BlockList, clear_input_blocks: bool, *_): ) inverted = False if sum(other_block_bytes) > sum(base_block_bytes): - # Make sure that other is the smaller dataset, so we minimize splitting - # work when aligning other with base. + # Make sure that other is the smaller datastream, so we minimize + # splitting work when aligning other with base. # TODO(Clark): Improve this heuristic for minimizing splitting work, # e.g. by generating the splitting plans for each route (via # _generate_per_block_split_indices) and choosing the plan that splits @@ -205,14 +205,14 @@ def do_zip_all(block_list: BlockList, clear_input_blocks: bool, *_): indices = list(itertools.accumulate(base_block_rows)) indices.pop(-1) - # Check that each dataset has the same number of rows. + # Check that each datastream has the same number of rows. # TODO(Clark): Support different number of rows via user-directed # dropping/padding. total_base_rows = sum(base_block_rows) total_other_rows = sum(other_block_rows) if total_base_rows != total_other_rows: raise ValueError( - "Cannot zip datasets of different number of rows: " + "Cannot zip datastreams of different number of rows: " f"{total_base_rows}, {total_other_rows}" ) @@ -311,16 +311,16 @@ def _do_zip( class SortStage(AllToAllStage): - """Implementation of `Dataset.sort()`.""" + """Implementation of `Datastream.sort()`.""" - def __init__(self, ds: "Dataset", key: Optional[KeyFn], descending: bool): + def __init__(self, ds: "Datastream", key: Optional[KeyFn], descending: bool): def do_sort( block_list, ctx: TaskContext, clear_input_blocks: bool, *_, ): - # Handle empty dataset. + # Handle empty datastream. if block_list.initial_num_blocks() == 0: return block_list, {} if clear_input_blocks: diff --git a/python/ray/data/_internal/stats.py b/python/ray/data/_internal/stats.py index c3627f089eda..c0621814a8a6 100644 --- a/python/ray/data/_internal/stats.py +++ b/python/ray/data/_internal/stats.py @@ -14,8 +14,8 @@ from ray.util.annotations import DeveloperAPI from ray.util.scheduling_strategies import NodeAffinitySchedulingStrategy -STATS_ACTOR_NAME = "datasets_stats_actor" -STATS_ACTOR_NAMESPACE = "_dataset_stats_actor" +STATS_ACTOR_NAME = "datastreams_stats_actor" +STATS_ACTOR_NAMESPACE = "_datastream_stats_actor" StatsDict = Dict[str, List[BlockMetadata]] @@ -76,24 +76,24 @@ def avg(self) -> float: return self._value / self._total_count if self._total_count else float("inf") -class _DatasetStatsBuilder: - """Helper class for building dataset stats. +class _DatastreamStatsBuilder: + """Helper class for building datastream stats. When this class is created, we record the start time. When build() is - called with the final blocks of the new dataset, the time delta is + called with the final blocks of the new datastream, the time delta is saved as part of the stats.""" def __init__( self, stage_name: str, - parent: "DatasetStats", + parent: "DatastreamStats", override_start_time: Optional[float], ): self.stage_name = stage_name self.parent = parent self.start_time = override_start_time or time.perf_counter() - def build_multistage(self, stages: StatsDict) -> "DatasetStats": + def build_multistage(self, stages: StatsDict) -> "DatastreamStats": stage_infos = {} for i, (k, v) in enumerate(stages.items()): capped_k = capfirst(k) @@ -104,7 +104,7 @@ def build_multistage(self, stages: StatsDict) -> "DatasetStats": stage_infos[self.stage_name.split("->")[-1] + capped_k] = v else: stage_infos[self.stage_name] = v - stats = DatasetStats( + stats = DatastreamStats( stages=stage_infos, parent=self.parent, base_name=self.stage_name, @@ -112,8 +112,8 @@ def build_multistage(self, stages: StatsDict) -> "DatasetStats": stats.time_total_s = time.perf_counter() - self.start_time return stats - def build(self, final_blocks: BlockList) -> "DatasetStats": - stats = DatasetStats( + def build(self, final_blocks: BlockList) -> "DatastreamStats": + stats = DatastreamStats( stages={self.stage_name: final_blocks.get_metadata()}, parent=self.parent, ) @@ -125,7 +125,7 @@ def build(self, final_blocks: BlockList) -> "DatasetStats": class _StatsActor: """Actor holding stats for blocks created by LazyBlockList. - This actor is shared across all datasets created in the same cluster. + This actor is shared across all datastreams created in the same cluster. In order to cap memory usage, we set a max number of stats to keep in the actor. When this limit is exceeded, the stats will be garbage collected in FIFO order. @@ -196,32 +196,32 @@ def _get_or_create_stats_actor(): ).remote() -class DatasetStats: - """Holds the execution times for a given Dataset. +class DatastreamStats: + """Holds the execution times for a given Datastream. - This object contains a reference to the parent Dataset's stats as well, - but not the Dataset object itself, to allow its blocks to be dropped from + This object contains a reference to the parent Datastream's stats as well, + but not the Datastream object itself, to allow its blocks to be dropped from memory.""" def __init__( self, *, stages: StatsDict, - parent: Union[Optional["DatasetStats"], List["DatasetStats"]], + parent: Union[Optional["DatastreamStats"], List["DatastreamStats"]], needs_stats_actor: bool = False, stats_uuid: str = None, base_name: str = None, ): - """Create dataset stats. + """Create datastream stats. Args: - stages: Dict of stages used to create this Dataset from the + stages: Dict of stages used to create this Datastream from the previous one. Typically one entry, e.g., {"map": [...]}. - parent: Reference to parent Dataset's stats, or a list of parents + parent: Reference to parent Datastream's stats, or a list of parents if there are multiple. - needs_stats_actor: Whether this Dataset's stats needs a stats actor for - stats collection. This is currently only used for Datasets using a lazy - datasource (i.e. a LazyBlockList). + needs_stats_actor: Whether this Datastream's stats needs a stats actor for + stats collection. This is currently only used for Datastreams using a + lazy datasource (i.e. a LazyBlockList). stats_uuid: The uuid for the stats, used to fetch the right stats from the stats actor. base_name: The name of the base operation for a multi-stage operation. @@ -230,20 +230,20 @@ def __init__( self.stages: StatsDict = stages if parent is not None and not isinstance(parent, list): parent = [parent] - self.parents: List["DatasetStats"] = parent or [] + self.parents: List["DatastreamStats"] = parent or [] self.number: int = ( 0 if not self.parents else max(p.number for p in self.parents) + 1 ) self.base_name = base_name - # TODO(ekl) deprecate and remove the notion of dataset UUID once we move + # TODO(ekl) deprecate and remove the notion of datastream UUID once we move # fully to streaming execution. - self.dataset_uuid: str = "unknown_uuid" + self.datastream_uuid: str = "unknown_uuid" self.time_total_s: float = 0 self.needs_stats_actor = needs_stats_actor self.stats_uuid = stats_uuid self._legacy_iter_batches = False - # Iteration stats, filled out if the user iterates over the dataset. + # Iteration stats, filled out if the user iterates over the datastream. self.iter_wait_s: Timer = Timer() self.iter_get_s: Timer = Timer() self.iter_next_batch_s: Timer = Timer() @@ -270,21 +270,21 @@ def stats_actor(self): def child_builder( self, name: str, override_start_time: Optional[float] = None - ) -> _DatasetStatsBuilder: + ) -> _DatastreamStatsBuilder: """Start recording stats for an op of the given name (e.g., map).""" - return _DatasetStatsBuilder(name, self, override_start_time) + return _DatastreamStatsBuilder(name, self, override_start_time) - def child_TODO(self, name: str) -> "DatasetStats": + def child_TODO(self, name: str) -> "DatastreamStats": """Placeholder for child ops not yet instrumented.""" - return DatasetStats(stages={name + "_TODO": []}, parent=self) + return DatastreamStats(stages={name + "_TODO": []}, parent=self) @staticmethod def TODO(): """Placeholder for ops not yet instrumented.""" - return DatasetStats(stages={"TODO": []}, parent=None) + return DatastreamStats(stages={"TODO": []}, parent=None) - def to_summary(self) -> "DatasetStatsSummary": - """Generate a `DatasetStatsSummary` object from the given `DatasetStats` + def to_summary(self) -> "DatastreamStatsSummary": + """Generate a `DatastreamStatsSummary` object from the given `DatastreamStats` object, which can be used to generate a summary string.""" if self.needs_stats_actor: ac = self.stats_actor @@ -330,12 +330,12 @@ def to_summary(self) -> "DatasetStatsSummary": stats_summary_parents = [] if self.parents is not None: stats_summary_parents = [p.to_summary() for p in self.parents] - return DatasetStatsSummary( + return DatastreamStatsSummary( stages_stats, iter_stats, stats_summary_parents, self.number, - self.dataset_uuid, + self.datastream_uuid, self.time_total_s, self.base_name, self.extra_metrics, @@ -344,12 +344,12 @@ def to_summary(self) -> "DatasetStatsSummary": @DeveloperAPI @dataclass -class DatasetStatsSummary: +class DatastreamStatsSummary: stages_stats: List["StageStatsSummary"] iter_stats: "IterStatsSummary" - parents: List["DatasetStatsSummary"] + parents: List["DatastreamStatsSummary"] number: int - dataset_uuid: str + datastream_uuid: str time_total_s: float base_name: str extra_metrics: Dict[str, Any] @@ -357,7 +357,7 @@ class DatasetStatsSummary: def to_string( self, already_printed: Optional[Set[str]] = None, include_parent: bool = True ) -> str: - """Return a human-readable summary of this Dataset's stats. + """Return a human-readable summary of this Datastream's stats. Args: already_printed: Set of stage IDs that have already had its stats printed @@ -365,7 +365,7 @@ def to_string( include_parent: If true, also include parent stats summary; otherwise, only log stats of the latest stage. Returns: - String with summary statistics for executing the Dataset. + String with summary statistics for executing the Datastream. """ if already_printed is None: already_printed = set() @@ -380,7 +380,7 @@ def to_string( if len(self.stages_stats) == 1: stage_stats_summary = self.stages_stats[0] stage_name = stage_stats_summary.stage_name - stage_uuid = self.dataset_uuid + stage_name + stage_uuid = self.datastream_uuid + stage_name out += "Stage {} {}: ".format(self.number, stage_name) if stage_uuid in already_printed: out += "[execution cached]\n" @@ -397,7 +397,7 @@ def to_string( ) for n, stage_stats_summary in enumerate(self.stages_stats): stage_name = stage_stats_summary.stage_name - stage_uuid = self.dataset_uuid + stage_name + stage_uuid = self.datastream_uuid + stage_name out += "\n" out += "\tSubstage {} {}: ".format(n, stage_name) if stage_uuid in already_printed: @@ -426,8 +426,8 @@ def __repr__(self, level=0) -> str: parent_stats = f"\n{parent_stats},\n{indent} " if parent_stats else "" extra_metrics = f"\n{extra_metrics}\n{indent} " if extra_metrics else "" return ( - f"{indent}DatasetStatsSummary(\n" - f"{indent} dataset_uuid={self.dataset_uuid},\n" + f"{indent}DatastreamStatsSummary(\n" + f"{indent} datastream_uuid={self.datastream_uuid},\n" f"{indent} base_name={self.base_name},\n" f"{indent} number={self.number},\n" f"{indent} extra_metrics={{{extra_metrics}}},\n" @@ -463,7 +463,7 @@ class StageStatsSummary: # Whether the stage associated with this StageStatsSummary object is a substage is_substage: bool # This is the total walltime of the entire stage, typically obtained from - # `DatasetStats.time_total_s`. An important distinction is that this is the + # `DatastreamStats.time_total_s`. An important distinction is that this is the # overall runtime of the stage, pulled from the stats actor, whereas the # computed walltimes in `self.wall_time` are calculated on a substage level. time_total_s: float @@ -730,7 +730,7 @@ class IterStatsSummary: block_time: Timer # Time spent in user code, in seconds user_time: Timer - # Total time taken by Dataset iterator, in seconds + # Total time taken by Datastream iterator, in seconds total_time: Timer # Num of blocks that are in local object store iter_blocks_local: int @@ -755,7 +755,7 @@ def to_string(self) -> str: or self.format_time.get() or self.collate_time.get() ): - out += "\nDataset iterator time breakdown:\n" + out += "\nDatastream iterator time breakdown:\n" if self.block_time.get(): out += "* Total time user code is blocked: {}\n".format( fmt(self.block_time.get()) @@ -822,7 +822,7 @@ def to_string_legacy(self) -> str: or self.format_time.get() or self.get_time.get() ): - out += "\nDataset iterator time breakdown:\n" + out += "\nDatastream iterator time breakdown:\n" out += "* In ray.wait(): {}\n".format(fmt(self.wait_time.get())) out += "* In ray.get(): {}\n".format(fmt(self.get_time.get())) out += "* Num blocks local: {}\n".format(self.iter_blocks_local) @@ -854,16 +854,16 @@ def __repr__(self, level=0) -> str: class DatasetPipelineStats: - """Holds the execution times for a pipeline of Datasets.""" + """Holds the execution times for a pipeline of Datastreams.""" def __init__(self, *, max_history: int = 3): - """Create a dataset pipeline stats object. + """Create a datastream pipeline stats object. Args: - max_history: The max number of dataset window stats to track. + max_history: The max number of datastream window stats to track. """ self.max_history: int = max_history - self.history_buffer: List[Tuple[int, DatasetStats]] = [] + self.history_buffer: List[Tuple[int, DatastreamStats]] = [] self.count = 0 self.wait_time_s = [] @@ -887,7 +887,7 @@ def __getattr__(self, name): return self._iter_stats[name] raise AttributeError - def add(self, stats: DatasetStats) -> None: + def add(self, stats: DatastreamStats) -> None: """Called to add stats for a newly computed window.""" self.history_buffer.append((self.count, stats)) if len(self.history_buffer) > self.max_history: @@ -900,8 +900,8 @@ def add_pipeline_stats(self, other_stats: "DatasetPipelineStats"): `other_stats` should cover a disjoint set of windows than the current stats. """ - for _, dataset_stats in other_stats.history_buffer: - self.add(dataset_stats) + for _, datastream_stats in other_stats.history_buffer: + self.add(datastream_stats) self.wait_time_s.extend(other_stats.wait_time_s) @@ -918,7 +918,7 @@ def _summarize_iter(self) -> str: or self.iter_get_s.get() ): out += "\nDatasetPipeline iterator time breakdown:\n" - out += "* Waiting for next dataset: {}\n".format( + out += "* Waiting for next datastream: {}\n".format( fmt(self.iter_ds_wait_s.get()) ) out += "* In ray.wait(): {}\n".format(fmt(self.iter_wait_s.get())) @@ -947,7 +947,7 @@ def summary_string(self, exclude_first_window: bool = True) -> str: wait_time_s = self.wait_time_s[1 if exclude_first_window else 0 :] if wait_time_s: out += ( - "* Time stalled waiting for next dataset: " + "* Time stalled waiting for next datastream: " "{} min, {} max, {} mean, {} total\n".format( fmt(min(wait_time_s)), fmt(max(wait_time_s)), diff --git a/python/ray/data/_internal/util.py b/python/ray/data/_internal/util.py index 38a132c8fda7..0edd3ab43558 100644 --- a/python/ray/data/_internal/util.py +++ b/python/ray/data/_internal/util.py @@ -63,7 +63,7 @@ def _check_pyarrow_version(): if parse_version(version) < parse_version(MIN_PYARROW_VERSION): raise ImportError( - f"Datasets requires pyarrow >= {MIN_PYARROW_VERSION}, but " + f"Datastream requires pyarrow >= {MIN_PYARROW_VERSION}, but " f"{version} is installed. Reinstall with " f'`pip install -U "pyarrow"`. ' "If you want to disable this pyarrow version check, set the " @@ -74,7 +74,7 @@ def _check_pyarrow_version(): "You are using the 'pyarrow' module, but the exact version is unknown " "(possibly carried as an internal component by another module). Please " f"make sure you are using pyarrow >= {MIN_PYARROW_VERSION} to ensure " - "compatibility with Ray Datasets. " + "compatibility with Ray Datastream. " "If you want to disable this pyarrow version check, set the " f"environment variable {RAY_DISABLE_PYARROW_VERSION_CHECK}=1." ) @@ -103,7 +103,7 @@ def _autodetect_parallelism( Args: parallelism: The user-requested parallelism, or -1 for auto-detection. cur_pg: The current placement group, to be used for avail cpu calculation. - ctx: The current Dataset context to use for configs. + ctx: The current Datastream context to use for configs. reader: The datasource reader, to be used for data size estimation. avail_cpus: Override avail cpus detection (for testing only). @@ -141,7 +141,7 @@ def _autodetect_parallelism( def _estimate_avail_cpus(cur_pg: Optional["PlacementGroup"]) -> int: - """Estimates the available CPU parallelism for this Dataset in the cluster. + """Estimates the available CPU parallelism for this Datastream in the cluster. If we aren't in a placement group, this is trivially the number of CPUs in the cluster. Otherwise, we try to calculate how large the placement group is relative @@ -155,7 +155,7 @@ def _estimate_avail_cpus(cur_pg: Optional["PlacementGroup"]) -> int: # If we're in a placement group, we shouldn't assume the entire cluster's # resources are available for us to use. Estimate an upper bound on what's - # reasonable to assume is available for datasets to use. + # reasonable to assume is available for datastreams to use. if cur_pg: pg_cpus = 0 for bundle in cur_pg.bundle_specs: @@ -175,7 +175,7 @@ def _estimate_avail_cpus(cur_pg: Optional["PlacementGroup"]) -> int: def _estimate_available_parallelism() -> int: - """Estimates the available CPU parallelism for this Dataset in the cluster. + """Estimates the available CPU parallelism for this Datastream in the cluster. If we are currently in a placement group, take that into account.""" cur_pg = ray.util.get_current_placement_group() return _estimate_avail_cpus(cur_pg) @@ -343,18 +343,18 @@ def _consumption_api( insert_after=False, ): """Annotate the function with an indication that it's a consumption API, and that it - will trigger Datasets execution. + will trigger Datastream execution. """ base = ( " will trigger execution of the lazy transformations performed on " - "this dataset." + "this datastream." ) if delegate: message = delegate + base elif not if_more_than_read: message = "This operation" + base else: - condition = "If this dataset consists of more than a read, " + condition = "If this datastream consists of more than a read, " if datasource_metadata is not None: condition += ( f"or if the {datasource_metadata} can't be determined from the " @@ -379,7 +379,7 @@ def wrap(obj): def ConsumptionAPI(*args, **kwargs): """Annotate the function with an indication that it's a consumption API, and that it - will trigger Datasets execution. + will trigger Datastream execution. """ if len(args) == 1 and len(kwargs) == 0 and callable(args[0]): return _consumption_api()(args[0]) diff --git a/python/ray/data/context.py b/python/ray/data/context.py index f807b61a4316..4a83402de834 100644 --- a/python/ray/data/context.py +++ b/python/ray/data/context.py @@ -37,10 +37,10 @@ # TODO (kfstorm): Remove this once stable. DEFAULT_ENABLE_PANDAS_BLOCK = True -# Whether to enable stage-fusion optimizations for dataset pipelines. +# Whether to enable stage-fusion optimizations for datastream pipelines. DEFAULT_OPTIMIZE_FUSE_STAGES = True -# Whether to enable stage-reorder optimizations for dataset pipelines. +# Whether to enable stage-reorder optimizations for datastream pipelines. DEFAULT_OPTIMIZE_REORDER_STAGES = True # Whether to furthermore fuse read stages. diff --git a/python/ray/data/dataset.py b/python/ray/data/dataset.py index 0ff0798d4ace..0beaad4739b1 100644 --- a/python/ray/data/dataset.py +++ b/python/ray/data/dataset.py @@ -85,7 +85,7 @@ from ray.data._internal.progress_bar import ProgressBar from ray.data._internal.remote_fn import cached_remote_fn from ray.data._internal.split import _split_at_index, _split_at_indices, _get_num_rows -from ray.data._internal.stats import DatasetStats, DatasetStatsSummary +from ray.data._internal.stats import DatastreamStats, DatastreamStatsSummary from ray.data.aggregate import AggregateFn, Max, Mean, Min, Std, Sum from ray.data.block import ( VALID_BATCH_FORMATS, @@ -1454,7 +1454,7 @@ def split_at_indices(self, indices: List[int]) -> List["MaterializedDatastream[T parent_stats = self._plan.stats() splits = [] for bs, ms in zip(blocks, metadata): - stats = DatasetStats(stages={"Split": ms}, parent=parent_stats) + stats = DatastreamStats(stages={"Split": ms}, parent=parent_stats) stats.time_total_s = split_duration splits.append( MaterializedDatastream( @@ -1520,10 +1520,10 @@ def split_proportionately( if any(p <= 0 for p in proportions): raise ValueError("proportions must be bigger than 0") - dataset_length = self.count() + datastream_length = self.count() cumulative_proportions = np.cumsum(proportions) split_indices = [ - int(dataset_length * proportion) for proportion in cumulative_proportions + int(datastream_length * proportion) for proportion in cumulative_proportions ] # Ensure each split has at least one element @@ -1677,7 +1677,7 @@ def union(self, *other: List["Datastream[T]"]) -> "Datastream[T]": "number {} will be used. This warning will not " "be shown again.".format(set(epochs), max_epoch) ) - stats = DatasetStats( + stats = DatastreamStats( stages={"Union": []}, parent=[d._plan.stats() for d in datastreams], ) @@ -2210,11 +2210,11 @@ def limit(self, limit: int) -> "Datastream[T]": ) for m in metadata ] - dataset_stats = DatasetStats( + datastream_stats = DatastreamStats( stages={"Limit": meta_for_stats}, parent=self._plan.stats(), ) - dataset_stats.time_total_s = split_duration + datastream_stats.time_total_s = split_duration return Datastream( ExecutionPlan( BlockList( @@ -2222,7 +2222,7 @@ def limit(self, limit: int) -> "Datastream[T]": metadata, owned_by_consumer=block_list._owned_by_consumer, ), - dataset_stats, + datastream_stats, run_by_consumer=block_list._owned_by_consumer, ), self._epoch, @@ -2490,7 +2490,7 @@ def write_parquet( ParquetDatasource(), ray_remote_args=ray_remote_args, path=path, - dataset_uuid=self._uuid, + datastream_uuid=self._uuid, filesystem=filesystem, try_create_dir=try_create_dir, open_stream_args=arrow_open_stream_args, @@ -2554,7 +2554,7 @@ def write_json( JSONDatasource(), ray_remote_args=ray_remote_args, path=path, - dataset_uuid=self._uuid, + datastream_uuid=self._uuid, filesystem=filesystem, try_create_dir=try_create_dir, open_stream_args=arrow_open_stream_args, @@ -2615,7 +2615,7 @@ def write_csv( CSVDatasource(), ray_remote_args=ray_remote_args, path=path, - dataset_uuid=self._uuid, + datastream_uuid=self._uuid, filesystem=filesystem, try_create_dir=try_create_dir, open_stream_args=arrow_open_stream_args, @@ -2683,7 +2683,7 @@ def write_tfrecords( TFRecordDatasource(), ray_remote_args=ray_remote_args, path=path, - dataset_uuid=self._uuid, + datastream_uuid=self._uuid, filesystem=filesystem, try_create_dir=try_create_dir, open_stream_args=arrow_open_stream_args, @@ -2753,7 +2753,7 @@ def write_webdataset( WebDatasetDatasource(), ray_remote_args=ray_remote_args, path=path, - dataset_uuid=self._uuid, + datastream_uuid=self._uuid, filesystem=filesystem, try_create_dir=try_create_dir, open_stream_args=arrow_open_stream_args, @@ -2809,7 +2809,7 @@ def write_numpy( NumpyDatasource(), ray_remote_args=ray_remote_args, path=path, - dataset_uuid=self._uuid, + datastream_uuid=self._uuid, column=column, filesystem=filesystem, try_create_dir=try_create_dir, @@ -3858,7 +3858,7 @@ def repeat(self, times: Optional[int] = None) -> "DatasetPipeline[T]": outer_stats = self._plan.stats() read_stage = None uuid = self._get_uuid() - outer_stats.dataset_uuid = uuid + outer_stats.datastream_uuid = uuid if times is not None and times < 1: raise ValueError("`times` must be >= 1, got {}".format(times)) @@ -3878,7 +3878,10 @@ def __next__(self) -> Callable[[], "Datastream[T]"]: def gen(): ds = Datastream( ExecutionPlan( - blocks, outer_stats, dataset_uuid=uuid, run_by_consumer=True + blocks, + outer_stats, + datastream_uuid=uuid, + run_by_consumer=True, ), epoch, lazy=False, @@ -4145,7 +4148,7 @@ def stats(self) -> str: """ return self._get_stats_summary().to_string() - def _get_stats_summary(self) -> DatasetStatsSummary: + def _get_stats_summary(self) -> DatastreamStatsSummary: return self._plan.stats_summary() @ConsumptionAPI(pattern="Time complexity:") diff --git a/python/ray/data/dataset_iterator.py b/python/ray/data/dataset_iterator.py index 25421f1c415a..ee742ccadfd8 100644 --- a/python/ray/data/dataset_iterator.py +++ b/python/ray/data/dataset_iterator.py @@ -20,7 +20,7 @@ from ray.util.annotations import PublicAPI from ray.data._internal.block_batching import batch_block_refs from ray.data._internal.block_batching.iter_batches import iter_batches -from ray.data._internal.stats import DatasetStats +from ray.data._internal.stats import DatastreamStats from ray.data._internal.util import _is_tensor_schema if TYPE_CHECKING: @@ -31,8 +31,8 @@ from ray.data.dataset import TensorFlowTensorBatchType -def _is_tensor_dataset(schema) -> bool: - """Return ``True`` if this is an iterator over a tensor dataset.""" +def _is_tensor_datastream(schema) -> bool: + """Return ``True`` if this is an iterator over a tensor datastream.""" if schema is None or isinstance(schema, type): return False return _is_tensor_schema(schema.names) @@ -40,13 +40,13 @@ def _is_tensor_dataset(schema) -> bool: @PublicAPI(stability="beta") class DataIterator(abc.ABC): - """An iterator for reading items from a :class:`~Dataset` or + """An iterator for reading items from a :class:`~Datastream` or :class:`~DatasetPipeline`. - For Datasets, each iteration call represents a complete read of all items in the - Dataset. For DatasetPipelines, each iteration call represents one pass (epoch) - over the base Dataset. Note that for DatasetPipelines, each pass iterates over - the original Dataset, instead of a window (if ``.window()`` was used). + For Datastreams, each iteration call represents a complete read of all items in the + Datastream. For DatasetPipelines, each iteration call represents one pass (epoch) + over the base Datastream. Note that for DatasetPipelines, each pass iterates over + the original Datastream, instead of a window (if ``.window()`` was used). If using Ray AIR, each trainer actor should get its own iterator by calling :meth:`session.get_dataset_shard("train") @@ -67,22 +67,24 @@ class DataIterator(abc.ABC): .. tip:: For debugging purposes, use :meth:`~ray.air.util.check_ingest.make_local_dataset_iterator` to create a - local `DataIterator` from a :class:`~ray.data.Dataset`, a - :class:`~ray.data.Preprocessor`, and a :class:`~ray.air.DatasetConfig`. + local `DataIterator` from a :class:`~ray.data.Datastream`, a + :class:`~ray.data.Preprocessor`, and a :class:`~ray.air.DatastreamConfig`. """ @abc.abstractmethod def _to_block_iterator( self, ) -> Tuple[ - Iterator[Tuple[ObjectRef[Block], BlockMetadata]], Optional[DatasetStats], bool + Iterator[Tuple[ObjectRef[Block], BlockMetadata]], + Optional[DatastreamStats], + bool, ]: """Returns the iterator to use for `iter_batches`. Returns: A tuple. The first item of the tuple is an iterator over pairs of Block object references and their corresponding metadata. The second item of the - tuple is a DatasetStats object used for recording stats during iteration. + tuple is a DatastreamStats object used for recording stats during iteration. The third item is a boolean indicating if the blocks can be safely cleared after use. """ @@ -101,7 +103,7 @@ def iter_batches( # Deprecated. prefetch_blocks: int = 0, ) -> Iterator[DataBatch]: - """Return a local batched iterator over the dataset. + """Return a local batched iterator over the datastream. Examples: >>> import ray @@ -127,8 +129,8 @@ def iter_batches( Specify "default" to use the default block format (promoting tables to Pandas and tensors to NumPy), "pandas" to select ``pandas.DataFrame``, "pyarrow" to select ``pyarrow.Table``, or "numpy" - to select ``numpy.ndarray`` for tensor datasets and - ``Dict[str, numpy.ndarray]`` for tabular datasets, or None to return + to select ``numpy.ndarray`` for tensor datastreams and + ``Dict[str, numpy.ndarray]`` for tabular datastreams, or None to return the underlying block exactly as is with no additional formatting. The default is "default". drop_last: Whether to drop the last batch if it's incomplete. @@ -199,16 +201,16 @@ def drop_metadata(block_iterator): stats.iter_total_s.add(time.perf_counter() - time_start) def iter_rows(self, *, prefetch_blocks: int = 0) -> Iterator[Union[T, TableRow]]: - """Return a local row iterator over the dataset. + """Return a local row iterator over the datastream. - If the dataset is a tabular dataset (Arrow/Pandas blocks), dict-like mappings - :py:class:`~ray.data.row.TableRow` are yielded for each row by the iterator. - If the dataset is not tabular, the raw row is yielded. + If the datastream is a tabular datastream (Arrow/Pandas blocks), dict-like + mappings :py:class:`~ray.data.row.TableRow` are yielded for each row by the + iterator. If the datastream is not tabular, the raw row is yielded. Examples: >>> import ray - >>> dataset = ray.data.range(10) - >>> next(iter(dataset.iterator().iter_rows())) + >>> datastream = ray.data.range(10) + >>> next(iter(datastream.iterator().iter_rows())) 0 Time complexity: O(1) @@ -218,7 +220,7 @@ def iter_rows(self, *, prefetch_blocks: int = 0) -> Iterator[Union[T, TableRow]] current block during the scan. Returns: - An iterator over rows of the dataset. + An iterator over rows of the datastream. """ iter_batch_args = {"batch_size": None, "batch_format": None} @@ -241,7 +243,7 @@ def stats(self) -> str: @abc.abstractmethod def schema(self) -> Union[type, "pyarrow.lib.Schema"]: - """Return the schema of the dataset iterated over.""" + """Return the schema of the datastream iterated over.""" raise NotImplementedError def iter_torch_batches( @@ -260,9 +262,9 @@ def iter_torch_batches( # Deprecated. prefetch_blocks: int = 0, ) -> Iterator["TorchTensorBatchType"]: - """Return a local batched iterator of Torch Tensors over the dataset. + """Return a local batched iterator of Torch Tensors over the datastream. - This iterator will yield single-tensor batches if the underlying dataset + This iterator will yield single-tensor batches if the underlying datastream consists of a single column; otherwise, it will yield a dictionary of column-tensors. If looking for more flexibility in the tensor conversion (e.g. casting dtypes) or the batch format, try using `.iter_batches` directly. @@ -361,15 +363,15 @@ def iter_tf_batches( # Deprecated. prefetch_blocks: int = 0, ) -> Iterator["TensorFlowTensorBatchType"]: - """Return a local batched iterator of TensorFlow Tensors over the dataset. + """Return a local batched iterator of TensorFlow Tensors over the datastream. - This iterator will yield single-tensor batches of the underlying dataset + This iterator will yield single-tensor batches of the underlying datastream consists of a single column; otherwise, it will yield a dictionary of column-tensors. .. tip:: If you don't need the additional flexibility provided by this method, - consider using :meth:`~ray.data.Dataset.to_tf` instead. It's easier + consider using :meth:`~ray.data.Datastream.to_tf` instead. It's easier to use. Examples: @@ -447,9 +449,9 @@ def to_torch( # Deprecated. prefetch_blocks: int = 0, ) -> "torch.utils.data.IterableDataset": - """Return a Torch IterableDataset over this dataset. + """Return a Torch IterableDataset over this datastream. - This is only supported for datasets convertible to Arrow records. + This is only supported for datastreams convertible to Arrow records. It is recommended to use the returned ``IterableDataset`` directly instead of passing it into a torch ``DataLoader``. @@ -479,10 +481,10 @@ def to_torch( If ``unsqueeze_label_tensor=True`` (default), the label tensor will be of shape (N, 1). Otherwise, it will be of shape (N,). If ``label_column`` is specified as ``None``, then no column from the - ``Dataset`` will be treated as the label, and the output label tensor + ``Datastream`` will be treated as the label, and the output label tensor will be ``None``. - Note that you probably want to call ``.split()`` on this dataset if + Note that you probably want to call ``.split()`` on this datastream if there are to be multiple Torch workers consuming the data. Time complexity: O(1) @@ -513,8 +515,8 @@ def to_torch( prefetching behavior that uses `prefetch_blocks` by setting `use_legacy_iter_batches` to True in the DataContext. drop_last: Set to True to drop the last incomplete batch, - if the dataset size is not divisible by the batch size. If - False and the size of dataset is not divisible by the batch + if the datastream size is not divisible by the batch size. If + False and the size of datastream is not divisible by the batch size, then the last batch will be smaller. Defaults to False. local_shuffle_buffer_size: If non-None, the data will be randomly shuffled using a local in-memory shuffle buffer, and this value will serve as the @@ -635,10 +637,10 @@ def to_tf( # Deprecated. prefetch_blocks: int = 0, ) -> "tf.data.Dataset": - """Return a TF Dataset over this dataset. + """Return a TF Dataset over this datastream. .. warning:: - If your dataset contains ragged tensors, this method errors. To prevent + If your datastream contains ragged tensors, this method errors. To prevent errors, resize tensors or :ref:`disable tensor extension casting `. @@ -670,7 +672,7 @@ def to_tf( >>> it.to_tf(["sepal length (cm)", "sepal width (cm)"], "target") # doctest: +SKIP <_OptionsDataset element_spec=({'sepal length (cm)': TensorSpec(shape=(None,), dtype=tf.float64, name='sepal length (cm)'), 'sepal width (cm)': TensorSpec(shape=(None,), dtype=tf.float64, name='sepal width (cm)')}, TensorSpec(shape=(None,), dtype=tf.int64, name='target'))> - If your dataset contains multiple features but your model accepts a single + If your datastream contains multiple features but your model accepts a single tensor as input, combine features with :class:`~ray.data.preprocessors.Concatenator`. @@ -708,8 +710,8 @@ def to_tf( `use_legacy_iter_batches` to True in the DataContext. batch_size: Record batch size. Defaults to 1. drop_last: Set to True to drop the last incomplete batch, - if the dataset size is not divisible by the batch size. If - False and the size of dataset is not divisible by the batch + if the datastream size is not divisible by the batch size. If + False and the size of datastream is not divisible by the batch size, then the last batch will be smaller. Defaults to False. local_shuffle_buffer_size: If non-None, the data will be randomly shuffled using a local in-memory shuffle buffer, and this value will serve as the @@ -737,15 +739,15 @@ def to_tf( schema = self.schema() - if _is_tensor_dataset(schema): + if _is_tensor_datastream(schema): raise NotImplementedError( - "`to_tf` doesn't support single-column tensor datasets. Call the " + "`to_tf` doesn't support single-column tensor datastreams. Call the " "more-flexible `iter_batches` instead." ) if isinstance(schema, type): raise NotImplementedError( - "`to_tf` doesn't support simple datasets. Call `map_batches` and " + "`to_tf` doesn't support simple datastreams. Call `map_batches` and " "convert your data to a tabular format. Alternatively, call the more-" "flexible `iter_batches` in place of `to_tf`." ) @@ -757,7 +759,7 @@ def validate_column(column: str) -> None: raise ValueError( f"You specified '{column}' in `feature_columns` or " f"`label_columns`, but there's no column named '{column}' in the " - f"dataset. Valid column names are: {valid_columns}." + f"datastream. Valid column names are: {valid_columns}." ) def validate_columns(columns: Union[str, List]) -> None: @@ -808,7 +810,7 @@ def generator(): label_type_spec = get_type_spec(schema, columns=label_columns) output_signature = (feature_type_spec, label_type_spec) - dataset = tf.data.Dataset.from_generator( + datastream = tf.data.Dataset.from_generator( generator, output_signature=output_signature ) @@ -816,7 +818,7 @@ def generator(): options.experimental_distribute.auto_shard_policy = ( tf.data.experimental.AutoShardPolicy.OFF ) - return dataset.with_options(options) + return datastream.with_options(options) def iter_epochs(self, max_epoch: int = -1) -> None: raise DeprecationWarning( diff --git a/python/ray/data/dataset_pipeline.py b/python/ray/data/dataset_pipeline.py index e1bb70059661..b196b8788efe 100644 --- a/python/ray/data/dataset_pipeline.py +++ b/python/ray/data/dataset_pipeline.py @@ -32,10 +32,10 @@ PipelinedDataIterator, ) from ray.data._internal.plan import ExecutionPlan -from ray.data._internal.stats import DatasetPipelineStats, DatasetStats +from ray.data._internal.stats import DatasetPipelineStats, DatastreamStats from ray.data.block import BatchUDF, Block, DataBatch, KeyFn, RowUDF, T, U from ray.data.context import DataContext -from ray.data.dataset import Dataset +from ray.data.dataset import Datastream from ray.data.dataset_iterator import DataIterator from ray.data.datasource import Datasource from ray.data.datasource.file_based_datasource import ( @@ -65,26 +65,26 @@ @PublicAPI class DatasetPipeline(Generic[T]): - """Implements a pipeline of Datasets. + """Implements a pipeline of Datastreams. DatasetPipelines implement pipelined execution. This allows for the overlapped execution of data input (e.g., reading files), computation (e.g. feature preprocessing), and output (e.g., distributed ML training). - A DatasetPipeline can be created by either repeating a Dataset - (``ds.repeat(times=None)``), by turning a single Dataset into a pipeline + A DatasetPipeline can be created by either repeating a Datastream + (``ds.repeat(times=None)``), by turning a single Datastream into a pipeline (``ds.window(blocks_per_window=10)``), or defined explicitly using ``DatasetPipeline.from_iterable()``. - DatasetPipeline supports the all the per-record transforms of Datasets + DatasetPipeline supports the all the per-record transforms of Datastreams (e.g., map, flat_map, filter), holistic transforms (e.g., repartition), and output methods (e.g., iter_rows, to_tf, to_torch, write_datasource). """ def __init__( self, - base_iterable: Iterable[Callable[[], Dataset[T]]], - stages: List[Callable[[Dataset[Any]], Dataset[Any]]] = None, + base_iterable: Iterable[Callable[[], Datastream[T]]], + stages: List[Callable[[Datastream[Any]], Datastream[Any]]] = None, length: Optional[int] = None, progress_bars: bool = progress_bar._enabled, _executed: List[bool] = None, @@ -92,7 +92,7 @@ def __init__( """Construct a DatasetPipeline (internal API). The constructor is not part of the DatasetPipeline API. Use the - ``Dataset.repeat()``, ``Dataset.window()``, or + ``Datastream.repeat()``, ``Datastream.window()``, or ``DatasetPipeline.from_iterable()`` methods to construct a pipeline. """ self._base_iterable = base_iterable @@ -104,17 +104,19 @@ def __init__( # Whether the pipeline execution has started. # This variable is shared across all pipelines descending from this. self._executed = _executed or [False] - self._first_dataset: Optional[Dataset] = None - self._remaining_datasets_iter: Optional[Iterator[Callable[[], Dataset]]] = None + self._first_datastream: Optional[Datastream] = None + self._remaining_datastreams_iter: Optional[ + Iterator[Callable[[], Datastream]] + ] = None self._schema = None self._stats = DatasetPipelineStats() def iterator(self) -> DataIterator: """Return a :class:`~ray.data.DataIterator` that - can be used to repeatedly iterate over the dataset. + can be used to repeatedly iterate over the datastream. - Note that each pass iterates over the entire original Dataset, even if - the dataset was windowed with ``.window()``. + Note that each pass iterates over the entire original Datastream, even if + the datastream was windowed with ``.window()``. Examples: >>> import ray @@ -132,9 +134,9 @@ def iterator(self) -> DataIterator: def iter_rows(self, *, prefetch_blocks: int = 0) -> Iterator[Union[T, TableRow]]: """Return a local row iterator over the data in the pipeline. - If the dataset is a tabular dataset (Arrow/Pandas blocks), dict-like mappings - :py:class:`~ray.data.row.TableRow` are yielded for each row by the iterator. - If the dataset is not tabular, the raw row is yielded. + If the datastream is a tabular datastream (Arrow/Pandas blocks), dict-like + mappings :py:class:`~ray.data.row.TableRow` are yielded for each row by the + iterator. If the datastream is not tabular, the raw row is yielded. Examples: >>> import ray @@ -199,8 +201,8 @@ def iter_batches( batch_format: Specify ``"default"`` to use the default block format (promotes tables to Pandas and tensors to NumPy), ``"pandas"`` to select ``pandas.DataFrame``, "pyarrow" to select ``pyarrow.Table``, or - ``"numpy"`` to select ``numpy.ndarray`` for tensor datasets and - ``Dict[str, numpy.ndarray]`` for tabular datasets, or None to return + ``"numpy"`` to select ``numpy.ndarray`` for tensor datastreams and + ``Dict[str, numpy.ndarray]`` for tabular datastreams, or None to return the underlying block exactly as is with no additional formatting. The default is "default". drop_last: Whether to drop the last batch if it's incomplete. @@ -226,9 +228,9 @@ def iter_batches( if self._executed[0]: raise RuntimeError("Pipeline cannot be read multiple times.") time_start = time.perf_counter() - if self._first_dataset is not None: + if self._first_datastream is not None: blocks_owned_by_consumer = ( - self._first_dataset._plan.execute()._owned_by_consumer + self._first_datastream._plan.execute()._owned_by_consumer ) else: blocks_owned_by_consumer = self._peek()._plan.execute()._owned_by_consumer @@ -299,15 +301,15 @@ def split( ) def split_at_indices(self, indices: List[int]) -> List["DatasetPipeline[T]"]: - """Split the datasets within the pipeline at the given indices + """Split the datastreams within the pipeline at the given indices (like np.split). - This will split each dataset contained within this pipeline, thereby + This will split each datastream contained within this pipeline, thereby producing len(indices) + 1 pipelines with the first pipeline containing - the [0, indices[0]) slice from each dataset, the second pipeline - containing the [indices[0], indices[1]) slice from each dataset, and so + the [0, indices[0]) slice from each datastream, the second pipeline + containing the [indices[0], indices[1]) slice from each datastream, and so on, with the final pipeline will containing the - [indices[-1], self.count()) slice from each dataset. + [indices[-1], self.count()) slice from each datastream. Examples: >>> import ray @@ -343,7 +345,7 @@ def split_at_indices(self, indices: List[int]) -> List["DatasetPipeline[T]"]: return self._split(len(indices) + 1, lambda ds: ds.split_at_indices(indices)) def _split( - self, n: int, splitter: Callable[[Dataset], List["Dataset[T]"]] + self, n: int, splitter: Callable[[Datastream], List["Datastream[T]"]] ) -> List["DatasetPipeline[T]"]: ctx = DataContext.get_current() scheduling_strategy = ctx.scheduling_strategy @@ -378,7 +380,9 @@ def __next__(self): tries = 0 while ds is None: ds = ray.get( - self.coordinator.next_dataset_if_ready.remote(self.split_index) + self.coordinator.next_datastream_if_ready.remote( + self.split_index + ) ) # Wait for other shards to catch up reading. if not ds: @@ -411,13 +415,13 @@ def __next__(self): def rewindow( self, *, blocks_per_window: int, preserve_epoch: bool = True ) -> "DatasetPipeline[T]": - """Change the windowing (blocks per dataset) of this pipeline. + """Change the windowing (blocks per datastream) of this pipeline. Changes the windowing of this pipeline to the specified size. For - example, if the current pipeline has two blocks per dataset, and - `.rewindow(blocks_per_window=4)` is requested, adjacent datasets will - be merged until each dataset is 4 blocks. If - `.rewindow(blocks_per_window)` was requested the datasets will be + example, if the current pipeline has two blocks per datastream, and + `.rewindow(blocks_per_window=4)` is requested, adjacent datastreams will + be merged until each datastream is 4 blocks. If + `.rewindow(blocks_per_window)` was requested the datastreams will be split into smaller windows. Args: @@ -429,9 +433,9 @@ def rewindow( class WindowIterator: def __init__(self, original_iter): self._original_iter = original_iter - self._buffer: Optional[Dataset[T]] = None + self._buffer: Optional[Datastream[T]] = None - def __next__(self) -> Dataset[T]: + def __next__(self) -> Datastream[T]: try: # Merge windows until we meet the requested window size. if self._buffer is None: @@ -476,7 +480,7 @@ def __iter__(self): length = None # The newly created DatasetPipeline will contain a PipelineExecutor (because - # this will execute the pipeline so far to iter the datasets). In order to + # this will execute the pipeline so far to iter the datastreams). In order to # make this new DatasetPipeline serializable, we need to make sure the # PipelineExecutor has not been iterated. So this uses # _iter_datasets_without_peek() instead of iter_datasets(). @@ -513,7 +517,7 @@ def __init__(self, original_iter): # This is calculated later. self._max_i = None - def __next__(self) -> Callable[[], Dataset[T]]: + def __next__(self) -> Callable[[], Datastream[T]]: # Still going through the original pipeline. if self._original_iter: try: @@ -571,10 +575,10 @@ def __iter__(self): def schema( self, fetch_if_missing: bool = False ) -> Union[type, "pyarrow.lib.Schema"]: - """Return the schema of the dataset pipeline. + """Return the schema of the datastream pipeline. - For datasets of Arrow records, this will return the Arrow schema. - For dataset of Python objects, this returns their Python type. + For datastreams of Arrow records, this will return the Arrow schema. + For datastream of Python objects, this returns their Python type. Note: This is intended to be a method for peeking schema before the execution of DatasetPipeline. If execution has already started, @@ -596,7 +600,7 @@ def schema( return self._schema def dataset_format(self) -> BlockFormat: - """The format of the dataset pipeline's underlying data blocks. Possible + """The format of the datastream pipeline's underlying data blocks. Possible values are: "arrow", "pandas" and "simple". This may block; if the schema is unknown, this will synchronously fetch @@ -607,8 +611,8 @@ def dataset_format(self) -> BlockFormat: schema = self.schema(fetch_if_missing=True) if schema is None: raise ValueError( - "Dataset is empty or cleared, can't determine the format of " - "the dataset." + "Datastream is empty or cleared, can't determine the format of " + "the datastream." ) try: @@ -625,14 +629,14 @@ def dataset_format(self) -> BlockFormat: return BlockFormat.SIMPLE def count(self) -> int: - """Count the number of records in the dataset pipeline. + """Count the number of records in the datastream pipeline. This blocks until the entire pipeline is fully executed. - Time complexity: O(dataset size / parallelism) + Time complexity: O(datastream size / parallelism) Returns: - The number of records in the dataset pipeline. + The number of records in the datastream pipeline. """ if self._length == float("inf"): raise ValueError("Cannot count a pipeline of infinite length.") @@ -644,14 +648,14 @@ def count(self) -> int: return total def sum(self) -> int: - """Sum the records in the dataset pipeline. + """Sum the records in the datastream pipeline. This blocks until the entire pipeline is fully executed. - Time complexity: O(dataset size / parallelism) + Time complexity: O(datastream size / parallelism) Returns: - The sum of the records in the dataset pipeline. + The sum of the records in the datastream pipeline. """ if self._length == float("inf"): raise ValueError("Cannot sum a pipeline of infinite length.") @@ -662,14 +666,14 @@ def sum(self) -> int: total += elem return total - def show_windows(self, limit_per_dataset: int = 10) -> None: - """Print up to the given number of records from each window/dataset. + def show_windows(self, limit_per_datastream: int = 10) -> None: + """Print up to the given number of records from each window/datastream. This is helpful as a debugging tool for understanding the structure of - dataset pipelines. + datastream pipelines. Args: - limit_per_dataset: Rows to print per window/dataset. + limit_per_datastream: Rows to print per window/datastream. """ epoch = None for i, ds in enumerate(self.iter_datasets()): @@ -677,12 +681,12 @@ def show_windows(self, limit_per_dataset: int = 10) -> None: epoch = ds._get_epoch() print("------ Epoch {} ------".format(epoch)) print("=== Window {} ===".format(i)) - ds.show(limit_per_dataset) + ds.show(limit_per_datastream) def iter_epochs(self, max_epoch: int = -1) -> Iterator["DatasetPipeline[T]"]: """Split this pipeline up by epoch. - This allows reading of data per-epoch for repeated Datasets, which is + This allows reading of data per-epoch for repeated Datastreams, which is useful for ML training. For example, ``ray.data.range(10).repeat(50)`` generates a pipeline with 500 rows total split across 50 epochs. This method allows iterating over the data individually per epoch @@ -732,11 +736,11 @@ def __next__(self) -> T: return item class SingleEpochIterator: - def __init__(self, peekable_iter: Iterator[Dataset[T]], epoch: int): + def __init__(self, peekable_iter: Iterator[Datastream[T]], epoch: int): self._iter = peekable_iter self._epoch = epoch - def __next__(self) -> Dataset[T]: + def __next__(self) -> Datastream[T]: if self._iter.peek()._get_epoch() > self._epoch: raise StopIteration ds = next(self._iter) @@ -784,7 +788,7 @@ def map( compute: Union[str, ComputeStrategy] = None, **ray_remote_args, ) -> "DatasetPipeline[U]": - """Apply :py:meth:`Dataset.map ` to each dataset/window + """Apply :py:meth:`Datastream.map ` to each datastream/window in this pipeline.""" return self.foreach_window( lambda ds: ds.map(fn, compute=compute, **ray_remote_args) @@ -803,8 +807,8 @@ def map_batches( fn_constructor_kwargs: Optional[Dict[str, Any]] = None, **ray_remote_args, ) -> "DatasetPipeline[U]": - """Apply :py:meth:`Dataset.map_batches ` to each - dataset/window in this pipeline.""" + """Apply :py:meth:`Datastream.map_batches ` to each + datastream/window in this pipeline.""" return self.foreach_window( lambda ds: ds.map_batches( fn, @@ -826,8 +830,8 @@ def flat_map( compute: Union[str, ComputeStrategy] = None, **ray_remote_args, ) -> "DatasetPipeline[U]": - """Apply :py:meth:`Dataset.flat_map ` to each - dataset/window in this pipeline.""" + """Apply :py:meth:`Datastream.flat_map ` to each + datastream/window in this pipeline.""" return self.foreach_window( lambda ds: ds.flat_map(fn, compute=compute, **ray_remote_args) ) @@ -839,8 +843,8 @@ def filter( compute: Union[str, ComputeStrategy] = None, **ray_remote_args, ) -> "DatasetPipeline[T]": - """Apply :py:meth:`Dataset.filter ` to each - dataset/window in this pipeline.""" + """Apply :py:meth:`Datastream.filter ` to each + datastream/window in this pipeline.""" return self.foreach_window( lambda ds: ds.filter(fn, compute=compute, **ray_remote_args) ) @@ -853,8 +857,8 @@ def add_column( compute: Optional[str] = None, **ray_remote_args, ) -> "DatasetPipeline[U]": - """Apply :py:meth:`Dataset.add_column ` to each - dataset/window in this pipeline.""" + """Apply :py:meth:`Datastream.add_column ` to each + datastream/window in this pipeline.""" return self.foreach_window( lambda ds: ds.add_column(col, fn, compute=compute, **ray_remote_args) ) @@ -866,8 +870,8 @@ def drop_columns( compute: Optional[str] = None, **ray_remote_args, ) -> "DatasetPipeline[U]": - """Apply :py:meth:`Dataset.drop_columns ` to - each dataset/window in this pipeline.""" + """Apply :py:meth:`Datastream.drop_columns ` to + each datastream/window in this pipeline.""" return self.foreach_window( lambda ds: ds.drop_columns(cols, compute=compute, **ray_remote_args) ) @@ -879,8 +883,8 @@ def select_columns( compute: Optional[str] = None, **ray_remote_args, ) -> "DatasetPipeline[U]": - """Apply :py:meth:`Dataset.select_columns ` to - each dataset/window in this pipeline.""" + """Apply :py:meth:`Datastream.select_columns ` to + each datastream/window in this pipeline.""" return self.foreach_window( lambda ds: ds.select_columns(cols, compute=compute, **ray_remote_args) ) @@ -888,8 +892,8 @@ def select_columns( def repartition_each_window( self, num_blocks: int, *, shuffle: bool = False ) -> "DatasetPipeline[U]": - """Apply :py:meth:`Dataset.repartition ` to each - dataset/window in this pipeline.""" + """Apply :py:meth:`Datastream.repartition ` to each + datastream/window in this pipeline.""" return self.foreach_window( lambda ds: ds.repartition(num_blocks, shuffle=shuffle) ) @@ -901,8 +905,8 @@ def random_shuffle_each_window( num_blocks: Optional[int] = None, **ray_remote_args, ) -> "DatasetPipeline[U]": - """Apply :py:meth:`Dataset.random_shuffle ` to - each dataset/window in this pipeline.""" + """Apply :py:meth:`Datastream.random_shuffle ` to + each datastream/window in this pipeline.""" return self.foreach_window( lambda ds: ds.random_shuffle( seed=seed, num_blocks=num_blocks, **ray_remote_args @@ -912,15 +916,15 @@ def random_shuffle_each_window( def sort_each_window( self, key: Optional[KeyFn] = None, descending: bool = False ) -> "DatasetPipeline[U]": - """Apply :py:meth:`Dataset.sort ` to each dataset/window + """Apply :py:meth:`Datastream.sort ` to each datastream/window in this pipeline.""" return self.foreach_window(lambda ds: ds.sort(key, descending)) def randomize_block_order_each_window( self, *, seed: Optional[int] = None ) -> "DatasetPipeline[U]": - """Apply :py:meth:`Dataset.randomize_block_order - ` to each dataset/window in this + """Apply :py:meth:`Datastream.randomize_block_order + ` to each datastream/window in this pipeline.""" return self.foreach_window(lambda ds: ds.randomize_block_order(seed=seed)) @@ -936,9 +940,9 @@ def write_json( ray_remote_args: Dict[str, Any] = None, **pandas_json_args, ) -> None: - """Call :py:meth:`Dataset.write_json ` on each - output dataset of this pipeline.""" - self._write_each_dataset( + """Call :py:meth:`Datastream.write_json ` on each + output datastream of this pipeline.""" + self._write_each_datastream( lambda ds: ds.write_json( path, filesystem=filesystem, @@ -963,9 +967,9 @@ def write_csv( ray_remote_args: Dict[str, Any] = None, **arrow_csv_args, ) -> None: - """Call :py:meth:`Dataset.write_csv ` on each - output dataset of this pipeline.""" - self._write_each_dataset( + """Call :py:meth:`Datastream.write_csv ` on each + output datastream of this pipeline.""" + self._write_each_datastream( lambda ds: ds.write_csv( path, filesystem=filesystem, @@ -990,9 +994,9 @@ def write_parquet( ray_remote_args: Dict[str, Any] = None, **arrow_parquet_args, ) -> None: - """Call :py:meth:`Dataset.write_parquet ` on - each output dataset of this pipeline.""" - self._write_each_dataset( + """Call :py:meth:`Datastream.write_parquet ` on + each output datastream of this pipeline.""" + self._write_each_datastream( lambda ds: ds.write_parquet( path, filesystem=filesystem, @@ -1015,9 +1019,9 @@ def write_tfrecords( block_path_provider: BlockWritePathProvider = DefaultBlockWritePathProvider(), ray_remote_args: Dict[str, Any] = None, ) -> None: - """Call :py:meth:`Dataset.write_tfrecords ` on - each output dataset of this pipeline.""" - self._write_each_dataset( + """Call :py:meth:`Datastream.write_tfrecords ` on + each output datastream of this pipeline.""" + self._write_each_datastream( lambda ds: ds.write_tfrecords( path, filesystem=filesystem, @@ -1035,9 +1039,9 @@ def write_datasource( ray_remote_args: Dict[str, Any] = None, **write_args, ) -> None: - """Call :py:meth:`Dataset.write_datasource ` - on each output dataset of this pipeline.""" - self._write_each_dataset( + """Call :py:meth:`Datastream.write_datasource ` + on each output datastream of this pipeline.""" + self._write_each_datastream( lambda ds: ds.write_datasource( datasource, ray_remote_args=ray_remote_args, @@ -1046,19 +1050,19 @@ def write_datasource( ) def take(self, limit: int = 20) -> List[T]: - """Call :py:meth:`Dataset.take ` over the stream of + """Call :py:meth:`Datastream.take ` over the stream of output batches from the pipeline""" - return Dataset.take(self, limit) + return Datastream.take(self, limit) def take_all(self, limit: Optional[int] = None) -> List[T]: - """Call :py:meth:`Dataset.take_all ` over the stream + """Call :py:meth:`Datastream.take_all ` over the stream of output batches from the pipeline""" - return Dataset.take_all(self, limit) + return Datastream.take_all(self, limit) def show(self, limit: int = 20) -> None: - """Call :py:meth:`Dataset.show ` over the stream of + """Call :py:meth:`Datastream.show ` over the stream of output batches from the pipeline""" - return Dataset.show(self, limit) + return Datastream.show(self, limit) def iter_tf_batches( self, @@ -1071,7 +1075,7 @@ def iter_tf_batches( local_shuffle_seed: Optional[int] = None, ) -> Iterator[Union["tf.Tensor", Dict[str, "tf.Tensor"]]]: """Call - :py:meth:`Dataset.iter_tf_batches ` + :py:meth:`Datastream.iter_tf_batches ` over the stream of output batches from the pipeline.""" return DataIterator.iter_tf_batches( self, @@ -1097,8 +1101,9 @@ def iter_torch_batches( local_shuffle_seed: Optional[int] = None, ) -> Iterator["TorchTensorBatchType"]: """Call - :py:meth:`Dataset.iter_torch_batches ` - over the stream of output batches from the pipeline.""" + :py:meth:`Datastream.iter_torch_batches + ` over the stream of output batches + from the pipeline.""" return DataIterator.iter_torch_batches( self, prefetch_blocks=prefetch_blocks, @@ -1122,7 +1127,7 @@ def to_tf( local_shuffle_buffer_size: Optional[int] = None, local_shuffle_seed: Optional[int] = None, ) -> "tf.data.Dataset": - """Call :py:meth:`Dataset.to_tf ` over the stream of + """Call :py:meth:`Datastream.to_tf ` over the stream of output batches from the pipeline""" return DataIterator.to_tf( self, @@ -1152,7 +1157,7 @@ def to_torch( unsqueeze_label_tensor: bool = True, unsqueeze_feature_tensors: bool = True, ) -> "torch.utils.data.IterableDataset": - """Call :py:meth:`Dataset.to_torch ` over the stream + """Call :py:meth:`Datastream.to_torch ` over the stream of output batches from the pipeline""" return DataIterator.to_torch( self, @@ -1172,17 +1177,17 @@ def _iter_datasets_without_peek(self): if self._executed[0]: raise RuntimeError("Pipeline cannot be read multiple times.") self._executed[0] = True - if self._first_dataset: + if self._first_datastream: raise RuntimeError("The pipeline has been peeked.") self._optimize_stages() return PipelineExecutor(self) @DeveloperAPI - def iter_datasets(self) -> Iterator[Dataset[T]]: - """Iterate over the output datasets of this pipeline. + def iter_datasets(self) -> Iterator[Datastream[T]]: + """Iterate over the output datastreams of this pipeline. Returns: - Iterator over the datasets outputted from this pipeline. + Iterator over the datastreams outputted from this pipeline. """ if self._executed[0]: raise RuntimeError("Pipeline cannot be read multiple times.") @@ -1190,10 +1195,10 @@ def iter_datasets(self) -> Iterator[Dataset[T]]: self._optimize_stages() - # If the first dataset has already been executed (via a peek operation), then - # we don't re-execute the first dataset when iterating through the pipeline. - # We re-use the saved _first_dataset and _remaining_dataset_iter. - if self._first_dataset is not None: + # If the first datastream has already been executed (via a peek operation), then + # we don't re-execute the first datastream when iterating through the pipeline. + # We re-use the saved _first_datastream and _remaining_datastream_iter. + if self._first_datastream is not None: class _IterableWrapper(Iterable): """Wrapper that takes an iterator and converts it to an @@ -1205,26 +1210,26 @@ def __init__(self, base_iterator): def __iter__(self): return self.base_iterator - # Update the base iterable to skip the first dataset. + # Update the base iterable to skip the first datastream. # It is ok to update the base iterable here since # the pipeline can never be executed again. - self._base_iterable = _IterableWrapper(self._remaining_datasets_iter) + self._base_iterable = _IterableWrapper(self._remaining_datastreams_iter) - iter = itertools.chain([self._first_dataset], PipelineExecutor(self)) - self._first_dataset = None - self._remaining_datasets_iter = None + iter = itertools.chain([self._first_datastream], PipelineExecutor(self)) + self._first_datastream = None + self._remaining_datastreams_iter = None return iter else: return PipelineExecutor(self) @DeveloperAPI def foreach_window( - self, fn: Callable[[Dataset[T]], Dataset[U]] + self, fn: Callable[[Datastream[T]], Datastream[U]] ) -> "DatasetPipeline[U]": - """Apply a transform to each dataset/window in this pipeline. + """Apply a transform to each datastream/window in this pipeline. Args: - fn: The function to transform each dataset with. + fn: The function to transform each datastream with. Returns: The transformed DatasetPipeline. @@ -1253,13 +1258,13 @@ def stats(self, exclude_first_window: bool = True) -> str: @staticmethod def from_iterable( - iterable: Iterable[Callable[[], Dataset[T]]], + iterable: Iterable[Callable[[], Datastream[T]]], ) -> "DatasetPipeline[T]": - """Create a pipeline from an sequence of Dataset producing functions. + """Create a pipeline from an sequence of Datastream producing functions. Args: iterable: A finite or infinite-length sequence of functions that - each produce a Dataset when called. + each produce a Datastream when called. """ if hasattr(iterable, "__len__"): length = len(iterable) @@ -1289,22 +1294,22 @@ def _optimize_stages(self): self._optimized_stages = self._stages return - # This dummy dataset will be used to get a set of optimized stages. - dummy_ds = Dataset( + # This dummy datastream will be used to get a set of optimized stages. + dummy_ds = Datastream( ExecutionPlan( BlockList([], [], owned_by_consumer=True), - DatasetStats(stages={}, parent=None), + DatastreamStats(stages={}, parent=None), run_by_consumer=True, ), 0, True, ) - # Apply all pipeline operations to the dummy dataset. + # Apply all pipeline operations to the dummy datastream. 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. + # Apply these optimized stages to the datastreams underlying the pipeline. # These optimized stages will be executed by the PipelineExecutor. optimized_stages = [] for stage in stages: @@ -1314,31 +1319,33 @@ def add_stage(ds, stage): return ds._plan.with_stage(stage) optimized_stages.append( - lambda ds, stage=stage: Dataset(add_stage(ds, stage), ds._epoch, True) + lambda ds, stage=stage: Datastream( + add_stage(ds, stage), ds._epoch, True + ) ) self._optimized_stages = optimized_stages - def _peek(self) -> Dataset[T]: - if self._first_dataset is None: - dataset_iter = iter(self._base_iterable) - first_dataset_gen = next(dataset_iter) + def _peek(self) -> Datastream[T]: + if self._first_datastream is None: + datastream_iter = iter(self._base_iterable) + first_datastream_gen = next(datastream_iter) peek_pipe = DatasetPipeline( - base_iterable=[first_dataset_gen], + base_iterable=[first_datastream_gen], stages=self._stages.copy(), length=1, progress_bars=True, ) - # Cache the executed _first_dataset. - self._first_dataset = next(peek_pipe.iter_datasets()) - self._remaining_datasets_iter = dataset_iter + # Cache the executed _first_datastream. + self._first_datastream = next(peek_pipe.iter_datasets()) + self._remaining_datastreams_iter = datastream_iter # Store the stats from the peek pipeline. self._stats.add_pipeline_stats(peek_pipe._stats) - return self._first_dataset + return self._first_datastream - def _write_each_dataset(self, write_fn: Callable[[Dataset[T]], None]) -> None: - """Write output for each dataset. + def _write_each_datastream(self, write_fn: Callable[[Datastream[T]], None]) -> None: + """Write output for each datastream. This is utility method used for write_json, write_csv, write_parquet, write_datasource, etc. diff --git a/python/ray/data/datasource/datasource.py b/python/ray/data/datasource/datasource.py index 6e2f2e187e28..af32be89191e 100644 --- a/python/ray/data/datasource/datasource.py +++ b/python/ray/data/datasource/datasource.py @@ -24,10 +24,10 @@ @PublicAPI class Datasource(Generic[T]): - """Interface for defining a custom ``ray.data.Dataset`` datasource. + """Interface for defining a custom ``ray.data.Datastream`` datasource. - To read a datasource into a dataset, use ``ray.data.read_datasource()``. - To write to a writable datasource, use ``Dataset.write_datasource()``. + To read a datasource into a datastream, use ``ray.data.read_datasource()``. + To write to a writable datasource, use ``Datastream.write_datasource()``. See ``RangeDatasource`` and ``DummyOutputDatasource`` for examples of how to implement readable and writable datasources. @@ -135,7 +135,7 @@ class Reader(Generic[T]): """A bound read operation for a datasource. This is a stateful class so that reads can be prepared in multiple stages. - For example, it is useful for Datasets to know the in-memory size of the read + For example, it is useful for Datastreams to know the in-memory size of the read prior to executing it. """ @@ -175,7 +175,7 @@ def get_read_tasks(self, parallelism: int) -> List["ReadTask[T]"]: @DeveloperAPI class ReadTask(Callable[[], Iterable[Block]]): - """A function used to read blocks from the dataset. + """A function used to read blocks from the datastream. Read tasks are generated by ``reader.get_read_tasks()``, and return a list of ``ray.data.Block`` when called. Initial metadata about the read diff --git a/python/ray/data/datasource/file_based_datasource.py b/python/ray/data/datasource/file_based_datasource.py index 2641131f81c2..03f514fadc50 100644 --- a/python/ray/data/datasource/file_based_datasource.py +++ b/python/ray/data/datasource/file_based_datasource.py @@ -63,7 +63,7 @@ @DeveloperAPI class BlockWritePathProvider: """Abstract callable that provides concrete output paths when writing - dataset blocks. + datastream blocks. Current subclasses: DefaultBlockWritePathProvider @@ -74,32 +74,32 @@ def _get_write_path_for_block( base_path: str, *, filesystem: Optional["pyarrow.fs.FileSystem"] = None, - dataset_uuid: Optional[str] = None, + datastream_uuid: Optional[str] = None, block: Optional[Block] = None, block_index: Optional[int] = None, file_format: Optional[str] = None, ) -> str: """ - Resolves and returns the write path for the given dataset block. When + Resolves and returns the write path for the given datastream block. When implementing this method, care should be taken to ensure that a unique - path is provided for every dataset block. + path is provided for every datastream block. Args: - base_path: The base path to write the dataset block out to. This is - expected to be the same for all blocks in the dataset, and may + base_path: The base path to write the datastream block out to. This is + expected to be the same for all blocks in the datastream, and may point to either a directory or file prefix. filesystem: The filesystem implementation that will be used to write a file out to the write path returned. - dataset_uuid: Unique identifier for the dataset that this block + datastream_uuid: Unique identifier for the datastream that this block belongs to. block: The block to write. block_index: Ordered index of the block to write within its parent - dataset. + datastream. file_format: File format string for the block that can be used as the file extension in the write path returned. Returns: - The dataset block write path. + The datastream block write path. """ raise NotImplementedError @@ -108,7 +108,7 @@ def __call__( base_path: str, *, filesystem: Optional["pyarrow.fs.FileSystem"] = None, - dataset_uuid: Optional[str] = None, + datastream_uuid: Optional[str] = None, block: Optional[Block] = None, block_index: Optional[int] = None, file_format: Optional[str] = None, @@ -116,7 +116,7 @@ def __call__( return self._get_write_path_for_block( base_path, filesystem=filesystem, - dataset_uuid=dataset_uuid, + datastream_uuid=datastream_uuid, block=block, block_index=block_index, file_format=file_format, @@ -126,8 +126,8 @@ def __call__( @DeveloperAPI class DefaultBlockWritePathProvider(BlockWritePathProvider): """Default block write path provider implementation that writes each - dataset block out to a file of the form: - {base_path}/{dataset_uuid}_{block_index}.{file_format} + datastream block out to a file of the form: + {base_path}/{datastream_uuid}_{block_index}.{file_format} """ def _get_write_path_for_block( @@ -135,12 +135,12 @@ def _get_write_path_for_block( base_path: str, *, filesystem: Optional["pyarrow.fs.FileSystem"] = None, - dataset_uuid: Optional[str] = None, + datastream_uuid: Optional[str] = None, block: Optional[ObjectRef[Block]] = None, block_index: Optional[int] = None, file_format: Optional[str] = None, ) -> str: - suffix = f"{dataset_uuid}_{block_index:06}.{file_format}" + suffix = f"{datastream_uuid}_{block_index:06}.{file_format}" # Uses POSIX path for cross-filesystem compatibility, since PyArrow # FileSystem paths are always forward slash separated, see: # https://arrow.apache.org/docs/python/filesystems.html @@ -277,7 +277,7 @@ def write( blocks: Iterable[Block], ctx: TaskContext, path: str, - dataset_uuid: str, + datastream_uuid: str, filesystem: Optional["pyarrow.fs.FileSystem"] = None, try_create_dir: bool = True, open_stream_args: Optional[Dict[str, Any]] = None, @@ -332,7 +332,7 @@ def write_block(write_path: str, block: Block): write_path = block_path_provider( path, filesystem=filesystem, - dataset_uuid=dataset_uuid, + datastream_uuid=datastream_uuid, block=block, block_index=ctx.task_idx, file_format=file_format, diff --git a/python/ray/data/datasource/file_meta_provider.py b/python/ray/data/datasource/file_meta_provider.py index 972053d77bab..1242d6ba5aab 100644 --- a/python/ray/data/datasource/file_meta_provider.py +++ b/python/ray/data/datasource/file_meta_provider.py @@ -25,7 +25,7 @@ @DeveloperAPI class FileMetadataProvider: - """Abstract callable that provides metadata for the files of a single dataset block. + """Abstract callable that provides metadata for the files of a single datastream block. Current subclasses: BaseFileMetadataProvider @@ -40,10 +40,10 @@ def _get_block_metadata( ) -> BlockMetadata: """Resolves and returns block metadata for files in the given paths. - All file paths provided should belong to a single dataset block. + All file paths provided should belong to a single datastream block. Args: - paths: The file paths for a single dataset block. + paths: The file paths for a single datastream block. schema: The user-provided or inferred schema for the given paths, if any. @@ -80,10 +80,10 @@ def _get_block_metadata( rows_per_file: Optional[int], file_sizes: List[Optional[int]], ) -> BlockMetadata: - """Resolves and returns block metadata for files of a single dataset block. + """Resolves and returns block metadata for files of a single datastream block. Args: - paths: The file paths for a single dataset block. These + paths: The file paths for a single datastream block. These paths will always be a subset of those previously returned from `expand_paths()`. schema: The user-provided or inferred schema for the given file @@ -206,7 +206,7 @@ def expand_paths( class ParquetMetadataProvider(FileMetadataProvider): """Abstract callable that provides block metadata for Arrow Parquet file fragments. - All file fragments should belong to a single dataset block. + All file fragments should belong to a single datastream block. Supports optional pre-fetching of ordered metadata for all file fragments in a single batch to help optimize metadata resolution. @@ -223,10 +223,10 @@ def _get_block_metadata( pieces: List["pyarrow.dataset.ParquetFileFragment"], prefetched_metadata: Optional[List[Any]], ) -> BlockMetadata: - """Resolves and returns block metadata for files of a single dataset block. + """Resolves and returns block metadata for files of a single datastream block. Args: - paths: The file paths for a single dataset block. + paths: The file paths for a single datastream block. schema: The user-provided or inferred schema for the given file paths, if any. pieces: The Parquet file fragments derived from the input file paths. @@ -269,7 +269,7 @@ class DefaultParquetMetadataProvider(ParquetMetadataProvider): """The default file metadata provider for ParquetDatasource. Aggregates total block bytes and number of rows using the Parquet file metadata - associated with a list of Arrow Parquet dataset file fragments. + associated with a list of Arrow Parquet datastream file fragments. """ def _get_block_metadata( diff --git a/python/ray/data/datasource/parquet_datasource.py b/python/ray/data/datasource/parquet_datasource.py index c6a06b2611fd..d9fa6f7ad992 100644 --- a/python/ray/data/datasource/parquet_datasource.py +++ b/python/ray/data/datasource/parquet_datasource.py @@ -41,19 +41,19 @@ # compared to Parquet encoded representation. Parquet file statistics only record # encoded (i.e. uncompressed) data size information. # -# To estimate real-time in-memory data size, Datasets will try to estimate the correct -# inflation ratio from Parquet to Arrow, using this constant as the default value for -# safety. See https://github.com/ray-project/ray/pull/26516 for more context. +# To estimate real-time in-memory data size, Datastreams will try to estimate the +# correct inflation ratio from Parquet to Arrow, using this constant as the default +# value for safety. See https://github.com/ray-project/ray/pull/26516 for more context. PARQUET_ENCODING_RATIO_ESTIMATE_DEFAULT = 5 # The lower bound size to estimate Parquet encoding ratio. PARQUET_ENCODING_RATIO_ESTIMATE_LOWER_BOUND = 2 -# The percentage of files (1% by default) to be sampled from the dataset to estimate +# The percentage of files (1% by default) to be sampled from the datastream to estimate # Parquet encoding ratio. PARQUET_ENCODING_RATIO_ESTIMATE_SAMPLING_RATIO = 0.01 -# The minimal and maximal number of file samples to take from the dataset to estimate +# The minimal and maximal number of file samples to take from the datastream to estimate # Parquet encoding ratio. # This is to restrict `PARQUET_ENCODING_RATIO_ESTIMATE_SAMPLING_RATIO` within the # proper boundary. @@ -146,8 +146,8 @@ class ParquetDatasource(ParquetBaseDatasource): """Parquet datasource, for reading and writing Parquet files. The primary difference from ParquetBaseDatasource is that this uses - PyArrow's `ParquetDataset` abstraction for dataset reads, and thus offers - automatic Arrow dataset schema inference and row count collection at the + PyArrow's `ParquetDataset` abstraction for datastream reads, and thus offers + automatic Arrow datastream schema inference and row count collection at the cost of some potential performance and/or compatibility penalties. Examples: @@ -202,7 +202,10 @@ def __init__( dataset_kwargs = reader_args.pop("dataset_kwargs", {}) try: pq_ds = pq.ParquetDataset( - paths, **dataset_kwargs, filesystem=filesystem, use_legacy_dataset=False + paths, + **dataset_kwargs, + filesystem=filesystem, + use_legacy_dataset=False, ) except OSError as e: _handle_read_os_error(e, paths) @@ -214,14 +217,14 @@ def __init__( ) if _block_udf is not None: - # Try to infer dataset schema by passing dummy table through UDF. + # Try to infer datastream schema by passing dummy table through UDF. dummy_table = schema.empty_table() try: inferred_schema = _block_udf(dummy_table).schema inferred_schema = inferred_schema.with_metadata(schema.metadata) except Exception: logger.debug( - "Failed to infer schema of dataset by passing dummy table " + "Failed to infer schema of datastream by passing dummy table " "through UDF due to the following exception:", exc_info=True, ) diff --git a/python/ray/data/datasource/partitioning.py b/python/ray/data/datasource/partitioning.py index 30462514c36a..0554a50c4a77 100644 --- a/python/ray/data/datasource/partitioning.py +++ b/python/ray/data/datasource/partitioning.py @@ -17,7 +17,7 @@ @DeveloperAPI class PartitionStyle(str, Enum): - """Supported dataset partition styles. + """Supported datastream partition styles. Inherits from `str` to simplify plain text serialization/deserialization. @@ -41,7 +41,7 @@ class Partitioning: """Partition scheme used to describe path-based partitions. Path-based partition formats embed all partition keys and values directly in - their dataset file paths. + their datastream file paths. """ #: The partition style - may be either HIVE or DIRECTORY. @@ -53,7 +53,7 @@ class Partitioning: #: directories. base_dir: Optional[str] = None #: The partition key field names (i.e. column names for tabular - #: datasets). When non-empty, the order and length of partition key + #: datastreams). When non-empty, the order and length of partition key #: field names must match the order and length of partition values. #: Required when parsing DIRECTORY partitioned paths or generating #: HIVE partitioned paths. @@ -112,7 +112,7 @@ class PathPartitionEncoder: """Callable that generates directory path strings for path-based partition formats. Path-based partition formats embed all partition keys and values directly in - their dataset file paths. + their datastream file paths. Two path partition formats are currently supported - HIVE and DIRECTORY. @@ -140,7 +140,7 @@ def of( base_dir: "/"-delimited base directory that all partition paths will be generated under (exclusive). field_names: The partition key field names (i.e. column names for tabular - datasets). Required for HIVE partition paths, optional for DIRECTORY + datastreams). Required for HIVE partition paths, optional for DIRECTORY partition paths. When non-empty, the order and length of partition key field names must match the order and length of partition values. filesystem: Filesystem that will be used for partition path file I/O. @@ -229,7 +229,7 @@ class PathPartitionParser: """Partition parser for path-based partition formats. Path-based partition formats embed all partition keys and values directly in - their dataset file paths. + their datastream file paths. Two path partition formats are currently supported - HIVE and DIRECTORY. @@ -274,7 +274,7 @@ def of( Optional for HIVE partitioning. When non-empty, the order and length of partition key field names must match the order and length of partition directories discovered. Partition key field names are not required to - exist in the dataset schema. + exist in the datastream schema. filesystem: Filesystem that will be used for partition path file I/O. Returns: @@ -452,7 +452,7 @@ def do_assert(val, msg): Optional for HIVE partitioning. When non-empty, the order and length of partition key field names must match the order and length of partition directories discovered. Partition key field names are not required to - exist in the dataset schema. + exist in the datastream schema. filesystem: Filesystem that will be used for partition path file I/O. Returns: diff --git a/python/ray/data/datasource/webdataset_datasource.py b/python/ray/data/datasource/webdataset_datasource.py index f0abe2935247..431dea55e6d6 100644 --- a/python/ray/data/datasource/webdataset_datasource.py +++ b/python/ray/data/datasource/webdataset_datasource.py @@ -295,7 +295,7 @@ def _make_iterable(block: BlockAccessor): This is a placeholder for dealing with more complex blocks. Args: - block: Ray Dataset block + block: Ray Datastream block Returns: Iterable[Dict[str,Any]]: Iterable of samples @@ -305,7 +305,7 @@ def _make_iterable(block: BlockAccessor): @PublicAPI(stability="alpha") class WebDatasetDatasource(FileBasedDatasource): - """A Datasource for WebDataset datasets (tar format with naming conventions).""" + """A Datasource for WebDataset datastreams (tar format with naming conventions).""" _FILE_EXTENSION = "tar" @@ -327,7 +327,7 @@ def _read_stream( Args: stream: File descriptor to read from. - path: Path to the dataset. + path: Path to the data. decoder: decoder or list of decoders to be applied to samples fileselect: Predicate for skipping files in tar decoder. Defaults to lambda_:False. diff --git a/python/ray/data/grouped_dataset.py b/python/ray/data/grouped_dataset.py index a873c87499d7..c625b75f4dec 100644 --- a/python/ray/data/grouped_dataset.py +++ b/python/ray/data/grouped_dataset.py @@ -119,23 +119,24 @@ class PushBasedGroupbyOp(_GroupbyOp, PushBasedShufflePlan): @PublicAPI class GroupedData(Generic[T]): - """Represents a grouped dataset created by calling ``Dataset.groupby()``. + """Represents a grouped datastream created by calling ``Datastream.groupby()``. The actual groupby is deferred until an aggregation is applied. """ - def __init__(self, dataset: Datastream[T], key: KeyFn): - """Construct a dataset grouped by key (internal API). + def __init__(self, datastream: Datastream[T], key: KeyFn): + """Construct a datastream grouped by key (internal API). The constructor is not part of the GroupedData API. - Use the ``Dataset.groupby()`` method to construct one. + Use the ``Datastream.groupby()`` method to construct one. """ - self._dataset = dataset + self._datastream = datastream self._key = key def __repr__(self) -> str: return ( - f"{self.__class__.__name__}(dataset={self._dataset}, " f"key={self._key!r})" + f"{self.__class__.__name__}(datastream={self._datastream}, " + f"key={self._key!r})" ) def aggregate(self, *aggs: AggregateFn) -> Datastream[U]: @@ -171,11 +172,11 @@ def aggregate(self, *aggs: AggregateFn) -> Datastream[U]: aggs: Aggregations to do. Returns: - If the input dataset is simple dataset then the output is a simple - dataset of ``(k, v_1, ..., v_n)`` tuples where ``k`` is the groupby + If the input datastream is simple datastream then the output is a simple + datastream of ``(k, v_1, ..., v_n)`` tuples where ``k`` is the groupby key and ``v_i`` is the result of the ith given aggregation. - If the input dataset is an Arrow dataset then the output is an - Arrow dataset of ``n + 1`` columns where the first column is the + If the input datastream is an Arrow datastream then the output is an + Arrow datastream of ``n + 1`` columns where the first column is the groupby key and the second through ``n + 1`` columns are the results of the aggregations. If groupby key is ``None`` then the key part of return is omitted. @@ -187,8 +188,8 @@ def do_agg(blocks, task_ctx: TaskContext, clear_input_blocks: bool, *_): if len(aggs) == 0: raise ValueError("Aggregate requires at least one aggregation") for agg in aggs: - agg._validate(self._dataset) - # Handle empty dataset. + agg._validate(self._datastream) + # Handle empty datastream. if blocks.initial_num_blocks() == 0: return blocks, stage_info @@ -221,7 +222,7 @@ def do_agg(blocks, task_ctx: TaskContext, clear_input_blocks: bool, *_): ctx=task_ctx, ) - plan = self._dataset._plan.with_stage( + plan = self._datastream._plan.with_stage( AllToAllStage( "Aggregate", None, @@ -230,7 +231,7 @@ def do_agg(blocks, task_ctx: TaskContext, clear_input_blocks: bool, *_): ) ) - logical_plan = self._dataset._logical_plan + logical_plan = self._datastream._logical_plan if logical_plan is not None: op = Aggregate( logical_plan.dag, @@ -240,8 +241,8 @@ def do_agg(blocks, task_ctx: TaskContext, clear_input_blocks: bool, *_): logical_plan = LogicalPlan(op) return Datastream( plan, - self._dataset._epoch, - self._dataset._lazy, + self._datastream._epoch, + self._datastream._lazy, logical_plan, ) @@ -253,14 +254,14 @@ def _aggregate_on( *args, **kwargs, ): - """Helper for aggregating on a particular subset of the dataset. + """Helper for aggregating on a particular subset of the datastream. This validates the `on` argument, and converts a list of column names or lambdas to a multi-aggregation. A null `on` results in a - multi-aggregation on all columns for an Arrow Dataset, and a single - aggregation on the entire row for a simple Dataset. + multi-aggregation on all columns for an Arrow Datastream, and a single + aggregation on the entire row for a simple Datastream. """ - aggs = self._dataset._build_multicolumn_aggs( + aggs = self._datastream._build_multicolumn_aggs( agg_cls, on, ignore_nulls, *args, skip_cols=self._key, **kwargs ) return self.aggregate(*aggs) @@ -275,7 +276,7 @@ def map_groups( ) -> "Datastream[Any]": # TODO AttributeError: 'GroupedData' object has no attribute 'map_groups' # in the example below. - """Apply the given function to each group of records of this dataset. + """Apply the given function to each group of records of this datastream. While map_groups() is very flexible, note that it comes with downsides: * It may be slower than using more specific methods such as min(), max(). @@ -327,8 +328,8 @@ def map_groups( batch_format: Specify ``"default"`` to use the default block format (promotes tables to Pandas and tensors to NumPy), ``"pandas"`` to select ``pandas.DataFrame``, "pyarrow" to select ``pyarrow.Table``, or - ``"numpy"`` to select ``numpy.ndarray`` for tensor datasets and - ``Dict[str, numpy.ndarray]`` for tabular datasets, or None + ``"numpy"`` to select ``numpy.ndarray`` for tensor datastreams and + ``Dict[str, numpy.ndarray]`` for tabular datastreams, or None to return the underlying block exactly as is with no additional formatting. The default is "default". ray_remote_args: Additional resource requirements to request from @@ -342,9 +343,9 @@ def map_groups( # Note that sort() will ensure that records of the same key partitioned # into the same block. if self._key is not None: - sorted_ds = self._dataset.sort(self._key) + sorted_ds = self._datastream.sort(self._key) else: - sorted_ds = self._dataset.repartition(1) + sorted_ds = self._datastream.repartition(1) # Returns the group boundaries. def get_key_boundaries(block_accessor: BlockAccessor): @@ -405,7 +406,7 @@ def count(self) -> Datastream[U]: ... "A").count() # doctest: +SKIP Returns: - A simple dataset of ``(k, v)`` pairs or an Arrow dataset of + A simple datastream of ``(k, v)`` pairs or an Arrow datastream of ``[k, v]`` columns where ``k`` is the groupby key and ``v`` is the number of rows with that key. If groupby key is ``None`` then the key part of return is omitted. @@ -435,9 +436,9 @@ def sum( Args: on: The data subset on which to compute the sum. - - For a simple dataset: it can be a callable or a list thereof, + - For a simple datastream: it can be a callable or a list thereof, and the default is to take a sum of all rows. - - For an Arrow dataset: it can be a column name or a list + - For an Arrow datastream: it can be a column name or a list thereof, and the default is to do a column-wise sum of all columns. ignore_nulls: Whether to ignore null values. If ``True``, null @@ -449,21 +450,21 @@ def sum( Returns: The sum result. - For a simple dataset, the output is: + For a simple datastream, the output is: - - ``on=None``: a simple dataset of ``(k, sum)`` tuples where ``k`` + - ``on=None``: a simple datastream of ``(k, sum)`` tuples where ``k`` is the groupby key and ``sum`` is sum of all rows in that group. - - ``on=[callable_1, ..., callable_n]``: a simple dataset of + - ``on=[callable_1, ..., callable_n]``: a simple datastream of ``(k, sum_1, ..., sum_n)`` tuples where ``k`` is the groupby key and ``sum_i`` is sum of the outputs of the ith callable called on each row in that group. - For an Arrow dataset, the output is: + For an Arrow datastream, the output is: - - ``on=None``: an Arrow dataset containing a groupby key column, + - ``on=None``: an Arrow datastream containing a groupby key column, ``"k"``, and a column-wise sum column for each original column - in the dataset. - - ``on=["col_1", ..., "col_n"]``: an Arrow dataset of ``n + 1`` + in the datastream. + - ``on=["col_1", ..., "col_n"]``: an Arrow datastream of ``n + 1`` columns where the first column is the groupby key and the second through ``n + 1`` columns are the results of the aggregations. @@ -494,9 +495,9 @@ def min( Args: on: The data subset on which to compute the min. - - For a simple dataset: it can be a callable or a list thereof, + - For a simple datastream: it can be a callable or a list thereof, and the default is to take a min of all rows. - - For an Arrow dataset: it can be a column name or a list + - For an Arrow datastream: it can be a column name or a list thereof, and the default is to do a column-wise min of all columns. ignore_nulls: Whether to ignore null values. If ``True``, null @@ -508,21 +509,21 @@ def min( Returns: The min result. - For a simple dataset, the output is: + For a simple datastream, the output is: - - ``on=None``: a simple dataset of ``(k, min)`` tuples where ``k`` + - ``on=None``: a simple datastream of ``(k, min)`` tuples where ``k`` is the groupby key and min is min of all rows in that group. - - ``on=[callable_1, ..., callable_n]``: a simple dataset of + - ``on=[callable_1, ..., callable_n]``: a simple datastream of ``(k, min_1, ..., min_n)`` tuples where ``k`` is the groupby key and ``min_i`` is min of the outputs of the ith callable called on each row in that group. - For an Arrow dataset, the output is: + For an Arrow datastream, the output is: - - ``on=None``: an Arrow dataset containing a groupby key column, + - ``on=None``: an Arrow datastream containing a groupby key column, ``"k"``, and a column-wise min column for each original column in - the dataset. - - ``on=["col_1", ..., "col_n"]``: an Arrow dataset of ``n + 1`` + the datastream. + - ``on=["col_1", ..., "col_n"]``: an Arrow datastream of ``n + 1`` columns where the first column is the groupby key and the second through ``n + 1`` columns are the results of the aggregations. @@ -553,9 +554,9 @@ def max( Args: on: The data subset on which to compute the max. - - For a simple dataset: it can be a callable or a list thereof, + - For a simple datastream: it can be a callable or a list thereof, and the default is to take a max of all rows. - - For an Arrow dataset: it can be a column name or a list + - For an Arrow datastream: it can be a column name or a list thereof, and the default is to do a column-wise max of all columns. ignore_nulls: Whether to ignore null values. If ``True``, null @@ -567,21 +568,21 @@ def max( Returns: The max result. - For a simple dataset, the output is: + For a simple datastream, the output is: - - ``on=None``: a simple dataset of ``(k, max)`` tuples where ``k`` + - ``on=None``: a simple datastream of ``(k, max)`` tuples where ``k`` is the groupby key and ``max`` is max of all rows in that group. - - ``on=[callable_1, ..., callable_n]``: a simple dataset of + - ``on=[callable_1, ..., callable_n]``: a simple datastream of ``(k, max_1, ..., max_n)`` tuples where ``k`` is the groupby key and ``max_i`` is max of the outputs of the ith callable called on each row in that group. - For an Arrow dataset, the output is: + For an Arrow datastream, the output is: - - ``on=None``: an Arrow dataset containing a groupby key column, + - ``on=None``: an Arrow datastream containing a groupby key column, ``"k"``, and a column-wise max column for each original column in - the dataset. - - ``on=["col_1", ..., "col_n"]``: an Arrow dataset of ``n + 1`` + the datastream. + - ``on=["col_1", ..., "col_n"]``: an Arrow datastream of ``n + 1`` columns where the first column is the groupby key and the second through ``n + 1`` columns are the results of the aggregations. @@ -612,9 +613,9 @@ def mean( Args: on: The data subset on which to compute the mean. - - For a simple dataset: it can be a callable or a list thereof, + - For a simple datastream: it can be a callable or a list thereof, and the default is to take a mean of all rows. - - For an Arrow dataset: it can be a column name or a list + - For an Arrow datastream: it can be a column name or a list thereof, and the default is to do a column-wise mean of all columns. ignore_nulls: Whether to ignore null values. If ``True``, null @@ -626,22 +627,22 @@ def mean( Returns: The mean result. - For a simple dataset, the output is: + For a simple datastream, the output is: - - ``on=None``: a simple dataset of ``(k, mean)`` tuples where ``k`` + - ``on=None``: a simple datastream of ``(k, mean)`` tuples where ``k`` is the groupby key and ``mean`` is mean of all rows in that group. - - ``on=[callable_1, ..., callable_n]``: a simple dataset of + - ``on=[callable_1, ..., callable_n]``: a simple datastream of ``(k, mean_1, ..., mean_n)`` tuples where ``k`` is the groupby key and ``mean_i`` is mean of the outputs of the ith callable called on each row in that group. - For an Arrow dataset, the output is: + For an Arrow datastream, the output is: - - ``on=None``: an Arrow dataset containing a groupby key column, + - ``on=None``: an Arrow datastream containing a groupby key column, ``"k"``, and a column-wise mean column for each original column - in the dataset. - - ``on=["col_1", ..., "col_n"]``: an Arrow dataset of ``n + 1`` + in the datastream. + - ``on=["col_1", ..., "col_n"]``: an Arrow datastream of ``n + 1`` columns where the first column is the groupby key and the second through ``n + 1`` columns are the results of the aggregations. @@ -683,9 +684,9 @@ def std( Args: on: The data subset on which to compute the std. - - For a simple dataset: it can be a callable or a list thereof, + - For a simple datastream: it can be a callable or a list thereof, and the default is to take a std of all rows. - - For an Arrow dataset: it can be a column name or a list + - For an Arrow datastream: it can be a column name or a list thereof, and the default is to do a column-wise std of all columns. ddof: Delta Degrees of Freedom. The divisor used in calculations @@ -699,21 +700,21 @@ def std( Returns: The standard deviation result. - For a simple dataset, the output is: + For a simple datastream, the output is: - - ``on=None``: a simple dataset of ``(k, std)`` tuples where ``k`` + - ``on=None``: a simple datastream of ``(k, std)`` tuples where ``k`` is the groupby key and ``std`` is std of all rows in that group. - - ``on=[callable_1, ..., callable_n]``: a simple dataset of + - ``on=[callable_1, ..., callable_n]``: a simple datastream of ``(k, std_1, ..., std_n)`` tuples where ``k`` is the groupby key and ``std_i`` is std of the outputs of the ith callable called on each row in that group. - For an Arrow dataset, the output is: + For an Arrow datastream, the output is: - - ``on=None``: an Arrow dataset containing a groupby key column, + - ``on=None``: an Arrow datastream containing a groupby key column, ``"k"``, and a column-wise std column for each original column in - the dataset. - - ``on=["col_1", ..., "col_n"]``: an Arrow dataset of ``n + 1`` + the datastream. + - ``on=["col_1", ..., "col_n"]``: an Arrow datastream of ``n + 1`` columns where the first column is the groupby key and the second through ``n + 1`` columns are the results of the aggregations. diff --git a/python/ray/data/preprocessors/batch_mapper.py b/python/ray/data/preprocessors/batch_mapper.py index 4479bf3e9478..4aca660cce90 100644 --- a/python/ray/data/preprocessors/batch_mapper.py +++ b/python/ray/data/preprocessors/batch_mapper.py @@ -18,12 +18,12 @@ @PublicAPI(stability="alpha") class BatchMapper(Preprocessor): - """Apply an arbitrary operation to a dataset. + """Apply an arbitrary operation to a datastream. - :class:`BatchMapper` applies a user-defined function to batches of a dataset. A + :class:`BatchMapper` applies a user-defined function to batches of a datastream. A batch is a Pandas ``DataFrame`` that represents a small amount of data. By modifying batches instead of individual records, this class can efficiently transform a - dataset with vectorized operations. + datastream with vectorized operations. Use this preprocessor to apply stateless operations that aren't already built-in. @@ -48,25 +48,25 @@ class BatchMapper(Preprocessor): >>> >>> preprocessor = BatchMapper(fn, batch_format="pandas") >>> preprocessor.transform(ds) # doctest: +SKIP - Dataset(num_blocks=1, num_rows=3, schema={X: int64}) + Datastream(num_blocks=1, num_rows=3, schema={X: int64}) >>> >>> def fn_numpy(batch: Dict[str, np.ndarray]) -> Dict[str, np.ndarray]: ... return {"X": batch["X"]} >>> preprocessor = BatchMapper(fn_numpy, batch_format="numpy") >>> preprocessor.transform(ds) # doctest: +SKIP - Dataset(num_blocks=1, num_rows=3, schema={X: int64}) + Datastream(num_blocks=1, num_rows=3, schema={X: int64}) Args: fn: The function to apply to data batches. batch_size: The desired number of rows in each data batch provided to ``fn``. - Semantics are the same as in ```dataset.map_batches()``: specifying + Semantics are the same as in ```datastream.map_batches()``: specifying ``None`` wil use the entire underlying blocks as batches (blocks may contain different number of rows) and the actual size of the batch provided to ``fn`` may be smaller than ``batch_size`` if ``batch_size`` doesn't evenly divide the block(s) sent to a given map task. Defaults to 4096, - which is the same default value as ``dataset.map_batches()``. + which is the same default value as ``datastream.map_batches()``. batch_format: The preferred batch format to use in UDF. If not given, - we will infer based on the input dataset data format. + we will infer based on the input datastream data format. """ _is_fittable = False diff --git a/python/ray/data/preprocessors/chain.py b/python/ray/data/preprocessors/chain.py index 48349813160a..4673b73b871e 100644 --- a/python/ray/data/preprocessors/chain.py +++ b/python/ray/data/preprocessors/chain.py @@ -1,6 +1,6 @@ from typing import TYPE_CHECKING, Union from ray.air.util.data_batch_conversion import BatchFormat -from ray.data import Dataset, DatasetPipeline +from ray.data import Datastream, DatasetPipeline from ray.data.preprocessor import Preprocessor from ray.util.annotations import PublicAPI @@ -12,7 +12,7 @@ class Chain(Preprocessor): """Combine multiple preprocessors into a single :py:class:`Preprocessor`. - When you call ``fit``, each preprocessor is fit on the dataset produced by the + When you call ``fit``, each preprocessor is fit on the datastream produced by the preceeding preprocessor's ``fit_transform``. Example: @@ -69,22 +69,22 @@ def fit_status(self): def __init__(self, *preprocessors: Preprocessor): self.preprocessors = preprocessors - def _fit(self, ds: Dataset) -> Preprocessor: + def _fit(self, ds: Datastream) -> Preprocessor: for preprocessor in self.preprocessors[:-1]: ds = preprocessor.fit_transform(ds) self.preprocessors[-1].fit(ds) return self - def fit_transform(self, ds: Dataset) -> Dataset: + def fit_transform(self, ds: Datastream) -> Datastream: for preprocessor in self.preprocessors: ds = preprocessor.fit_transform(ds) return ds def _transform( - self, ds: Union[Dataset, DatasetPipeline] - ) -> Union[Dataset, DatasetPipeline]: + self, ds: Union[Datastream, DatasetPipeline] + ) -> Union[Datastream, DatasetPipeline]: for preprocessor in self.preprocessors: - if isinstance(ds, Dataset): + if isinstance(ds, Datastream): ds = preprocessor.transform(ds) elif isinstance(ds, DatasetPipeline): ds = preprocessor._transform_pipeline(ds) diff --git a/python/ray/data/preprocessors/concatenator.py b/python/ray/data/preprocessors/concatenator.py index 31d0621cf7fb..cc4bbdd38aa5 100644 --- a/python/ray/data/preprocessors/concatenator.py +++ b/python/ray/data/preprocessors/concatenator.py @@ -51,7 +51,7 @@ class Concatenator(Preprocessor): 2 [1.0, 0.9] Sometimes, you might not want to concatenate all of of the columns in your - dataset. In this case, you can exclude columns with the ``exclude`` parameter. + datastream. In this case, you can exclude columns with the ``exclude`` parameter. >>> df = pd.DataFrame({"X0": [0, 3, 1], "X1": [0.5, 0.2, 0.9], "Y": ["blue", "orange", "blue"]}) >>> ds = ray.data.from_pandas(df) # doctest: +SKIP @@ -88,7 +88,7 @@ class Concatenator(Preprocessor): >>> concatenator = Concatenator(include=["X0", "X1"], dtype=np.float32) >>> concatenator.fit_transform(ds) # doctest: +SKIP - Dataset(num_blocks=1, num_rows=3, schema={Y: object, concat_out: TensorDtype(shape=(2,), dtype=float32)}) + Datastream(num_blocks=1, num_rows=3, schema={Y: object, concat_out: TensorDtype(shape=(2,), dtype=float32)}) Args: output_column_name: The desired name for the new column. @@ -106,7 +106,7 @@ class Concatenator(Preprocessor): Raises: ValueError: if `raise_if_missing` is `True` and a column in `include` or - `exclude` doesn't exist in the dataset. + `exclude` doesn't exist in the datastream. """ # noqa: E501 _is_fittable = False diff --git a/python/ray/data/preprocessors/discretizer.py b/python/ray/data/preprocessors/discretizer.py index 36e34f805ef1..0ef294b7a8c1 100644 --- a/python/ray/data/preprocessors/discretizer.py +++ b/python/ray/data/preprocessors/discretizer.py @@ -3,7 +3,7 @@ import pandas as pd import numpy as np -from ray.data import Dataset +from ray.data import Datastream from ray.data.aggregate import Max, Min from ray.data.preprocessor import Preprocessor from ray.util.annotations import PublicAPI @@ -253,7 +253,7 @@ def __init__( self.duplicates = duplicates self.dtypes = dtypes - def _fit(self, dataset: Dataset) -> Preprocessor: + def _fit(self, datastream: Datastream) -> Preprocessor: self._validate_on_fit() stats = {} aggregates = [] @@ -267,7 +267,7 @@ def _fit(self, dataset: Dataset) -> Preprocessor: self._fit_uniform_covert_bin_to_aggregate_if_needed(column) ) - aggregate_stats = dataset.aggregate(*aggregates) + aggregate_stats = datastream.aggregate(*aggregates) mins = {} maxes = {} for key, value in aggregate_stats.items(): diff --git a/python/ray/data/preprocessors/encoder.py b/python/ray/data/preprocessors/encoder.py index dc3a156f77c5..18f2ce115c7f 100644 --- a/python/ray/data/preprocessors/encoder.py +++ b/python/ray/data/preprocessors/encoder.py @@ -6,7 +6,7 @@ import pandas as pd import pandas.api.types -from ray.data import Dataset +from ray.data import Datastream from ray.data.preprocessor import Preprocessor from ray.util.annotations import PublicAPI @@ -43,7 +43,7 @@ class OrdinalEncoder(Preprocessor): 2 1 0 3 0 1 - If you transform a value not present in the original dataset, then the value + If you transform a value not present in the original datastream, then the value is encoded as ``float("nan")``. >>> df = pd.DataFrame({"sex": ["female"], "level": ["L6"]}) @@ -87,9 +87,9 @@ def __init__(self, columns: List[str], *, encode_lists: bool = True): self.columns = columns self.encode_lists = encode_lists - def _fit(self, dataset: Dataset) -> Preprocessor: + def _fit(self, datastream: Datastream) -> Preprocessor: self.stats_ = _get_unique_value_indices( - dataset, self.columns, encode_lists=self.encode_lists + datastream, self.columns, encode_lists=self.encode_lists ) return self @@ -135,7 +135,7 @@ class OneHotEncoder(Preprocessor): 1 if the category matches and 0 otherwise. If you encode an infrequent category (see ``max_categories``) or a category - that isn't in the fitted dataset, then the category is encoded as all 0s. + that isn't in the fitted datastream, then the category is encoded as all 0s. Columns must contain hashable objects or lists of hashable objects. @@ -160,7 +160,7 @@ class OneHotEncoder(Preprocessor): 4 1 0 0 5 0 1 0 - If you one-hot encode a value that isn't in the fitted dataset, then the + If you one-hot encode a value that isn't in the fitted datastream, then the value is encoded with zeros. >>> df = pd.DataFrame({"color": ["yellow"]}) @@ -206,9 +206,9 @@ def __init__( self.columns = columns self.max_categories = max_categories - def _fit(self, dataset: Dataset) -> Preprocessor: + def _fit(self, datastream: Datastream) -> Preprocessor: self.stats_ = _get_unique_value_indices( - dataset, + datastream, self.columns, max_categories=self.max_categories, encode_lists=False, @@ -315,9 +315,12 @@ def __init__( self.columns = columns self.max_categories = max_categories - def _fit(self, dataset: Dataset) -> Preprocessor: + def _fit(self, datastream: Datastream) -> Preprocessor: self.stats_ = _get_unique_value_indices( - dataset, self.columns, max_categories=self.max_categories, encode_lists=True + datastream, + self.columns, + max_categories=self.max_categories, + encode_lists=True, ) return self @@ -374,7 +377,7 @@ class LabelEncoder(Preprocessor): 2 4.9 3.0 0 3 6.2 3.4 2 - If you transform a label not present in the original dataset, then the new + If you transform a label not present in the original datastream, then the new label is encoded as ``float("nan")``. >>> df = pd.DataFrame({ @@ -400,8 +403,8 @@ class LabelEncoder(Preprocessor): def __init__(self, label_column: str): self.label_column = label_column - def _fit(self, dataset: Dataset) -> Preprocessor: - self.stats_ = _get_unique_value_indices(dataset, [self.label_column]) + def _fit(self, datastream: Datastream) -> Preprocessor: + self.stats_ = _get_unique_value_indices(datastream, [self.label_column]) return self def _transform_pandas(self, df: pd.DataFrame): @@ -428,7 +431,7 @@ class Categorizer(Preprocessor): .. warning:: If you don't specify ``dtypes``, fit this preprocessor before splitting - your dataset into train and test splits. This ensures categories are + your datastream into train and test splits. This ensures categories are consistent across splits. Examples: @@ -474,13 +477,13 @@ def __init__( self.columns = columns self.dtypes = dtypes - def _fit(self, dataset: Dataset) -> Preprocessor: + def _fit(self, datastream: Datastream) -> Preprocessor: columns_to_get = [ column for column in self.columns if column not in self.dtypes ] if columns_to_get: unique_indices = _get_unique_value_indices( - dataset, columns_to_get, drop_na_values=True, key_format="{0}" + datastream, columns_to_get, drop_na_values=True, key_format="{0}" ) unique_indices = { column: pd.CategoricalDtype(values_indices.keys()) @@ -504,7 +507,7 @@ def __repr__(self): def _get_unique_value_indices( - dataset: Dataset, + datastream: Datastream, columns: List[str], drop_na_values: bool = False, key_format: str = "unique_values({0})", @@ -551,7 +554,7 @@ def get_pd_value_counts(df: pd.DataFrame) -> List[Dict[str, Counter]]: ) return [result] - value_counts = dataset.map_batches(get_pd_value_counts, batch_format="pandas") + value_counts = datastream.map_batches(get_pd_value_counts, batch_format="pandas") final_counters = {col: Counter() for col in columns} for batch in value_counts.iter_batches(batch_size=None): for col_value_counts in batch: diff --git a/python/ray/data/preprocessors/imputer.py b/python/ray/data/preprocessors/imputer.py index cadfdf16f306..23c7f232a9c8 100644 --- a/python/ray/data/preprocessors/imputer.py +++ b/python/ray/data/preprocessors/imputer.py @@ -5,7 +5,7 @@ import pandas as pd from pandas.api.types import is_categorical_dtype -from ray.data import Dataset +from ray.data import Datastream from ray.data.aggregate import Mean from ray.data.preprocessor import Preprocessor from ray.util.annotations import PublicAPI @@ -106,12 +106,12 @@ def __init__( '`fill_value` must be set when using "constant" strategy.' ) - def _fit(self, dataset: Dataset) -> Preprocessor: + def _fit(self, datastream: Datastream) -> Preprocessor: if self.strategy == "mean": aggregates = [Mean(col) for col in self.columns] - self.stats_ = dataset.aggregate(*aggregates) + self.stats_ = datastream.aggregate(*aggregates) elif self.strategy == "most_frequent": - self.stats_ = _get_most_frequent_values(dataset, *self.columns) + self.stats_ = _get_most_frequent_values(datastream, *self.columns) return self @@ -142,14 +142,14 @@ def __repr__(self): def _get_most_frequent_values( - dataset: Dataset, *columns: str + datastream: Datastream, *columns: str ) -> Dict[str, Union[str, Number]]: columns = list(columns) def get_pd_value_counts(df: pd.DataFrame) -> List[Dict[str, Counter]]: return [{col: Counter(df[col].value_counts().to_dict()) for col in columns}] - value_counts = dataset.map_batches(get_pd_value_counts, batch_format="pandas") + value_counts = datastream.map_batches(get_pd_value_counts, batch_format="pandas") final_counters = {col: Counter() for col in columns} for batch in value_counts.iter_batches(batch_size=None): for col_value_counts in batch: diff --git a/python/ray/data/preprocessors/scaler.py b/python/ray/data/preprocessors/scaler.py index b53415c954cd..713542fea8e3 100644 --- a/python/ray/data/preprocessors/scaler.py +++ b/python/ray/data/preprocessors/scaler.py @@ -3,7 +3,7 @@ import numpy as np import pandas as pd -from ray.data import Dataset +from ray.data import Datastream from ray.data.aggregate import Mean, Std, Min, Max, AbsMax from ray.data.preprocessor import Preprocessor from ray.util.annotations import PublicAPI @@ -66,10 +66,10 @@ class StandardScaler(Preprocessor): def __init__(self, columns: List[str]): self.columns = columns - def _fit(self, dataset: Dataset) -> Preprocessor: + def _fit(self, datastream: Datastream) -> Preprocessor: mean_aggregates = [Mean(col) for col in self.columns] std_aggregates = [Std(col, ddof=0) for col in self.columns] - self.stats_ = dataset.aggregate(*mean_aggregates, *std_aggregates) + self.stats_ = datastream.aggregate(*mean_aggregates, *std_aggregates) return self def _transform_pandas(self, df: pd.DataFrame): @@ -150,9 +150,9 @@ class MinMaxScaler(Preprocessor): def __init__(self, columns: List[str]): self.columns = columns - def _fit(self, dataset: Dataset) -> Preprocessor: + def _fit(self, datastream: Datastream) -> Preprocessor: aggregates = [Agg(col) for Agg in [Min, Max] for col in self.columns] - self.stats_ = dataset.aggregate(*aggregates) + self.stats_ = datastream.aggregate(*aggregates) return self def _transform_pandas(self, df: pd.DataFrame): @@ -230,9 +230,9 @@ class MaxAbsScaler(Preprocessor): def __init__(self, columns: List[str]): self.columns = columns - def _fit(self, dataset: Dataset) -> Preprocessor: + def _fit(self, datastream: Datastream) -> Preprocessor: aggregates = [AbsMax(col) for col in self.columns] - self.stats_ = dataset.aggregate(*aggregates) + self.stats_ = datastream.aggregate(*aggregates) return self def _transform_pandas(self, df: pd.DataFrame): @@ -315,12 +315,12 @@ def __init__( self.columns = columns self.quantile_range = quantile_range - def _fit(self, dataset: Dataset) -> Preprocessor: + def _fit(self, datastream: Datastream) -> Preprocessor: low = self.quantile_range[0] med = 0.50 high = self.quantile_range[1] - num_records = dataset.count() + num_records = datastream.count() max_index = num_records - 1 split_indices = [int(percentile * max_index) for percentile in (low, med, high)] @@ -328,15 +328,15 @@ def _fit(self, dataset: Dataset) -> Preprocessor: # TODO(matt): Handle case where quantile lands between 2 numbers. # The current implementation will simply choose the closest index. - # This will affect the results of small datasets more than large datasets. + # This will affect the results of small datastreams more than large datastreams. for col in self.columns: - filtered_dataset = dataset.map_batches( + filtered_datastream = datastream.map_batches( lambda df: df[[col]], batch_format="pandas" ) - sorted_dataset = filtered_dataset.sort(col) - _, low, med, high = sorted_dataset.split_at_indices(split_indices) + sorted_datastream = filtered_datastream.sort(col) + _, low, med, high = sorted_datastream.split_at_indices(split_indices) - def _get_first_value(ds: Dataset, c: str): + def _get_first_value(ds: Datastream, c: str): return ds.take(1)[0][c] low_val = _get_first_value(low, col) diff --git a/python/ray/data/preprocessors/torch.py b/python/ray/data/preprocessors/torch.py index fc2196bd7298..cdede252be56 100644 --- a/python/ray/data/preprocessors/torch.py +++ b/python/ray/data/preprocessors/torch.py @@ -18,9 +18,9 @@ class TorchVisionPreprocessor(Preprocessor): Examples: >>> import ray - >>> dataset = ray.data.read_images("s3://anonymous@air-example-data-2/imagenet-sample-images") - >>> dataset # doctest: +ellipsis - Dataset(num_blocks=..., num_rows=..., schema={image: numpy.ndarray(shape=(..., 3), dtype=float)}) + >>> datastream = ray.data.read_images("s3://anonymous@air-example-data-2/imagenet-sample-images") + >>> datastream # doctest: +ellipsis + Datastream(num_blocks=..., num_rows=..., schema={image: numpy.ndarray(shape=(..., 3), dtype=float)}) Torch models expect inputs of shape :math:`(B, C, H, W)` in the range :math:`[0.0, 1.0]`. To convert images to this format, add ``ToTensor`` to your @@ -33,9 +33,9 @@ class TorchVisionPreprocessor(Preprocessor): ... transforms.Resize((224, 224)), ... ]) >>> preprocessor = TorchVisionPreprocessor(["image"], transform=transform) - >>> dataset = preprocessor.transform(dataset) # doctest: +ellipsis - >>> dataset # doctest: +ellipsis - Dataset(num_blocks=..., num_rows=..., schema={image: numpy.ndarray(shape=(3, 224, 224), dtype=float)}) + >>> datastream = preprocessor.transform(datastream) # doctest: +ellipsis + >>> datastream # doctest: +ellipsis + Datastream(num_blocks=..., num_rows=..., schema={image: numpy.ndarray(shape=(3, 224, 224), dtype=float)}) For better performance, set ``batched`` to ``True`` and replace ``ToTensor`` with a batch-supporting ``Lambda``. @@ -54,9 +54,9 @@ class TorchVisionPreprocessor(Preprocessor): >>> preprocessor = TorchVisionPreprocessor( ... ["image"], transform=transform, batched=True ... ) - >>> dataset = preprocessor.transform(dataset) # doctest: +ellipsis - >>> dataset # doctest: +ellipsis - Dataset(num_blocks=..., num_rows=..., schema={image: numpy.ndarray(shape=(3, 224, 224), dtype=float)}) + >>> datastream = preprocessor.transform(datastream) # doctest: +ellipsis + >>> datastream # doctest: +ellipsis + Datastream(num_blocks=..., num_rows=..., schema={image: numpy.ndarray(shape=(3, 224, 224), dtype=float)}) Args: columns: The columns to apply the TorchVision transform to. diff --git a/python/ray/data/preprocessors/vectorizer.py b/python/ray/data/preprocessors/vectorizer.py index 0d21d8b2a76e..6949ade9275b 100644 --- a/python/ray/data/preprocessors/vectorizer.py +++ b/python/ray/data/preprocessors/vectorizer.py @@ -3,7 +3,7 @@ import pandas as pd -from ray.data import Dataset +from ray.data import Datastream from ray.data.preprocessor import Preprocessor from ray.data.preprocessors.utils import simple_split_tokenizer, simple_hash from ray.util.annotations import PublicAPI @@ -201,7 +201,7 @@ class CountVectorizer(Preprocessor): output. If unspecified, the tokenizer uses a function equivalent to ``lambda s: s.split(" ")``. max_features: The maximum number of tokens to encode in the transformed - dataset. If specified, only the most frequent tokens are encoded. + datastream. If specified, only the most frequent tokens are encoded. """ # noqa: E501 @@ -217,7 +217,7 @@ def __init__( self.tokenization_fn = tokenization_fn or simple_split_tokenizer self.max_features = max_features - def _fit(self, dataset: Dataset) -> Preprocessor: + def _fit(self, datastream: Datastream) -> Preprocessor: def get_pd_value_counts(df: pd.DataFrame) -> List[Counter]: def get_token_counts(col): token_series = df[col].apply(self.tokenization_fn) @@ -226,7 +226,9 @@ def get_token_counts(col): return [{col: get_token_counts(col) for col in self.columns}] - value_counts = dataset.map_batches(get_pd_value_counts, batch_format="pandas") + value_counts = datastream.map_batches( + get_pd_value_counts, batch_format="pandas" + ) total_counts = {col: Counter() for col in self.columns} for batch in value_counts.iter_batches(batch_size=None): for x in batch: diff --git a/python/ray/data/read_api.py b/python/ray/data/read_api.py index 6c92276ab772..64b30e3d975b 100644 --- a/python/ray/data/read_api.py +++ b/python/ray/data/read_api.py @@ -38,7 +38,7 @@ from ray.data._internal.pandas_block import PandasRow from ray.data._internal.plan import ExecutionPlan from ray.data._internal.remote_fn import cached_remote_fn -from ray.data._internal.stats import DatasetStats +from ray.data._internal.stats import DatastreamStats from ray.data._internal.util import ( _lazy_import_pyarrow_dataset, _autodetect_parallelism, @@ -183,7 +183,7 @@ def from_items( return MaterializedDatastream( ExecutionPlan( BlockList(blocks, metadata, owned_by_consumer=False), - DatasetStats(stages={"FromItems": metadata}, parent=None), + DatastreamStats(stages={"FromItems": metadata}, parent=None), run_by_consumer=False, ), 0, @@ -483,7 +483,7 @@ def read_mongo( ... ) Args: - uri: The URI of the source MongoDB where the dataset will be + uri: The URI of the source MongoDB where the datastream will be read from. For the URI format, see details in https://www.mongodb.com/docs/manual/reference/connection-string/. database: The name of the database hosted in the MongoDB. This database @@ -1590,7 +1590,7 @@ def from_pandas_refs( return MaterializedDatastream( ExecutionPlan( BlockList(dfs, metadata, owned_by_consumer=False), - DatasetStats(stages={"FromPandasRefs": metadata}, parent=None), + DatastreamStats(stages={"FromPandasRefs": metadata}, parent=None), run_by_consumer=False, ), 0, @@ -1606,7 +1606,7 @@ def from_pandas_refs( return MaterializedDatastream( ExecutionPlan( BlockList(blocks, metadata, owned_by_consumer=False), - DatasetStats(stages={"FromPandasRefs": metadata}, parent=None), + DatastreamStats(stages={"FromPandasRefs": metadata}, parent=None), run_by_consumer=False, ), 0, @@ -1672,7 +1672,7 @@ def from_numpy_refs( return MaterializedDatastream( ExecutionPlan( BlockList(blocks, metadata, owned_by_consumer=False), - DatasetStats(stages={"FromNumpyRefs": metadata}, parent=None), + DatastreamStats(stages={"FromNumpyRefs": metadata}, parent=None), run_by_consumer=False, ), 0, @@ -1727,7 +1727,7 @@ def from_arrow_refs( return MaterializedDatastream( ExecutionPlan( BlockList(tables, metadata, owned_by_consumer=False), - DatasetStats(stages={"FromArrowRefs": metadata}, parent=None), + DatastreamStats(stages={"FromArrowRefs": metadata}, parent=None), run_by_consumer=False, ), 0, @@ -1899,7 +1899,7 @@ def _get_read_tasks( Args: ds: Datasource to read from. - ctx: Dataset config to use. + ctx: Datastream 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. diff --git a/python/ray/data/tests/block_batching/test_util.py b/python/ray/data/tests/block_batching/test_util.py index 86a4a12f2f07..67eeffa3e777 100644 --- a/python/ray/data/tests/block_batching/test_util.py +++ b/python/ray/data/tests/block_batching/test_util.py @@ -47,15 +47,15 @@ def test_blocks_to_batches(block_size, drop_last): full_batches = 0 leftover_batches = 0 - dataset_size = block_size * num_blocks + datastream_size = block_size * num_blocks for batch in batch_iter: if len(batch.data) == batch_size: full_batches += 1 - if len(batch.data) == (dataset_size % batch_size): + if len(batch.data) == (datastream_size % batch_size): leftover_batches += 1 assert leftover_batches == 1 - assert full_batches == (dataset_size // batch_size) + assert full_batches == (datastream_size // batch_size) assert [batch.batch_idx for batch in batch_iter] == list(range(len(batch_iter))) diff --git a/python/ray/data/tests/conftest.py b/python/ray/data/tests/conftest.py index ece8609500ea..d5bb43a57b6b 100644 --- a/python/ray/data/tests/conftest.py +++ b/python/ray/data/tests/conftest.py @@ -157,14 +157,15 @@ def _get_write_path_for_block( base_path, *, filesystem=None, - dataset_uuid=None, + datastream_uuid=None, block=None, block_index=None, file_format=None, ): num_rows = BlockAccessor.for_block(block).num_rows() suffix = ( - f"{block_index:06}_{num_rows:02}_{dataset_uuid}" f".test.{file_format}" + f"{block_index:06}_{num_rows:02}_{datastream_uuid}" + f".test.{file_format}" ) return posixpath.join(base_path, suffix) @@ -252,7 +253,7 @@ def _assert_base_partitioned_ds( actual_input_files = ds.input_files() assert len(actual_input_files) == num_input_files, actual_input_files - # For Datasets with long string representations, the format will include + # For Datastreams with long string representations, the format will include # whitespace and newline characters, which is difficult to generalize # without implementing the formatting logic again (from # `ExecutionPlan.get_plan_as_string()`). Therefore, we remove whitespace @@ -293,7 +294,7 @@ def _remove_whitespace(ds_str): @pytest.fixture -def restore_dataset_context(request): +def restore_data_context(request): """Restore any DataContext changes after the test runs""" original = copy.deepcopy(ray.data.context.DataContext.get_current()) yield @@ -369,7 +370,7 @@ def enable_streaming_executor(): ctx.use_streaming_executor = use_streaming_executor -# ===== Pandas dataset formats ===== +# ===== Pandas datastream formats ===== @pytest.fixture(scope="function") def ds_pandas_single_column_format(ray_start_regular_shared): in_df = pd.DataFrame({"column_1": [1, 2, 3, 4]}) @@ -388,7 +389,7 @@ def ds_pandas_list_multi_column_format(ray_start_regular_shared): yield ray.data.from_pandas([in_df] * 4) -# ===== Arrow dataset formats ===== +# ===== Arrow datastream formats ===== @pytest.fixture(scope="function") def ds_arrow_single_column_format(ray_start_regular_shared): yield ray.data.from_arrow(pa.table({"column_1": [1, 2, 3, 4]})) @@ -424,7 +425,7 @@ def ds_list_arrow_multi_column_format(ray_start_regular_shared): yield ray.data.from_arrow([pa.table({"column_1": [1], "column_2": [1]})] * 4) -# ===== Numpy dataset formats ===== +# ===== Numpy datastream formats ===== @pytest.fixture(scope="function") def ds_numpy_single_column_tensor_format(ray_start_regular_shared): yield ray.data.from_numpy(np.arange(16).reshape((4, 2, 2))) diff --git a/python/ray/data/tests/mock_server.py b/python/ray/data/tests/mock_server.py index 42dffd7f0690..c2a81c2b5546 100644 --- a/python/ray/data/tests/mock_server.py +++ b/python/ray/data/tests/mock_server.py @@ -67,7 +67,7 @@ def stop_process(process): # being unreachable). This appears to only be an issue when using the tmp_dir # fixture as the S3 dir path. We should fix this since "session" scope should # reduce a lot of the per-test overhead (2x faster execution for IO methods in -# test_dataset.py). +# test_datastream.py). @pytest.fixture(scope="function") def s3_server(): host = "localhost" diff --git a/python/ray/data/tests/preprocessors/test_preprocessors.py b/python/ray/data/tests/preprocessors/test_preprocessors.py index 6107c742bc9b..f03fd14c2294 100644 --- a/python/ray/data/tests/preprocessors/test_preprocessors.py +++ b/python/ray/data/tests/preprocessors/test_preprocessors.py @@ -187,7 +187,7 @@ def test_pipeline_fail(): class FittablePreprocessor(Preprocessor): _is_fittable = True - def _fit(self, dataset): + def _fit(self, datastream): self.fitted_ = True return self @@ -234,7 +234,7 @@ def test_transform_all_formats(create_dummy_preprocessors, pipeline, dataset_for if pipeline: patcher = patch.object(ray.data.dataset_pipeline.DatasetPipeline, "map_batches") else: - patcher = patch.object(ray.data.dataset.Dataset, "map_batches") + patcher = patch.object(ray.data.dataset.Datastream, "map_batches") with patcher as mock_map_batches: _apply_transform(with_pandas, ds) @@ -263,7 +263,7 @@ def test_transform_all_formats(create_dummy_preprocessors, pipeline, dataset_for def test_numpy_pandas_support_transform_batch_wrong_format(create_dummy_preprocessors): - # Case 1: simple dataset. No support + # Case 1: simple datastream. No support ( with_nothing, with_pandas, @@ -290,7 +290,7 @@ def test_numpy_pandas_support_transform_batch_wrong_format(create_dummy_preproce def test_numpy_pandas_support_transform_batch_pandas(create_dummy_preprocessors): - # Case 2: pandas dataset + # Case 2: pandas datastream ( with_nothing, with_pandas, @@ -328,7 +328,7 @@ def test_numpy_pandas_support_transform_batch_pandas(create_dummy_preprocessors) def test_numpy_pandas_support_transform_batch_arrow(create_dummy_preprocessors): - # Case 3: arrow dataset + # Case 3: arrow datastream ( with_nothing, with_pandas, @@ -371,7 +371,7 @@ def test_numpy_pandas_support_transform_batch_arrow(create_dummy_preprocessors): def test_numpy_pandas_support_transform_batch_tensor(create_dummy_preprocessors): - # Case 4: tensor dataset created by from numpy data directly + # Case 4: tensor datastream created by from numpy data directly ( with_nothing, with_pandas, diff --git a/python/ray/data/tests/preprocessors/test_torch.py b/python/ray/data/tests/preprocessors/test_torch.py index d5efeb3d678a..78455bc6753f 100644 --- a/python/ray/data/tests/preprocessors/test_torch.py +++ b/python/ray/data/tests/preprocessors/test_torch.py @@ -32,7 +32,7 @@ def __repr__(self): ], ) def test_transform_images(self, transform): - dataset = ray.data.from_items( + datastream = ray.data.from_items( [ {"image": np.zeros((32, 32, 3)), "label": 0}, {"image": np.zeros((32, 32, 3)), "label": 1}, @@ -40,19 +40,19 @@ def test_transform_images(self, transform): ) preprocessor = TorchVisionPreprocessor(columns=["image"], transform=transform) - transformed_dataset = preprocessor.transform(dataset) + transformed_datastream = preprocessor.transform(datastream) - assert transformed_dataset.schema().names == ["image", "label"] + assert transformed_datastream.schema().names == ["image", "label"] transformed_images = [ - record["image"] for record in transformed_dataset.take_all() + record["image"] for record in transformed_datastream.take_all() ] assert all(image.shape == (3, 32, 32) for image in transformed_images) assert all(image.dtype == np.double for image in transformed_images) - labels = {record["label"] for record in transformed_dataset.take_all()} + labels = {record["label"] for record in transformed_datastream.take_all()} assert labels == {0, 1} def test_batch_transform_images(self): - dataset = ray.data.from_items( + datastream = ray.data.from_items( [ {"image": np.zeros((32, 32, 3)), "label": 0}, {"image": np.zeros((32, 32, 3)), "label": 1}, @@ -70,19 +70,19 @@ def test_batch_transform_images(self): columns=["image"], transform=transform, batched=True ) - transformed_dataset = preprocessor.transform(dataset) + transformed_datastream = preprocessor.transform(datastream) - assert transformed_dataset.schema().names == ["image", "label"] + assert transformed_datastream.schema().names == ["image", "label"] transformed_images = [ - record["image"] for record in transformed_dataset.take_all() + record["image"] for record in transformed_datastream.take_all() ] assert all(image.shape == (3, 64, 64) for image in transformed_images) assert all(image.dtype == np.double for image in transformed_images) - labels = {record["label"] for record in transformed_dataset.take_all()} + labels = {record["label"] for record in transformed_datastream.take_all()} assert labels == {0, 1} def test_transform_ragged_images(self): - dataset = ray.data.from_items( + datastream = ray.data.from_items( [ {"image": np.zeros((16, 16, 3)), "label": 0}, {"image": np.zeros((32, 32, 3)), "label": 1}, @@ -91,22 +91,22 @@ def test_transform_ragged_images(self): transform = transforms.ToTensor() preprocessor = TorchVisionPreprocessor(columns=["image"], transform=transform) - transformed_dataset = preprocessor.transform(dataset) + transformed_datastream = preprocessor.transform(datastream) - assert transformed_dataset.schema().names == ["image", "label"] + assert transformed_datastream.schema().names == ["image", "label"] transformed_images = [ - record["image"] for record in transformed_dataset.take_all() + record["image"] for record in transformed_datastream.take_all() ] assert sorted(image.shape for image in transformed_images) == [ (3, 16, 16), (3, 32, 32), ] assert all(image.dtype == np.double for image in transformed_images) - labels = {record["label"] for record in transformed_dataset.take_all()} + labels = {record["label"] for record in transformed_datastream.take_all()} assert labels == {0, 1} def test_invalid_transform_raises_value_error(self): - dataset = ray.data.from_items( + datastream = ray.data.from_items( [ {"image": np.zeros((32, 32, 3)), "label": 0}, {"image": np.zeros((32, 32, 3)), "label": 1}, @@ -118,7 +118,7 @@ def test_invalid_transform_raises_value_error(self): preprocessor = TorchVisionPreprocessor(columns=["image"], transform=transform) with pytest.raises(ValueError): - preprocessor.transform(dataset).materialize() + preprocessor.transform(datastream).materialize() if __name__ == "__main__": diff --git a/python/ray/data/tests/test_dataset_all_to_all.py b/python/ray/data/tests/test_dataset_all_to_all.py index f7120da3dbfc..c98021da8b13 100644 --- a/python/ray/data/tests/test_dataset_all_to_all.py +++ b/python/ray/data/tests/test_dataset_all_to_all.py @@ -212,13 +212,13 @@ def test_repartition_shuffle_arrow(ray_start_regular_shared): assert large._block_num_rows() == [500] * 20 -def test_grouped_dataset_repr(ray_start_regular_shared): +def test_grouped_datastream_repr(ray_start_regular_shared): ds = ray.data.from_items([{"key": "spam"}, {"key": "ham"}, {"key": "spam"}]) - assert repr(ds.groupby("key")) == f"GroupedData(dataset={ds!r}, key='key')" + assert repr(ds.groupby("key")) == f"GroupedData(datastream={ds!r}, key='key')" def test_groupby_arrow(ray_start_regular_shared, use_push_based_shuffle): - # Test empty dataset. + # Test empty datastream. agg_ds = ( ray.data.range_table(10) .filter(lambda r: r["value"] > 10) @@ -416,7 +416,7 @@ def _to_pandas(ds): ds = _to_pandas(ds) assert ds.sum("A") == 4950 - # Test empty dataset + # Test empty datastream ds = ray.data.range_table(10) if ds_format == "pandas": ds = _to_pandas(ds) @@ -776,7 +776,7 @@ def test_groupby_agg_bad_on(ray_start_regular_shared): ray.data.from_pandas(df).groupby("A").mean("D").materialize() with pytest.raises(ValueError): ray.data.from_pandas(df).groupby("A").mean(["B", "D"]).materialize() - # Columns for simple Dataset. + # Columns for simple Datastream. with pytest.raises(ValueError): ray.data.from_items(xs).groupby(lambda x: x % 3 == 0).mean("A").materialize() @@ -794,7 +794,7 @@ def test_groupby_agg_bad_on(ray_start_regular_shared): ray.data.from_pandas(df).mean("D").materialize() with pytest.raises(ValueError): ray.data.from_pandas(df).mean(["B", "D"]).materialize() - # Columns for simple Dataset. + # Columns for simple Datastream. with pytest.raises(ValueError): ray.data.from_items(xs).mean("A").materialize() @@ -905,7 +905,7 @@ def test_groupby_simple(ray_start_regular_shared): ("None", 3), ] - # Test empty dataset. + # Test empty datastream. ds = ray.data.from_items([]) agg_ds = ds.groupby(lambda r: r[0]).aggregate( AggregateFn( @@ -995,7 +995,7 @@ def test_groupby_simple_sum(ray_start_regular_shared, num_parts): assert nan_ds.sum() is None -def test_groupby_map_groups_for_empty_dataset(ray_start_regular_shared): +def test_groupby_map_groups_for_empty_datastream(ray_start_regular_shared): ds = ray.data.from_items([]) mapped = ds.groupby(lambda x: x % 3).map_groups(lambda x: [min(x) * min(x)]) assert mapped.count() == 0 @@ -1304,7 +1304,7 @@ def test_groupby_simple_mean(ray_start_regular_shared, num_parts): # Test built-in global mean aggregation assert ray.data.from_items(xs).repartition(num_parts).mean() == 49.5 - # Test empty dataset + # Test empty datastream assert ray.data.range(10).filter(lambda r: r > 10).mean() is None # Test built-in global mean aggregation with nans @@ -1403,7 +1403,7 @@ def test_groupby_simple_std(ray_start_regular_shared, num_parts): pd.Series(xs).std(ddof=0), ) - # Test empty dataset + # Test empty datastream assert ray.data.from_items([]).std() is None # Test edge cases assert ray.data.from_items([3]).std() == 0 @@ -1521,7 +1521,7 @@ def test_random_block_order_schema(ray_start_regular_shared): ds.schema().names == ["a", "b"] -def test_random_block_order(ray_start_regular_shared, restore_dataset_context): +def test_random_block_order(ray_start_regular_shared, restore_data_context): ctx = DataContext.get_current() ctx.execution_options.preserve_order = True @@ -1604,7 +1604,7 @@ def range(n, parallelism=200): r2 = range(100).random_shuffle().take(999) assert r1 != r2, (r1, r2) - # Test empty dataset. + # Test empty datastream. ds = ray.data.from_items([]) r1 = ds.random_shuffle() assert r1.count() == 0 @@ -1670,7 +1670,7 @@ def test_random_shuffle_with_custom_resource(ray_start_cluster): ray.init(cluster.address) - # Run dataset in "bar" nodes. + # Run datastream in "bar" nodes. ds = ray.data.read_parquet( "example://parquet_images_mini", parallelism=2, diff --git a/python/ray/data/tests/test_dataset_consumption.py b/python/ray/data/tests/test_dataset_consumption.py index 17fb18f33faa..9ca6081ee0f2 100644 --- a/python/ray/data/tests/test_dataset_consumption.py +++ b/python/ray/data/tests/test_dataset_consumption.py @@ -13,7 +13,7 @@ import ray from ray.data._internal.arrow_block import ArrowRow from ray.data._internal.block_builder import BlockBuilder -from ray.data._internal.dataset_logger import DatasetLogger +from ray.data._internal.dataset_logger import DatastreamLogger from ray.data._internal.lazy_block_list import LazyBlockList from ray.data._internal.pandas_block import PandasRow from ray.data.block import BlockAccessor, BlockMetadata @@ -66,7 +66,7 @@ def test_dataset_lineage_serialization(shutdown_only): ds = ds.random_shuffle() epoch = ds._get_epoch() uuid = ds._get_uuid() - plan_uuid = ds._plan._dataset_uuid + plan_uuid = ds._plan._datastream_uuid serialized_ds = ds.serialize_lineage() # Confirm that the original Dataset was properly copied before clearing/mutating. @@ -83,7 +83,7 @@ def test_dataset_lineage_serialization(shutdown_only): # Check Dataset state. assert ds._get_epoch() == epoch assert ds._get_uuid() == uuid - assert ds._plan._dataset_uuid == plan_uuid + assert ds._plan._datastream_uuid == plan_uuid # Check Dataset content. assert ds.count() == 10 assert sorted(ds.take()) == list(range(2, 12)) @@ -1566,7 +1566,7 @@ def test_dataset_retry_exceptions(ray_start_regular, local_path): path1 = os.path.join(local_path, "test1.csv") df1.to_csv(path1, index=False, storage_options={}) ds1 = ray.data.read_datasource(FlakyCSVDatasource(), parallelism=1, paths=path1) - ds1.write_datasource(FlakyCSVDatasource(), path=local_path, dataset_uuid="data") + ds1.write_datasource(FlakyCSVDatasource(), path=local_path, datastream_uuid="data") assert df1.equals( pd.read_csv(os.path.join(local_path, "data_000000.csv"), storage_options={}) ) @@ -1736,7 +1736,7 @@ def test_warning_execute_with_no_cpu(ray_start_cluster): cluster = ray_start_cluster cluster.add_node(num_cpus=0) - logger = DatasetLogger("ray.data._internal.plan").get_logger() + logger = DatastreamLogger("ray.data._internal.plan").get_logger() with patch.object( logger, "warning", @@ -1767,7 +1767,7 @@ def test_nowarning_execute_with_cpu(ray_start_cluster): # Create one node with CPUs to avoid triggering the Dataset warning ray.init(ray_start_cluster.address) - logger = DatasetLogger("ray.data._internal.plan").get_logger() + logger = DatastreamLogger("ray.data._internal.plan").get_logger() with patch.object( logger, "warning", diff --git a/python/ray/data/tests/test_dataset_logger.py b/python/ray/data/tests/test_dataset_logger.py index 834a8a788296..df72fb95d9b9 100644 --- a/python/ray/data/tests/test_dataset_logger.py +++ b/python/ray/data/tests/test_dataset_logger.py @@ -8,18 +8,18 @@ from datetime import datetime import ray -from ray.data._internal.dataset_logger import DatasetLogger +from ray.data._internal.dataset_logger import DatastreamLogger def test_dataset_logger(shutdown_only): ray.init() log_name, msg = "test_name", "test_message_1234" - logger = DatasetLogger(log_name) + logger = DatastreamLogger(log_name) logger.get_logger().info(msg) # Read from log file, and parse each component of emitted log row session_dir = ray._private.worker._global_node.get_session_dir_path() - log_file_path = os.path.join(session_dir, DatasetLogger.DEFAULT_DATASET_LOG_PATH) + log_file_path = os.path.join(session_dir, DatastreamLogger.DEFAULT_DATASET_LOG_PATH) with open(log_file_path, "r") as f: raw_logged_msg = f.read() ( diff --git a/python/ray/data/tests/test_dataset_map.py b/python/ray/data/tests/test_dataset_map.py index 5666e5b6536f..c71b4d973cd1 100644 --- a/python/ray/data/tests/test_dataset_map.py +++ b/python/ray/data/tests/test_dataset_map.py @@ -260,7 +260,7 @@ def test_select_columns(ray_start_regular_shared): ds3.select_columns(cols=[]).materialize() -def test_map_batches_basic(ray_start_regular_shared, tmp_path, restore_dataset_context): +def test_map_batches_basic(ray_start_regular_shared, tmp_path, restore_data_context): ctx = DataContext.get_current() ctx.execution_options.preserve_order = True @@ -614,7 +614,7 @@ def test_map_batches_actors_preserves_order(shutdown_only): ], ) def test_map_batches_batch_mutation( - ray_start_regular_shared, num_rows, num_blocks, batch_size, restore_dataset_context + ray_start_regular_shared, num_rows, num_blocks, batch_size, restore_data_context ): ctx = DataContext.get_current() ctx.execution_options.preserve_order = True diff --git a/python/ray/data/tests/test_dataset_pipeline.py b/python/ray/data/tests/test_dataset_pipeline.py index 644512dcd962..6d0e591c2650 100644 --- a/python/ray/data/tests/test_dataset_pipeline.py +++ b/python/ray/data/tests/test_dataset_pipeline.py @@ -456,19 +456,19 @@ def test_schema_peek(ray_start_regular_shared): # Multiple datasets pipe = ray.data.range(6, parallelism=6).window(blocks_per_window=2) assert pipe.schema() == int - assert pipe._first_dataset is not None + assert pipe._first_datastream is not None dss = list(pipe.iter_datasets()) assert len(dss) == 3, dss - assert pipe._first_dataset is None + assert pipe._first_datastream is None assert pipe.schema() == int # Only 1 dataset pipe = ray.data.range(1).window(blocks_per_window=2) assert pipe.schema() == int - assert pipe._first_dataset is not None + assert pipe._first_datastream is not None dss = list(pipe.iter_datasets()) assert len(dss) == 1, dss - assert pipe._first_dataset is None + assert pipe._first_datastream is None assert pipe.schema() == int # Empty datasets @@ -478,10 +478,10 @@ def test_schema_peek(ray_start_regular_shared): .window(blocks_per_window=2) ) assert pipe.schema() is None - assert pipe._first_dataset is not None + assert pipe._first_datastream is not None dss = list(pipe.iter_datasets()) assert len(dss) == 3, dss - assert pipe._first_dataset is None + assert pipe._first_datastream is None assert pipe.schema() is None diff --git a/python/ray/data/tests/test_split.py b/python/ray/data/tests/test_split.py index 9c3183548153..5442d31474ba 100644 --- a/python/ray/data/tests/test_split.py +++ b/python/ray/data/tests/test_split.py @@ -14,7 +14,7 @@ _equalize, ) from ray.data._internal.plan import ExecutionPlan -from ray.data._internal.stats import DatasetStats +from ray.data._internal.stats import DatastreamStats from ray.data._internal.split import ( _drop_empty_block_split, _generate_valid_indices, @@ -103,7 +103,7 @@ def _test_equal_split_balanced(block_sizes, num_splits): total_rows += block_size block_list = BlockList(blocks, metadata, owned_by_consumer=True) ds = Dataset( - ExecutionPlan(block_list, DatasetStats.TODO(), run_by_consumer=True), + ExecutionPlan(block_list, DatastreamStats.TODO(), run_by_consumer=True), 0, False, ) diff --git a/python/ray/data/tests/test_stats.py b/python/ray/data/tests/test_stats.py index a57c3036afc1..7e211a5a9934 100644 --- a/python/ray/data/tests/test_stats.py +++ b/python/ray/data/tests/test_stats.py @@ -5,8 +5,8 @@ import pytest import ray -from ray.data._internal.stats import _StatsActor, DatasetStats -from ray.data._internal.dataset_logger import DatasetLogger +from ray.data._internal.stats import _StatsActor, DatastreamStats +from ray.data._internal.dataset_logger import DatastreamLogger from ray.data.block import BlockMetadata from ray.data.context import DataContext from ray.tests.conftest import * # noqa @@ -39,19 +39,19 @@ def test_dataset_stats_basic(ray_start_regular_shared, enable_auto_log_stats): if context.new_execution_backend: if context.use_streaming_executor: - logger = DatasetLogger( + logger = DatastreamLogger( "ray.data._internal.execution.streaming_executor" ).get_logger( log_to_stdout=enable_auto_log_stats, ) else: - logger = DatasetLogger( + logger = DatastreamLogger( "ray.data._internal.execution.bulk_executor" ).get_logger( log_to_stdout=enable_auto_log_stats, ) else: - logger = DatasetLogger("ray.data._internal.plan").get_logger( + logger = DatastreamLogger("ray.data._internal.plan").get_logger( log_to_stdout=enable_auto_log_stats, ) with patch.object(logger, "info") as mock_logger: @@ -146,7 +146,7 @@ def test_dataset_stats_basic(ray_start_regular_shared, enable_auto_log_stats): * Extra metrics: {'obj_store_mem_alloc': N, 'obj_store_mem_freed': N, \ 'obj_store_mem_peak': N} -Dataset iterator time breakdown: +Datastream iterator time breakdown: * Total time user code is blocked: T * Total time in user code: T * Total time overall: T @@ -182,7 +182,7 @@ def test_dataset_stats_basic(ray_start_regular_shared, enable_auto_log_stats): * Extra metrics: {'obj_store_mem_alloc': N, 'obj_store_mem_freed': N, \ 'obj_store_mem_peak': N} -Dataset iterator time breakdown: +Datastream iterator time breakdown: * In ray.wait(): T * In ray.get(): T * Num blocks local: Z @@ -218,7 +218,7 @@ def test_dataset_stats_basic(ray_start_regular_shared, enable_auto_log_stats): * Extra metrics: {'obj_store_mem_alloc': N, 'obj_store_mem_freed': N, \ 'obj_store_mem_peak': N} -Dataset iterator time breakdown: +Datastream iterator time breakdown: * Total time user code is blocked: T * Total time in user code: T * Total time overall: T @@ -250,7 +250,7 @@ def test_dataset_stats_basic(ray_start_regular_shared, enable_auto_log_stats): * 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: +Datastream iterator time breakdown: * In ray.wait(): T * In ray.get(): T * In next_batch(): T @@ -274,8 +274,8 @@ def test_dataset__repr__(ray_start_regular_shared): ss2 = ds2._plan.stats().to_summary() assert canonicalize(repr(ss)) == ( - "DatasetStatsSummary(\n" - " dataset_uuid=U,\n" + "DatastreamStatsSummary(\n" + " datastream_uuid=U,\n" " base_name=None,\n" " number=N,\n" " extra_metrics={},\n" @@ -308,8 +308,8 @@ def test_dataset__repr__(ray_start_regular_shared): ")" ) assert canonicalize(repr(ss2)) == ( - "DatasetStatsSummary(\n" - " dataset_uuid=U,\n" + "DatastreamStatsSummary(\n" + " datastream_uuid=U,\n" " base_name=MapBatches(),\n" " number=N,\n" " extra_metrics={\n" @@ -343,8 +343,8 @@ def test_dataset__repr__(ray_start_regular_shared): " total_time=T,\n" " ),\n" " parents=[\n" - " DatasetStatsSummary(\n" - " dataset_uuid=U,\n" + " DatastreamStatsSummary(\n" + " datastream_uuid=U,\n" " base_name=None,\n" " number=N,\n" " extra_metrics={},\n" @@ -573,19 +573,19 @@ def test_dataset_pipeline_stats_basic(ray_start_regular_shared, enable_auto_log_ if context.new_execution_backend: if context.use_streaming_executor: - logger = DatasetLogger( + logger = DatastreamLogger( "ray.data._internal.execution.streaming_executor" ).get_logger( log_to_stdout=enable_auto_log_stats, ) else: - logger = DatasetLogger( + logger = DatastreamLogger( "ray.data._internal.execution.bulk_executor" ).get_logger( log_to_stdout=enable_auto_log_stats, ) else: - logger = DatasetLogger("ray.data._internal.plan").get_logger( + logger = DatastreamLogger("ray.data._internal.plan").get_logger( log_to_stdout=enable_auto_log_stats, ) @@ -746,10 +746,10 @@ def test_dataset_pipeline_stats_basic(ray_start_regular_shared, enable_auto_log_ 'obj_store_mem_peak': N} ##### Overall Pipeline Time Breakdown ##### -* Time stalled waiting for next dataset: T min, T max, T mean, T total +* Time stalled waiting for next datastream: T min, T max, T mean, T total DatasetPipeline iterator time breakdown: -* Waiting for next dataset: T +* Waiting for next datastream: T * In ray.wait(): T * In ray.get(): T * In next_batch(): T @@ -801,10 +801,10 @@ def test_dataset_pipeline_stats_basic(ray_start_regular_shared, enable_auto_log_ * 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 +* Time stalled waiting for next datastream: T min, T max, T mean, T total DatasetPipeline iterator time breakdown: -* Waiting for next dataset: T +* Waiting for next datastream: T * In ray.wait(): T * In ray.get(): T * In next_batch(): T @@ -877,10 +877,10 @@ def consume(split): 'obj_store_mem_peak': N} ##### Overall Pipeline Time Breakdown ##### -* Time stalled waiting for next dataset: T min, T max, T mean, T total +* Time stalled waiting for next datastream: T min, T max, T mean, T total DatasetPipeline iterator time breakdown: -* Waiting for next dataset: T +* Waiting for next datastream: T * In ray.wait(): T * In ray.get(): T * In next_batch(): T @@ -911,10 +911,10 @@ def consume(split): * 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 +* Time stalled waiting for next datastream: T min, T max, T mean, T total DatasetPipeline iterator time breakdown: -* Waiting for next dataset: T +* Waiting for next datastream: T * In ray.wait(): T * In ray.get(): T * In next_batch(): T @@ -930,7 +930,7 @@ def test_calculate_blocks_stats(ray_start_regular_shared, stage_two_block): context.optimize_fuse_stages = True block_params, block_meta_list = stage_two_block - stats = DatasetStats( + stats = DatastreamStats( stages={"Read": block_meta_list}, parent=None, ) @@ -975,11 +975,11 @@ def test_summarize_blocks(ray_start_regular_shared, stage_two_block): context.optimize_fuse_stages = True block_params, block_meta_list = stage_two_block - stats = DatasetStats( + stats = DatastreamStats( stages={"Read": block_meta_list}, parent=None, ) - stats.dataset_uuid = "test-uuid" + stats.datastream_uuid = "test-uuid" calculated_stats = stats.to_summary() summarized_lines = calculated_stats.to_string().split("\n") @@ -1050,14 +1050,14 @@ def test_summarize_blocks(ray_start_regular_shared, stage_two_block): def test_get_total_stats(ray_start_regular_shared, stage_two_block): """Tests a set of similar getter methods which pull aggregated statistics values after calculating stage-level stats: - `DatasetStats.get_max_wall_time()`, - `DatasetStats.get_total_cpu_time()`, - `DatasetStats.get_max_heap_memory()`.""" + `DatastreamStats.get_max_wall_time()`, + `DatastreamStats.get_total_cpu_time()`, + `DatastreamStats.get_max_heap_memory()`.""" context = DataContext.get_current() context.optimize_fuse_stages = True block_params, block_meta_list = stage_two_block - stats = DatasetStats( + stats = DatastreamStats( stages={"Read": block_meta_list}, parent=None, ) @@ -1074,7 +1074,7 @@ def test_get_total_stats(ray_start_regular_shared, stage_two_block): assert dataset_stats_summary.get_max_heap_memory() == peak_memory_stats.get("max") -def test_streaming_stats_full(ray_start_regular_shared, restore_dataset_context): +def test_streaming_stats_full(ray_start_regular_shared, restore_data_context): DataContext.get_current().new_execution_backend = True DataContext.get_current().use_streaming_executor = True @@ -1093,7 +1093,7 @@ def test_streaming_stats_full(ray_start_regular_shared, restore_dataset_context) * Extra metrics: \ {'obj_store_mem_alloc': N, 'obj_store_mem_freed': N, 'obj_store_mem_peak': N} -Dataset iterator time breakdown: +Datastream iterator time breakdown: * Total time user code is blocked: T * Total time in user code: T * Total time overall: T diff --git a/python/ray/data/tests/test_streaming_integration.py b/python/ray/data/tests/test_streaming_integration.py index 1513022f4848..42676deda296 100644 --- a/python/ray/data/tests/test_streaming_integration.py +++ b/python/ray/data/tests/test_streaming_integration.py @@ -212,7 +212,7 @@ def consume(x, times): ) -def test_e2e_option_propagation(ray_start_10_cpus_shared, restore_dataset_context): +def test_e2e_option_propagation(ray_start_10_cpus_shared, restore_data_context): DataContext.get_current().new_execution_backend = True DataContext.get_current().use_streaming_executor = True @@ -229,7 +229,7 @@ def run(): run() -def test_configure_spread_e2e(ray_start_10_cpus_shared, restore_dataset_context): +def test_configure_spread_e2e(ray_start_10_cpus_shared, restore_data_context): from ray import remote_function tasks = [] @@ -251,7 +251,7 @@ def _test_hook(fn, args, strategy): def test_scheduling_progress_when_output_blocked( - ray_start_10_cpus_shared, restore_dataset_context + ray_start_10_cpus_shared, restore_data_context ): # Processing stages should fully finish even if output is completely stalled. @@ -288,7 +288,7 @@ def func(x): assert list(it) == [[x] for x in range(1, 100)] -def test_backpressure_from_output(ray_start_10_cpus_shared, restore_dataset_context): +def test_backpressure_from_output(ray_start_10_cpus_shared, restore_data_context): # Here we set the memory limit low enough so the output getting blocked will # actually stall execution. @@ -333,7 +333,7 @@ def func(x): def test_e2e_liveness_with_output_backpressure_edge_case( - ray_start_10_cpus_shared, restore_dataset_context + ray_start_10_cpus_shared, restore_data_context ): # At least one operator is ensured to be running, if the output becomes idle. ctx = DataContext.get_current() @@ -346,7 +346,7 @@ def test_e2e_liveness_with_output_backpressure_edge_case( assert ds.take_all() == list(range(10000)) -def test_e2e_autoscaling_up(ray_start_10_cpus_shared, restore_dataset_context): +def test_e2e_autoscaling_up(ray_start_10_cpus_shared, restore_data_context): DataContext.get_current().new_execution_backend = True DataContext.get_current().use_streaming_executor = True @@ -421,7 +421,7 @@ def barrier3(x): ).take_all() -def test_e2e_autoscaling_down(ray_start_10_cpus_shared, restore_dataset_context): +def test_e2e_autoscaling_down(ray_start_10_cpus_shared, restore_data_context): DataContext.get_current().new_execution_backend = True DataContext.get_current().use_streaming_executor = True @@ -439,7 +439,7 @@ def f(x): ).map_batches(lambda x: x, batch_size=None, num_cpus=2).take_all() -def test_can_pickle(ray_start_10_cpus_shared, restore_dataset_context): +def test_can_pickle(ray_start_10_cpus_shared, restore_data_context): DataContext.get_current().new_execution_backend = True DataContext.get_current().use_streaming_executor = True @@ -452,7 +452,7 @@ def test_can_pickle(ray_start_10_cpus_shared, restore_dataset_context): assert ds2.count() == 1000000 -def test_streaming_fault_tolerance(ray_start_10_cpus_shared, restore_dataset_context): +def test_streaming_fault_tolerance(ray_start_10_cpus_shared, restore_data_context): DataContext.get_current().new_execution_backend = True DataContext.get_current().use_streaming_executor = True diff --git a/python/ray/train/examples/horovod/horovod_cifar_pbt_example.py b/python/ray/train/examples/horovod/horovod_cifar_pbt_example.py deleted file mode 120000 index bfb284ffe23d..000000000000 --- a/python/ray/train/examples/horovod/horovod_cifar_pbt_example.py +++ /dev/null @@ -1 +0,0 @@ -../../../../../release/air_tests/horovod/workloads/horovod_tune_test.py \ No newline at end of file diff --git a/python/ray/train/examples/horovod/horovod_cifar_pbt_example.py b/python/ray/train/examples/horovod/horovod_cifar_pbt_example.py new file mode 100755 index 000000000000..58e32e8e2253 --- /dev/null +++ b/python/ray/train/examples/horovod/horovod_cifar_pbt_example.py @@ -0,0 +1,195 @@ +import numpy as np +import torch +import torch.nn as nn +from torch.utils.data import DataLoader +import torchvision +import torchvision.transforms as transforms +from torchvision.models import resnet18 + +import ray +from ray.air import RunConfig, session +from ray.air.config import ScalingConfig, FailureConfig, CheckpointConfig +from ray.air.checkpoint import Checkpoint +import ray.train.torch +from ray.train.horovod import HorovodTrainer +from ray import tune +from ray.tune.schedulers import create_scheduler +from ray.tune.tune_config import TuneConfig +from ray.tune.tuner import Tuner +from ray.tune.utils.release_test_util import ProgressCallback + +# The long running version starts 4 trials while only 2 can be run at a time. +# Thus trials are paused and restored at all times so that every trial can make +# progress. The PBT scheduler also applies perturbation and mutation, +# which also involves pausing and restoring. +# The intention is to stress test the pausing and restoring of trials, +# especially that there should be no GPU memory leak. + +# TODO(ml-team): This test is very low signal at the moment. +# We should further trim it down. + +CIFAR10_STATS = { + "mean": (0.4914, 0.4822, 0.4465), + "std": (0.2023, 0.1994, 0.2010), +} + + +def train_loop_per_worker(config): + import horovod.torch as hvd + + hvd.init() + device = ray.train.torch.get_device() + net = resnet18().to(device) + optimizer = torch.optim.SGD( + net.parameters(), + lr=config["lr"], + ) + epoch = 0 + + checkpoint = session.get_checkpoint() + if checkpoint: + checkpoint_dict = checkpoint.to_dict() + model_state = checkpoint_dict["model_state"] + optimizer_state = checkpoint_dict["optimizer_state"] + epoch = checkpoint_dict["epoch"] + 1 + + net.load_state_dict(model_state) + optimizer.load_state_dict(optimizer_state) + + criterion = nn.CrossEntropyLoss() + optimizer = hvd.DistributedOptimizer(optimizer) + np.random.seed(1 + hvd.rank()) + torch.manual_seed(1234) + # To ensure consistent initialization across workers, + hvd.broadcast_parameters(net.state_dict(), root_rank=0) + hvd.broadcast_optimizer_state(optimizer, root_rank=0) + + trainset = ray.get(config["data"]) + + train_sampler = torch.utils.data.distributed.DistributedSampler( + trainset, num_replicas=hvd.size(), rank=hvd.rank() + ) + + # Note, don't set `num_workers` in DataLoader (not even 1), + # as that will separately start multiple processes (each corresponding to 1 worker) + # to load the data. This is known to cause issues with Ray. + trainloader = DataLoader( + trainset, batch_size=int(config["batch_size"]), sampler=train_sampler + ) + + for epoch in range(epoch, 40): # loop over the dataset multiple times + running_loss = 0.0 + epoch_steps = 0 + for i, data in enumerate(trainloader): + # get the inputs; data is a list of [inputs, labels] + inputs, labels = data + inputs, labels = inputs.to(device), labels.to(device) + + # zero the parameter gradients + optimizer.zero_grad() + + # forward + backward + optimize + outputs = net(inputs) + loss = criterion(outputs, labels) + loss.backward() + optimizer.step() + + # print statistics + running_loss += loss.item() + epoch_steps += 1 + + if i % 2000 == 1999: # print every 2000 mini-batches + print( + "[%d, %5d] loss: %.3f" + % (epoch + 1, i + 1, running_loss / epoch_steps) + ) + + if config["smoke_test"]: + break + + checkpoint = Checkpoint.from_dict( + dict( + model_state=net.state_dict(), + optimizer_state=optimizer.state_dict(), + epoch=epoch, + ) + ) + session.report(dict(loss=running_loss / epoch_steps), checkpoint=checkpoint) + + +if __name__ == "__main__": + import argparse + + parser = argparse.ArgumentParser() + parser.add_argument( + "--smoke-test", action="store_true", help=("Finish quickly for testing.") + ) + args = parser.parse_args() + + if args.smoke_test: + ray.init() + else: + ray.init(address="auto") # assumes ray is started with ray up + + transform_train = transforms.Compose( + [ + transforms.RandomCrop(32, padding=4), + transforms.RandomHorizontalFlip(), + transforms.ToTensor(), + transforms.Normalize(CIFAR10_STATS["mean"], CIFAR10_STATS["std"]), + ] + ) # meanstd transformation + + dataset = torchvision.datasets.CIFAR10( + root="/tmp/data_cifar", train=True, download=True, transform=transform_train + ) + + horovod_trainer = HorovodTrainer( + train_loop_per_worker=train_loop_per_worker, + scaling_config=ScalingConfig( + use_gpu=False if args.smoke_test else True, + num_workers=2, + ), + train_loop_config={"batch_size": 64, "data": ray.put(dataset)}, + ) + + # ensure that checkpointing works. + pbt = create_scheduler( + "pbt", + perturbation_interval=1, # To make perturb more often. + hyperparam_mutations={ + "train_loop_config": {"lr": tune.uniform(0.001, 0.1)}, + }, + ) + + tuner = Tuner( + horovod_trainer, + param_space={ + "train_loop_config": { + "lr": 0.1 + if args.smoke_test + else tune.grid_search([0.1 * i for i in range(1, 5)]), # 4 trials + "smoke_test": args.smoke_test, + } + }, + tune_config=TuneConfig( + num_samples=2 if args.smoke_test else 1, + metric="loss", + mode="min", + scheduler=pbt, + ), + run_config=RunConfig( + stop={"training_iteration": 1} if args.smoke_test else None, + failure_config=FailureConfig(fail_fast=False), + checkpoint_config=CheckpointConfig(num_to_keep=1), + callbacks=[ProgressCallback()], + ), + ) + + result_grid = tuner.fit() + + # Make sure trials do not fail. + for result in result_grid: + assert not result.error + + print("Best hyperparameters found were: ", result_grid.get_best_result().config) diff --git a/python/ray/train/tests/pytorch_pbt_failure.py b/python/ray/train/tests/pytorch_pbt_failure.py deleted file mode 120000 index ea70152e5b3c..000000000000 --- a/python/ray/train/tests/pytorch_pbt_failure.py +++ /dev/null @@ -1 +0,0 @@ -../../../../release/long_running_distributed_tests/workloads/pytorch_pbt_failure.py \ No newline at end of file diff --git a/python/ray/train/tests/pytorch_pbt_failure.py b/python/ray/train/tests/pytorch_pbt_failure.py new file mode 100644 index 000000000000..3dfbc9801436 --- /dev/null +++ b/python/ray/train/tests/pytorch_pbt_failure.py @@ -0,0 +1,79 @@ +import argparse +import sys + +import numpy as np + +import ray +from ray import tune +from ray.air.config import RunConfig, ScalingConfig, FailureConfig +from ray.train.examples.pytorch.tune_cifar_torch_pbt_example import train_func +from ray.train.torch import TorchConfig, TorchTrainer +from ray.tune.schedulers import PopulationBasedTraining +from ray.tune.tune_config import TuneConfig +from ray.tune.tuner import Tuner +from ray.tune.utils.mock import FailureInjectorCallback +from ray.tune.utils.release_test_util import ProgressCallback + +parser = argparse.ArgumentParser() +parser.add_argument( + "--smoke-test", + action="store_true", + default=False, + help="Finish quickly for training.", +) +args = parser.parse_args() + +ray.init(address="auto" if not args.smoke_test else None, log_to_driver=True) +num_training_workers = 1 if args.smoke_test else 3 + +trainer = TorchTrainer( + train_func, + scaling_config=ScalingConfig( + num_workers=num_training_workers, + use_gpu=not args.smoke_test, + ), + torch_config=TorchConfig(backend="gloo"), +) + + +pbt_scheduler = PopulationBasedTraining( + time_attr="training_iteration", + perturbation_interval=1, + hyperparam_mutations={ + "train_loop_config": { + # distribution for resampling + "lr": lambda: np.random.uniform(0.001, 1), + # allow perturbations within this set of categorical values + "momentum": [0.8, 0.9, 0.99], + } + }, +) + +tuner = Tuner( + trainer, + param_space={ + "train_loop_config": { + "lr": tune.choice([0.001, 0.01, 0.1]), + "momentum": 0.8, + "head_location": None, + "worker_locations": None, + "test_mode": args.smoke_test, + "batch_size": 128 * num_training_workers, + # For the long running test, we want the training to run forever, + # and it will be terminated by the release test infra. + "epochs": 1 if args.smoke_test else sys.maxsize, + } + }, + tune_config=TuneConfig( + num_samples=4, metric="loss", mode="min", scheduler=pbt_scheduler + ), + run_config=RunConfig( + stop={"training_iteration": 1} if args.smoke_test else None, + failure_config=FailureConfig(max_failures=-1), + callbacks=[FailureInjectorCallback(time_between_checks=90), ProgressCallback()], + ), +) + +results = tuner.fit() + +print(results.get_best_result(metric="loss", mode="min")) diff --git a/release/air_examples/gptj_deepspeed_finetuning/test_myst_doc.py b/release/air_examples/gptj_deepspeed_finetuning/test_myst_doc.py deleted file mode 120000 index c265ccc7b062..000000000000 --- a/release/air_examples/gptj_deepspeed_finetuning/test_myst_doc.py +++ /dev/null @@ -1 +0,0 @@ -../../../doc/test_myst_doc.py \ No newline at end of file diff --git a/release/air_examples/gptj_deepspeed_finetuning/test_myst_doc.py b/release/air_examples/gptj_deepspeed_finetuning/test_myst_doc.py new file mode 100644 index 000000000000..279e70969b15 --- /dev/null +++ b/release/air_examples/gptj_deepspeed_finetuning/test_myst_doc.py @@ -0,0 +1,76 @@ +"""Convert a jupytext-compliant format in to a python script +and execute it with parsed arguments. + +Any cell with 'remove-cell-ci' tag in metadata will not be included +in the converted python script. +""" + +import argparse +import subprocess +import sys +import tempfile +from pathlib import Path + +import jupytext + +parser = argparse.ArgumentParser(description=__doc__) +parser.add_argument( + "--path", + help="path to the jupytext-compatible file", +) +parser.add_argument( + "--find-recursively", + action="store_true", + help="if true, will attempt to find path recursively in cwd", +) +parser.add_argument( + "--no-postprocess", + action="store_true", + help="if true, will not postprocess the notebook", +) + + +def filter_out_cells_with_remove_cell_ci_tag(cells: list): + """Filters out cells which contain the 'remove-cell-ci' tag in metadata""" + + def should_keep_cell(cell): + tags = cell.metadata.get("tags") + if tags: + # Both - and _ for consistent behavior with built-in tags + return "remove_cell_ci" not in tags and "remove-cell-ci" not in tags + return True + + return [cell for cell in cells if should_keep_cell(cell)] + + +def postprocess_notebook(notebook): + notebook.cells = filter_out_cells_with_remove_cell_ci_tag(notebook.cells) + return notebook + + +if __name__ == "__main__": + + args, remainder = parser.parse_known_args() + + path = Path(args.path) + cwd = Path.cwd() + if args.find_recursively and not path.exists(): + path = next((p for p in cwd.rglob("*") if str(p).endswith(args.path)), None) + assert path and path.exists() + + with open(path, "r") as f: + notebook = jupytext.read(f) + + if not args.no_postprocess: + notebook = postprocess_notebook(notebook) + + name = "" + with tempfile.NamedTemporaryFile("w", delete=False) as f: + jupytext.write(notebook, f, fmt="py:percent") + name = f.name + + remainder.insert(0, name) + remainder.insert(0, sys.executable) + + # Run the notebook + subprocess.run(remainder, check=True) diff --git a/scripts/bazel.py b/scripts/bazel.py deleted file mode 120000 index e73f36e03dca..000000000000 --- a/scripts/bazel.py +++ /dev/null @@ -1 +0,0 @@ -../ci/run/bazel.py \ No newline at end of file diff --git a/scripts/bazel.py b/scripts/bazel.py new file mode 100755 index 000000000000..30549037fcb9 --- /dev/null +++ b/scripts/bazel.py @@ -0,0 +1,265 @@ +#!/usr/bin/env python + +import ast +import errno +import json +import os +import re +import subprocess +import stat +import sys + +from collections import defaultdict, OrderedDict + + +def textproto_format(space, key, value, json_encoder): + """Rewrites a key-value pair from textproto as JSON.""" + if value.startswith(b'"'): + evaluated = ast.literal_eval(value.decode("utf-8")) + value = json_encoder.encode(evaluated).encode("utf-8") + return b'%s["%s", %s]' % (space, key, value) + + +def textproto_split(input_lines, json_encoder): + """When given e.g. the output of "bazel aquery --output=textproto", + yields each top-level item as a string formatted as JSON (if an encoder is + given) or Python AST. + The input MUST be formatted neatly line-by-line, as follows: + actions { + mnemonic: "Genrule" + environment_variables { + key: "CC" + value: "clang" + } + ... + } + targets { + id: "0" + label: "//:target" + rule_class_id: "0" + } + """ + outputs = [] + re_flags = re.M + pat_open = re.compile(b"^(\\s*)([-\\w:]+)(\\s*){$", flags=re_flags) + pat_line = re.compile(b"^(\\s*)([-\\w]+): (.*)$", flags=re_flags) + pat_close = re.compile(b"}$", flags=re_flags) + prev_comma = False + prev_tail = b"" + for full_line in input_lines: + pieces = re.split(b"(\\r|\\n)", full_line, 1) + pieces[1:] = [b"".join(pieces[1:])] + [line, tail] = pieces + next_line = pat_open.sub(b'\\1["\\2",\\3[', line) + outputs.append( + b"" if not prev_comma else b"]" if next_line.endswith(b"}") else b"," + ) + next_line = pat_close.sub(b"]", next_line) + next_line = pat_line.sub( + lambda m: textproto_format(*(m.groups() + (json_encoder,))), next_line + ) + outputs.append(prev_tail + next_line) + if line == b"}": + yield b"".join(outputs) + del outputs[:] + prev_comma = line != b"}" and ( + next_line.endswith(b"]") or next_line.endswith(b'"') + ) + prev_tail = tail + if len(outputs) > 0: + yield b"".join(outputs) + del outputs[:] + + +def textproto_parse(stream, encoding, json_encoder): + for item in textproto_split(stream, json_encoder): + yield json.loads(item.decode(encoding)) + + +class Bazel(object): + encoding = "utf-8" + + def __init__(self, program=None): + if program is None: + program = os.getenv("BAZEL_EXECUTABLE", "bazel") + self.argv = (program,) + self.extra_args = ("--show_progress=no",) + + def _call(self, command, *args): + return subprocess.check_output( + self.argv + (command,) + args[:1] + self.extra_args + args[1:], + stdin=subprocess.PIPE, + ) + + def info(self, *args): + result = OrderedDict() + for line in self._call("info", *args).splitlines(): + (key, value) = line.split(b":", 1) + if value.startswith(b" "): + value = value[1:] + result[key.decode(self.encoding)] = value.decode(self.encoding) + return result + + def aquery(self, *args): + out = self._call("aquery", "--output=jsonproto", *args) + return json.loads(out.decode(self.encoding)) + + +def parse_aquery_shell_calls(aquery_results): + """Extracts and yields the command lines representing the genrule() rules + from Bazel aquery results. + """ + for action in aquery_results["actions"]: + if action["mnemonic"] != "Genrule": + continue + yield action["arguments"] + + +def parse_aquery_output_artifacts(aquery_results): + """Extracts and yields the file paths representing the output artifact + from the provided Bazel aquery results. + + To understand the output of aquery command in textproto format, try: + bazel aquery --include_artifacts=true --output=jsonproto \ + 'mnemonic("Genrule", deps(//:*))' + """ + fragments = {} + for fragment in aquery_results["pathFragments"]: + fragments[fragment["id"]] = fragment + + artifacts = {} + for artifact in aquery_results["artifacts"]: + artifacts[artifact["id"]] = artifact + + def _path(fragment_id): + fragment = fragments[fragment_id] + parent = _path(fragment["parentId"]) if "parentId" in fragment else [] + return parent + [fragment["label"]] + + for action in aquery_results["actions"]: + for output_id in action["outputIds"]: + path = os.path.join(*_path(artifacts[output_id]["pathFragmentId"])) + yield path + + +def textproto2json(infile, outfile): + """Translates the output of bazel aquery --output=textproto into JSON. + Useful for later command-line manipulation. + + Args: + infile: The binary input stream. + outfile: The binary output stream. + """ + json_encoder = json.JSONEncoder(indent=2) + encoding = "utf-8" + for obj in textproto_parse(infile, encoding, json_encoder): + outfile.write((json_encoder.encode(obj) + "\n").encode(encoding)) + + +def preclean(bazel_aquery): + """Cleans up any genrule() outputs for the provided target(s). + + This is useful for forcing genrule actions to re-run, because the _true_ + outputs of those actions can include a larger set of files (e.g. files + copied to the workspace) which Bazel is unable to detect changes to (or + delete changes of). + + Usually, you would run this script along with 'git clean -f', to make sure + Bazel re-copies outputs the next time a build occurs. + """ + result = 0 + bazel = Bazel() + aquery_results = bazel.aquery("--include_artifacts=true", bazel_aquery) + for path in parse_aquery_output_artifacts(aquery_results): + try: + if sys.platform == "win32": + os.chmod(path, stat.S_IWRITE) # Needed to remove read-only bit + os.remove(path) + except IOError as ex: + if ex.errno != errno.ENOENT: + sys.stderr.write(str(ex) + "\n") + result = result or ex.errno + return result + + +def shellcheck(bazel_aquery, *shellcheck_argv): + """Runs shellcheck with the provided argument(s) on all targets that match + the given Bazel aquery. + + Args: + bazel_aquery: A Bazel aquery expression (e.g. "//:*") + shellcheck_argv: The command-line arguments to call for shellcheck. + Note that the first entry should be the shellcheck program itself. + If omitted, will simply call "shellcheck". + + Returns: + The exit code of shellcheck. + """ + bazel = Bazel() + shellcheck_argv = list(shellcheck_argv) or ["shellcheck"] + all_script_infos = defaultdict(lambda: []) + aquery_results = bazel.aquery("--include_artifacts=false", bazel_aquery) + shell_calls = list(parse_aquery_shell_calls(aquery_results)) + for shell_args in shell_calls: + shname = os.path.basename(os.path.splitext(shell_args[0])[0]).lower() + finished_options = False + i = 1 + while i < len(shell_args): + if finished_options or not shell_args[i].startswith("-"): + all_script_infos[shname].append((shell_args[i], None)) + elif shell_args[i] == "--": + finished_options = True + elif shell_args[i] in ("-o", "+o"): + i += 1 + elif shell_args[i] == "-c": + all_script_infos[shname].append((None, shell_args[i + 1])) + break + i += 1 + + result = 0 + bazel_execution_root = None + for shell, script_infos in all_script_infos.items(): + scripts_combined = [] + has_stdin = False + filenames = [] + for script_file, script_text in script_infos: + if script_file is not None: + filenames.append(script_file) + if script_text is not None: + has_stdin = True + flatc = "host/bin/external/com_github_google_flatbuffers/flatc" + if flatc not in script_text: + statements = ["if test -t 0; then", script_text, "fi"] + scripts_combined.append("\n".join(statements)) + if has_stdin: + filenames.insert(0, "-") + if shell.endswith("sh"): + if bazel_execution_root is None: + bazel_execution_root = bazel.info()["execution_root"] + cwd = bazel_execution_root + cmdargs = ["--shell=" + shell, "--external-sources"] + filenames + cmdargs = shellcheck_argv + cmdargs + proc = subprocess.Popen(cmdargs, stdin=subprocess.PIPE, cwd=cwd) + try: + proc.communicate("\n".join(scripts_combined).encode("utf-8")) + finally: + proc.wait() + result = result or proc.returncode + return result + + +def main(program, command, *command_args): + result = 0 + if command == textproto2json.__name__: + result = textproto2json(sys.stdin.buffer, sys.stdout.buffer, *command_args) + elif command == shellcheck.__name__: + result = shellcheck(*command_args) + elif command == preclean.__name__: + result = preclean(*command_args) + else: + raise ValueError("Unrecognized command: " + command) + return result + + +if __name__ == "__main__": + sys.exit(main(*sys.argv) or 0) diff --git a/scripts/bazel_sharding.py b/scripts/bazel_sharding.py deleted file mode 120000 index d16853e6f089..000000000000 --- a/scripts/bazel_sharding.py +++ /dev/null @@ -1 +0,0 @@ -../ci/run/bazel_sharding/bazel_sharding.py \ No newline at end of file diff --git a/scripts/bazel_sharding.py b/scripts/bazel_sharding.py new file mode 100644 index 000000000000..93be178fd5a9 --- /dev/null +++ b/scripts/bazel_sharding.py @@ -0,0 +1,427 @@ +#!/usr/bin/env python3 +# +# Copyright 2021 Google Inc. All rights reserved. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# BASED ON https://github.com/philwo/bazel-utils/blob/main/sharding/sharding.py + +from collections import defaultdict +from dataclasses import dataclass +from typing import Iterable, List, Optional, Set, Tuple +import argparse +import os +import re +import shlex +import subprocess +import sys +import xml.etree.ElementTree as ET + + +@dataclass +class BazelRule: + """ + Dataclass representing a bazel py_test rule (BUILD entry). + + Only the subset of fields we care about is included. + """ + + name: str + size: str + timeout: Optional[str] = None + + def __post_init__(self): + assert self.size in ("small", "medium", "large", "enormous") + assert self.timeout in (None, "short", "moderate", "long", "eternal") + + @property + def actual_timeout_s(self) -> float: + # See https://bazel.build/reference/be/common-definitions + # Timeout takes priority over size + if self.timeout == "short": + return 60 + if self.timeout == "moderate": + return 60 * 5 + if self.timeout == "long": + return 60 * 15 + if self.timeout == "eternal": + return 60 * 60 + if self.size == "small": + return 60 + if self.size == "medium": + return 60 * 5 + if self.size == "large": + return 60 * 15 + if self.size == "enormous": + return 60 * 60 + + def __lt__(self, other: "BazelRule") -> bool: + return (self.name, self.actual_timeout_s) < (other.name, other.actual_timeout_s) + + def __hash__(self) -> int: + return self.name.__hash__() + + @classmethod + def from_xml_element(cls, element: ET.Element) -> "BazelRule": + """Create a BazelRule from an XML element. + + The XML element is expected to be produced by the + ``bazel query --output=xml`` command. + """ + name = element.get("name") + all_string_tags = element.findall("string") + size = next( + (tag.get("value") for tag in all_string_tags if tag.get("name") == "size"), + "medium", + ) + timeout = next( + ( + tag.get("value") + for tag in all_string_tags + if tag.get("name") == "timeout" + ), + None, + ) + return cls(name=name, size=size, timeout=timeout) + + +def quote_targets(targets: Iterable[str]) -> str: + """Quote each target in a list so that it can be passed used in subprocess.""" + return (" ".join(shlex.quote(t) for t in targets)) if targets else "" + + +def partition_targets(targets: Iterable[str]) -> Tuple[List[str], List[str]]: + """ + Given a list of string targets, partition them into included and excluded + lists depending on whether they start with a - (exclude) or not (include). + """ + included_targets, excluded_targets = set(), set() + for target in targets: + if target[0] == "-": + assert not target[1] == "-", f"Double negation is not allowed: {target}" + excluded_targets.add(target[1:]) + else: + included_targets.add(target) + return included_targets, excluded_targets + + +def split_tag_filters(tag_str: str) -> Tuple[Set[str], Set[str]]: + """Split tag_filters string into include & exclude tags.""" + split_tags = tag_str.split(",") if tag_str else [] + return partition_targets(split_tags) + + +def generate_regex_from_tags(tags: Iterable[str]) -> str: + """Turn tag filters into a regex used in bazel query.""" + return "|".join([f"(\\b{re.escape(tag)}\\b)" for tag in tags]) + + +def get_target_expansion_query( + targets: Iterable[str], + tests_only: bool, + exclude_manual: bool, + include_tags: Optional[Iterable[str]] = None, + exclude_tags: Optional[Iterable[str]] = None, +) -> str: + """Generate the bazel query to obtain individual rules.""" + included_targets, excluded_targets = partition_targets(targets) + + included_targets = quote_targets(included_targets) + excluded_targets = quote_targets(excluded_targets) + + query = f"set({included_targets})" + + if include_tags: + tags_regex = generate_regex_from_tags(include_tags) + # Each rule has to have at least one tag from + # include_tags + query = f'attr("tags", "{tags_regex}", {query})' + + if tests_only: + # Discard any non-test rules + query = f"tests({query})" + + if excluded_targets: + # Exclude the targets we do not want + excluded_set = f"set({excluded_targets})" + query = f"{query} except {excluded_set}" + + if exclude_manual: + # Exclude targets with 'manual' tag + exclude_tags = exclude_tags or set() + exclude_tags.add("manual") + + if exclude_tags: + # Exclude targets which have at least one exclude_tag + tags_regex = generate_regex_from_tags(exclude_tags) + query = f'{query} except attr("tags", "{tags_regex}", set({included_targets}))' + + return query + + +def run_bazel_query(query: str, debug: bool) -> ET.Element: + """Runs bazel query with XML output format. + + We need the XML to obtain rule metadata such as + size, timeout, etc. + """ + args = ["bazel", "query", "--output=xml", query] + if debug: + print(f"$ {args}", file=sys.stderr) + sys.stderr.flush() + p = subprocess.run( + args, + check=True, + stdout=subprocess.PIPE, + errors="replace", + universal_newlines=True, + ) + output = p.stdout.strip() + return ET.fromstring(output) if output else None + + +def extract_rules_from_xml(element: ET.Element) -> List[BazelRule]: + """Extract BazelRules from the XML obtained from ``bazel query --output=xml``.""" + xml_rules = element.findall("rule") + return [BazelRule.from_xml_element(element) for element in xml_rules] + + +def group_rules_by_time_needed( + rules: List[BazelRule], +) -> List[Tuple[float, List[BazelRule]]]: + """ + Return a list of tuples of (timeout in seconds, list of rules) + sorted descending. + """ + grouped_rules = defaultdict(list) + for rule in rules: + grouped_rules[rule.actual_timeout_s].append(rule) + for timeout in grouped_rules: + grouped_rules[timeout] = sorted(grouped_rules[timeout]) + return sorted(grouped_rules.items(), key=lambda x: x[0], reverse=True) + + +def get_rules_for_shard_naive( + rules_grouped_by_time: List[Tuple[float, List[BazelRule]]], index: int, count: int +) -> List[str]: + """Create shards by assigning the same number of rules to each shard.""" + all_rules = [] + for _, rules in rules_grouped_by_time: + all_rules.extend(rules) + shard = sorted(all_rules)[index::count] + return [rule.name for rule in shard] + + +def add_rule_to_best_shard( + rule_to_add: BazelRule, shards: List[List[BazelRule]], optimum: float +): + """Adds a rule to the best shard. + + The best shard is determined in the following fashion: + 1. Pick first shard which is below optimum, + 2. If no shard is below optimum, pick the shard closest + to optimum. + """ + first_shard_index_below_optimum = None + shard_index_right_above_optimum = None + shard_index_right_above_optimum_time = None + for i, shard in enumerate(shards): + # Total time the shard needs to run so far + shard_time = sum(rule.actual_timeout_s for rule in shard) + # Total time the shard would need to run with the rule_to_add + shard_time_with_item = shard_time + rule_to_add.actual_timeout_s + + if shard_time_with_item < optimum: + # If there's a shard below optimum, just use that + first_shard_index_below_optimum = i + break + elif ( + shard_index_right_above_optimum is None + or shard_index_right_above_optimum_time > shard_time_with_item + ): + # Otherwise, pick the shard closest to optimum + shard_index_right_above_optimum = i + shard_index_right_above_optimum_time = shard_time_with_item + if first_shard_index_below_optimum is not None: + best_shard_index = first_shard_index_below_optimum + else: + best_shard_index = shard_index_right_above_optimum + + shards[best_shard_index].append(rule_to_add) + + +def get_rules_for_shard_optimal( + rules_grouped_by_time: List[Tuple[float, List[BazelRule]]], index: int, count: int +) -> List[str]: + """Creates shards by trying to make sure each shard takes around the same time. + + We use a simple heuristic here (as this problem is NP-complete): + 1. Determine how long one shard would take if they were ideally balanced + (this may be impossible to attain, but that's fine). + 2. Allocate the next biggest item into the first shard that is below the optimum. + 3. If there's no shard below optimium, choose the shard closest to optimum. + + This works very well for our usecase and is fully deterministic. + + ``rules_grouped_by_time`` is expected to be a list of tuples of + (timeout in seconds, list of rules) sorted by timeout descending. + """ + # For sanity checks later. + all_rules = [] + for _, rules in rules_grouped_by_time: + all_rules.extend(rules) + + # Instantiate the shards, each represented by a list. + shards: List[List[BazelRule]] = [list() for _ in range(count)] + + # The theoretical optimum we are aiming for. Note that this may be unattainable + # as it doesn't take into account that tests are discrete and cannot be split. + # This is however fine, because it should only serve as a guide which shard to + # add the next test to. + optimum = ( + sum(timeout * len(rules) for timeout, rules in rules_grouped_by_time) / count + ) + + def get_next_longest_rule() -> BazelRule: + """ + Get the next longest (taking up the most time) BazelRule from the + ``rules_grouped_by_time`` list. + """ + item = None + for _, items in rules_grouped_by_time: + if items: + return items.pop() + return item + + rule_to_add = get_next_longest_rule() + while rule_to_add: + add_rule_to_best_shard(rule_to_add, shards, optimum) + rule_to_add = get_next_longest_rule() + + # Sanity checks. + num_all_rules = sum(len(shard) for shard in shards) + + # Make sure that there are no duplicate rules. + all_rules_set = set() + for shard in shards: + all_rules_set = all_rules_set.union(set(shard)) + assert len(all_rules_set) == num_all_rules, ( + f"num of unique rules {len(all_rules_set)} " + f"doesn't match num of rules {num_all_rules}" + ) + + # Make sure that all rules have been included in the shards. + assert all_rules_set == set(all_rules_set), ( + f"unique rules after sharding {len(all_rules_set)} " + f"doesn't match unique rules after sharding {num_all_rules}" + ) + + print( + f"get_rules_for_shard statistics:\n\tOptimum: {optimum} seconds\n" + + "\n".join( + ( + f"\tShard {i}: {len(shard)} rules, " + f"{sum(rule.actual_timeout_s for rule in shard)} seconds" + ) + for i, shard in enumerate(shards) + ), + file=sys.stderr, + ) + return sorted([rule.name for rule in shards[index]]) + + +def main( + targets: List[str], + *, + index: int, + count: int, + tests_only: bool = False, + exclude_manual: bool = False, + tag_filters: Optional[str] = None, + sharding_strategy: str = "optimal", + debug: bool = False, +) -> List[str]: + include_tags, exclude_tags = split_tag_filters(tag_filters) + + query = get_target_expansion_query( + targets, tests_only, exclude_manual, include_tags, exclude_tags + ) + xml_output = run_bazel_query(query, debug) + rules = extract_rules_from_xml(xml_output) + rules_grouped_by_time = group_rules_by_time_needed(rules) + if sharding_strategy == "optimal": + rules_for_this_shard = get_rules_for_shard_optimal( + rules_grouped_by_time, index, count + ) + else: + rules_for_this_shard = get_rules_for_shard_naive( + rules_grouped_by_time, index, count + ) + return rules_for_this_shard + + +if __name__ == "__main__": + parser = argparse.ArgumentParser(description="Expand and shard Bazel targets.") + parser.add_argument("--debug", action="store_true") + parser.add_argument("--tests_only", action="store_true") + parser.add_argument("--exclude_manual", action="store_true") + parser.add_argument( + "--index", type=int, default=os.getenv("BUILDKITE_PARALLEL_JOB", 1) + ) + parser.add_argument( + "--count", type=int, default=os.getenv("BUILDKITE_PARALLEL_JOB_COUNT", 1) + ) + parser.add_argument( + "--tag_filters", + type=str, + help=( + "Accepts the same string as in bazel test --test_tag_filters " + "to apply the filters during gathering targets here." + ), + ) + parser.add_argument( + "--sharding_strategy", + type=str, + default="optimal", + help=( + "What sharding strategy to use. Can be 'optimal' (try to make sure each " + "shard takes up around the same time) or 'naive' (assign the same number " + "of targets to each shard)." + ), + ) + parser.add_argument("targets", nargs="+") + args, extra_args = parser.parse_known_args() + args.targets = list(args.targets) + list(extra_args) + if args.index >= args.count: + parser.error(f"--index must be between 0 and {args.count - 1}") + + if args.sharding_strategy not in ("optimal", "naive"): + parser.error( + "--sharding_strategy must be either 'optimal' or 'naive', " + f"got {args.sharding_strategy}" + ) + + my_targets = main( + targets=args.targets, + index=args.index, + count=args.count, + tests_only=args.tests_only, + exclude_manual=args.exclude_manual, + tag_filters=args.tag_filters, + sharding_strategy=args.sharding_strategy, + debug=args.debug, + ) + + # Print so we can capture the stdout and pipe it somewhere. + print(" ".join(my_targets)) + sys.exit(0) diff --git a/scripts/build-docker-images.py b/scripts/build-docker-images.py deleted file mode 120000 index 9a3fb9d3aecf..000000000000 --- a/scripts/build-docker-images.py +++ /dev/null @@ -1 +0,0 @@ -../ci/build/build-docker-images.py \ No newline at end of file diff --git a/scripts/build-docker-images.py b/scripts/build-docker-images.py new file mode 100644 index 000000000000..24ff33efd6c8 --- /dev/null +++ b/scripts/build-docker-images.py @@ -0,0 +1,885 @@ +import datetime +import json +import functools +import glob +import itertools +import os +import platform +import re +import shutil +import subprocess +import sys +from collections import defaultdict +from typing import List, Optional, Tuple + +import click +import docker + +print = functools.partial(print, file=sys.stderr, flush=True) +DOCKER_USERNAME = "raytravisbot" +DOCKER_CLIENT = docker.from_env() +PYTHON_WHL_VERSION = "cp3" +ADDITIONAL_PLATFORMS = ["aarch64"] + +DOCKER_HUB_DESCRIPTION = { + "base-deps": ( + "Internal Image, refer to " "https://hub.docker.com/r/rayproject/ray" + ), + "ray-deps": ("Internal Image, refer to " "https://hub.docker.com/r/rayproject/ray"), + "ray": "Official Docker Images for Ray, the distributed computing API.", + "ray-ml": "Developer ready Docker Image for Ray.", + "ray-worker-container": "Internal Image for CI test", +} + +PY_MATRIX = { + "py36": "3.6", + "py37": "3.7", + "py38": "3.8", + "py39": "3.9", + "py310": "3.10", +} + +BASE_IMAGES = { + "cu118": "nvidia/cuda:11.8.0-cudnn8-devel-ubuntu20.04", + "cu116": "nvidia/cuda:11.6.1-cudnn8-devel-ubuntu20.04", + "cu113": "nvidia/cuda:11.3.1-cudnn8-devel-ubuntu20.04", + "cu112": "nvidia/cuda:11.2.0-cudnn8-devel-ubuntu20.04", + "cu111": "nvidia/cuda:11.1.1-cudnn8-devel-ubuntu20.04", + "cu110": "nvidia/cuda:11.0.3-cudnn8-devel-ubuntu20.04", + # there is no ubuntu20.04 image for cuda 10.2 and 10.1 + "cu102": "nvidia/cuda:10.2-cudnn8-devel-ubuntu18.04", + "cu101": "nvidia/cuda:10.1-cudnn8-devel-ubuntu18.04", + "cpu": "ubuntu:focal", +} + +CUDA_FULL = { + "cu118": "CUDA 11.8", + "cu116": "CUDA 11.6", + "cu113": "CUDA 11.3", + "cu112": "CUDA 11.2", + "cu111": "CUDA 11.1", + "cu110": "CUDA 11.0", + "cu102": "CUDA 10.2", + "cu101": "CUDA 10.1", +} + +# The CUDA version to use for the ML Docker image. +# If changing the CUDA version in the below line, you should also change the base Docker +# image being used in ~/ci/docker/Dockerfile.base.gpu to match the same image being used +# here. +ML_CUDA_VERSION = "cu116" + +DEFAULT_PYTHON_VERSION = "py37" + +IMAGE_NAMES = list(DOCKER_HUB_DESCRIPTION.keys()) + + +def _with_suffix(tag: str, suffix: Optional[str] = None): + if suffix: + return tag + "-" + suffix + return tag + + +def _get_branch(): + branch = os.environ.get("TRAVIS_BRANCH") or os.environ.get("BUILDKITE_BRANCH") + if not branch: + print("Branch not found!") + print(os.environ) + print("Environment is above ^^") + return branch + + +def _release_build(): + branch = _get_branch() + if branch is None: + return False + return branch != "master" and branch.startswith("releases") + + +def _valid_branch(): + branch = _get_branch() + if branch is None: + return False + return branch == "master" or _release_build() + + +def _get_curr_dir(): + return os.path.dirname(os.path.realpath(__file__)) + + +def _get_root_dir(): + return os.path.join(_get_curr_dir(), "../../") + + +def _get_commit_sha(): + sha = os.environ.get("TRAVIS_COMMIT") or os.environ.get("BUILDKITE_COMMIT") or "" + if len(sha) < 6: + print("INVALID SHA FOUND") + return "ERROR" + return sha[:6] + + +def _configure_human_version(): + global _get_branch + global _get_commit_sha + fake_branch_name = input( + "Provide a 'branch name'. For releases, it " "should be `releases/x.x.x`" + ) + _get_branch = lambda: fake_branch_name # noqa: E731 + fake_sha = input("Provide a SHA (used for tag value)") + _get_commit_sha = lambda: fake_sha # noqa: E731 + + +def _get_wheel_name(minor_version_number): + if minor_version_number: + matches = [ + file + for file in glob.glob( + f"{_get_root_dir()}/.whl/ray-*{PYTHON_WHL_VERSION}" + f"{minor_version_number}*-manylinux*" + ) + if "+" not in file # Exclude dbg, asan builds + ] + assert len(matches) == 1, ( + f"Found ({len(matches)}) matches for 'ray-*{PYTHON_WHL_VERSION}" + f"{minor_version_number}*-manylinux*' instead of 1.\n" + f"wheel matches: {matches}" + ) + return os.path.basename(matches[0]) + else: + matches = glob.glob(f"{_get_root_dir()}/.whl/*{PYTHON_WHL_VERSION}*-manylinux*") + return [os.path.basename(i) for i in matches] + + +def _check_if_docker_files_modified(): + stdout = subprocess.check_output( + [ + sys.executable, + f"{_get_curr_dir()}/../pipeline/determine_tests_to_run.py", + "--output=json", + ] + ) + affected_env_var_list = json.loads(stdout) + affected = ( + "RAY_CI_DOCKER_AFFECTED" in affected_env_var_list + or "RAY_CI_PYTHON_DEPENDENCIES_AFFECTED" in affected_env_var_list + ) + print(f"Docker affected: {affected}") + return affected + + +def _build_docker_image( + image_name: str, + py_version: str, + image_type: str, + suffix: Optional[str] = None, + no_cache=True, +): + """Builds Docker image with the provided info. + + image_name: The name of the image to build. Must be one of + IMAGE_NAMES. + py_version: The Python version to build the image for. + Must be one of PY_MATRIX.keys() + image_type: The image type to build. Must be one of + BASE_IMAGES.keys() + suffix: Suffix to add to the tags (e.g. "aarch64" for "ray:sha256-aarch64") + no_cache: If True, don't use caching when building the image. + """ + + if image_name not in IMAGE_NAMES: + raise ValueError( + f"The provided image name {image_name} is not " + f"recognized. Image names must be one of {IMAGE_NAMES}" + ) + + if py_version not in PY_MATRIX.keys(): + raise ValueError( + f"The provided python version {py_version} is not " + f"recognized. Python version must be one of" + f" {PY_MATRIX.keys()}" + ) + + if image_type not in BASE_IMAGES.keys(): + raise ValueError( + f"The provided CUDA version {image_type} is not " + f"recognized. CUDA version must be one of" + f" {BASE_IMAGES.keys()}" + ) + + build_args = {} + build_args["PYTHON_VERSION"] = PY_MATRIX[py_version] + # I.e. "py310"[3:] == 10 + build_args["PYTHON_MINOR_VERSION"] = py_version[3:] + + if platform.processor() in ADDITIONAL_PLATFORMS: + build_args["HOSTTYPE"] = platform.processor() + + device_tag = f"{image_type}" + + if image_name == "base-deps": + base_image = BASE_IMAGES[image_type] + else: + base_image = f"-{py_version}-{device_tag}" + + base_image = _with_suffix(base_image, suffix=suffix) + + if image_name != "ray-worker-container": + build_args["BASE_IMAGE"] = base_image + + if image_name in ["ray", "ray-deps", "ray-worker-container"]: + wheel = _get_wheel_name(build_args["PYTHON_MINOR_VERSION"]) + build_args["WHEEL_PATH"] = f".whl/{wheel}" + # Add pip option "--find-links .whl/" to ensure ray-cpp wheel + # can be found. + build_args["FIND_LINKS_PATH"] = ".whl" + + tagged_name = f"rayproject/{image_name}:nightly-{py_version}-{device_tag}" + + tagged_name = _with_suffix(tagged_name, suffix=suffix) + + for i in range(2): + cleanup = DOCKER_CLIENT.containers.prune().get("SpaceReclaimed") + if cleanup is not None: + print(f"Cleaned up {cleanup / (2 ** 20)}MB") + + labels = { + "image-name": image_name, + "python-version": PY_MATRIX[py_version], + "ray-commit": _get_commit_sha(), + } + if image_type in CUDA_FULL: + labels["cuda-version"] = CUDA_FULL[image_type] + + output = DOCKER_CLIENT.api.build( + path=os.path.join(_get_root_dir(), "docker", image_name), + tag=tagged_name, + nocache=no_cache, + labels=labels, + buildargs=build_args, + ) + + cmd_output = [] + try: + start = datetime.datetime.now() + current_iter = start + for line in output: + cmd_output.append(line.decode("utf-8")) + if datetime.datetime.now() - current_iter >= datetime.timedelta( + minutes=5 + ): + current_iter = datetime.datetime.now() + elapsed = datetime.datetime.now() - start + print( + f"Still building {tagged_name} after " + f"{elapsed.seconds} seconds" + ) + if elapsed >= datetime.timedelta(minutes=15): + print("Additional build output:") + print(*cmd_output, sep="\n") + # Clear cmd_output after printing, so the next + # iteration will not print out the same lines. + cmd_output = [] + except Exception as e: + print(f"FAILURE with error {e}") + + if len(DOCKER_CLIENT.api.images(tagged_name)) == 0: + print(f"ERROR building: {tagged_name}. Output below:") + print(*cmd_output, sep="\n") + if i == 1: + raise Exception("FAILED TO BUILD IMAGE") + print("TRYING AGAIN") + else: + break + + print("BUILT: ", tagged_name) + + +def copy_wheels(human_build): + if human_build: + print( + "Please download images using:\n" + "`pip download --python-version ray==" + ) + root_dir = _get_root_dir() + wheels = _get_wheel_name(None) + for wheel in wheels: + source = os.path.join(root_dir, ".whl", wheel) + ray_dst = os.path.join(root_dir, "docker/ray/.whl/") + ray_dep_dst = os.path.join(root_dir, "docker/ray-deps/.whl/") + ray_worker_container_dst = os.path.join( + root_dir, "docker/ray-worker-container/.whl/" + ) + os.makedirs(ray_dst, exist_ok=True) + shutil.copy(source, ray_dst) + os.makedirs(ray_dep_dst, exist_ok=True) + shutil.copy(source, ray_dep_dst) + os.makedirs(ray_worker_container_dst, exist_ok=True) + shutil.copy(source, ray_worker_container_dst) + + +def check_staleness(repository, tag): + DOCKER_CLIENT.api.pull(repository=repository, tag=tag) + + age = DOCKER_CLIENT.api.inspect_image(f"{repository}:{tag}")["Created"] + short_date = datetime.datetime.strptime(age.split("T")[0], "%Y-%m-%d") + is_stale = (datetime.datetime.now() - short_date) > datetime.timedelta(days=14) + return is_stale + + +def build_for_all_versions(image_name, py_versions, image_types, suffix, **kwargs): + """Builds the given Docker image for all Python & CUDA versions""" + for py_version in py_versions: + for image_type in image_types: + _build_docker_image( + image_name, + py_version=py_version, + image_type=image_type, + suffix=suffix, + **kwargs, + ) + + +def build_base_images(py_versions, image_types, suffix): + build_for_all_versions( + "base-deps", py_versions, image_types, suffix=suffix, no_cache=False + ) + build_for_all_versions( + "ray-deps", py_versions, image_types, suffix=suffix, no_cache=False + ) + + +def build_or_pull_base_images( + py_versions: List[str], + image_types: List[str], + rebuild_base_images: bool = True, + suffix: Optional[str] = None, +) -> bool: + """Returns images to tag and build.""" + repositories = ["rayproject/base-deps", "rayproject/ray-deps"] + tags = [ + f"nightly-{py_version}-{image_type}" + for py_version, image_type in itertools.product(py_versions, image_types) + ] + + try: + is_stale = check_staleness(repositories[0], tags[0]) + + # We still pull even if we have to rebuild the base images to help with + # caching. + for repository in repositories: + for tag in tags: + DOCKER_CLIENT.api.pull(repository=repository, tag=tag) + except Exception as e: + print(e) + is_stale = True + + if rebuild_base_images or _release_build() or is_stale: + build_base_images(py_versions, image_types, suffix=suffix) + return True + else: + print("Just pulling images!") + return False + + +def prep_ray_ml(): + root_dir = _get_root_dir() + + requirements_files = ["python/requirements.txt"] + ml_requirements_files = [ + "python/requirements/ml/requirements_ml_docker.txt", + "python/requirements/ml/requirements_dl.txt", + "python/requirements/ml/requirements_tune.txt", + "python/requirements/ml/requirements_rllib.txt", + "python/requirements/ml/requirements_train.txt", + "python/requirements/ml/requirements_upstream.txt", + ] + # We don't need these in the ml docker image + ignore_requirements = [ + "python/requirements/compat/requirements_legacy_compat.txt", + "python/requirements/compat/requirements_py36_compat.txt", + ] + + files_on_disk = glob.glob(f"{root_dir}/python/**/requirements*.txt", recursive=True) + for file_on_disk in files_on_disk: + rel = os.path.relpath(file_on_disk, start=root_dir) + print(rel) + if not rel.startswith("python/requirements/ml"): + continue + elif rel not in ml_requirements_files and rel not in ignore_requirements: + raise RuntimeError( + f"A new requirements file was found in the repository, but it has " + f"not been added to `build-docker-images.py` " + f"(and the `ray-ml/Dockerfile`): {rel}" + ) + + for requirement_file in requirements_files + ml_requirements_files: + shutil.copy( + os.path.join(root_dir, requirement_file), + os.path.join(root_dir, "docker/ray-ml/"), + ) + + +def _get_docker_creds() -> Tuple[str, str]: + docker_password = os.environ.get("DOCKER_PASSWORD") + assert docker_password, "DOCKER_PASSWORD not set." + return DOCKER_USERNAME, docker_password + + +def _docker_push(image, tag): + print(f"PUSHING: {image}:{tag}, result:") + # This docker API is janky. Without "stream=True" it returns a + # massive string filled with every progress bar update, which can + # cause CI to back up. + # + # With stream=True, it's a line-at-a-time generator of the same + # info. So we can slow it down by printing every couple hundred + # lines + i = 0 + for progress_line in DOCKER_CLIENT.api.push(image, tag=tag, stream=True): + if i % 100 == 0: + print(progress_line) + + +def _tag_and_push(full_image_name, old_tag, new_tag, merge_build=False): + # Do not tag release builds because they are no longer up to + # date after the branch cut. + if "nightly" in new_tag and _release_build(): + return + if old_tag != new_tag: + DOCKER_CLIENT.api.tag( + image=f"{full_image_name}:{old_tag}", + repository=full_image_name, + tag=new_tag, + ) + if not merge_build: + print( + "This is a PR Build! On a merge build, we would normally push" + f"to: {full_image_name}:{new_tag}" + ) + else: + _docker_push(full_image_name, new_tag) + + +def _create_new_tags(all_tags, old_str, new_str): + new_tags = [] + for full_tag in all_tags: + new_tag = full_tag.replace(old_str, new_str) + new_tags.append(new_tag) + return new_tags + + +def create_image_tags( + image_name: str, + py_versions: List[str], + image_types: List[str], + specific_tag: Optional[str] = None, + version: str = "nightly", + suffix: Optional[str] = None, +): + # Mapping from old tags to new tags. + # These are the tags we will push. + # The key is the full image name, and the values are all the tags + # for that image. + tag_mapping = defaultdict(list) + for py_name in py_versions: + for image_type in image_types: + if image_name == "ray-ml" and image_type not in [ + ML_CUDA_VERSION, + "cpu", + ]: + print( + "ML Docker image is not built for the following " + f"device type: {image_type}" + ) + continue + + tag = _with_suffix(f"{version}-{py_name}-{image_type}", suffix=suffix) + + tag_mapping[tag].append(tag) + + # If no device is specified, it should map to CPU image. + # For ray-ml image, if no device specified, it should map to GPU image. + # "-gpu" tag should refer to the ML_CUDA_VERSION + for old_tag in tag_mapping.keys(): + if "cpu" in old_tag and image_name != "ray-ml": + new_tags = _create_new_tags( + tag_mapping[old_tag], old_str="-cpu", new_str="" + ) + tag_mapping[old_tag].extend(new_tags) + elif ML_CUDA_VERSION in old_tag: + new_tags = _create_new_tags( + tag_mapping[old_tag], old_str=f"-{ML_CUDA_VERSION}", new_str="-gpu" + ) + tag_mapping[old_tag].extend(new_tags) + + if image_name == "ray-ml": + new_tags = _create_new_tags( + tag_mapping[old_tag], old_str=f"-{ML_CUDA_VERSION}", new_str="" + ) + tag_mapping[old_tag].extend(new_tags) + + # No Python version specified should refer to DEFAULT_PYTHON_VERSION + for old_tag in tag_mapping.keys(): + if DEFAULT_PYTHON_VERSION in old_tag: + new_tags = _create_new_tags( + tag_mapping[old_tag], + old_str=f"-{DEFAULT_PYTHON_VERSION}", + new_str="", + ) + tag_mapping[old_tag].extend(new_tags) + + # For all tags, create Date/Sha tags + if specific_tag: + for old_tag in tag_mapping.keys(): + new_tags = _create_new_tags( + tag_mapping[old_tag], + old_str=version, + new_str=specific_tag, + ) + tag_mapping[old_tag].extend(new_tags) + + return tag_mapping + + +# For non-release builds, push "nightly" & "sha" +# For release builds, push "nightly" & "latest" & "x.x.x" +def push_and_tag_images( + py_versions: List[str], + image_types: List[str], + merge_build: bool = False, + image_list: Optional[List[str]] = None, + suffix: Optional[str] = None, +): + + date_tag = datetime.datetime.now().strftime("%Y-%m-%d") + sha_tag = _get_commit_sha() + if _release_build(): + release_name = re.search("[0-9]+\.[0-9]+\.[0-9].*", _get_branch()).group(0) + date_tag = release_name + sha_tag = release_name + + for image_name in image_list: + full_image_name = f"rayproject/{image_name}" + + tag_mapping = create_image_tags( + image_name=image_name, + py_versions=py_versions, + image_types=image_types, + specific_tag=date_tag if "-deps" in image_name else sha_tag, + version="nightly", + suffix=suffix, + ) + + print(f"These tags will be created for {image_name}: ", tag_mapping) + + # Sanity checking. + for old_tag in tag_mapping.keys(): + if DEFAULT_PYTHON_VERSION in old_tag: + if "-cpu" in old_tag: + assert ( + _with_suffix("nightly-cpu", suffix=suffix) + in tag_mapping[old_tag] + ) + if "-deps" in image_name: + assert ( + _with_suffix("nightly", suffix=suffix) + in tag_mapping[old_tag] + ) + assert ( + _with_suffix(f"{date_tag}-cpu", suffix=suffix) + in tag_mapping[old_tag] + ) + assert ( + _with_suffix(f"{date_tag}", suffix=suffix) + in tag_mapping[old_tag] + ) + elif image_name == "ray": + assert ( + _with_suffix("nightly", suffix=suffix) + in tag_mapping[old_tag] + ) + assert ( + _with_suffix(f"{sha_tag}-cpu", suffix=suffix) + in tag_mapping[old_tag] + ) + assert ( + _with_suffix(f"{sha_tag}", suffix=suffix) + in tag_mapping[old_tag] + ) + # For ray-ml, nightly should refer to the GPU image. + elif image_name == "ray-ml": + assert ( + _with_suffix(f"{sha_tag}-cpu", suffix=suffix) + in tag_mapping[old_tag] + ) + else: + raise RuntimeError(f"Invalid image name: {image_name}") + + elif ML_CUDA_VERSION in old_tag: + assert ( + _with_suffix("nightly-gpu", suffix=suffix) + in tag_mapping[old_tag] + ) + if "-deps" in image_name: + assert ( + _with_suffix(f"{date_tag}-gpu", suffix=suffix) + in tag_mapping[old_tag] + ) + elif image_name == "ray": + assert ( + _with_suffix(f"{sha_tag}-gpu", suffix=suffix) + in tag_mapping[old_tag] + ) + # For ray-ml, nightly should refer to the GPU image. + elif image_name == "ray-ml": + assert ( + _with_suffix("nightly", suffix=suffix) + in tag_mapping[old_tag] + ) + assert ( + _with_suffix(f"{sha_tag}", suffix=suffix) + in tag_mapping[old_tag] + ) + assert ( + _with_suffix(f"{sha_tag}-gpu", suffix=suffix) + in tag_mapping[old_tag] + ) + else: + raise RuntimeError(f"Invalid image name: {image_name}") + + # Tag and push all images. + for old_tag in tag_mapping.keys(): + for new_tag in tag_mapping[old_tag]: + _tag_and_push( + full_image_name, + old_tag=old_tag, + new_tag=new_tag, + merge_build=merge_build, + ) + + +# Push infra here: +# https://github.com/christian-korneck/docker-pushrm/blob/master/README-containers.md#push-a-readme-file-to-dockerhub # noqa +def push_readmes(merge_build: bool): + if not merge_build: + print("Not pushing README because this is a PR build.") + return + username, password = _get_docker_creds() + for image, tag_line in DOCKER_HUB_DESCRIPTION.items(): + environment = { + "DOCKER_USER": username, + "DOCKER_PASS": password, + "PUSHRM_FILE": f"/myvol/docker/{image}/README.md", + "PUSHRM_DEBUG": 1, + "PUSHRM_SHORT": tag_line, + } + cmd_string = f"rayproject/{image}" + + print( + DOCKER_CLIENT.containers.run( + "chko/docker-pushrm:1", + command=cmd_string, + volumes={ + os.path.abspath(_get_root_dir()): { + "bind": "/myvol", + "mode": "rw", + } + }, + environment=environment, + remove=True, + detach=False, + stderr=True, + stdout=True, + tty=False, + ) + ) + + +# Build base-deps/ray-deps only on file change, 2 weeks, per release +# Build ray, ray-ml every time +# build-docker-images.py --py-versions PY37 --build-type PR --rebuild-all +MERGE = "MERGE" +HUMAN = "HUMAN" +PR = "PR" +BUILDKITE = "BUILDKITE" +LOCAL = "LOCAL" +BUILD_TYPES = [MERGE, HUMAN, PR, BUILDKITE, LOCAL] + + +@click.command() +@click.option( + "--py-versions", + "-V", + default=["py37"], + type=click.Choice(list(PY_MATRIX.keys())), + multiple=True, + help="Which python versions to build. " + "Must be in (py36, py37, py38, py39, py310, py311)", +) +@click.option( + "--device-types", + "-T", + default=[], + type=click.Choice(list(BASE_IMAGES.keys())), + multiple=True, + help="Which device types (CPU/CUDA versions) to build images for. " + "If not specified, images will be built for all device types.", +) +@click.option( + "--build-type", + type=click.Choice(BUILD_TYPES), + required=True, + help="Whether to bypass checking if docker is affected", +) +@click.option( + "--suffix", + type=click.Choice(ADDITIONAL_PLATFORMS), + help="Suffix to append to the build tags", +) +@click.option( + "--build-base/--no-build-base", + default=True, + help="Whether to build base-deps & ray-deps", +) +@click.option( + "--only-build-worker-container/--no-only-build-worker-container", + default=False, + help="Whether only to build ray-worker-container", +) +def main( + py_versions: Tuple[str], + device_types: Tuple[str], + build_type: str, + suffix: Optional[str] = None, + build_base: bool = True, + only_build_worker_container: bool = False, +): + py_versions = ( + list(py_versions) if isinstance(py_versions, (list, tuple)) else [py_versions] + ) + image_types = ( + list(device_types) + if isinstance(device_types, (list, tuple)) + else list(BASE_IMAGES.keys()) + ) + + assert set(list(CUDA_FULL.keys()) + ["cpu"]) == set(BASE_IMAGES.keys()) + + # Make sure the python images and cuda versions we build here are + # consistent with the ones used with fix-latest-docker.sh script. + py_version_file = os.path.join( + _get_root_dir(), "docker/retag-lambda", "python_versions.txt" + ) + with open(py_version_file) as f: + py_file_versions = f.read().splitlines() + assert set(PY_MATRIX.keys()) == set(py_file_versions), ( + PY_MATRIX.keys(), + py_file_versions, + ) + + cuda_version_file = os.path.join( + _get_root_dir(), "docker/retag-lambda", "cuda_versions.txt" + ) + + with open(cuda_version_file) as f: + cuda_file_versions = f.read().splitlines() + assert set(BASE_IMAGES.keys()) == set(cuda_file_versions + ["cpu"]), ( + BASE_IMAGES.keys(), + cuda_file_versions + ["cpu"], + ) + + print( + "Building the following python versions: ", + [PY_MATRIX[py_version] for py_version in py_versions], + ) + print("Building images for the following devices: ", image_types) + print("Building base images: ", build_base) + + is_buildkite = build_type == BUILDKITE + is_local = build_type == LOCAL + + if build_type == BUILDKITE: + if os.environ.get("BUILDKITE_PULL_REQUEST", "") == "false": + build_type = MERGE + else: + build_type = PR + + if build_type == HUMAN: + # If manually triggered, request user for branch and SHA value to use. + _configure_human_version() + if ( + build_type in {HUMAN, MERGE, BUILDKITE, LOCAL} + or _check_if_docker_files_modified() + or only_build_worker_container + ): + is_merge = build_type == MERGE + # Buildkite is authenticated in the background. + if is_merge and not is_buildkite and not is_local: + # We do this here because we want to be authenticated for + # Docker pulls as well as pushes (to avoid rate-limits). + username, password = _get_docker_creds() + DOCKER_CLIENT.api.login(username=username, password=password) + copy_wheels(build_type == HUMAN) + is_base_images_built = build_or_pull_base_images( + py_versions, image_types, build_base, suffix=suffix + ) + + if only_build_worker_container: + build_for_all_versions( + "ray-worker-container", py_versions, image_types, suffix=suffix + ) + # TODO Currently don't push ray_worker_container + else: + # Build Ray Docker images. + build_for_all_versions("ray", py_versions, image_types, suffix=suffix) + + # List of images to tag and push to docker hub + images_to_tag_and_push = [] + + if is_base_images_built: + images_to_tag_and_push += ["base-deps", "ray-deps"] + + # Always tag/push ray + images_to_tag_and_push += ["ray"] + + # Only build ML Docker images for ML_CUDA_VERSION or cpu. + if platform.processor() not in ADDITIONAL_PLATFORMS: + ml_image_types = [ + image_type + for image_type in image_types + if image_type in [ML_CUDA_VERSION, "cpu"] + ] + else: + # Do not build ray-ml e.g. for arm64 + ml_image_types = [] + + if len(ml_image_types) > 0: + prep_ray_ml() + build_for_all_versions( + "ray-ml", + py_versions, + image_types=ml_image_types, + suffix=suffix, + ) + images_to_tag_and_push += ["ray-ml"] + + if build_type in {MERGE, PR}: + valid_branch = _valid_branch() + if (not valid_branch) and is_merge: + print(f"Invalid Branch found: {_get_branch()}") + push_and_tag_images( + py_versions, + image_types, + merge_build=valid_branch and is_merge, + image_list=images_to_tag_and_push, + suffix=suffix, + ) + + # TODO(ilr) Re-Enable Push READMEs by using a normal password + # (not auth token :/) + # push_readmes(build_type is MERGE) + + +if __name__ == "__main__": + main() diff --git a/scripts/build-multinode-image.py b/scripts/build-multinode-image.py deleted file mode 120000 index 5a75e9e516ca..000000000000 --- a/scripts/build-multinode-image.py +++ /dev/null @@ -1 +0,0 @@ -../ci/build/build-multinode-image.py \ No newline at end of file diff --git a/scripts/build-multinode-image.py b/scripts/build-multinode-image.py new file mode 100644 index 000000000000..971eb7ebe7ad --- /dev/null +++ b/scripts/build-multinode-image.py @@ -0,0 +1,35 @@ +import argparse +import os +import shutil +import subprocess +import tempfile + + +def build_multinode_image(source_image: str, target_image: str): + """Build docker image from source_image. + + This docker image will contain packages needed for the fake multinode + docker cluster to work. + """ + tempdir = tempfile.mkdtemp() + + dockerfile = os.path.join(tempdir, "Dockerfile") + with open(dockerfile, "wt") as f: + f.write(f"FROM {source_image}\n") + f.write("RUN sudo apt update\n") + f.write("RUN sudo apt install -y openssh-server\n") + + subprocess.check_output( + f"docker build -t {target_image} .", shell=True, cwd=tempdir + ) + + shutil.rmtree(tempdir) + + +if __name__ == "__main__": + parser = argparse.ArgumentParser() + parser.add_argument("source_image", type=str) + parser.add_argument("target_image", type=str) + args = parser.parse_args() + + build_multinode_image(args.source_image, args.target_image) diff --git a/scripts/check-bazel-team-owner.py b/scripts/check-bazel-team-owner.py deleted file mode 120000 index 7ec3cc17b348..000000000000 --- a/scripts/check-bazel-team-owner.py +++ /dev/null @@ -1 +0,0 @@ -../ci/lint/check-bazel-team-owner.py \ No newline at end of file diff --git a/scripts/check-bazel-team-owner.py b/scripts/check-bazel-team-owner.py new file mode 100644 index 000000000000..11bc69d0417e --- /dev/null +++ b/scripts/check-bazel-team-owner.py @@ -0,0 +1,45 @@ +"""Used to check bazel output for team's test owner tags + +The bazel output looks like + + + + + + + + +... + +""" +import sys +import xml.etree.ElementTree as ET + + +def perform_check(raw_xml_string: str): + tree = ET.fromstring(raw_xml_string) + owners = {} + missing_owners = [] + for rule in tree.findall("rule"): + test_name = rule.attrib["name"] + tags = [child.attrib["value"] for child in rule.find("list").getchildren()] + team_owner = [t for t in tags if t.startswith("team")] + if len(team_owner) == 0: + missing_owners.append(test_name) + owners[test_name] = team_owner + + if len(missing_owners): + raise Exception( + f"Cannot find owner for tests {missing_owners}, please add " + "`team:*` to the tags." + ) + + print(owners) + + +if __name__ == "__main__": + raw_xml_string = sys.stdin.read() + perform_check(raw_xml_string) diff --git a/scripts/check-test-run.py b/scripts/check-test-run.py deleted file mode 120000 index c5954de3539b..000000000000 --- a/scripts/check-test-run.py +++ /dev/null @@ -1 +0,0 @@ -../ci/pipeline/check-test-run.py \ No newline at end of file diff --git a/scripts/check-test-run.py b/scripts/check-test-run.py new file mode 100644 index 000000000000..87ac0a36a08e --- /dev/null +++ b/scripts/check-test-run.py @@ -0,0 +1,37 @@ +"""Make sure tests will be run by CI. +""" + +import glob +import subprocess +import xml.etree.ElementTree as ET + +if __name__ == "__main__": + # Make sure python unit tests have corresponding bazel targets that will run them. + xml_string = subprocess.run( + ["bazel", "query", 'kind("py_test", //...)', "--output=xml"], + stdout=subprocess.PIPE, + ).stdout.decode("utf-8") + root_element = ET.fromstring(xml_string) + src_files = set() + for src_element in root_element.findall(".//*[@name='srcs']/label"): + src_file = src_element.attrib["value"][2:].replace(":", "/") + src_files.add(src_file) + + missing_bazel_targets = [] + for f in glob.glob("python/**/tests/test_*.py", recursive=True): + if f.startswith("python/build/") or f.startswith( + "python/ray/thirdparty_files/" + ): + continue + # TODO(jiaodong) Remove this once experimental module is tested + if f.startswith("python/ray/experimental"): + continue + if f not in src_files: + missing_bazel_targets.append(f) + + if missing_bazel_targets: + raise Exception( + f"Cannot find bazel targets for tests {missing_bazel_targets} " + f"so they won't be run automatically by CI, " + f"please add them to BUILD files." + ) diff --git a/scripts/check_import_order.py b/scripts/check_import_order.py deleted file mode 120000 index a479de4036fd..000000000000 --- a/scripts/check_import_order.py +++ /dev/null @@ -1 +0,0 @@ -../ci/lint/check_import_order.py \ No newline at end of file diff --git a/scripts/check_import_order.py b/scripts/check_import_order.py new file mode 100644 index 000000000000..ce73f639e6f9 --- /dev/null +++ b/scripts/check_import_order.py @@ -0,0 +1,85 @@ +""" +This script ensures python files conform to ray's import ordering rules. +In particular, we make sure psutil and setproctitle is imported _after_ +importing ray due to our bundling of the two libraries. + +Usage: +$ python check_import_order.py SOURCE_DIR -s SKIP_DIR +some/file/path.py:23 import psutil without explicitly import ray before it. +""" + +import argparse +import glob +import io +import re +import sys +from pathlib import Path + +exit_with_error = False + + +def check_import(file): + check_to_lines = {"import ray": -1, "import psutil": -1, "import setproctitle": -1} + + with io.open(file, "r", encoding="utf-8") as f: + for i, line in enumerate(f): + for check in check_to_lines.keys(): + # This regex will match the following case + # - the string itself: `import psutil` + # - white space/indentation + the string:` import psutil` + # - the string and arbitrary whitespace: `import psutil ` + # - the string and the noqa flag to silent pylint + # `import psutil # noqa F401 import-ordering` + # It will not match the following + # - submodule import: `import ray.constants as ray_constants` + # - submodule import: `from ray import xyz` + if ( + re.search(r"^\s*" + check + r"(\s*|\s+# noqa F401.*)$", line) + and check_to_lines[check] == -1 + ): + check_to_lines[check] = i + + for import_lib in ["import psutil", "import setproctitle"]: + if check_to_lines[import_lib] != -1: + import_psutil_line = check_to_lines[import_lib] + import_ray_line = check_to_lines["import ray"] + if import_ray_line == -1 or import_ray_line > import_psutil_line: + print( + "{}:{}".format(str(file), import_psutil_line + 1), + "{} without explicitly import ray before it.".format(import_lib), + ) + global exit_with_error + exit_with_error = True + + +if __name__ == "__main__": + parser = argparse.ArgumentParser() + parser.add_argument("path", help="File path to check. e.g. '.' or './src'") + # TODO(simon): For the future, consider adding a feature to explicitly + # white-list the path instead of skipping them. + parser.add_argument("-s", "--skip", action="append", help="Skip certian directory") + args = parser.parse_args() + + file_path = Path(args.path) + if file_path.is_dir(): + all_py_files = glob.glob("*.py", recursive=True) + else: + all_py_files = [file_path] + + if args.skip is not None: + filtered_py_files = [] + for py_file in all_py_files: + should_skip = False + for skip_dir in args.skip: + if str(py_file).startswith(skip_dir): + should_skip = True + if not should_skip: + filtered_py_files.append(py_file) + all_py_files = filtered_py_files + + for py_file in all_py_files: + check_import(py_file) + + if exit_with_error: + print("check import ordering failed") + sys.exit(1) diff --git a/scripts/check_minimal_install.py b/scripts/check_minimal_install.py deleted file mode 120000 index 51a913d13d3b..000000000000 --- a/scripts/check_minimal_install.py +++ /dev/null @@ -1 +0,0 @@ -../ci/env/check_minimal_install.py \ No newline at end of file diff --git a/scripts/check_minimal_install.py b/scripts/check_minimal_install.py new file mode 100644 index 000000000000..01beb5adf4c7 --- /dev/null +++ b/scripts/check_minimal_install.py @@ -0,0 +1,50 @@ +""" +This script ensures that some dependencies are _not_ installed in the +current python environment. + +This is to ensure that tests with minimal dependencies are not tainted +by too many installed packages. +""" + +from typing import List + +# These are taken from `setup.py` for ray[default] +DEFAULT_BLACKLIST = [ + "aiohttp", + "aiohttp_cors", + "colorful", + "py-spy", + "gpustat", + "opencensus", + "prometheus_client", + "smart_open", + "torch", + "tensorflow", + "jax", +] + + +def assert_packages_not_installed(blacklist: List[str]): + try: + from pip._internal.operations import freeze + except ImportError: # pip < 10.0 + from pip.operations import freeze + + installed_packages = [p.split("==")[0].split(" @ ")[0] for p in freeze.freeze()] + + assert not any(p in installed_packages for p in blacklist), ( + f"Found blacklisted packages in installed python packages: " + f"{[p for p in blacklist if p in installed_packages]}. " + f"Minimal dependency tests could be tainted by this. " + f"Check the install logs and primary dependencies if any of these " + f"packages were installed as part of another install step." + ) + + print( + f"Confirmed that blacklisted packages are not installed in " + f"current Python environment: {blacklist}" + ) + + +if __name__ == "__main__": + assert_packages_not_installed(DEFAULT_BLACKLIST) diff --git a/scripts/clang-tidy-diff.py b/scripts/clang-tidy-diff.py deleted file mode 120000 index 5d95690f11e9..000000000000 --- a/scripts/clang-tidy-diff.py +++ /dev/null @@ -1 +0,0 @@ -../ci/lint/clang-tidy-diff.py \ No newline at end of file diff --git a/scripts/clang-tidy-diff.py b/scripts/clang-tidy-diff.py new file mode 100755 index 000000000000..3831d5d9def5 --- /dev/null +++ b/scripts/clang-tidy-diff.py @@ -0,0 +1,306 @@ +#!/usr/bin/env python +# +# This file is based on +# https://github.com/llvm-mirror/clang-tools-extra/blob/5c40544fa40bfb85ec888b6a03421b3905e4a4e7/clang-tidy/tool/clang-tidy-diff.py +# +# ===- clang-tidy-diff.py - ClangTidy Diff Checker ----------*- python -*--===# +# +# Part of the LLVM Project, under the Apache License v2.0 with LLVM Exceptions. +# See https://llvm.org/LICENSE.txt for license information. +# SPDX-License-Identifier: Apache-2.0 WITH LLVM-exception +# +# ===----------------------------------------------------------------------===# +r""" +ClangTidy Diff Checker +====================== +This script reads input from a unified diff, runs clang-tidy on all changed +files and outputs clang-tidy warnings in changed lines only. This is useful to +detect clang-tidy regressions in the lines touched by a specific patch. +Example usage for git/svn users: + git diff -U0 HEAD^ | clang-tidy-diff.py -p1 + svn diff --diff-cmd=diff -x-U0 | \ + clang-tidy-diff.py -fix -checks=-*,modernize-use-override +""" + +import argparse +import glob +import json +import multiprocessing +import os +import re +import shutil +import subprocess +import sys +import tempfile +import threading +import traceback + +try: + import yaml +except ImportError: + yaml = None + +is_py2 = sys.version[0] == "2" + +if is_py2: + import Queue as queue +else: + import queue as queue + + +def run_tidy(task_queue, lock, timeout): + watchdog = None + while True: + command = task_queue.get() + try: + proc = subprocess.Popen( + command, stdout=subprocess.PIPE, stderr=subprocess.PIPE + ) + + if timeout is not None: + watchdog = threading.Timer(timeout, proc.kill) + watchdog.start() + + stdout, stderr = proc.communicate() + + with lock: + sys.stdout.write(stdout.decode("utf-8") + "\n") + sys.stdout.flush() + if stderr: + sys.stderr.write(stderr.decode("utf-8") + "\n") + sys.stderr.flush() + except Exception as e: + with lock: + sys.stderr.write("Failed: " + str(e) + ": ".join(command) + "\n") + finally: + with lock: + if timeout is not None and watchdog is not None: + if not watchdog.is_alive(): + sys.stderr.write( + "Terminated by timeout: " + " ".join(command) + "\n" + ) + watchdog.cancel() + task_queue.task_done() + + +def start_workers(max_tasks, tidy_caller, task_queue, lock, timeout): + for _ in range(max_tasks): + t = threading.Thread(target=tidy_caller, args=(task_queue, lock, timeout)) + t.daemon = True + t.start() + + +def merge_replacement_files(tmpdir, mergefile): + """Merge all replacement files in a directory into a single file""" + # The fixes suggested by clang-tidy >= 4.0.0 are given under + # the top level key 'Diagnostics' in the output yaml files + mergekey = "Diagnostics" + merged = [] + for replacefile in glob.iglob(os.path.join(tmpdir, "*.yaml")): + content = yaml.safe_load(open(replacefile, "r")) + if not content: + continue # Skip empty files. + merged.extend(content.get(mergekey, [])) + + if merged: + # MainSourceFile: The key is required by the definition inside + # include/clang/Tooling/ReplacementsYaml.h, but the value + # is actually never used inside clang-apply-replacements, + # so we set it to '' here. + output = {"MainSourceFile": "", mergekey: merged} + with open(mergefile, "w") as out: + yaml.safe_dump(output, out) + else: + # Empty the file: + open(mergefile, "w").close() + + +def main(): + parser = argparse.ArgumentParser( + description="Run clang-tidy against changed files, and " + "output diagnostics only for modified " + "lines." + ) + parser.add_argument( + "-clang-tidy-binary", + metavar="PATH", + default="clang-tidy", + help="path to clang-tidy binary", + ) + parser.add_argument( + "-p", + metavar="NUM", + default=0, + help="strip the smallest prefix containing P slashes", + ) + parser.add_argument( + "-regex", + metavar="PATTERN", + default=None, + help="custom pattern selecting file paths to check " + "(case sensitive, overrides -iregex)", + ) + parser.add_argument( + "-iregex", + metavar="PATTERN", + default=r".*\.(cpp|cc|c\+\+|cxx|c|cl|h|hpp|m|mm|inc)", + help="custom pattern selecting file paths to check " + "(case insensitive, overridden by -regex)", + ) + parser.add_argument( + "-j", + type=int, + default=1, + help="number of tidy instances to be run in parallel.", + ) + parser.add_argument( + "-timeout", type=int, default=None, help="timeout per each file in seconds." + ) + parser.add_argument( + "-fix", action="store_true", default=False, help="apply suggested fixes" + ) + parser.add_argument( + "-checks", + help="checks filter, when not specified, use clang-tidy " "default", + default="", + ) + parser.add_argument( + "-path", dest="build_path", help="Path used to read a compile command database." + ) + if yaml: + parser.add_argument( + "-export-fixes", + metavar="FILE", + dest="export_fixes", + help="Create a yaml file to store suggested fixes in, " + "which can be applied with clang-apply-replacements.", + ) + parser.add_argument( + "-extra-arg", + dest="extra_arg", + action="append", + default=[], + help="Additional argument to append to the compiler " "command line.", + ) + parser.add_argument( + "-extra-arg-before", + dest="extra_arg_before", + action="append", + default=[], + help="Additional argument to prepend to the compiler " "command line.", + ) + parser.add_argument( + "-quiet", + action="store_true", + default=False, + help="Run clang-tidy in quiet mode", + ) + clang_tidy_args = [] + argv = sys.argv[1:] + if "--" in argv: + clang_tidy_args.extend(argv[argv.index("--") :]) + argv = argv[: argv.index("--")] + + args = parser.parse_args(argv) + + # Extract changed lines for each file. + filename = None + lines_by_file = {} + for line in sys.stdin: + match = re.search('^\+\+\+\ "?(.*?/){%s}([^ \t\n"]*)' % args.p, line) + if match: + filename = match.group(2) + if filename is None: + continue + + if args.regex is not None: + if not re.match("^%s$" % args.regex, filename): + continue + else: + if not re.match("^%s$" % args.iregex, filename, re.IGNORECASE): + continue + + match = re.search("^@@.*\+(\d+)(,(\d+))?", line) + if match: + start_line = int(match.group(1)) + line_count = 1 + if match.group(3): + line_count = int(match.group(3)) + if line_count == 0: + continue + end_line = start_line + line_count - 1 + lines_by_file.setdefault(filename, []).append([start_line, end_line]) + + if not any(lines_by_file): + print("No relevant changes found.") + sys.exit(0) + + max_task_count = args.j + if max_task_count == 0: + max_task_count = multiprocessing.cpu_count() + max_task_count = min(len(lines_by_file), max_task_count) + + tmpdir = None + if yaml and args.export_fixes: + tmpdir = tempfile.mkdtemp() + + # Tasks for clang-tidy. + task_queue = queue.Queue(max_task_count) + # A lock for console output. + lock = threading.Lock() + + # Run a pool of clang-tidy workers. + start_workers(max_task_count, run_tidy, task_queue, lock, args.timeout) + + # Form the common args list. + common_clang_tidy_args = [] + if args.fix: + common_clang_tidy_args.append("-fix") + if args.checks != "": + common_clang_tidy_args.append("-checks=" + args.checks) + if args.quiet: + common_clang_tidy_args.append("-quiet") + if args.build_path is not None: + common_clang_tidy_args.append("-p=%s" % args.build_path) + for arg in args.extra_arg: + common_clang_tidy_args.append("-extra-arg=%s" % arg) + for arg in args.extra_arg_before: + common_clang_tidy_args.append("-extra-arg-before=%s" % arg) + + for name in lines_by_file: + line_filter_json = json.dumps( + [{"name": name, "lines": lines_by_file[name]}], separators=(",", ":") + ) + + # Run clang-tidy on files containing changes. + command = [args.clang_tidy_binary] + command.append("-line-filter=" + line_filter_json) + if yaml and args.export_fixes: + # Get a temporary file. We immediately close the handle so + # clang-tidy can overwrite it. + (handle, tmp_name) = tempfile.mkstemp(suffix=".yaml", dir=tmpdir) + os.close(handle) + command.append("-export-fixes=" + tmp_name) + command.extend(common_clang_tidy_args) + command.append(name) + command.extend(clang_tidy_args) + + task_queue.put(command) + + # Wait for all threads to be done. + task_queue.join() + + if yaml and args.export_fixes: + print("Writing fixes to " + args.export_fixes + " ...") + try: + merge_replacement_files(tmpdir, args.export_fixes) + except Exception: + sys.stderr.write("Error exporting fixes.\n") + traceback.print_exc() + + if tmpdir: + shutil.rmtree(tmpdir) + + +if __name__ == "__main__": + main() diff --git a/scripts/determine_tests_to_run.py b/scripts/determine_tests_to_run.py deleted file mode 120000 index 95f7150bb005..000000000000 --- a/scripts/determine_tests_to_run.py +++ /dev/null @@ -1 +0,0 @@ -../ci/pipeline/determine_tests_to_run.py \ No newline at end of file diff --git a/scripts/determine_tests_to_run.py b/scripts/determine_tests_to_run.py new file mode 100644 index 000000000000..bed9110be938 --- /dev/null +++ b/scripts/determine_tests_to_run.py @@ -0,0 +1,366 @@ +# Script used for checking changes for incremental testing cases +from __future__ import absolute_import, division, print_function + +import argparse +import json +import os +import re +import subprocess +import sys +from pprint import pformat +import traceback + + +# NOTE(simon): do not add type hint here because it's ran using python2 in CI. +def list_changed_files(commit_range): + """Returns a list of names of files changed in the given commit range. + + The function works by opening a subprocess and running git. If an error + occurs while running git, the script will abort. + + Args: + commit_range: The commit range to diff, consisting of the two + commit IDs separated by \"..\" + + Returns: + list: List of changed files within the commit range + """ + base_branch = os.environ.get("BUILDKITE_PULL_REQUEST_BASE_BRANCH") + if base_branch: + pull_command = ["git", "fetch", "origin", base_branch] + subprocess.check_call(pull_command) + + command = ["git", "diff", "--name-only", commit_range, "--"] + out = subprocess.check_output(command) + return [s.strip() for s in out.decode().splitlines() if s is not None] + + +def is_pull_request(): + event_type = None + + for key in ["GITHUB_EVENT_NAME", "TRAVIS_EVENT_TYPE"]: + event_type = os.getenv(key, event_type) + + if ( + os.environ.get("BUILDKITE") + and os.environ.get("BUILDKITE_PULL_REQUEST") != "false" + ): + event_type = "pull_request" + + return event_type == "pull_request" + + +def get_commit_range(): + commit_range = None + + if os.environ.get("TRAVIS"): + commit_range = os.environ["TRAVIS_COMMIT_RANGE"] + elif os.environ.get("GITHUB_EVENT_PATH"): + with open(os.environ["GITHUB_EVENT_PATH"], "rb") as f: + event = json.loads(f.read()) + base = event["pull_request"]["base"]["sha"] + commit_range = "{}...{}".format(base, event.get("after", "")) + elif os.environ.get("BUILDKITE"): + commit_range = "origin/{}...{}".format( + os.environ["BUILDKITE_PULL_REQUEST_BASE_BRANCH"], + os.environ["BUILDKITE_COMMIT"], + ) + + assert commit_range is not None + return commit_range + + +if __name__ == "__main__": + parser = argparse.ArgumentParser() + parser.add_argument("--output", type=str, help="json or envvars", default="envvars") + args = parser.parse_args() + + RAY_CI_BRANCH_BUILD = int( + os.environ.get("BUILDKITE_PULL_REQUEST", "false") == "false" + ) + RAY_CI_ML_AFFECTED = 0 + RAY_CI_TUNE_AFFECTED = 0 + RAY_CI_TRAIN_AFFECTED = 0 + # Whether only the most important (high-level) RLlib tests should be run. + # Set to 1 for any changes to Ray Tune or python source files that are + # NOT related to Serve, Dashboard or Train. + RAY_CI_RLLIB_AFFECTED = 0 + # Whether all RLlib tests should be run. + # Set to 1 only when a source file in `ray/rllib` has been changed. + RAY_CI_RLLIB_DIRECTLY_AFFECTED = 0 + RAY_CI_SERVE_AFFECTED = 0 + RAY_CI_CORE_CPP_AFFECTED = 0 + RAY_CI_CPP_AFFECTED = 0 + RAY_CI_JAVA_AFFECTED = 0 + RAY_CI_PYTHON_AFFECTED = 0 + RAY_CI_LINUX_WHEELS_AFFECTED = 0 + RAY_CI_MACOS_WHEELS_AFFECTED = 0 + RAY_CI_DASHBOARD_AFFECTED = 0 + RAY_CI_DOCKER_AFFECTED = 0 + RAY_CI_DOC_AFFECTED = 0 + RAY_CI_PYTHON_DEPENDENCIES_AFFECTED = 0 + RAY_CI_TOOLS_AFFECTED = 0 + RAY_CI_DATA_AFFECTED = 0 + RAY_CI_WORKFLOW_AFFECTED = 0 + RAY_CI_RELEASE_TESTS_AFFECTED = 0 + RAY_CI_COMPILED_PYTHON_AFFECTED = 0 + + if is_pull_request(): + commit_range = get_commit_range() + files = list_changed_files(commit_range) + print(pformat(commit_range), file=sys.stderr) + print(pformat(files), file=sys.stderr) + + # Dry run py_dep_analysis.py to see which tests we would have run. + try: + import py_dep_analysis as pda + + graph = pda.build_dep_graph() + rllib_tests = pda.list_rllib_tests() + print("Total # of RLlib tests: ", len(rllib_tests), file=sys.stderr) + + impacted = {} + for test in rllib_tests: + for file in files: + if pda.test_depends_on_file(graph, test, file): + impacted[test[0]] = True + + print("RLlib tests impacted: ", len(impacted), file=sys.stderr) + for test in impacted.keys(): + print(" ", test, file=sys.stderr) + except Exception: + print("Failed to dry run py_dep_analysis.py", file=sys.stderr) + traceback.print_exc(file=sys.stderr) + # End of dry run. + + skip_prefix_list = [ + ".buildkite/", + "doc/", + "examples/", + "dev/", + "kubernetes/", + "site/", + ] + + for changed_file in files: + if changed_file.startswith("python/ray/air"): + RAY_CI_ML_AFFECTED = 1 + RAY_CI_TRAIN_AFFECTED = 1 + RAY_CI_TUNE_AFFECTED = 1 + RAY_CI_RLLIB_AFFECTED = 1 + RAY_CI_DATA_AFFECTED = 1 + RAY_CI_LINUX_WHEELS_AFFECTED = 1 + RAY_CI_MACOS_WHEELS_AFFECTED = 1 + elif changed_file.startswith("python/ray/data"): + RAY_CI_DATA_AFFECTED = 1 + RAY_CI_ML_AFFECTED = 1 + RAY_CI_TRAIN_AFFECTED = 1 + RAY_CI_LINUX_WHEELS_AFFECTED = 1 + RAY_CI_MACOS_WHEELS_AFFECTED = 1 + elif changed_file.startswith("python/ray/workflow"): + RAY_CI_WORKFLOW_AFFECTED = 1 + RAY_CI_LINUX_WHEELS_AFFECTED = 1 + RAY_CI_MACOS_WHEELS_AFFECTED = 1 + elif changed_file.startswith("python/ray/tune"): + RAY_CI_ML_AFFECTED = 1 + RAY_CI_DOC_AFFECTED = 1 + RAY_CI_TUNE_AFFECTED = 1 + RAY_CI_RLLIB_AFFECTED = 1 + RAY_CI_TRAIN_AFFECTED = 1 + RAY_CI_LINUX_WHEELS_AFFECTED = 1 + RAY_CI_MACOS_WHEELS_AFFECTED = 1 + elif changed_file.startswith("python/ray/train"): + RAY_CI_ML_AFFECTED = 1 + RAY_CI_TRAIN_AFFECTED = 1 + RAY_CI_LINUX_WHEELS_AFFECTED = 1 + RAY_CI_MACOS_WHEELS_AFFECTED = 1 + elif re.match("^(python/ray/)?rllib/", changed_file): + RAY_CI_RLLIB_AFFECTED = 1 + RAY_CI_RLLIB_DIRECTLY_AFFECTED = 1 + RAY_CI_LINUX_WHEELS_AFFECTED = 1 + RAY_CI_MACOS_WHEELS_AFFECTED = 1 + elif changed_file.startswith("python/ray/serve"): + RAY_CI_DOC_AFFECTED = 1 + RAY_CI_SERVE_AFFECTED = 1 + RAY_CI_LINUX_WHEELS_AFFECTED = 1 + RAY_CI_MACOS_WHEELS_AFFECTED = 1 + RAY_CI_JAVA_AFFECTED = 1 + elif changed_file.startswith("python/ray/dashboard"): + RAY_CI_DASHBOARD_AFFECTED = 1 + # https://github.com/ray-project/ray/pull/15981 + RAY_CI_LINUX_WHEELS_AFFECTED = 1 + RAY_CI_MACOS_WHEELS_AFFECTED = 1 + elif changed_file.startswith("dashboard"): + RAY_CI_DASHBOARD_AFFECTED = 1 + # https://github.com/ray-project/ray/pull/15981 + RAY_CI_LINUX_WHEELS_AFFECTED = 1 + RAY_CI_MACOS_WHEELS_AFFECTED = 1 + elif changed_file.startswith("python/"): + RAY_CI_ML_AFFECTED = 1 + RAY_CI_TUNE_AFFECTED = 1 + RAY_CI_TRAIN_AFFECTED = 1 + RAY_CI_RLLIB_AFFECTED = 1 + RAY_CI_SERVE_AFFECTED = 1 + RAY_CI_WORKFLOW_AFFECTED = 1 + RAY_CI_DATA_AFFECTED = 1 + RAY_CI_PYTHON_AFFECTED = 1 + RAY_CI_DASHBOARD_AFFECTED = 1 + RAY_CI_LINUX_WHEELS_AFFECTED = 1 + RAY_CI_MACOS_WHEELS_AFFECTED = 1 + RAY_CI_DOC_AFFECTED = 1 + # Python changes might impact cross language stack in Java. + # Java also depends on Python CLI to manage processes. + RAY_CI_JAVA_AFFECTED = 1 + if changed_file.startswith("python/setup.py") or re.match( + ".*requirements.*\.txt", changed_file + ): + RAY_CI_PYTHON_DEPENDENCIES_AFFECTED = 1 + for compiled_extension in (".pxd", ".pyi", ".pyx", ".so"): + if changed_file.endswith(compiled_extension): + RAY_CI_COMPILED_PYTHON_AFFECTED = 1 + break + elif changed_file.startswith("java/"): + RAY_CI_JAVA_AFFECTED = 1 + elif changed_file.startswith("cpp/"): + RAY_CI_CPP_AFFECTED = 1 + elif changed_file.startswith("docker/"): + RAY_CI_DOCKER_AFFECTED = 1 + RAY_CI_LINUX_WHEELS_AFFECTED = 1 + elif changed_file.startswith("doc/"): + if ( + changed_file.endswith(".py") + or changed_file.endswith(".ipynb") + or changed_file.endswith("BUILD") + ): + RAY_CI_DOC_AFFECTED = 1 + # Else, this affects only a rst file or so. In that case, + # we pass, as the flag RAY_CI_DOC_AFFECTED is only + # used to indicate that tests/examples should be run + # (documentation will be built always) + elif changed_file.startswith("release/"): + if changed_file.startswith("release/ray_release"): + # Release test unit tests are ALWAYS RUN, so pass + pass + elif not changed_file.endswith(".yaml") and not changed_file.endswith( + ".md" + ): + # Do not run on config changes + RAY_CI_RELEASE_TESTS_AFFECTED = 1 + elif any(changed_file.startswith(prefix) for prefix in skip_prefix_list): + # nothing is run but linting in these cases + pass + elif changed_file.startswith("ci/lint"): + # Linter will always be run + RAY_CI_TOOLS_AFFECTED = 1 + elif changed_file.startswith("ci/pipeline"): + # These scripts are always run as part of the build process + RAY_CI_TOOLS_AFFECTED = 1 + elif changed_file.endswith("build-docker-images.py"): + RAY_CI_DOCKER_AFFECTED = 1 + RAY_CI_LINUX_WHEELS_AFFECTED = 1 + RAY_CI_TOOLS_AFFECTED = 1 + elif changed_file.startswith("ci/run"): + RAY_CI_TOOLS_AFFECTED = 1 + elif changed_file.startswith("src/"): + RAY_CI_ML_AFFECTED = 1 + RAY_CI_TUNE_AFFECTED = 1 + RAY_CI_TRAIN_AFFECTED = 1 + RAY_CI_RLLIB_AFFECTED = 1 + RAY_CI_SERVE_AFFECTED = 1 + RAY_CI_CORE_CPP_AFFECTED = 1 + RAY_CI_CPP_AFFECTED = 1 + RAY_CI_JAVA_AFFECTED = 1 + RAY_CI_PYTHON_AFFECTED = 1 + RAY_CI_LINUX_WHEELS_AFFECTED = 1 + RAY_CI_MACOS_WHEELS_AFFECTED = 1 + RAY_CI_DASHBOARD_AFFECTED = 1 + RAY_CI_DOC_AFFECTED = 1 + RAY_CI_RELEASE_TESTS_AFFECTED = 1 + else: + print( + "Unhandled source code change: {changed_file}".format( + changed_file=changed_file + ), + file=sys.stderr, + ) + + RAY_CI_ML_AFFECTED = 1 + RAY_CI_TUNE_AFFECTED = 1 + RAY_CI_TRAIN_AFFECTED = 1 + RAY_CI_RLLIB_AFFECTED = 1 + RAY_CI_SERVE_AFFECTED = 1 + RAY_CI_CORE_CPP_AFFECTED = 1 + RAY_CI_CPP_AFFECTED = 1 + RAY_CI_JAVA_AFFECTED = 1 + RAY_CI_PYTHON_AFFECTED = 1 + RAY_CI_DOC_AFFECTED = 1 + RAY_CI_LINUX_WHEELS_AFFECTED = 1 + RAY_CI_MACOS_WHEELS_AFFECTED = 1 + RAY_CI_DASHBOARD_AFFECTED = 1 + RAY_CI_TOOLS_AFFECTED = 1 + RAY_CI_RELEASE_TESTS_AFFECTED = 1 + RAY_CI_COMPILED_PYTHON_AFFECTED = 1 + + else: + RAY_CI_ML_AFFECTED = 1 + RAY_CI_TUNE_AFFECTED = 1 + RAY_CI_TRAIN_AFFECTED = 1 + RAY_CI_RLLIB_AFFECTED = 1 + RAY_CI_RLLIB_DIRECTLY_AFFECTED = 1 + RAY_CI_SERVE_AFFECTED = 1 + RAY_CI_CPP_AFFECTED = 1 + RAY_CI_CORE_CPP_AFFECTED = 1 + RAY_CI_JAVA_AFFECTED = 1 + RAY_CI_PYTHON_AFFECTED = 1 + RAY_CI_DOC_AFFECTED = 1 + RAY_CI_LINUX_WHEELS_AFFECTED = 1 + RAY_CI_MACOS_WHEELS_AFFECTED = 1 + RAY_CI_DASHBOARD_AFFECTED = 1 + RAY_CI_TOOLS_AFFECTED = 1 + RAY_CI_WORKFLOW_AFFECTED = 1 + RAY_CI_DATA_AFFECTED = 1 + RAY_CI_RELEASE_TESTS_AFFECTED = 1 + RAY_CI_COMPILED_PYTHON_AFFECTED = 1 + + # Log the modified environment variables visible in console. + output_string = " ".join( + [ + "RAY_CI_BRANCH_BUILD={}".format(RAY_CI_BRANCH_BUILD), + "RAY_CI_ML_AFFECTED={}".format(RAY_CI_ML_AFFECTED), + "RAY_CI_TUNE_AFFECTED={}".format(RAY_CI_TUNE_AFFECTED), + "RAY_CI_TRAIN_AFFECTED={}".format(RAY_CI_TRAIN_AFFECTED), + "RAY_CI_RLLIB_AFFECTED={}".format(RAY_CI_RLLIB_AFFECTED), + "RAY_CI_RLLIB_DIRECTLY_AFFECTED={}".format(RAY_CI_RLLIB_DIRECTLY_AFFECTED), + "RAY_CI_SERVE_AFFECTED={}".format(RAY_CI_SERVE_AFFECTED), + "RAY_CI_DASHBOARD_AFFECTED={}".format(RAY_CI_DASHBOARD_AFFECTED), + "RAY_CI_DOC_AFFECTED={}".format(RAY_CI_DOC_AFFECTED), + "RAY_CI_CORE_CPP_AFFECTED={}".format(RAY_CI_CORE_CPP_AFFECTED), + "RAY_CI_CPP_AFFECTED={}".format(RAY_CI_CPP_AFFECTED), + "RAY_CI_JAVA_AFFECTED={}".format(RAY_CI_JAVA_AFFECTED), + "RAY_CI_PYTHON_AFFECTED={}".format(RAY_CI_PYTHON_AFFECTED), + "RAY_CI_LINUX_WHEELS_AFFECTED={}".format(RAY_CI_LINUX_WHEELS_AFFECTED), + "RAY_CI_MACOS_WHEELS_AFFECTED={}".format(RAY_CI_MACOS_WHEELS_AFFECTED), + "RAY_CI_DOCKER_AFFECTED={}".format(RAY_CI_DOCKER_AFFECTED), + "RAY_CI_PYTHON_DEPENDENCIES_AFFECTED={}".format( + RAY_CI_PYTHON_DEPENDENCIES_AFFECTED + ), + "RAY_CI_TOOLS_AFFECTED={}".format(RAY_CI_TOOLS_AFFECTED), + "RAY_CI_WORKFLOW_AFFECTED={}".format(RAY_CI_WORKFLOW_AFFECTED), + "RAY_CI_DATA_AFFECTED={}".format(RAY_CI_DATA_AFFECTED), + "RAY_CI_RELEASE_TESTS_AFFECTED={}".format(RAY_CI_RELEASE_TESTS_AFFECTED), + "RAY_CI_COMPILED_PYTHON_AFFECTED={}".format( + RAY_CI_COMPILED_PYTHON_AFFECTED + ), + ] + ) + + # Debug purpose + print(output_string, file=sys.stderr) + + # Used by buildkite log format + if args.output.lower() == "json": + pairs = [item.split("=") for item in output_string.split(" ")] + affected_vars = [key for key, affected in pairs if affected == "1"] + print(json.dumps(affected_vars)) + else: + print(output_string) diff --git a/scripts/get_build_info.py b/scripts/get_build_info.py deleted file mode 120000 index 4329ff77814e..000000000000 --- a/scripts/get_build_info.py +++ /dev/null @@ -1 +0,0 @@ -../ci/build/get_build_info.py \ No newline at end of file diff --git a/scripts/get_build_info.py b/scripts/get_build_info.py new file mode 100755 index 000000000000..f9e6f452c4c0 --- /dev/null +++ b/scripts/get_build_info.py @@ -0,0 +1,87 @@ +#!/usr/bin/env python +""" +This script gathers build metadata from Travis environment variables and Travis +APIs. + +Usage: +$ python get_build_info.py +{ + "json": ["containing", "build", "metadata"] +} +""" + +import os +import sys +import json + + +def gha_get_self_url(): + import requests + + # stringed together api call to get the current check's html url. + sha = os.environ["GITHUB_SHA"] + repo = os.environ["GITHUB_REPOSITORY"] + resp = requests.get( + "https://api.github.com/repos/{}/commits/{}/check-suites".format(repo, sha) + ) + data = resp.json() + for check in data["check_suites"]: + slug = check["app"]["slug"] + if slug == "github-actions": + run_url = check["check_runs_url"] + html_url = requests.get(run_url).json()["check_runs"][0]["html_url"] + return html_url + + # Return a fallback url + return "https://github.com/ray-project/ray/actions" + + +def get_build_env(): + if os.environ.get("GITHUB_ACTION"): + return { + "TRAVIS_COMMIT": os.environ["GITHUB_SHA"], + "TRAVIS_JOB_WEB_URL": gha_get_self_url(), + "TRAVIS_OS_NAME": "windows", + } + + if os.environ.get("BUILDKITE"): + return { + "TRAVIS_COMMIT": os.environ["BUILDKITE_COMMIT"], + "TRAVIS_JOB_WEB_URL": ( + os.environ["BUILDKITE_BUILD_URL"] + "#" + os.environ["BUILDKITE_JOB_ID"] + ), + "TRAVIS_OS_NAME": { # The map is used to stay consistent with Travis + "linux": "linux", + "darwin": "osx", + "win32": "windows", + }[sys.platform], + } + + keys = [ + "TRAVIS_COMMIT", + "TRAVIS_JOB_WEB_URL", + "TRAVIS_OS_NAME", + ] + return {key: os.environ.get(key) for key in keys} + + +def get_build_config(): + if os.environ.get("GITHUB_ACTION"): + return {"config": {"env": "Windows CI"}} + + if os.environ.get("BUILDKITE"): + return {"config": {"env": "Buildkite " + os.environ["BUILDKITE_LABEL"]}} + + import requests + + url = "https://api.travis-ci.com/job/{job_id}?include=job.config" + url = url.format(job_id=os.environ["TRAVIS_JOB_ID"]) + resp = requests.get(url, headers={"Travis-API-Version": "3"}) + return resp.json() + + +if __name__ == "__main__": + build_env = get_build_env() + build_config = get_build_config() + + print(json.dumps({"build_env": build_env, "build_config": build_config}, indent=2)) diff --git a/scripts/py_dep_analysis.py b/scripts/py_dep_analysis.py deleted file mode 120000 index a24c022ef41e..000000000000 --- a/scripts/py_dep_analysis.py +++ /dev/null @@ -1 +0,0 @@ -../ci/pipeline/py_dep_analysis.py \ No newline at end of file diff --git a/scripts/py_dep_analysis.py b/scripts/py_dep_analysis.py new file mode 100644 index 000000000000..c9aa6a701b55 --- /dev/null +++ b/scripts/py_dep_analysis.py @@ -0,0 +1,384 @@ +#!/usr/bin/env python +# +# This file contains utilities for understanding dependencies between python +# source files and tests. +# +# Utils are assumed to be used from top level ray/ folder, since that is how +# our tests are defined today. +# +# Example usage: +# To find all circular dependencies under ray/python/: +# python ci/pipeline/py_dep_analysis.py --mode=circular-dep +# To find all the RLlib tests that depend on a file: +# python ci/pipeline/py_dep_analysis.py --mode=test-dep \ +# --file=python/ray/tune/tune.py +# For testing, add --smoke-test to any commands, so it doesn't spend +# tons of time querying for available RLlib tests. + +import argparse +import ast +import os +import re +import subprocess +import sys +from typing import Dict, List, Tuple + + +class DepGraph(object): + def __init__(self): + self.edges: Dict[str, Dict[str, bool]] = {} + self.ids: Dict[str, int] = {} + self.inv_ids: Dict[int, str] = {} + + +def _run_shell(args: List[str]) -> str: + return subprocess.check_output(args).decode(sys.stdout.encoding) + + +def list_rllib_tests(n: int = -1, test: str = None) -> Tuple[str, List[str]]: + """List RLlib tests. + + Args: + n: return at most n tests. all tests if n = -1. + test: only return information about a specific test. + """ + tests_res = _run_shell( + ["bazel", "query", "tests(//python/ray/rllib:*)", "--output", "label"] + ) + + all_tests = [] + + # Strip, also skip any empty lines + tests = [t.strip() for t in tests_res.splitlines() if t.strip()] + for t in tests: + if test and t != test: + continue + + src_out = _run_shell( + [ + "bazel", + "query", + 'kind("source file", deps({}))'.format(t), + "--output", + "label", + ] + ) + + srcs = [f.strip() for f in src_out.splitlines()] + srcs = [f for f in srcs if f.startswith("//python") and f.endswith(".py")] + if srcs: + all_tests.append((t, srcs)) + + # Break early if smoke test. + if n > 0 and len(all_tests) >= n: + break + + return all_tests + + +def _new_dep(graph: DepGraph, src_module: str, dep: str): + """Create a new dependency between src_module and dep.""" + if dep not in graph.ids: + graph.ids[dep] = len(graph.ids) + + src_id = graph.ids[src_module] + dep_id = graph.ids[dep] + + if src_id not in graph.edges: + graph.edges[src_id] = {} + graph.edges[src_id][dep_id] = True + + +def _new_import(graph: DepGraph, src_module: str, dep_module: str): + """Process a new import statement in src_module.""" + # We don't care about system imports. + if not dep_module.startswith("ray"): + return + + _new_dep(graph, src_module, dep_module) + + +def _is_path_module(module: str, name: str, _base_dir: str) -> bool: + """Figure out if base.sub is a python module or not.""" + # Special handling for _raylet, which is a C++ lib. + if module == "ray._raylet": + return False + + bps = ["python"] + module.split(".") + path = os.path.join(_base_dir, os.path.join(*bps), name + ".py") + if os.path.isfile(path): + return True # file module + return False + + +def _new_from_import( + graph: DepGraph, src_module: str, dep_module: str, dep_name: str, _base_dir: str +): + """Process a new "from ... import ..." statement in src_module.""" + # We don't care about imports outside of ray package. + if not dep_module or not dep_module.startswith("ray"): + return + + if _is_path_module(dep_module, dep_name, _base_dir): + # dep_module.dep_name points to a file. + _new_dep(graph, src_module, _full_module_path(dep_module, dep_name)) + else: + # sub is an obj on base dir/file. + _new_dep(graph, src_module, dep_module) + + +def _process_file(graph: DepGraph, src_path: str, src_module: str, _base_dir=""): + """Create dependencies from src_module to all the valid imports in src_path. + + Args: + graph: the DepGraph to be added to. + src_path: .py file to be processed. + src_module: full module path of the source file. + _base_dir: use a different base dir than current dir. For unit testing. + """ + with open(os.path.join(_base_dir, src_path), "r") as in_f: + tree = ast.parse(in_f.read()) + + for node in ast.walk(tree): + if isinstance(node, ast.Import): + for alias in node.names: + _new_import(graph, src_module, alias.name) + elif isinstance(node, ast.ImportFrom): + for alias in node.names: + _new_from_import( + graph, src_module, node.module, alias.name, _base_dir + ) + + +def build_dep_graph() -> DepGraph: + """Build index from py files to their immediate dependees.""" + graph = DepGraph() + + # Assuming we run from root /ray directory. + # Follow links since rllib is linked to /rllib. + for root, sub_dirs, files in os.walk("python", followlinks=True): + if _should_skip(root): + continue + + module = _bazel_path_to_module_path(root) + + # Process files first. + for f in files: + if not f.endswith(".py"): + continue + + full = _full_module_path(module, f) + + if full not in graph.ids: + graph.ids[full] = len(graph.ids) + + # Process file: + _process_file(graph, os.path.join(root, f), full) + + # Build reverse index for convenience. + graph.inv_ids = {v: k for k, v in graph.ids.items()} + + return graph + + +def _full_module_path(module, f) -> str: + if f == "__init__.py": + # __init__ file for this module. + # Full path is the same as the module name. + return module + + fn = re.sub(r"\.py$", "", f) + + if not module: + return fn + return module + "." + fn + + +def _should_skip(d: str) -> bool: + """Skip directories that should not contain py sources.""" + if d.startswith("python/.eggs/"): + return True + if d.startswith("python/."): + return True + if d.startswith("python/build"): + return True + if d.startswith("python/ray/cpp"): + return True + return False + + +def _bazel_path_to_module_path(d: str) -> str: + """Convert a Bazel file path to python module path. + + Example: //python/ray/rllib:xxx/yyy/dd -> ray.rllib.xxx.yyy.dd + """ + # Do this in 3 steps, so all of 'python:', 'python/', or '//python', etc + # will get stripped. + d = re.sub(r"^\/\/", "", d) + d = re.sub(r"^python", "", d) + d = re.sub(r"^[\/:]", "", d) + return d.replace("/", ".").replace(":", ".") + + +def _file_path_to_module_path(f: str) -> str: + """Return the corresponding module path for a .py file.""" + dir, fn = os.path.split(f) + return _full_module_path(_bazel_path_to_module_path(dir), fn) + + +def _depends( + graph: DepGraph, visited: Dict[int, bool], tid: int, qid: int +) -> List[int]: + """Whether there is a dependency path from module tid to module qid. + + Given graph, and without going through visited. + """ + if tid not in graph.edges or qid not in graph.edges: + return [] + if qid in graph.edges[tid]: + # tid directly depends on qid. + return [tid, qid] + for c in graph.edges[tid]: + if c in visited: + continue + visited[c] = True + # Reduce to a question of whether there is a path from c to qid. + ds = _depends(graph, visited, c, qid) + if ds: + # From tid -> c -> qid. + return [tid] + ds + return [] + + +def test_depends_on_file( + graph: DepGraph, test: Tuple[str, Tuple[str]], path: str +) -> List[int]: + """Give dependency graph, check if a test depends on a specific .py file. + + Args: + graph: the dependency graph. + test: information about a test, in the format of: + [test_name, (src files for the test)] + """ + query = _file_path_to_module_path(path) + if query not in graph.ids: + # Not a file that we care about. + return [] + + t, srcs = test + + # Skip tuned_examples/ and examples/ tests. + if t.startswith("//python/ray/rllib:examples/"): + return [] + + for src in srcs: + if src == "ray.rllib.tests.run_regression_tests": + return [] + + tid = _file_path_to_module_path(src) + if tid not in graph.ids: + # Not a test that we care about. + # TODO(jungong): What tests are these????? + continue + + branch = _depends(graph, {}, graph.ids[tid], graph.ids[query]) + if branch: + return branch + + # Does not depend on file. + return [] + + +def _find_circular_dep_impl(graph: DepGraph, id: str, branch: str) -> bool: + if id not in graph.edges: + return False + for c in graph.edges[id]: + if c in branch: + # Found a circle. + branch.append(c) + return True + branch.append(c) + if _find_circular_dep_impl(graph, c, branch): + return True + branch.pop() + return False + + +def find_circular_dep(graph: DepGraph) -> Dict[str, List[int]]: + """Find circular dependencies among a dependency graph.""" + known = {} + circles = {} + for m, id in graph.ids.items(): + branch = [] + if _find_circular_dep_impl(graph, id, branch): + if branch[-1] in known: + # Already knew, skip. + continue + # Since this is a cycle dependency, any step along the circle + # will form a different circle. + # So we mark every entry on this circle known. + for n in branch: + known[n] = True + # Mark that module m contains a potential circular dep. + circles[m] = branch + return circles + + +if __name__ == "__main__": + parser = argparse.ArgumentParser() + + parser.add_argument( + "--mode", + type=str, + default="test-dep", + help=( + "test-dep: find dependencies for a specified test. " + "circular-dep: find circular dependencies in " + "the specific codebase." + ), + ) + parser.add_argument( + "--file", type=str, help="Path of a .py source file relative to --base_dir." + ) + parser.add_argument("--test", type=str, help="Specific test to check.") + parser.add_argument( + "--smoke-test", action="store_true", help="Load only a few tests for testing." + ) + + args = parser.parse_args() + + print("building dep graph ...") + graph = build_dep_graph() + print( + "done. total {} files, {} of which have dependencies.".format( + len(graph.ids), len(graph.edges) + ) + ) + + if args.mode == "circular-dep": + circles = find_circular_dep(graph) + print("Found following circular dependencies: \n") + for m, b in circles.items(): + print(m) + for n in b: + print(" ", graph.inv_ids[n]) + print() + + if args.mode == "test-dep": + assert args.file, "Must specify --file for the query." + + # Only support RLlib tests for now. + # The way Tune tests are defined, they all depend on + # the entire tune codebase. + tests = list_rllib_tests(5 if args.smoke_test else -1, args.test) + print("Total # of tests: ", len(tests)) + + for t in tests: + branch = test_depends_on_file(graph, t, args.file) + if branch: + print("{} depends on {}".format(t[0], args.file)) + # Print some debugging info. + for n in branch: + print(" ", graph.inv_ids[n]) + else: + print("{} does not depend on {}".format(t[0], args.file)) diff --git a/scripts/py_dep_analysis_test.py b/scripts/py_dep_analysis_test.py deleted file mode 120000 index c52009fa068e..000000000000 --- a/scripts/py_dep_analysis_test.py +++ /dev/null @@ -1 +0,0 @@ -../ci/pipeline/py_dep_analysis_test.py \ No newline at end of file diff --git a/scripts/py_dep_analysis_test.py b/scripts/py_dep_analysis_test.py new file mode 100644 index 000000000000..5a03650b0d01 --- /dev/null +++ b/scripts/py_dep_analysis_test.py @@ -0,0 +1,152 @@ +import os +import tempfile +import unittest + +import py_dep_analysis as pda + + +class TestPyDepAnalysis(unittest.TestCase): + def create_tmp_file(self, path: str, content: str): + with open(path, "w") as f: + f.write(content) + + def test_full_module_path(self): + self.assertEqual(pda._full_module_path("aa.bb.cc", "__init__.py"), "aa.bb.cc") + self.assertEqual(pda._full_module_path("aa.bb.cc", "dd.py"), "aa.bb.cc.dd") + self.assertEqual(pda._full_module_path("", "dd.py"), "dd") + + def test_bazel_path_to_module_path(self): + self.assertEqual( + pda._bazel_path_to_module_path("//python/ray/rllib:xxx/yyy/dd"), + "ray.rllib.xxx.yyy.dd", + ) + self.assertEqual( + pda._bazel_path_to_module_path("python:ray/rllib/xxx/yyy/dd"), + "ray.rllib.xxx.yyy.dd", + ) + self.assertEqual( + pda._bazel_path_to_module_path("python/ray/rllib:xxx/yyy/dd"), + "ray.rllib.xxx.yyy.dd", + ) + + def test_file_path_to_module_path(self): + self.assertEqual( + pda._file_path_to_module_path("python/ray/rllib/env/env.py"), + "ray.rllib.env.env", + ) + self.assertEqual( + pda._file_path_to_module_path("python/ray/rllib/env/__init__.py"), + "ray.rllib.env", + ) + + def test_import_line_continuation(self): + graph = pda.DepGraph() + graph.ids["ray"] = 0 + + with tempfile.TemporaryDirectory() as tmpdir: + src_path = os.path.join(tmpdir, "continuation1.py") + self.create_tmp_file( + src_path, + """ +import ray.rllib.env.\\ + mock_env +b = 2 +""", + ) + pda._process_file(graph, src_path, "ray") + + self.assertEqual(len(graph.ids), 2) + print(graph.ids) + # Shoud pick up the full module name. + self.assertEqual(graph.ids["ray.rllib.env.mock_env"], 1) + self.assertEqual(graph.edges[0], {1: True}) + + def test_import_line_continuation_parenthesis(self): + graph = pda.DepGraph() + graph.ids["ray"] = 0 + + with tempfile.TemporaryDirectory() as tmpdir: + src_path = os.path.join(tmpdir, "continuation1.py") + self.create_tmp_file( + src_path, + """ +from ray.rllib.env import (ClassName, + module1, module2) +b = 2 +""", + ) + pda._process_file(graph, src_path, "ray") + + self.assertEqual(len(graph.ids), 2) + print(graph.ids) + # Shoud pick up the full module name without trailing (. + self.assertEqual(graph.ids["ray.rllib.env"], 1) + self.assertEqual(graph.edges[0], {1: True}) + + def test_from_import_file_module(self): + graph = pda.DepGraph() + graph.ids["ray"] = 0 + + with tempfile.TemporaryDirectory() as tmpdir: + src_path = "multi_line_comment_3.py" + self.create_tmp_file( + os.path.join(tmpdir, src_path), + """ +from ray.rllib.env import mock_env +a = 1 +b = 2 +""", + ) + # Touch ray/rllib/env/mock_env.py in tmpdir, + # so that it looks like a module. + module_dir = os.path.join(tmpdir, "python", "ray", "rllib", "env") + os.makedirs(module_dir, exist_ok=True) + f = open(os.path.join(module_dir, "mock_env.py"), "w") + f.write("print('hello world!')") + f.close + + pda._process_file(graph, src_path, "ray", _base_dir=tmpdir) + + self.assertEqual(len(graph.ids), 2) + self.assertEqual(graph.ids["ray.rllib.env.mock_env"], 1) + # Only 1 edge from ray to ray.rllib.env.mock_env + # ray.tune.tune is ignored. + self.assertEqual(graph.edges[0], {1: True}) + + def test_from_import_class_object(self): + graph = pda.DepGraph() + graph.ids["ray"] = 0 + + with tempfile.TemporaryDirectory() as tmpdir: + src_path = "multi_line_comment_3.py" + self.create_tmp_file( + os.path.join(tmpdir, src_path), + """ +from ray.rllib.env import MockEnv +a = 1 +b = 2 +""", + ) + # Touch ray/rllib/env.py in tmpdir, + # MockEnv is a class on env module. + module_dir = os.path.join(tmpdir, "python", "ray", "rllib") + os.makedirs(module_dir, exist_ok=True) + f = open(os.path.join(module_dir, "env.py"), "w") + f.write("print('hello world!')") + f.close + + pda._process_file(graph, src_path, "ray", _base_dir=tmpdir) + + self.assertEqual(len(graph.ids), 2) + # Should depend on env.py instead. + self.assertEqual(graph.ids["ray.rllib.env"], 1) + # Only 1 edge from ray to ray.rllib.env.mock_env + # ray.tune.tune is ignored. + self.assertEqual(graph.edges[0], {1: True}) + + +if __name__ == "__main__": + import pytest + import sys + + sys.exit(pytest.main(["-v", __file__]))