forked from ray-project/ray
-
Notifications
You must be signed in to change notification settings - Fork 0
/
dataset.py
5104 lines (4285 loc) · 200 KB
/
dataset.py
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
import collections
import copy
import html
import itertools
import logging
import time
import warnings
from typing import (
TYPE_CHECKING,
Any,
Callable,
Dict,
Generic,
Iterable,
List,
Literal,
Mapping,
Optional,
Tuple,
TypeVar,
Union,
)
import numpy as np
import ray
import ray.cloudpickle as pickle
from ray._private.thirdparty.tabulate.tabulate import tabulate
from ray._private.usage import usage_lib
from ray.air.util.tensor_extensions.utils import _create_possibly_ragged_ndarray
from ray.data._internal.block_list import BlockList
from ray.data._internal.compute import ComputeStrategy
from ray.data._internal.delegating_block_builder import DelegatingBlockBuilder
from ray.data._internal.equalize import _equalize
from ray.data._internal.execution.interfaces import RefBundle
from ray.data._internal.execution.legacy_compat import _block_list_to_bundles
from ray.data._internal.iterator.iterator_impl import DataIteratorImpl
from ray.data._internal.iterator.stream_split_iterator import StreamSplitDataIterator
from ray.data._internal.lazy_block_list import LazyBlockList
from ray.data._internal.logical.operators.all_to_all_operator import (
RandomizeBlocks,
RandomShuffle,
Repartition,
Sort,
)
from ray.data._internal.logical.operators.input_data_operator import InputData
from ray.data._internal.logical.operators.map_operator import (
Filter,
FlatMap,
MapBatches,
MapRows,
)
from ray.data._internal.logical.operators.n_ary_operator import (
Union as UnionLogicalOperator,
)
from ray.data._internal.logical.operators.n_ary_operator import Zip
from ray.data._internal.logical.operators.one_to_one_operator import Limit
from ray.data._internal.logical.operators.write_operator import Write
from ray.data._internal.logical.optimizers import LogicalPlan
from ray.data._internal.pandas_block import PandasBlockSchema
from ray.data._internal.plan import ExecutionPlan
from ray.data._internal.planner.exchange.sort_task_spec import SortKey
from ray.data._internal.remote_fn import cached_remote_fn
from ray.data._internal.split import _get_num_rows, _split_at_indices
from ray.data._internal.stats import DatasetStats, DatasetStatsSummary, StatsManager
from ray.data._internal.util import (
AllToAllAPI,
ConsumptionAPI,
_is_local_scheme,
get_compute_strategy,
)
from ray.data.aggregate import AggregateFn, Max, Mean, Min, Std, Sum
from ray.data.block import (
VALID_BATCH_FORMATS,
Block,
BlockAccessor,
BlockMetadata,
BlockPartition,
DataBatch,
T,
U,
UserDefinedFunction,
_apply_strict_mode_batch_format,
_apply_strict_mode_batch_size,
)
from ray.data.context import DataContext
from ray.data.datasource import (
BlockWritePathProvider,
Connection,
Datasink,
Datasource,
FilenameProvider,
ReadTask,
_BigQueryDatasink,
_CSVDatasink,
_ImageDatasink,
_JSONDatasink,
_MongoDatasink,
_NumpyDatasink,
_ParquetDatasink,
_SQLDatasink,
_TFRecordDatasink,
_WebDatasetDatasink,
)
from ray.data.iterator import DataIterator
from ray.data.random_access_dataset import RandomAccessDataset
from ray.types import ObjectRef
from ray.util.annotations import Deprecated, DeveloperAPI, PublicAPI
from ray.util.scheduling_strategies import NodeAffinitySchedulingStrategy
from ray.widgets import Template
from ray.widgets.util import repr_with_fallback
if TYPE_CHECKING:
import dask
import mars
import modin
import pandas
import pyarrow
import pyspark
import tensorflow as tf
import torch
import torch.utils.data
from tensorflow_metadata.proto.v0 import schema_pb2
from ray.data._internal.execution.interfaces import Executor, NodeIdStr
from ray.data.grouped_data import GroupedData
logger = logging.getLogger(__name__)
TensorflowFeatureTypeSpec = Union[
"tf.TypeSpec", List["tf.TypeSpec"], Dict[str, "tf.TypeSpec"]
]
TensorFlowTensorBatchType = Union["tf.Tensor", Dict[str, "tf.Tensor"]]
CollatedData = TypeVar("CollatedData")
TorchBatchType = Union[Dict[str, "torch.Tensor"], CollatedData]
@PublicAPI
class Dataset:
"""A Dataset is a distributed data collection for data loading and processing.
Datasets are distributed pipelines that produce ``ObjectRef[Block]`` outputs,
where each block holds data in Arrow format, representing a shard of the overall
data collection. The block also determines the unit of parallelism. For more
details, see :ref:`Ray Data Internals <dataset_concept>`.
Datasets can be created in multiple ways: from synthetic data via ``range_*()``
APIs, from existing memory data via ``from_*()`` APIs (this creates a subclass
of Dataset called ``MaterializedDataset``), or from external storage
systems such as local disk, S3, HDFS etc. via the ``read_*()`` APIs. The
(potentially processed) Dataset can be saved back to external storage systems
via the ``write_*()`` APIs.
Examples:
.. testcode::
:skipif: True
import ray
# Create dataset from synthetic data.
ds = ray.data.range(1000)
# Create dataset from in-memory data.
ds = ray.data.from_items(
[{"col1": i, "col2": i * 2} for i in range(1000)]
)
# Create dataset from external storage system.
ds = ray.data.read_parquet("s3://bucket/path")
# Save dataset back to external storage system.
ds.write_csv("s3://bucket/output")
Dataset has two kinds of operations: transformation, which takes in Dataset
and outputs a new Dataset (e.g. :py:meth:`.map_batches()`); and consumption,
which produces values (not a data stream) as output
(e.g. :meth:`.iter_batches()`).
Dataset transformations are lazy, with execution of the transformations being
triggered by downstream consumption.
Dataset supports parallel processing at scale: transformations such as
:py:meth:`.map_batches()`, aggregations such as
:py:meth:`.min()`/:py:meth:`.max()`/:py:meth:`.mean()`, grouping via
:py:meth:`.groupby()`, shuffling operations such as :py:meth:`.sort()`,
:py:meth:`.random_shuffle()`, and :py:meth:`.repartition()`.
Examples:
>>> import ray
>>> ds = ray.data.range(1000)
>>> # Transform batches (Dict[str, np.ndarray]) with map_batches().
>>> ds.map_batches(lambda batch: {"id": batch["id"] * 2}) # doctest: +ELLIPSIS
MapBatches(<lambda>)
+- Dataset(num_blocks=..., num_rows=1000, schema={id: int64})
>>> # Compute the maximum.
>>> ds.max("id")
999
>>> # Shuffle this dataset randomly.
>>> ds.random_shuffle() # doctest: +ELLIPSIS
RandomShuffle
+- Dataset(num_blocks=..., num_rows=1000, schema={id: int64})
>>> # Sort it back in order.
>>> ds.sort("id") # doctest: +ELLIPSIS
Sort
+- Dataset(num_blocks=..., num_rows=1000, schema={id: int64})
Both unexecuted and materialized Datasets can be passed between Ray tasks and
actors without incurring a copy. Dataset supports conversion to/from several
more featureful dataframe libraries (e.g., Spark, Dask, Modin, MARS), and are also
compatible with distributed TensorFlow / PyTorch.
"""
def __init__(
self,
plan: ExecutionPlan,
logical_plan: LogicalPlan,
):
"""Construct a Dataset (internal API).
The constructor is not part of the Dataset API. Use the ``ray.data.*``
read methods to construct a dataset.
"""
assert isinstance(plan, ExecutionPlan), type(plan)
usage_lib.record_library_usage("dataset") # Legacy telemetry name.
self._plan = plan
self._logical_plan = logical_plan
self._plan.link_logical_plan(logical_plan)
# Handle to currently running executor for this dataset.
self._current_executor: Optional["Executor"] = None
self._write_ds = None
self._set_uuid(StatsManager.get_dataset_id_from_stats_actor())
@staticmethod
def copy(
ds: "Dataset", _deep_copy: bool = False, _as: Optional[type] = None
) -> "Dataset":
if not _as:
_as = type(ds)
if _deep_copy:
return _as(ds._plan.deep_copy(), ds._logical_plan)
else:
return _as(ds._plan.copy(), ds._logical_plan)
def map(
self,
fn: UserDefinedFunction[Dict[str, Any], Dict[str, Any]],
*,
compute: Optional[ComputeStrategy] = None,
fn_args: Optional[Iterable[Any]] = None,
fn_kwargs: Optional[Dict[str, Any]] = None,
fn_constructor_args: Optional[Iterable[Any]] = None,
fn_constructor_kwargs: Optional[Dict[str, Any]] = None,
num_cpus: Optional[float] = None,
num_gpus: Optional[float] = None,
concurrency: Optional[Union[int, Tuple[int, int]]] = None,
**ray_remote_args,
) -> "Dataset":
"""Apply the given function to each row of this dataset.
Use this method to transform your data. To learn more, see
:ref:`Transforming rows <transforming_rows>`.
You can use either a function or a callable class to perform the transformation.
For functions, Ray Data uses stateless Ray tasks. For classes, Ray Data uses
stateful Ray actors. For more information, see
:ref:`Stateful Transforms <stateful_transforms>`.
.. tip::
If your transformation is vectorized like most NumPy or pandas operations,
:meth:`~Dataset.map_batches` might be faster.
Examples:
.. testcode::
import os
from typing import Any, Dict
import ray
def parse_filename(row: Dict[str, Any]) -> Dict[str, Any]:
row["filename"] = os.path.basename(row["path"])
return row
ds = (
ray.data.read_images("s3://anonymous@ray-example-data/image-datasets/simple", include_paths=True)
.map(parse_filename)
)
print(ds.schema())
.. testoutput::
Column Type
------ ----
image numpy.ndarray(shape=(32, 32, 3), dtype=uint8)
path string
filename string
Time complexity: O(dataset size / parallelism)
Args:
fn: The function to apply to each row, or a class type
that can be instantiated to create such a callable.
compute: This argument is deprecated. Use ``concurrency`` argument.
fn_args: Positional arguments to pass to ``fn`` after the first argument.
These arguments are top-level arguments to the underlying Ray task.
fn_kwargs: Keyword arguments to pass to ``fn``. These arguments are
top-level arguments to the underlying Ray task.
fn_constructor_args: Positional arguments to pass to ``fn``'s constructor.
You can only provide this if ``fn`` is a callable class. These arguments
are top-level arguments in the underlying Ray actor construction task.
fn_constructor_kwargs: Keyword arguments to pass to ``fn``'s constructor.
This can only be provided if ``fn`` is a callable class. These arguments
are top-level arguments in the underlying Ray actor construction task.
num_cpus: The number of CPUs to reserve for each parallel map worker.
num_gpus: The number of GPUs to reserve for each parallel map worker. For
example, specify `num_gpus=1` to request 1 GPU for each parallel map
worker.
concurrency: The number of Ray workers to use concurrently. For a fixed-sized
worker pool of size ``n``, specify ``concurrency=n``. For an autoscaling
worker pool from ``m`` to ``n`` workers, specify ``concurrency=(m, n)``.
ray_remote_args: Additional resource requirements to request from
Ray for each map worker.
.. seealso::
:meth:`~Dataset.flat_map`
Call this method to create new rows from existing ones. Unlike
:meth:`~Dataset.map`, a function passed to
:meth:`~Dataset.flat_map` can return multiple rows.
:meth:`~Dataset.map_batches`
Call this method to transform batches of data.
""" # noqa: E501
compute = get_compute_strategy(
fn,
fn_constructor_args=fn_constructor_args,
compute=compute,
concurrency=concurrency,
)
if num_cpus is not None:
ray_remote_args["num_cpus"] = num_cpus
if num_gpus is not None:
ray_remote_args["num_gpus"] = num_gpus
plan = self._plan.copy()
map_op = MapRows(
self._logical_plan.dag,
fn,
fn_args=fn_args,
fn_kwargs=fn_kwargs,
fn_constructor_args=fn_constructor_args,
fn_constructor_kwargs=fn_constructor_kwargs,
compute=compute,
ray_remote_args=ray_remote_args,
)
logical_plan = LogicalPlan(map_op)
return Dataset(plan, logical_plan)
def _set_name(self, name: Optional[str]):
"""Set the name of the dataset.
Used as a prefix for metrics tags.
"""
self._plan._dataset_name = name
@property
def _name(self) -> Optional[str]:
"""Returns the dataset name"""
return self._plan._dataset_name
def map_batches(
self,
fn: UserDefinedFunction[DataBatch, DataBatch],
*,
batch_size: Union[int, None, Literal["default"]] = "default",
compute: Optional[ComputeStrategy] = None,
batch_format: Optional[str] = "default",
zero_copy_batch: bool = False,
fn_args: Optional[Iterable[Any]] = None,
fn_kwargs: Optional[Dict[str, Any]] = None,
fn_constructor_args: Optional[Iterable[Any]] = None,
fn_constructor_kwargs: Optional[Dict[str, Any]] = None,
num_cpus: Optional[float] = None,
num_gpus: Optional[float] = None,
concurrency: Optional[Union[int, Tuple[int, int]]] = None,
**ray_remote_args,
) -> "Dataset":
"""Apply the given function to batches of data.
This method is useful for preprocessing data and performing inference. To learn
more, see :ref:`Transforming batches <transforming_batches>`.
You can use either a function or a callable class to perform the transformation.
For functions, Ray Data uses stateless Ray tasks. For classes, Ray Data uses
stateful Ray actors. For more information, see
:ref:`Stateful Transforms <stateful_transforms>`.
.. tip::
If ``fn`` doesn't mutate its input, set ``zero_copy_batch=True`` to improve
performance and decrease memory utilization.
Examples:
Call :meth:`~Dataset.map_batches` to transform your data.
.. testcode::
from typing import Dict
import numpy as np
import ray
def add_dog_years(batch: Dict[str, np.ndarray]) -> Dict[str, np.ndarray]:
batch["age_in_dog_years"] = 7 * batch["age"]
return batch
ds = (
ray.data.from_items([
{"name": "Luna", "age": 4},
{"name": "Rory", "age": 14},
{"name": "Scout", "age": 9},
])
.map_batches(add_dog_years)
)
ds.show()
.. testoutput::
{'name': 'Luna', 'age': 4, 'age_in_dog_years': 28}
{'name': 'Rory', 'age': 14, 'age_in_dog_years': 98}
{'name': 'Scout', 'age': 9, 'age_in_dog_years': 63}
If your function returns large objects, yield outputs in chunks.
.. testcode::
from typing import Dict
import ray
import numpy as np
def map_fn_with_large_output(batch: Dict[str, np.ndarray]) -> Dict[str, np.ndarray]:
for i in range(3):
yield {"large_output": np.ones((100, 1000))}
ds = (
ray.data.from_items([1])
.map_batches(map_fn_with_large_output)
)
If you require stateful transfomation,
use Python callable class. Here is an example showing how to use stateful transforms to create model inference workers, without having to reload the model on each call.
.. testcode::
from typing import Dict
import numpy as np
import torch
import ray
class TorchPredictor:
def __init__(self):
self.model = torch.nn.Identity().cuda()
self.model.eval()
def __call__(self, batch: Dict[str, np.ndarray]) -> Dict[str, np.ndarray]:
inputs = torch.as_tensor(batch["data"], dtype=torch.float32).cuda()
with torch.inference_mode():
batch["output"] = self.model(inputs).detach().cpu().numpy()
return batch
ds = (
ray.data.from_numpy(np.ones((32, 100)))
.map_batches(
TorchPredictor,
# Two workers with one GPU each
concurrency=2,
# Batch size is required if you're using GPUs.
batch_size=4,
num_gpus=1
)
)
To learn more, see
:ref:`End-to-end: Offline Batch Inference <batch_inference_home>`.
Args:
fn: The function or generator to apply to a record batch, or a class type
that can be instantiated to create such a callable. Note ``fn`` must be
pickle-able.
batch_size: The desired number of rows in each batch, or ``None`` to use
entire blocks as batches (blocks may contain different numbers of rows).
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. Default batch_size is 1024 with "default".
compute: This argument is deprecated. Use ``concurrency`` argument.
batch_format: If ``"default"`` or ``"numpy"``, batches are
``Dict[str, numpy.ndarray]``. If ``"pandas"``, batches are
``pandas.DataFrame``.
zero_copy_batch: Whether ``fn`` should be provided zero-copy, read-only
batches. If this is ``True`` and no copy is required for the
``batch_format`` conversion, the batch is a zero-copy, read-only
view on data in Ray's object store, which can decrease memory
utilization and improve performance. If this is ``False``, the batch
is writable, which requires an extra copy to guarantee.
If ``fn`` mutates its input, this needs to be ``False`` in order to
avoid "assignment destination is read-only" or "buffer source array is
read-only" errors. Default is ``False``.
fn_args: Positional arguments to pass to ``fn`` after the first argument.
These arguments are top-level arguments to the underlying Ray task.
fn_kwargs: Keyword arguments to pass to ``fn``. These arguments are
top-level arguments to the underlying Ray task.
fn_constructor_args: Positional arguments to pass to ``fn``'s constructor.
You can only provide this if ``fn`` is a callable class. These arguments
are top-level arguments in the underlying Ray actor construction task.
fn_constructor_kwargs: Keyword arguments to pass to ``fn``'s constructor.
This can only be provided if ``fn`` is a callable class. These arguments
are top-level arguments in the underlying Ray actor construction task.
num_cpus: The number of CPUs to reserve for each parallel map worker.
num_gpus: The number of GPUs to reserve for each parallel map worker. For
example, specify `num_gpus=1` to request 1 GPU for each parallel map worker.
concurrency: The number of Ray workers to use concurrently. For a fixed-sized
worker pool of size ``n``, specify ``concurrency=n``. For an autoscaling
worker pool from ``m`` to ``n`` workers, specify ``concurrency=(m, n)``.
ray_remote_args: Additional resource requirements to request from
ray for each map worker.
.. note::
The size of the batches provided to ``fn`` might be smaller than the
specified ``batch_size`` if ``batch_size`` doesn't evenly divide the
block(s) sent to a given map task.
.. seealso::
:meth:`~Dataset.iter_batches`
Call this function to iterate over batches of data.
:meth:`~Dataset.flat_map`
Call this method to create new records from existing ones. Unlike
:meth:`~Dataset.map`, a function passed to :meth:`~Dataset.flat_map`
can return multiple records.
:meth:`~Dataset.map`
Call this method to transform one record at time.
""" # noqa: E501
compute = get_compute_strategy(
fn,
fn_constructor_args=fn_constructor_args,
compute=compute,
concurrency=concurrency,
)
if num_cpus is not None:
ray_remote_args["num_cpus"] = num_cpus
if num_gpus is not None:
ray_remote_args["num_gpus"] = num_gpus
batch_format = _apply_strict_mode_batch_format(batch_format)
if batch_format == "native":
logger.warning("The 'native' batch format has been renamed 'default'.")
min_rows_per_bundled_input = None
if batch_size is not None and batch_size != "default":
if batch_size < 1:
raise ValueError("Batch size cannot be negative or 0")
# Enable blocks bundling when batch_size is specified by caller.
min_rows_per_bundled_input = batch_size
batch_size = _apply_strict_mode_batch_size(
batch_size, use_gpu="num_gpus" in ray_remote_args
)
if batch_format not in VALID_BATCH_FORMATS:
raise ValueError(
f"The batch format must be one of {VALID_BATCH_FORMATS}, got: "
f"{batch_format}"
)
plan = self._plan.copy()
map_batches_op = MapBatches(
self._logical_plan.dag,
fn,
batch_size=batch_size,
batch_format=batch_format,
zero_copy_batch=zero_copy_batch,
min_rows_per_bundled_input=min_rows_per_bundled_input,
fn_args=fn_args,
fn_kwargs=fn_kwargs,
fn_constructor_args=fn_constructor_args,
fn_constructor_kwargs=fn_constructor_kwargs,
compute=compute,
ray_remote_args=ray_remote_args,
)
logical_plan = LogicalPlan(map_batches_op)
return Dataset(plan, logical_plan)
def add_column(
self,
col: str,
fn: Callable[["pandas.DataFrame"], "pandas.Series"],
*,
compute: Optional[str] = None,
concurrency: Optional[Union[int, Tuple[int, int]]] = None,
**ray_remote_args,
) -> "Dataset":
"""Add the given column to the dataset.
A function generating the new column values given the batch in pandas
format must be specified.
Examples:
>>> import ray
>>> ds = ray.data.range(100)
>>> ds.schema()
Column Type
------ ----
id int64
Add a new column equal to ``id * 2``.
>>> ds.add_column("new_id", lambda df: df["id"] * 2).schema()
Column Type
------ ----
id int64
new_id int64
Overwrite the existing values with zeros.
>>> ds.add_column("id", lambda df: 0).take(3)
[{'id': 0}, {'id': 0}, {'id': 0}]
Time complexity: O(dataset size / parallelism)
Args:
col: Name of the column to add. If the name already exists, the
column is overwritten.
fn: Map function generating the column values given a batch of
records in pandas format.
compute: This argument is deprecated. Use ``concurrency`` argument.
concurrency: The number of Ray workers to use concurrently. For a
fixed-sized worker pool of size ``n``, specify ``concurrency=n``. For
an autoscaling worker pool from ``m`` to ``n`` workers, specify
``concurrency=(m, n)``.
ray_remote_args: Additional resource requirements to request from
ray (e.g., num_gpus=1 to request GPUs for the map tasks).
"""
compute = get_compute_strategy(
fn,
compute=compute,
concurrency=concurrency,
)
def process_batch(batch: "pandas.DataFrame") -> "pandas.DataFrame":
batch.loc[:, col] = fn(batch)
return batch
if not callable(fn):
raise ValueError("`fn` must be callable, got {}".format(fn))
return self.map_batches(
process_batch,
batch_format="pandas", # TODO(ekl) we should make this configurable.
compute=compute,
zero_copy_batch=False,
**ray_remote_args,
)
def drop_columns(
self,
cols: List[str],
*,
compute: Optional[str] = None,
concurrency: Optional[Union[int, Tuple[int, int]]] = None,
**ray_remote_args,
) -> "Dataset":
"""Drop one or more columns from the dataset.
Examples:
>>> import ray
>>> ds = ray.data.read_parquet("s3://anonymous@ray-example-data/iris.parquet")
>>> ds.schema()
Column Type
------ ----
sepal.length double
sepal.width double
petal.length double
petal.width double
variety string
>>> ds.drop_columns(["variety"]).schema()
Column Type
------ ----
sepal.length double
sepal.width double
petal.length double
petal.width double
Time complexity: O(dataset size / parallelism)
Args:
cols: Names of the columns to drop. If any name does not exist,
an exception is raised.
compute: This argument is deprecated. Use ``concurrency`` argument.
concurrency: The number of Ray workers to use concurrently. For a fixed-sized
worker pool of size ``n``, specify ``concurrency=n``. For an autoscaling
worker pool from ``m`` to ``n`` workers, specify ``concurrency=(m, n)``.
ray_remote_args: Additional resource requirements to request from
ray (e.g., num_gpus=1 to request GPUs for the map tasks).
""" # noqa: E501
def fn(batch):
return batch.drop(columns=cols)
compute = get_compute_strategy(
fn,
compute=compute,
concurrency=concurrency,
)
return self.map_batches(
fn,
batch_format="pandas",
zero_copy_batch=True,
compute=compute,
**ray_remote_args,
)
def select_columns(
self,
cols: List[str],
*,
compute: Union[str, ComputeStrategy] = None,
concurrency: Optional[Union[int, Tuple[int, int]]] = None,
**ray_remote_args,
) -> "Dataset":
"""Select one or more columns from the dataset.
Specified columns must be in the dataset schema.
Examples:
>>> import ray
>>> ds = ray.data.read_parquet("s3://anonymous@ray-example-data/iris.parquet")
>>> ds.schema()
Column Type
------ ----
sepal.length double
sepal.width double
petal.length double
petal.width double
variety string
>>> ds.select_columns(["sepal.length", "sepal.width"]).schema()
Column Type
------ ----
sepal.length double
sepal.width double
Time complexity: O(dataset size / parallelism)
Args:
cols: Names of the columns to select. If a name isn't in the
dataset schema, an exception is raised.
compute: This argument is deprecated. Use ``concurrency`` argument.
concurrency: The number of Ray workers to use concurrently. For a fixed-sized
worker pool of size ``n``, specify ``concurrency=n``. For an autoscaling
worker pool from ``m`` to ``n`` workers, specify ``concurrency=(m, n)``.
ray_remote_args: Additional resource requirements to request from
ray (e.g., num_gpus=1 to request GPUs for the map tasks).
""" # noqa: E501
def fn(batch):
return BlockAccessor.for_block(batch).select(columns=cols)
compute = get_compute_strategy(
fn,
compute=compute,
concurrency=concurrency,
)
return self.map_batches(
fn,
batch_format="pandas",
zero_copy_batch=True,
compute=compute,
**ray_remote_args,
)
def flat_map(
self,
fn: UserDefinedFunction[Dict[str, Any], List[Dict[str, Any]]],
*,
compute: Optional[ComputeStrategy] = None,
fn_args: Optional[Iterable[Any]] = None,
fn_kwargs: Optional[Dict[str, Any]] = None,
fn_constructor_args: Optional[Iterable[Any]] = None,
fn_constructor_kwargs: Optional[Dict[str, Any]] = None,
num_cpus: Optional[float] = None,
num_gpus: Optional[float] = None,
concurrency: Optional[Union[int, Tuple[int, int]]] = None,
**ray_remote_args,
) -> "Dataset":
"""Apply the given function to each row and then flatten results.
Use this method if your transformation returns multiple rows for each input
row.
You can use either a function or a callable class to perform the transformation.
For functions, Ray Data uses stateless Ray tasks. For classes, Ray Data uses
stateful Ray actors. For more information, see
:ref:`Stateful Transforms <stateful_transforms>`.
.. tip::
:meth:`~Dataset.map_batches` can also modify the number of rows. If your
transformation is vectorized like most NumPy and pandas operations,
it might be faster.
Examples:
.. testcode::
from typing import Any, Dict, List
import ray
def duplicate_row(row: Dict[str, Any]) -> List[Dict[str, Any]]:
return [row] * 2
print(
ray.data.range(3)
.flat_map(duplicate_row)
.take_all()
)
.. testoutput::
[{'id': 0}, {'id': 0}, {'id': 1}, {'id': 1}, {'id': 2}, {'id': 2}]
Time complexity: O(dataset size / parallelism)
Args:
fn: The function or generator to apply to each record, or a class type
that can be instantiated to create such a callable.
compute: This argument is deprecated. Use ``concurrency`` argument.
fn_args: Positional arguments to pass to ``fn`` after the first argument.
These arguments are top-level arguments to the underlying Ray task.
fn_kwargs: Keyword arguments to pass to ``fn``. These arguments are
top-level arguments to the underlying Ray task.
fn_constructor_args: Positional arguments to pass to ``fn``'s constructor.
You can only provide this if ``fn`` is a callable class. These arguments
are top-level arguments in the underlying Ray actor construction task.
fn_constructor_kwargs: Keyword arguments to pass to ``fn``'s constructor.
This can only be provided if ``fn`` is a callable class. These arguments
are top-level arguments in the underlying Ray actor construction task.
num_cpus: The number of CPUs to reserve for each parallel map worker.
num_gpus: The number of GPUs to reserve for each parallel map worker. For
example, specify `num_gpus=1` to request 1 GPU for each parallel map
worker.
concurrency: The number of Ray workers to use concurrently. For a
fixed-sized worker pool of size ``n``, specify ``concurrency=n``.
For an autoscaling worker pool from ``m`` to ``n`` workers, specify
``concurrency=(m, n)``.
ray_remote_args: Additional resource requirements to request from
ray for each map worker.
.. seealso::
:meth:`~Dataset.map_batches`
Call this method to transform batches of data.
:meth:`~Dataset.map`
Call this method to transform one row at time.
"""
compute = get_compute_strategy(
fn,
fn_constructor_args=fn_constructor_args,
compute=compute,
concurrency=concurrency,
)
if num_cpus is not None:
ray_remote_args["num_cpus"] = num_cpus
if num_gpus is not None:
ray_remote_args["num_gpus"] = num_gpus
plan = self._plan.copy()
op = FlatMap(
input_op=self._logical_plan.dag,
fn=fn,
fn_args=fn_args,
fn_kwargs=fn_kwargs,
fn_constructor_args=fn_constructor_args,
fn_constructor_kwargs=fn_constructor_kwargs,
compute=compute,
ray_remote_args=ray_remote_args,
)
logical_plan = LogicalPlan(op)
return Dataset(plan, logical_plan)
def filter(
self,
fn: UserDefinedFunction[Dict[str, Any], bool],
*,
compute: Union[str, ComputeStrategy] = None,
concurrency: Optional[Union[int, Tuple[int, int]]] = None,
**ray_remote_args,
) -> "Dataset":
"""Filter out rows that don't satisfy the given predicate.
You can use either a function or a callable class to perform the transformation.
For functions, Ray Data uses stateless Ray tasks. For classes, Ray Data uses
stateful Ray actors. For more information, see
:ref:`Stateful Transforms <stateful_transforms>`.
.. tip::
If you can represent your predicate with NumPy or pandas operations,
:meth:`Dataset.map_batches` might be faster. You can implement filter by
dropping rows.
Examples:
>>> import ray
>>> ds = ray.data.range(100)
>>> ds.filter(lambda row: row["id"] % 2 == 0).take_all()
[{'id': 0}, {'id': 2}, {'id': 4}, ...]
Time complexity: O(dataset size / parallelism)
Args:
fn: The predicate to apply to each row, or a class type
that can be instantiated to create such a callable.
compute: This argument is deprecated. Use ``concurrency`` argument.
concurrency: The number of Ray workers to use concurrently. For a
fixed-sized worker pool of size ``n``, specify ``concurrency=n``.
For an autoscaling worker pool from ``m`` to ``n`` workers, specify
``concurrency=(m, n)``.
ray_remote_args: Additional resource requirements to request from
ray (e.g., num_gpus=1 to request GPUs for the map tasks).
"""
compute = get_compute_strategy(
fn,
compute=compute,
concurrency=concurrency,
)
plan = self._plan.copy()
op = Filter(
input_op=self._logical_plan.dag,
fn=fn,
compute=compute,
ray_remote_args=ray_remote_args,
)
logical_plan = LogicalPlan(op)
return Dataset(plan, logical_plan)
@AllToAllAPI
def repartition(
self,
num_blocks: int,
*,
shuffle: bool = False,
) -> "Dataset":
"""Repartition the :class:`Dataset` into exactly this number of :ref:`blocks <dataset_concept>`.
This method can be useful to tune the performance of your pipeline. To learn
more, see :ref:`Advanced: Performance Tips and Tuning <data_performance_tips>`.
If you're writing data to files, you can also use this method to change the
number of output files. To learn more, see
:ref:`Changing the number of output files <changing-number-output-files>`.
.. note::
Repartition has two modes. If ``shuffle=False``, Ray Data performs the
minimal data movement needed to equalize block sizes. Otherwise, Ray Data
performs a full distributed shuffle.
.. image:: /data/images/dataset-shuffle.svg
:align: center
..
https://docs.google.com/drawings/d/132jhE3KXZsf29ho1yUdPrCHB9uheHBWHJhDQMXqIVPA/edit
Examples:
>>> import ray
>>> ds = ray.data.range(100)
>>> ds.repartition(10).num_blocks()
10
Time complexity: O(dataset size / parallelism)
Args:
num_blocks: The number of blocks.
shuffle: Whether to perform a distributed shuffle during the