-
Notifications
You must be signed in to change notification settings - Fork 186
/
Copy pathray_runner.py
1391 lines (1169 loc) · 54.8 KB
/
ray_runner.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
from __future__ import annotations
import contextlib
import dataclasses
import logging
import threading
import time
import uuid
from datetime import datetime
from queue import Full, Queue
from typing import TYPE_CHECKING, Any, Generator, Iterable, Iterator
# The ray runner is not a top-level module, so we don't need to lazily import pyarrow to minimize
# import times. If this changes, we first need to make the daft.lazy_import.LazyImport class
# serializable before importing pa from daft.dependencies.
import pyarrow as pa # noqa: TID253
import ray.experimental # noqa: TID253
from daft.arrow_utils import ensure_array
from daft.context import execution_config_ctx, get_context
from daft.daft import PyRecordBatch as _PyRecordBatch
from daft.dependencies import np
from daft.recordbatch import RecordBatch
from daft.runners import ray_tracing
from daft.runners.progress_bar import ProgressBar
from daft.series import Series, item_to_series
logger = logging.getLogger(__name__)
try:
import ray
except ImportError:
logger.error(
"Error when importing Ray. Please ensure that getdaft was installed with the Ray extras tag: getdaft[ray] (https://www.getdaft.io/projects/docs/en/latest/learn/install.html)"
)
raise
from daft.daft import (
FileFormatConfig,
FileInfos,
IOConfig,
PyDaftExecutionConfig,
ResourceRequest,
)
from daft.datatype import DataType
from daft.execution.execution_step import (
ActorPoolProject,
FanoutInstruction,
Instruction,
MultiOutputPartitionTask,
PartitionTask,
ReduceInstruction,
ScanWithTask,
SingleOutputPartitionTask,
)
from daft.execution.physical_plan import ActorPoolManager
from daft.expressions import ExpressionsProjection
from daft.filesystem import glob_path_with_stats
from daft.recordbatch import MicroPartition
from daft.runners import runner_io
from daft.runners.partitioning import (
LocalPartitionSet,
MaterializedResult,
PartID,
PartitionCacheEntry,
PartitionMetadata,
PartitionSet,
PartitionSetCache,
)
from daft.runners.profiler import profiler
from daft.runners.runner import Runner
if TYPE_CHECKING:
import dask
import pandas as pd
from ray.data.block import Block as RayDatasetBlock
from ray.data.dataset import Dataset as RayDataset
from daft.logical.builder import LogicalPlanBuilder
from daft.plan_scheduler import PhysicalPlanScheduler
from daft.runners.ray_tracing import RunnerTracer
_RAY_FROM_ARROW_REFS_AVAILABLE = True
try:
from ray.data import from_arrow_refs
except ImportError:
_RAY_FROM_ARROW_REFS_AVAILABLE = False
from daft.logical.schema import Schema
RAY_VERSION = tuple(int(s) for s in ray.__version__.split(".")[0:3])
_RAY_DATA_ARROW_TENSOR_TYPE_AVAILABLE = True
try:
from ray.data.extensions import ArrowTensorArray, ArrowTensorType
except ImportError:
_RAY_DATA_ARROW_TENSOR_TYPE_AVAILABLE = False
_RAY_DATA_EXTENSIONS_AVAILABLE = True
_TENSOR_EXTENSION_TYPES = []
try:
import ray
except ImportError:
_RAY_DATA_EXTENSIONS_AVAILABLE = False
else:
_RAY_VERSION = tuple(int(s) for s in ray.__version__.split(".")[0:3])
try:
# Variable-shaped tensor column support was added in Ray 2.1.0.
if _RAY_VERSION >= (2, 2, 0):
from ray.data.extensions import (
ArrowTensorType,
ArrowVariableShapedTensorType,
)
_TENSOR_EXTENSION_TYPES = [ArrowTensorType, ArrowVariableShapedTensorType]
else:
from ray.data.extensions import ArrowTensorType
_TENSOR_EXTENSION_TYPES = [ArrowTensorType]
except ImportError:
_RAY_DATA_EXTENSIONS_AVAILABLE = False
@ray.remote
def _glob_path_into_file_infos(
paths: list[str],
file_format_config: FileFormatConfig | None,
io_config: IOConfig | None,
) -> FileInfos:
file_infos = FileInfos()
file_format = file_format_config.file_format() if file_format_config is not None else None
for path in paths:
path_file_infos = glob_path_with_stats(path, file_format=file_format, io_config=io_config)
if len(path_file_infos) == 0:
raise FileNotFoundError(f"No files found at {path}")
file_infos.extend(path_file_infos)
return file_infos
@ray.remote
def _make_ray_block_from_micropartition(partition: MicroPartition) -> RayDatasetBlock:
try:
daft_schema = partition.schema()
arrow_tbl = partition.to_arrow()
# Convert arrays to Ray Data's native ArrowTensorType arrays
new_arrs = {}
for idx, field in enumerate(arrow_tbl.schema):
if daft_schema[field.name].dtype._is_fixed_shape_tensor_type():
assert isinstance(field.type, pa.FixedShapeTensorType)
new_dtype = ArrowTensorType(field.type.shape, field.type.value_type)
arrow_arr = arrow_tbl[field.name].combine_chunks()
storage_arr = arrow_arr.storage
list_size = storage_arr.type.list_size
new_storage_arr = pa.ListArray.from_arrays(
pa.array(
list(range(0, (len(arrow_arr) + 1) * list_size, list_size)),
pa.int32(),
),
storage_arr.values,
)
new_arrs[idx] = (
field.name,
pa.ExtensionArray.from_storage(new_dtype, new_storage_arr),
)
elif daft_schema[field.name].dtype._is_tensor_type():
assert isinstance(field.type, pa.ExtensionType)
new_arrs[idx] = (field.name, ArrowTensorArray.from_numpy(partition.get_column(field.name).to_pylist()))
for idx, (field_name, arr) in new_arrs.items():
arrow_tbl = arrow_tbl.set_column(idx, pa.field(field_name, arr.type), arr)
return arrow_tbl
except pa.ArrowInvalid:
return partition.to_pylist()
def _series_from_arrow_with_ray_data_extensions(
array: pa.Array | pa.ChunkedArray, name: str = "arrow_series"
) -> Series:
if isinstance(array, pa.Array):
# TODO(desmond): This might be dead code since `ArrayTensorType`s are `numpy.ndarray` under
# the hood and are not instances of `pyarrow.Array`. Should follow up and check if this code
# can be removed.
array = ensure_array(array)
if _RAY_DATA_EXTENSIONS_AVAILABLE and isinstance(array.type, ArrowTensorType):
storage_series = _series_from_arrow_with_ray_data_extensions(array.storage, name=name)
series = storage_series.cast(
DataType.fixed_size_list(
_from_arrow_type_with_ray_data_extensions(array.type.scalar_type),
int(np.prod(array.type.shape)),
)
)
return series.cast(DataType.from_arrow_type(array.type))
elif _RAY_DATA_EXTENSIONS_AVAILABLE and isinstance(array.type, ArrowVariableShapedTensorType):
return Series.from_numpy(array.to_numpy(zero_copy_only=False), name=name)
return Series.from_arrow(array, name)
def _micropartition_from_arrow_with_ray_data_extensions(arrow_table: pa.Table) -> MicroPartition:
assert isinstance(arrow_table, pa.Table)
non_native_fields = []
for arrow_field in arrow_table.schema:
dt = _from_arrow_type_with_ray_data_extensions(arrow_field.type)
if dt == DataType.python() or dt._is_tensor_type() or dt._is_fixed_shape_tensor_type():
non_native_fields.append(arrow_field.name)
if non_native_fields:
# If there are any contained Arrow types that are not natively supported, convert each
# series while checking for ray data extension types.
logger.debug("Unsupported Arrow types detected for columns: %s", non_native_fields)
series_dict = dict()
for name, column in zip(arrow_table.column_names, arrow_table.columns):
series = (
_series_from_arrow_with_ray_data_extensions(column, name)
if isinstance(column, (pa.Array, pa.ChunkedArray))
else item_to_series(name, column)
)
series_dict[name] = series._series
return MicroPartition._from_tables([RecordBatch._from_pytable(_PyRecordBatch.from_pylist_series(series_dict))])
return MicroPartition.from_arrow(arrow_table)
@ray.remote
def _make_daft_partition_from_ray_dataset_blocks(
ray_dataset_block: pa.MicroPartition, daft_schema: Schema
) -> MicroPartition:
return _micropartition_from_arrow_with_ray_data_extensions(ray_dataset_block)
@ray.remote(num_returns=2)
def _make_daft_partition_from_dask_dataframe_partitions(
dask_df_partition: pd.DataFrame,
) -> tuple[MicroPartition, pa.Schema]:
vpart = MicroPartition.from_pandas(dask_df_partition)
return vpart, vpart.schema()
def _to_pandas_ref(df: pd.DataFrame | ray.ObjectRef[pd.DataFrame]) -> ray.ObjectRef[pd.DataFrame]:
"""Ensures that the provided pandas DataFrame partition is in the Ray object store."""
import pandas as pd
if isinstance(df, pd.DataFrame):
return ray.put(df)
elif isinstance(df, ray.ObjectRef):
return df
else:
raise ValueError("Expected a Ray object ref or a Pandas DataFrame, " f"got {type(df)}")
class RayPartitionSet(PartitionSet[ray.ObjectRef]):
_results: dict[PartID, RayMaterializedResult]
def __init__(self) -> None:
super().__init__()
self._results = {}
def items(self) -> list[tuple[PartID, MaterializedResult[ray.ObjectRef]]]:
return [(pid, result) for pid, result in sorted(self._results.items())]
def _get_merged_micropartition(self) -> MicroPartition:
ids_and_partitions = self.items()
assert ids_and_partitions[0][0] == 0
assert ids_and_partitions[-1][0] + 1 == len(ids_and_partitions)
all_partitions = ray.get([part.partition() for id, part in ids_and_partitions])
return MicroPartition.concat(all_partitions)
def _get_preview_micropartitions(self, num_rows: int) -> list[MicroPartition]:
ids_and_partitions = self.items()
preview_parts = []
for _, mat_result in ids_and_partitions:
ref: ray.ObjectRef = mat_result.partition()
part: MicroPartition = ray.get(ref)
part_len = len(part)
if part_len >= num_rows: # if this part has enough rows, take what we need and break
preview_parts.append(part.slice(0, num_rows))
break
else: # otherwise, take the whole part and keep going
num_rows -= part_len
preview_parts.append(part)
return preview_parts
def to_ray_dataset(self) -> RayDataset:
if not _RAY_FROM_ARROW_REFS_AVAILABLE:
raise ImportError(
"Unable to import `ray.data.from_arrow_refs`. Please ensure that you have a compatible version of Ray >= 1.10 installed."
)
blocks = [
_make_ray_block_from_micropartition.remote(self._results[k].partition()) for k in self._results.keys()
]
# NOTE: although the Ray method is called `from_arrow_refs`, this method works also when the blocks are List[T] types
# instead of Arrow tables as the codepath for Dataset creation is the same.
return from_arrow_refs(blocks)
def to_dask_dataframe(
self,
meta: (pd.DataFrame | pd.Series | dict[str, Any] | Iterable[Any] | tuple[Any] | None) = None,
) -> dask.DataFrame:
import dask
import dask.dataframe as dd
from ray.util.dask import ray_dask_get
dask.config.set(scheduler=ray_dask_get)
@dask.delayed
def _make_dask_dataframe_partition_from_micropartition(partition: MicroPartition) -> pd.DataFrame:
return partition.to_pandas()
ddf_parts = [
_make_dask_dataframe_partition_from_micropartition(self._results[k].partition())
for k in self._results.keys()
]
return dd.from_delayed(ddf_parts, meta=meta)
def get_partition(self, idx: PartID) -> ray.ObjectRef:
return self._results[idx].partition()
def set_partition(self, idx: PartID, result: MaterializedResult[ray.ObjectRef]) -> None:
assert isinstance(result, RayMaterializedResult)
self._results[idx] = result
def delete_partition(self, idx: PartID) -> None:
del self._results[idx]
def has_partition(self, idx: PartID) -> bool:
return idx in self._results
def __len__(self) -> int:
return sum(result.metadata().num_rows for result in self._results.values())
def size_bytes(self) -> int | None:
size_bytes_ = [result.metadata().size_bytes for result in self._results.values()]
size_bytes: list[int] = [size for size in size_bytes_ if size is not None]
if len(size_bytes) != len(size_bytes_):
return None
else:
return sum(size_bytes)
def num_partitions(self) -> int:
return len(self._results)
def wait(self) -> None:
deduped_object_refs = {r.partition() for r in self._results.values()}
ray.wait(list(deduped_object_refs), fetch_local=False, num_returns=len(deduped_object_refs))
def _from_arrow_type_with_ray_data_extensions(arrow_type: pa.lib.DataType) -> DataType:
if _RAY_DATA_EXTENSIONS_AVAILABLE and isinstance(arrow_type, tuple(_TENSOR_EXTENSION_TYPES)):
scalar_dtype = _from_arrow_type_with_ray_data_extensions(arrow_type.scalar_type)
shape = arrow_type.shape if isinstance(arrow_type, ArrowTensorType) else None
return DataType.tensor(scalar_dtype, shape)
return DataType.from_arrow_type(arrow_type)
class RayRunnerIO(runner_io.RunnerIO):
def __init__(self, *args, **kwargs):
super().__init__(*args, **kwargs)
def glob_paths_details(
self,
source_paths: list[str],
file_format_config: FileFormatConfig | None = None,
io_config: IOConfig | None = None,
) -> FileInfos:
# Synchronously fetch the file infos, for now.
return ray.get(_glob_path_into_file_infos.remote(source_paths, file_format_config, io_config=io_config))
def partition_set_from_ray_dataset(
self,
ds: RayDataset,
) -> tuple[RayPartitionSet, Schema]:
arrow_schema = ds.schema(fetch_if_missing=True)
if not isinstance(arrow_schema, pa.Schema):
# Convert Dataset to an Arrow dataset.
extra_kwargs = {}
if RAY_VERSION >= (2, 3, 0):
# The zero_copy_batch kwarg was added in Ray 2.3.0.
extra_kwargs["zero_copy_batch"] = True
ds = ds.map_batches(
lambda x: x,
batch_size=None,
batch_format="pyarrow",
**extra_kwargs,
)
arrow_schema = ds.schema(fetch_if_missing=True)
# Ray 2.5.0 broke the API by using its own `ray.data.dataset.Schema` instead of PyArrow schemas
if RAY_VERSION >= (2, 5, 0):
arrow_schema = pa.schema({name: t for name, t in zip(arrow_schema.names, arrow_schema.types)})
daft_schema = Schema._from_field_name_and_types(
[
(arrow_field.name, _from_arrow_type_with_ray_data_extensions(arrow_field.type))
for arrow_field in arrow_schema
]
)
block_refs = ds.get_internal_block_refs()
# NOTE: This materializes the entire Ray Dataset - we could make this more intelligent by creating a new RayDatasetScan node
# which can iterate on Ray Dataset blocks and materialize as-needed
daft_micropartitions = [
_make_daft_partition_from_ray_dataset_blocks.remote(block, daft_schema) for block in block_refs
]
pset = RayPartitionSet()
for i, obj in enumerate(daft_micropartitions):
pset.set_partition(i, RayMaterializedResult(obj))
return (
pset,
daft_schema,
)
def partition_set_from_dask_dataframe(
self,
ddf: dask.DataFrame,
) -> tuple[RayPartitionSet, Schema]:
import dask
from ray.util.dask import ray_dask_get
partitions = ddf.to_delayed()
if not partitions:
raise ValueError("Can't convert an empty Dask DataFrame (with no partitions) to a Daft DataFrame.")
persisted_partitions = dask.persist(*partitions, scheduler=ray_dask_get)
parts = [_to_pandas_ref(next(iter(part.dask.values()))) for part in persisted_partitions]
daft_micropartitions, schemas = zip(
*(_make_daft_partition_from_dask_dataframe_partitions.remote(p) for p in parts)
)
schemas = ray.get(list(schemas))
# Dask shouldn't allow inconsistent schemas across partitions, but we double-check here.
if not all(schemas[0] == schema for schema in schemas[1:]):
raise ValueError(
"Can't convert a Dask DataFrame with inconsistent schemas across partitions to a Daft DataFrame:",
schemas,
)
pset = RayPartitionSet()
for i, obj in enumerate(daft_micropartitions):
pset.set_partition(i, RayMaterializedResult(obj))
return (
pset,
schemas[0],
)
def _get_ray_task_options(resource_request: ResourceRequest) -> dict[str, Any]:
options = {}
# FYI: Ray's default resource behaviour is documented here:
# https://docs.ray.io/en/latest/ray-core/tasks/resources.html
if resource_request.num_cpus is not None:
# Ray worker pool will thrash if a request comes in for fractional cpus,
# so we floor the request to at least 1 cpu here.
options["num_cpus"] = max(1, resource_request.num_cpus)
if resource_request.num_gpus:
options["num_gpus"] = resource_request.num_gpus
if resource_request.memory_bytes:
# Note that lower versions of Ray do not accept a value of 0 here,
# so the if-clause is load-bearing.
options["memory"] = resource_request.memory_bytes
return options
def build_partitions(
instruction_stack: list[Instruction], partial_metadatas: list[PartitionMetadata], *inputs: MicroPartition
) -> list[list[PartitionMetadata] | MicroPartition]:
partitions = list(inputs)
for instruction in instruction_stack:
partitions = instruction.run(partitions)
assert len(partial_metadatas) == len(partitions), f"{len(partial_metadatas)} vs {len(partitions)}"
metadatas = [PartitionMetadata.from_table(p).merge_with_partial(m) for p, m in zip(partitions, partial_metadatas)]
return [metadatas, *partitions]
@dataclasses.dataclass(frozen=True)
class PartitionTaskContext:
job_id: str
task_id: str
stage_id: int
# Give the same function different names to aid in profiling data distribution.
@ray_tracing.ray_remote_traced
@ray.remote
def single_partition_pipeline(
task_context: PartitionTaskContext,
daft_execution_config: PyDaftExecutionConfig,
instruction_stack: list[Instruction],
partial_metadatas: list[PartitionMetadata],
*inputs: MicroPartition,
) -> list[list[PartitionMetadata] | MicroPartition]:
with execution_config_ctx(
config=daft_execution_config,
), ray_tracing.collect_ray_task_metrics(
task_context.job_id, task_context.task_id, task_context.stage_id, daft_execution_config
):
return build_partitions(instruction_stack, partial_metadatas, *inputs)
@ray_tracing.ray_remote_traced
@ray.remote
def fanout_pipeline(
task_context: PartitionTaskContext,
daft_execution_config: PyDaftExecutionConfig,
instruction_stack: list[Instruction],
partial_metadatas: list[PartitionMetadata],
*inputs: MicroPartition,
) -> list[list[PartitionMetadata] | MicroPartition]:
with execution_config_ctx(config=daft_execution_config), ray_tracing.collect_ray_task_metrics(
task_context.job_id, task_context.task_id, task_context.stage_id, daft_execution_config
):
return build_partitions(instruction_stack, partial_metadatas, *inputs)
@ray_tracing.ray_remote_traced
@ray.remote
def reduce_pipeline(
task_context: PartitionTaskContext,
daft_execution_config: PyDaftExecutionConfig,
instruction_stack: list[Instruction],
partial_metadatas: list[PartitionMetadata],
inputs: list,
) -> list[list[PartitionMetadata] | MicroPartition]:
import ray
with execution_config_ctx(config=daft_execution_config), ray_tracing.collect_ray_task_metrics(
task_context.job_id, task_context.task_id, task_context.stage_id, daft_execution_config
):
return build_partitions(instruction_stack, partial_metadatas, *ray.get(inputs))
@ray_tracing.ray_remote_traced
@ray.remote
def reduce_and_fanout(
task_context: PartitionTaskContext,
daft_execution_config: PyDaftExecutionConfig,
instruction_stack: list[Instruction],
partial_metadatas: list[PartitionMetadata],
inputs: list,
) -> list[list[PartitionMetadata] | MicroPartition]:
import ray
with execution_config_ctx(config=daft_execution_config), ray_tracing.collect_ray_task_metrics(
task_context.job_id, task_context.task_id, task_context.stage_id, daft_execution_config
):
return build_partitions(instruction_stack, partial_metadatas, *ray.get(inputs))
@ray.remote
def get_metas(*partitions: MicroPartition) -> list[PartitionMetadata]:
return [PartitionMetadata.from_table(partition) for partition in partitions]
def _ray_num_cpus_provider(ttl_seconds: int = 1) -> Generator[int, None, None]:
"""Helper that gets the number of CPUs from Ray.
Used as a generator as it provides a guard against calling ray.cluster_resources()
more than once per `ttl_seconds`.
Example:
>>> p = _ray_num_cpus_provider()
>>> next(p)
"""
last_checked_time = time.time()
last_num_cpus_queried = int(ray.cluster_resources().get("CPU", 0))
while True:
currtime = time.time()
if currtime - last_checked_time < ttl_seconds:
yield last_num_cpus_queried
else:
last_checked_time = currtime
last_num_cpus_queried = int(ray.cluster_resources().get("CPU", 0))
yield last_num_cpus_queried
class Scheduler(ActorPoolManager):
def __init__(self, max_task_backlog: int | None, use_ray_tqdm: bool) -> None:
"""max_task_backlog: Max number of inflight tasks waiting for cores."""
# As of writing, Ray does not seem to be guaranteed to support
# more than this number of pending scheduling tasks.
# Ray has an internal proto that reports backlogged tasks [1],
# and each task proto can be up to 10 MiB [2],
# and protobufs have a max size of 2GB (from errors empirically encountered).
#
# https://github.com/ray-project/ray/blob/8427de2776717b30086c277e5e8e140316dbd193/src/ray/protobuf/node_manager.proto#L32
# https://github.com/ray-project/ray/blob/fb95f03f05981f232aa7a9073dd2c2512729e99a/src/ray/common/ray_config_def.h#LL513C1-L513C1
self.max_task_backlog = max_task_backlog if max_task_backlog is not None else 180
self.reserved_cores = 0
self.threads_by_df: dict[str, threading.Thread] = dict()
self.results_by_df: dict[str, Queue] = {}
self.active_by_df: dict[str, bool] = dict()
self.results_buffer_size_by_df: dict[str, int | None] = dict()
self._actor_pools: dict[str, RayRoundRobinActorPool] = {}
self.use_ray_tqdm = use_ray_tqdm
def next(self, result_uuid: str) -> RayMaterializedResult | StopIteration:
# Case: thread is terminated and no longer exists.
# Should only be hit for repeated calls to next() after StopIteration.
if result_uuid not in self.threads_by_df:
return StopIteration()
# Case: thread needs to be terminated
if not self.active_by_df.get(result_uuid, False):
return StopIteration()
# Common case: get the next result from the thread.
result = self.results_by_df[result_uuid].get()
return result
def start_plan(
self,
plan_scheduler: PhysicalPlanScheduler,
psets: dict[str, ray.ObjectRef],
result_uuid: str,
daft_execution_config: PyDaftExecutionConfig,
results_buffer_size: int | None = None,
) -> None:
self.results_by_df[result_uuid] = Queue(maxsize=1 if results_buffer_size is not None else -1)
self.active_by_df[result_uuid] = True
self.results_buffer_size_by_df[result_uuid] = results_buffer_size
t = threading.Thread(
target=self._run_plan,
name=result_uuid,
kwargs={
"plan_scheduler": plan_scheduler,
"psets": psets,
"result_uuid": result_uuid,
"daft_execution_config": daft_execution_config,
},
)
t.start()
self.threads_by_df[result_uuid] = t
def active_plans(self) -> list[str]:
return [r_uuid for r_uuid, is_active in self.active_by_df.items() if is_active]
def stop_plan(self, result_uuid: str) -> None:
if result_uuid in self.active_by_df:
# Mark df as non-active
self.active_by_df[result_uuid] = False
# wait till thread gracefully completes
self.threads_by_df[result_uuid].join()
# remove thread and history of df
del self.threads_by_df[result_uuid]
del self.active_by_df[result_uuid]
del self.results_by_df[result_uuid]
del self.results_buffer_size_by_df[result_uuid]
def get_actor_pool(
self,
name: str,
resource_request: ResourceRequest,
num_actors: int,
projection: ExpressionsProjection,
execution_config: PyDaftExecutionConfig,
) -> str:
actor_pool = RayRoundRobinActorPool(name, num_actors, resource_request, projection, execution_config)
self._actor_pools[name] = actor_pool
self._actor_pools[name].setup()
return name
def teardown_actor_pool(self, name: str) -> None:
if name in self._actor_pools:
self._actor_pools[name].teardown()
del self._actor_pools[name]
def _construct_dispatch_batch(
self,
execution_id: str,
tasks: ray_tracing.MaterializedPhysicalPlanWrapper,
dispatches_allowed: int,
runner_tracer: RunnerTracer,
) -> tuple[list[PartitionTask], bool]:
"""Constructs a batch of PartitionTasks that should be dispatched.
Args:
execution_id: The ID of the current execution.
tasks: The iterator over the physical plan.
dispatches_allowed (int): The maximum number of tasks that can be dispatched in this batch.
runner_tracer: The tracer to capture information about the dispatch batching process
Returns:
tuple[list[PartitionTask], bool]: A tuple containing:
- A list of PartitionTasks to be dispatched.
- A pagination boolean indicating whether or not there are more tasks to be had by calling _construct_dispatch_batch again
"""
with runner_tracer.dispatch_batching():
tasks_to_dispatch: list[PartitionTask] = []
# Loop until:
# - Reached the limit of the number of tasks we are allowed to dispatch
# - Encounter a `None` as the next step (short-circuit and return has_next=False)
while len(tasks_to_dispatch) < dispatches_allowed and self._is_active(execution_id):
next_step = next(tasks)
# CASE: Blocked on already dispatched tasks
# Early terminate and mark has_next=False
if next_step is None:
return tasks_to_dispatch, False
# CASE: A final result
# Place it in the result queue (potentially block on space to be available)
elif isinstance(next_step, MaterializedResult):
self._place_in_queue(execution_id, next_step)
# CASE: No-op task
# Just run it locally immediately.
elif len(next_step.instructions) == 0:
logger.debug("Running task synchronously in main thread: %s", next_step)
assert (
len(next_step.partial_metadatas) == 1
), "No-op tasks must have one output by definition, since there are no instructions to run"
[single_partial] = next_step.partial_metadatas
if single_partial.num_rows is None:
[single_meta] = ray.get(get_metas.remote(next_step.inputs))
accessor = PartitionMetadataAccessor.from_metadata_list(
[single_meta.merge_with_partial(single_partial)]
)
else:
accessor = PartitionMetadataAccessor.from_metadata_list(
[
PartitionMetadata(
num_rows=single_partial.num_rows,
size_bytes=single_partial.size_bytes,
boundaries=single_partial.boundaries,
)
]
)
next_step.set_result(
[RayMaterializedResult(partition, accessor, 0) for partition in next_step.inputs]
)
next_step.set_done()
# CASE: Actual task that needs to be dispatched
else:
tasks_to_dispatch.append(next_step)
return tasks_to_dispatch, True
def _dispatch_tasks(
self,
execution_id: str,
tasks_to_dispatch: list[PartitionTask],
daft_execution_config_objref: ray.ObjectRef,
runner_tracer: RunnerTracer,
) -> Iterator[tuple[PartitionTask, list[ray.ObjectRef]]]:
"""Iteratively Dispatches a batch of tasks to the Ray backend."""
with runner_tracer.dispatching():
for task in tasks_to_dispatch:
if task.actor_pool_id is None:
results = _build_partitions(execution_id, daft_execution_config_objref, task, runner_tracer)
else:
actor_pool = self._actor_pools.get(task.actor_pool_id)
assert actor_pool is not None, "Ray actor pool must live for as long as the tasks."
# TODO: Add tracing for submissions to actor pool
results = _build_partitions_on_actor_pool(task, actor_pool)
logger.debug("%s -> %s", task, results)
yield task, results
def _await_tasks(
self,
inflight_ref_to_task_id: dict[ray.ObjectRef, str],
inflight_tasks: dict[str, PartitionTask],
runner_tracer: RunnerTracer,
) -> list[ray.ObjectRef]:
"""Awaits for tasks to be completed. Returns tasks that are ready.
NOTE: This method blocks until at least 1 task is ready. Then it will return as many ready tasks as it can.
"""
if len(inflight_ref_to_task_id) == 0:
return []
# Await on (any) task to be ready with an unlimited timeout
with runner_tracer.awaiting(1, None):
ray.wait(
list(inflight_ref_to_task_id.keys()),
num_returns=1,
timeout=None,
fetch_local=False,
)
# Now, grab as many ready tasks as possible with a 0.01s timeout
timeout = 0.01
num_returns = len(inflight_ref_to_task_id)
with runner_tracer.awaiting(num_returns, timeout):
readies, _ = ray.wait(
list(inflight_ref_to_task_id.keys()),
num_returns=num_returns,
timeout=timeout,
fetch_local=False,
)
# Update traces
for ready in readies:
if ready in inflight_ref_to_task_id:
task_id = inflight_ref_to_task_id[ready]
runner_tracer.task_received_as_ready(task_id, inflight_tasks[task_id].stage_id)
return readies
def _is_active(self, execution_id: str):
"""Checks if the execution for the provided `execution_id` is still active."""
return self.active_by_df.get(execution_id, False)
def _place_in_queue(self, execution_id: str, item: ray.ObjectRef):
"""Places a result into the queue for the provided `execution_id.
NOTE: This will block and poll busily until space is available on the queue
`
"""
while self._is_active(execution_id):
try:
self.results_by_df[execution_id].put(item, timeout=0.1)
break
except Full:
pass
def _run_plan(
self,
plan_scheduler: PhysicalPlanScheduler,
psets: dict[str, ray.ObjectRef],
result_uuid: str,
daft_execution_config: PyDaftExecutionConfig,
) -> None:
# Put execution config into cluster once to share it amongst all tasks
daft_execution_config_objref = ray.put(daft_execution_config)
# Get executable tasks from plan scheduler.
results_buffer_size = self.results_buffer_size_by_df[result_uuid]
inflight_tasks: dict[str, PartitionTask[ray.ObjectRef]] = dict()
inflight_ref_to_task: dict[ray.ObjectRef, str] = dict()
pbar = ProgressBar(use_ray_tqdm=self.use_ray_tqdm)
num_cpus_provider = _ray_num_cpus_provider()
start = datetime.now()
profile_filename = (
f"profile_RayRunner.run()_"
f"{datetime.replace(datetime.now(), second=0, microsecond=0).isoformat()[:-3]}.json"
)
with profiler(profile_filename), ray_tracing.ray_tracer(result_uuid, daft_execution_config) as runner_tracer:
raw_tasks = plan_scheduler.to_partition_tasks(
psets,
self,
# Attempt to subtract 1 from results_buffer_size because the return Queue size is already 1
# If results_buffer_size=1 though, we can't do much and the total buffer size actually has to be >= 2
# because we have two buffers (the Queue and the buffer inside the `materialize` generator)
None if results_buffer_size is None else max(results_buffer_size - 1, 1),
)
tasks = ray_tracing.MaterializedPhysicalPlanWrapper(raw_tasks, runner_tracer)
try:
###
# Scheduling Loop:
#
# DispatchBatching ─► Dispatch
# ▲ │ ───────► Await
# └────────────────────────┘ │
# ▲ │
# └───────────────────────────────┘
###
wave_count = 0
while self._is_active(result_uuid):
###
# Dispatch Loop:
#
# DispatchBatching ─► Dispatch
# ▲ │
# └────────────────────────┘
###
wave_count += 1
with runner_tracer.dispatch_wave(wave_count):
while self._is_active(result_uuid):
# Update available cluster resources
# TODO: improve control loop code to be more understandable and dynamically adjust backlog
cores: int = max(
next(num_cpus_provider) - self.reserved_cores, 1
) # assume at least 1 CPU core for bootstrapping clusters that scale from zero
max_inflight_tasks = cores + self.max_task_backlog
dispatches_allowed = max_inflight_tasks - len(inflight_tasks)
dispatches_allowed = min(cores, dispatches_allowed)
# Dispatch Batching
tasks_to_dispatch, has_next = self._construct_dispatch_batch(
result_uuid,
tasks,
dispatches_allowed,
runner_tracer,
)
logger.debug(
"%ss: RayRunner dispatching %s tasks",
(datetime.now() - start).total_seconds(),
len(tasks_to_dispatch),
)
if not self._is_active(result_uuid):
break
# Dispatch
for task, result_obj_refs in self._dispatch_tasks(
result_uuid,
tasks_to_dispatch,
daft_execution_config_objref,
runner_tracer,
):
inflight_tasks[task.id()] = task
for result in result_obj_refs:
inflight_ref_to_task[result] = task.id()
pbar.mark_task_start(task)
# Break the dispatch batching/dispatch loop if no more dispatches allowed, or physical plan
# needs work for forward progress
if dispatches_allowed == 0 or not has_next:
break
###
# Await:
# Wait for some work to be completed from the current wave's dispatch
# Then we perform the necessary record-keeping on tasks that were retrieved as ready.
###
readies = self._await_tasks(
inflight_ref_to_task,
inflight_tasks,
runner_tracer,
)
for ready in readies:
if ready in inflight_ref_to_task:
task_id = inflight_ref_to_task[ready]
# Mark the entire task associated with the result as done.
task = inflight_tasks[task_id]
task.set_done()
if isinstance(task, SingleOutputPartitionTask):
del inflight_ref_to_task[ready]
elif isinstance(task, MultiOutputPartitionTask):
for partition in task.partitions():
del inflight_ref_to_task[partition]
pbar.mark_task_done(task)
del inflight_tasks[task_id]
except StopIteration as e:
self._place_in_queue(result_uuid, e)
# Ensure that all Exceptions are correctly propagated to the consumer before reraising to kill thread
except Exception as e:
self._place_in_queue(result_uuid, e)
pbar.close()
raise
pbar.close()
@contextlib.contextmanager
def actor_pool_context(
self,
name: str,
actor_resource_request: ResourceRequest,
task_resource_request: ResourceRequest,
num_actors: PartID,
projection: ExpressionsProjection,
) -> Iterator[str]:
# Ray runs actor methods serially, so the resource request for an actor should be both the actor's resources and the task's resources
resource_request = actor_resource_request + task_resource_request
execution_config = get_context().daft_execution_config
try:
yield self.get_actor_pool(name, resource_request, num_actors, projection, execution_config)
finally:
self.teardown_actor_pool(name)
SCHEDULER_ACTOR_NAME = "scheduler"
SCHEDULER_ACTOR_NAMESPACE = "daft"
@ray.remote(num_cpus=1)
class SchedulerActor(Scheduler):
def __init__(self, *n, **kw) -> None:
super().__init__(*n, **kw)
self.reserved_cores = 1
def _build_partitions(
job_id: str,
daft_execution_config_objref: ray.ObjectRef,