-
Notifications
You must be signed in to change notification settings - Fork 417
Expand file tree
/
Copy pathdataframe.py
More file actions
5340 lines (4550 loc) · 235 KB
/
dataframe.py
File metadata and controls
5340 lines (4550 loc) · 235 KB
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
# ruff: noqa: I002
# isort: dont-add-import: from __future__ import annotations
#
# This file uses strings for forward type annotations in public APIs,
# in order to support runtime typechecking across different Python versions.
# For technical details, see https://github.com/Eventual-Inc/Daft/pull/630
import io
import multiprocessing
import os
import pathlib
import typing
import warnings
from collections.abc import Callable, Iterable, Iterator, Mapping
from dataclasses import dataclass
from datetime import datetime, timezone
from functools import partial, reduce
from typing import TYPE_CHECKING, Any, Concatenate, Literal, ParamSpec, TypeVar, Union, overload
from daft.api_annotations import DataframePublicAPI
from daft.context import get_context
from daft.convert import InputListType
from daft.daft import (
DistributedPhysicalPlan,
FileFormat,
IOConfig,
JoinStrategy,
JoinType,
PyFormatSinkOption,
WriteMode,
)
from daft.dataframe.display import MermaidOptions
from daft.dataframe.preview import Preview, PreviewAlign, PreviewColumn, PreviewFormat, PreviewFormatter
from daft.datatype import DataType
from daft.errors import ExpressionTypeError
from daft.execution.native_executor import NativeExecutor
from daft.expressions import Expression, ExpressionsProjection, col, lit
from daft.logical.builder import LogicalPlanBuilder
from daft.recordbatch import MicroPartition, RecordBatch
from daft.runners import get_or_create_runner
from daft.runners.partitioning import (
LocalPartitionSet,
MaterializedResult,
PartitionCacheEntry,
PartitionSet,
PartitionT,
)
from daft.utils import (
ColumnInputType,
ManyColumnsInputType,
column_input_to_expression,
column_inputs_to_expressions,
in_notebook,
)
if TYPE_CHECKING:
import dask
import deltalake
import pandas
import pyarrow
import pyiceberg
import ray
import torch
from sqlalchemy.engine import Connection
from daft.execution.metadata import ExecutionMetadata
from daft.io import DataSink
from daft.io.catalog import DataCatalogTable
from daft.io.lance.rest_config import LanceRestConfig
from daft.io.sink import WriteResultType
from daft.unity_catalog import UnityCatalogTable
from daft.schema import Schema
UDFReturnType = TypeVar("UDFReturnType", covariant=True)
T = TypeVar("T")
R = TypeVar("R")
P = ParamSpec("P")
def to_logical_plan_builder(*parts: MicroPartition) -> LogicalPlanBuilder:
"""Creates a Daft DataFrame from a single RecordBatch.
Args:
parts: The Tables that we wish to convert into a Daft DataFrame.
Returns:
DataFrame: Daft DataFrame created from the provided Table.
"""
if not parts:
raise ValueError("Can't create a DataFrame from an empty list of tables.")
result_pset = LocalPartitionSet()
for i, part in enumerate(parts):
result_pset.set_partition_from_table(i, part)
cache_entry = get_or_create_runner().put_partition_set_into_cache(result_pset)
size_bytes = result_pset.size_bytes()
num_rows = len(result_pset)
assert size_bytes is not None, "In-memory data should always have non-None size in bytes"
return LogicalPlanBuilder.from_in_memory_scan(
cache_entry, parts[0].schema(), result_pset.num_partitions(), size_bytes, num_rows=num_rows
)
def _utc_now() -> datetime:
return datetime.now(timezone.utc)
class DataFrame:
"""A Daft DataFrame is a table of data.
It has columns, where each column has a type and the same number of items (rows) as all other columns.
"""
def __init__(self, builder: LogicalPlanBuilder) -> None:
"""Constructs a DataFrame according to a given LogicalPlan.
Users are expected instead to call the classmethods on DataFrame to create a DataFrame.
Args:
builder: LogicalPlan describing the steps required to arrive at this DataFrame
"""
if not isinstance(builder, LogicalPlanBuilder):
if isinstance(builder, dict):
raise ValueError(
"DataFrames should be constructed with a dictionary of columns using `daft.from_pydict`"
)
if isinstance(builder, list):
raise ValueError(
"DataFrames should be constructed with a list of dictionaries using `daft.from_pylist`"
)
raise ValueError(f"Expected DataFrame to be constructed with a LogicalPlanBuilder, received: {builder}")
self.__builder = builder
self._result_cache: PartitionCacheEntry | None = None
self._preview = Preview(partition=None, total_rows=None)
self._metadata: ExecutionMetadata | None = None
self._num_preview_rows = get_context().daft_execution_config.num_preview_rows
@property
def _builder(self) -> LogicalPlanBuilder:
if self._result_cache is None:
return self.__builder
else:
num_partitions = self._result_cache.num_partitions()
size_bytes = self._result_cache.size_bytes()
num_rows = self._result_cache.num_rows()
# Partition set should always be set on cache entry.
assert num_partitions is not None and size_bytes is not None and num_rows is not None, (
"Partition set should always be set on cache entry"
)
return self.__builder.from_in_memory_scan(
self._result_cache,
self.__builder.schema(),
num_partitions=num_partitions,
size_bytes=size_bytes,
num_rows=num_rows,
)
def _get_current_builder(self) -> LogicalPlanBuilder:
"""Returns the current logical plan builder, without any caching optimizations."""
return self.__builder
@property
def _result(self) -> PartitionSet[PartitionT] | None:
if self._result_cache is None:
return None
else:
return self._result_cache.value
@property
def metrics(self) -> RecordBatch | None:
if self._result_cache is None:
raise ValueError("Metrics are not available until the DataFrame has been materialized")
else:
return self._metadata.to_recordbatch() if self._metadata else None
def pipe(
self,
function: Callable[Concatenate["DataFrame", P], T],
*args: P.args,
**kwargs: P.kwargs,
) -> T:
"""Apply the function to this DataFrame.
Args:
function (Callable[Concatenate["DataFrame", P], T]): Function to apply.
*args (P.args): Positional arguments to pass to the function.
**kwargs (P.kwargs): Keyword arguments to pass to the function.
Returns:
Result of applying the function on this DataFrame.
Examples:
>>> import daft
>>>
>>> df = daft.from_pydict({"x": [1, 2, 3]})
>>>
>>> def square(df, column: str):
... return df.select((df[column] * df[column]).alias(column))
>>>
>>> df.pipe(square, "x").show()
╭───────╮
│ x │
│ --- │
│ Int64 │
╞═══════╡
│ 1 │
├╌╌╌╌╌╌╌┤
│ 4 │
├╌╌╌╌╌╌╌┤
│ 9 │
╰───────╯
<BLANKLINE>
(Showing first 3 of 3 rows)
"""
return function(self, *args, **kwargs)
@DataframePublicAPI
def explain(
self, show_all: bool = False, format: str = "ascii", simple: bool = False, file: io.IOBase | None = None
) -> Any:
r"""Prints the (logical and physical) plans that will be executed to produce this DataFrame.
Defaults to showing the unoptimized logical plan. Use `show_all=True` to show the unoptimized logical plan,
the optimized logical plan, and the physical plan.
Args:
show_all (bool): Whether to show the optimized logical plan and the physical plan in addition to the
unoptimized logical plan.
format (str): The format to print the plan in. one of 'ascii' or 'mermaid'
simple (bool): Whether to only show the type of op for each node in the plan, rather than showing details
of how each op is configured.
file (Optional[io.IOBase]): Location to print the output to, or defaults to None which defaults to the default location for
print (in Python, that should be sys.stdout)
Returns:
Union[None, str, MermaidFormatter]:
- If `format="mermaid"` and running in a notebook, returns a `MermaidFormatter` instance for rich rendering.
- If `format="mermaid"` and not in a notebook, returns a string representation of the plan.
- Otherwise, prints the plan(s) to the specified file or stdout and returns `None`.
Examples:
>>> import daft
>>>
>>> df = daft.from_pydict({"x": [1, 2, 3]})
>>>
>>> def double(df, column: str):
... return df.select((df[column] * df[column]).alias(column))
>>>
>>> df = df.pipe(double, "x")
>>>
>>> df.explain()
== Unoptimized Logical Plan ==
<BLANKLINE>
* Project: col(x) * col(x) as x
|
* Source:
| Number of partitions = 1
| Output schema = x#Int64
<BLANKLINE>
<BLANKLINE>
<BLANKLINE>
Set `show_all=True` to also see the Optimized and Physical plans. This will run the query optimizer.
"""
is_cached = self._result_cache is not None
if format == "mermaid":
from daft.dataframe.display import MermaidFormatter
from daft.utils import in_notebook
instance = MermaidFormatter(self.__builder, show_all, simple, is_cached)
if file is not None:
# if we are printing to a file, we print the markdown representation of the plan
text = instance._repr_markdown_()
print(text, file=file)
if in_notebook():
# if in a notebook, we return the class instance and let jupyter display it
return instance
else:
# if we are not in a notebook, we return the raw markdown instead of the class instance
return repr(instance)
print_to_file = partial(print, file=file)
if self._result_cache is not None:
print_to_file("Result is cached and will skip computation\n")
print_to_file(self._builder.pretty_print(simple, format=format))
print_to_file("However here is the logical plan used to produce this result:\n", file=file)
builder = self.__builder
print_to_file("== Unoptimized Logical Plan ==\n")
print_to_file(builder.pretty_print(simple, format=format))
if show_all:
print_to_file("\n== Optimized Logical Plan ==\n")
execution_config = get_context().daft_execution_config
builder = builder.optimize(execution_config)
print_to_file(builder.pretty_print(simple))
print_to_file("\n== Physical Plan ==\n")
if get_or_create_runner().name != "native":
from daft.daft import DistributedPhysicalPlan
distributed_plan = DistributedPhysicalPlan.from_logical_plan_builder(
builder._builder, "<tmp>", execution_config
)
if format == "ascii":
print_to_file(distributed_plan.repr_ascii(simple))
elif format == "mermaid":
print_to_file(distributed_plan.repr_mermaid(MermaidOptions(simple)))
else:
native_executor = NativeExecutor()
print_to_file(
native_executor.pretty_print(builder, get_context().daft_execution_config, simple, format=format)
)
else:
print_to_file(
"\n \nSet `show_all=True` to also see the Optimized and Physical plans. This will run the query optimizer.",
)
return None
def num_partitions(self) -> int | None:
"""Returns the number of partitions that will be used to execute this DataFrame.
The query optimizer may change the partitioning strategy. This method runs the optimizer
and then inspects the resulting physical plan scheduler to determine how many partitions
the execution will use.
Returns:
int: The number of partitions in the optimized physical execution plan.
Examples:
>>> import daft
>>>
>>> daft.set_runner_ray() # doctest: +SKIP
>>>
>>> # Create a DataFrame with 1000 rows
>>> df = daft.from_pydict({"x": list(range(1000))})
>>>
>>> # Partition count may depend on default config or optimizer decisions
>>> df.num_partitions() # doctest: +SKIP
1
>>>
>>> # You can repartition manually (if supported), and then inspect again:
>>> df2 = df.repartition(10) # doctest: +SKIP
>>> df2.num_partitions() # doctest: +SKIP
10
"""
runner_name = get_or_create_runner().name
# Native runner does not support num_partitions
if runner_name == "native":
return None
else:
execution_config = get_context().daft_execution_config
optimized = self._builder.optimize(execution_config)
distributed_plan = DistributedPhysicalPlan.from_logical_plan_builder(
optimized._builder, "<tmp>", execution_config
)
return distributed_plan.num_partitions()
@DataframePublicAPI
def schema(self) -> Schema:
"""Returns the Schema of the DataFrame, which provides information about each column, as a Python object.
Returns:
Schema: schema of the DataFrame
Examples:
>>> import daft
>>>
>>> df = daft.from_pydict({"x": [1, 2, 3], "y": ["a", "b", "c"]})
>>> df.schema()
╭─────────────┬────────╮
│ column_name ┆ type │
╞═════════════╪════════╡
│ x ┆ Int64 │
├╌╌╌╌╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌┤
│ y ┆ String │
╰─────────────┴────────╯
<BLANKLINE>
"""
return self.__builder.schema()
@property
def column_names(self) -> list[str]:
"""Returns column names of DataFrame as a list of strings.
Returns:
List[str]: Column names of this DataFrame.
"""
return self.__builder.schema().column_names()
@property
def columns(self) -> list[Expression]:
"""Returns column of DataFrame as a list of Expressions.
Returns:
List[Expression]: Columns of this DataFrame.
"""
return [col(field.name) for field in self.__builder.schema()]
@DataframePublicAPI
def __iter__(self) -> Iterator[dict[str, Any]]:
"""Alias of `self.iter_rows()` with default arguments for convenient access of data.
Returns:
Iterator[dict[str, Any]]: An iterator over the rows of the DataFrame, where each row is a dictionary
mapping column names to values.
Examples:
>>> import daft
>>> df = daft.from_pydict({"foo": [1, 2, 3], "bar": ["a", "b", "c"]})
>>> for row in df:
... print(row)
{'foo': 1, 'bar': 'a'}
{'foo': 2, 'bar': 'b'}
{'foo': 3, 'bar': 'c'}
Tip:
See also [`df.iter_rows()`][daft.DataFrame.iter_rows]: iterator over rows with more options
"""
return self.iter_rows(results_buffer_size=None)
@DataframePublicAPI
def iter_rows(
self,
results_buffer_size: int | None | Literal["num_cpus"] = "num_cpus",
column_format: Literal["python", "arrow"] = "python",
) -> Iterator[dict[str, Any]]:
"""Return an iterator of rows for this dataframe.
Each row will be a Python dictionary of the form `{ "key" : value, ...}`. If you are instead looking to iterate over
entire partitions of data, see [`df.iter_partitions()`][daft.DataFrame.iter_partitions].
By default, Daft will convert the columns to Python lists for easy consumption. Datatypes with Python equivalents will be converted accordingly, e.g. timestamps to datetime, tensors to numpy arrays.
For nested data such as List or Struct arrays, however, this can be expensive. You may wish to set `column_format` to "arrow" such that the nested data is returned as Arrow scalars.
Args:
results_buffer_size: how many partitions to allow in the results buffer (defaults to the total number of CPUs
available on the machine).
column_format: the format of the columns to iterate over. One of "python" or "arrow". Defaults to "python".
Note: A quick note on configuring asynchronous/parallel execution using `results_buffer_size`.
The `results_buffer_size` kwarg controls how many results Daft will allow to be in the buffer while iterating.
Once this buffer is filled, Daft will not run any more work until some partition is consumed from the buffer.
* Increasing this value means the iterator will consume more memory and CPU resources but have higher throughput
* Decreasing this value means the iterator will consume lower memory and CPU resources, but have lower throughput
* Setting this value to `None` means the iterator will consume as much resources as it deems appropriate per-iteration
The default value is the total number of CPUs available on the current machine.
Returns:
Iterator[dict[str, Any]]: An iterator over the rows of the DataFrame, where each row is a dictionary
mapping column names to values.
Examples:
>>> import daft
>>>
>>> df = daft.from_pydict({"foo": [1, 2, 3], "bar": ["a", "b", "c"]})
>>> for row in df.iter_rows():
... print(row)
{'foo': 1, 'bar': 'a'}
{'foo': 2, 'bar': 'b'}
{'foo': 3, 'bar': 'c'}
Tip:
See also [`df.iter_partitions()`][daft.DataFrame.iter_partitions]: iterator over entire partitions instead of single rows
"""
if results_buffer_size == "num_cpus":
results_buffer_size = multiprocessing.cpu_count()
def arrow_iter_rows(table: "pyarrow.Table") -> Iterator[dict[str, Any]]:
columns = table.columns
for i in range(len(table)):
row = {col._name: col[i] for col in columns}
yield row
def python_iter_rows(pydict: dict[str, list[Any]], num_rows: int) -> Iterator[dict[str, Any]]:
for i in range(num_rows):
row = {key: value[i] for (key, value) in pydict.items()}
yield row
if self._result is not None:
# If the dataframe has already finished executing,
# use the precomputed results.
if column_format == "python":
yield from python_iter_rows(self.to_pydict(), len(self))
elif column_format == "arrow":
yield from arrow_iter_rows(self.to_arrow())
else:
raise ValueError(
f"Unsupported column_format: {column_format}, supported formats are 'python' and 'arrow'"
)
else:
# Execute the dataframe in a streaming fashion.
partitions_iter = get_or_create_runner().run_iter_tables(
self._builder, results_buffer_size=results_buffer_size
)
# Iterate through partitions.
for partition in partitions_iter:
if column_format == "python":
yield from python_iter_rows(partition.to_pydict(), len(partition))
elif column_format == "arrow":
yield from arrow_iter_rows(partition.to_arrow())
else:
raise ValueError(
f"Unsupported column_format: {column_format}, supported formats are 'python' and 'arrow'"
)
@DataframePublicAPI
def to_arrow_iter(
self,
results_buffer_size: int | None | Literal["num_cpus"] = "num_cpus",
) -> Iterator["pyarrow.RecordBatch"]:
"""Return an iterator of pyarrow recordbatches for this dataframe.
Args:
results_buffer_size: how many partitions to allow in the results buffer (defaults to the total number of CPUs
available on the machine).
Note: A quick note on configuring asynchronous/parallel execution using `results_buffer_size`.
The `results_buffer_size` kwarg controls how many results Daft will allow to be in the buffer while iterating.
Once this buffer is filled, Daft will not run any more work until some partition is consumed from the buffer.
* Increasing this value means the iterator will consume more memory and CPU resources but have higher throughput
* Decreasing this value means the iterator will consume lower memory and CPU resources, but have lower throughput
* Setting this value to `None` means the iterator will consume as much resources as it deems appropriate per-iteration
The default value is the total number of CPUs available on the current machine.
Returns:
Iterator[pyarrow.RecordBatch]: An iterator over the RecordBatches of the DataFrame.
Examples:
>>> import daft
>>>
>>> df = daft.from_pydict({"foo": [1, 2, 3], "bar": ["a", "b", "c"]})
>>> for batch in df.to_arrow_iter():
... print(batch)
pyarrow.RecordBatch
foo: int64
bar: large_string
----
foo: [1,2,3]
bar: ["a","b","c"]
"""
if results_buffer_size == "num_cpus":
results_buffer_size = multiprocessing.cpu_count()
if results_buffer_size is not None and not results_buffer_size > 0:
raise ValueError(f"Provided `results_buffer_size` value must be > 0, received: {results_buffer_size}")
results = self._result
if results is not None:
# If the dataframe has already finished executing,
# use the precomputed results.
for _, result in results.items():
yield from (result.micropartition().to_arrow().to_batches())
else:
# Execute the dataframe in a streaming fashion.
partitions_iter = get_or_create_runner().run_iter_tables(
self._builder, results_buffer_size=results_buffer_size
)
# Iterate through partitions.
for partition in partitions_iter:
yield from partition.to_arrow().to_batches()
@DataframePublicAPI
def iter_partitions(
self, results_buffer_size: int | None | Literal["num_cpus"] = "num_cpus"
) -> Iterator[Union[MicroPartition, "ray.ObjectRef"]]:
"""Begin executing this dataframe and return an iterator over the partitions.
Each partition will be returned as a daft.recordbatch object (if using Python runner backend)
or a ray ObjectRef (if using Ray runner backend).
Args:
results_buffer_size: how many partitions to allow in the results buffer (defaults to the total number of CPUs
available on the machine).
Note: A quick note on configuring asynchronous/parallel execution using `results_buffer_size`.
The `results_buffer_size` kwarg controls how many results Daft will allow to be in the buffer while iterating.
Once this buffer is filled, Daft will not run any more work until some partition is consumed from the buffer.
* Increasing this value means the iterator will consume more memory and CPU resources but have higher throughput
* Decreasing this value means the iterator will consume lower memory and CPU resources, but have lower throughput
* Setting this value to `None` means the iterator will consume as much resources as it deems appropriate per-iteration
The default value is the total number of CPUs available on the current machine.
Returns:
Iterator[Union[MicroPartition, ray.ObjectRef]]: An iterator over the partitions of the DataFrame.
Each partition is a MicroPartition object (if using Python runner backend) or a ray ObjectRef
(if using Ray runner backend).
Examples:
>>> import daft
>>>
>>> daft.set_runner_ray() # doctest: +SKIP
>>>
>>> df = daft.from_pydict({"foo": [1, 2, 3], "bar": ["a", "b", "c"]}).into_partitions(2)
>>> for part in df.iter_partitions():
... print(part) # doctest: +SKIP
MicroPartition with 3 rows:
TableState: Loaded. 1 tables
╭───────┬────────╮
│ foo ┆ bar │
│ --- ┆ --- │
│ Int64 ┆ String │
╞═══════╪════════╡
│ 1 ┆ a │
├╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌┤
│ 2 ┆ b │
├╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌┤
│ 3 ┆ c │
╰───────┴────────╯
<BLANKLINE>
<BLANKLINE>
Statistics: missing
"""
if results_buffer_size == "num_cpus":
results_buffer_size = multiprocessing.cpu_count()
elif results_buffer_size is not None and not results_buffer_size > 0:
raise ValueError(f"Provided `results_buffer_size` value must be > 0, received: {results_buffer_size}")
results = self._result
if results is not None:
# If the dataframe has already finished executing,
# use the precomputed results.
for mat_result in results.values():
yield mat_result.partition()
else:
# Execute the dataframe in a streaming fashion.
results_iter: Iterator[MaterializedResult[Any]] = get_or_create_runner().run_iter(
self._builder, results_buffer_size=results_buffer_size
)
for result in results_iter:
yield result.partition()
def _populate_preview(self) -> None:
"""Populates the preview of the DataFrame, if it is not already populated."""
results = self._result
if results is None:
return
num_preview_rows = min(self._num_preview_rows, len(self))
preview_partition_invalid = self._preview.partition is None or len(self._preview.partition) != num_preview_rows
if preview_partition_invalid:
preview_parts = results._get_preview_micropartitions(num_preview_rows)
preview_results = LocalPartitionSet()
for i, part in enumerate(preview_parts):
preview_results.set_partition_from_table(i, part)
preview_partition = preview_results._get_merged_micropartition(self.schema())
self._preview = Preview(
partition=preview_partition,
total_rows=len(self),
)
@DataframePublicAPI
def __repr__(self) -> str:
self._populate_preview()
preview = PreviewFormatter(self._preview, self.schema())
return preview.__repr__()
@DataframePublicAPI
def _repr_html_(self) -> str:
self._populate_preview()
preview = PreviewFormatter(self._preview, self.schema())
try:
if in_notebook() and self._preview.partition is not None:
try:
interactive_html = preview._generate_interactive_html()
return interactive_html
except Exception:
pass
return preview._repr_html_()
except ImportError:
return preview._repr_html_()
@DataframePublicAPI
def _repr_mimebundle_(
self, include: Iterable[str] | None = None, exclude: Iterable[str] | None = None
) -> dict[str, str]:
mimebundle = {
"text/plain": self.__repr__(),
"text/html": self._repr_html_(),
}
if include is not None:
include_set = set(include)
mimebundle = {k: v for k, v in mimebundle.items() if k in include_set}
if exclude is not None:
exclude_set = set(exclude)
mimebundle = {k: v for k, v in mimebundle.items() if k not in exclude_set}
return mimebundle
###
# Creation methods
###
@classmethod
def _from_pylist(cls, data: list[dict[str, Any]]) -> "DataFrame":
"""Creates a DataFrame from a list of dictionaries."""
headers: set[str] = set()
for row in data:
if not isinstance(row, dict):
raise ValueError(f"Expected list of dictionaries of {{column_name: value}}, received: {type(row)}")
headers.update(row.keys())
headers_ordered = sorted(list(headers))
return cls._from_pydict(data={header: [row.get(header, None) for row in data] for header in headers_ordered})
@classmethod
def _from_pydict(cls, data: Mapping[str, InputListType]) -> "DataFrame":
"""Creates a DataFrame from a Python dictionary."""
column_lengths = {key: len(data[key]) for key in data}
if len(set(column_lengths.values())) > 1:
raise ValueError(
f"Expected all columns to be of the same length, but received columns with lengths: {column_lengths}"
)
data_micropartition = MicroPartition.from_pydict(data)
return cls._from_micropartitions(data_micropartition)
@classmethod
def _from_arrow(cls, data: Union["pyarrow.Table", list["pyarrow.Table"], Iterable["pyarrow.Table"]]) -> "DataFrame":
"""Creates a DataFrame from a `pyarrow Table <https://arrow.apache.org/docs/python/generated/pyarrow.Table.html>`__."""
if isinstance(data, Iterable):
data = list(data)
if not isinstance(data, list):
data = [data]
parts = [MicroPartition.from_arrow(table) for table in data]
return cls._from_micropartitions(*parts)
@classmethod
def _from_pandas(cls, data: Union["pandas.DataFrame", list["pandas.DataFrame"]]) -> "DataFrame":
"""Creates a Daft DataFrame from a `pandas DataFrame <https://pandas.pydata.org/docs/reference/api/pandas.DataFrame.html>`__."""
if not isinstance(data, list):
data = [data]
parts = [MicroPartition.from_pandas(df) for df in data]
return cls._from_micropartitions(*parts)
@classmethod
def _from_micropartitions(cls, *parts: MicroPartition) -> "DataFrame":
"""Creates a Daft DataFrame from MicroPartition(s).
Args:
parts: The Tables that we wish to convert into a Daft DataFrame.
Returns:
DataFrame: Daft DataFrame created from the provided Table.
"""
if not parts:
raise ValueError("Can't create a DataFrame from an empty list of tables.")
result_pset = LocalPartitionSet()
for i, part in enumerate(parts):
result_pset.set_partition_from_table(i, part)
cache_entry = get_or_create_runner().put_partition_set_into_cache(result_pset)
size_bytes = result_pset.size_bytes()
num_rows = len(result_pset)
assert size_bytes is not None, "In-memory data should always have non-None size in bytes"
builder = LogicalPlanBuilder.from_in_memory_scan(
cache_entry, parts[0].schema(), result_pset.num_partitions(), size_bytes, num_rows=num_rows
)
df = cls(builder)
df._result_cache = cache_entry
# build preview
df._populate_preview()
return df
@classmethod
def _from_schema(cls, schema: Schema) -> "DataFrame":
"""Creates a Daft DataFrame from a Schema.
Args:
schema: The Schema to convert into a DataFrame.
Returns:
DataFrame: Daft DataFrame with "column_name" and "type" fields.
"""
pydict: dict[str, list[str]] = {"column_name": [], "type": []}
for field in schema:
pydict["column_name"].append(field.name)
pydict["type"].append(str(field.dtype))
return DataFrame._from_pydict(pydict)
###
# Write methods
###
@DataframePublicAPI
def write_sql(
self,
table_name: str,
conn: str | Callable[[], "Connection"],
write_mode: Literal["append", "overwrite", "fail"] = "append",
column_types: dict[str, Any] | None = None,
non_primitive_handling: Literal["bytes", "str", "error"] | None = None,
) -> "DataFrame":
"""Write the DataFrame to a SQL database and return write metrics.
The write is executed via :meth:`daft.DataFrame.write_sink` using an internal
:class:`daft.io._sql.SQLDataSink`.
Primitive columns (ints, floats, bools, strings, binary, dates, timestamps) are written by converting to a pandas DataFrame and calling :meth:`pandas.DataFrame.to_sql`, letting SQLAlchemy or ``column_types`` choose concrete SQL types.
Non-primitive columns (lists, structs, maps, tensors, images, embeddings, python objects, etc.) are normalized according to ``non_primitive_handling`` (default ``None`` behaves like ``"str"``): ``"str"`` serializes values to text (JSON for arrays/maps and other containers, ``str(..)`` otherwise), ``"bytes"`` writes UTF-8 bytes of that text, and ``"error"`` fails if such columns are present.
Args:
table_name (str): Name of the table to write to.
conn (str | Callable[[], "Connection"]): Connection string or factory.
write_mode (str): Mode to write to the table. "append", "overwrite", or "fail". Defaults to "append".
column_types (Optional[Dict[str, Any]]): Optional mapping from column names to
SQLAlchemy types to use when creating the table or casting columns.
Passed through to the underlying SQL engine when creating or writing
the table.
non_primitive_handling (Literal["bytes", "str", "error"] | None):
Controls how non-primitive columns are normalized before reaching SQL; default ``None`` behaves like ``"str"``. Accepted values are ``"str"``, ``"bytes"``, and ``"error"``.
Returns:
DataFrame: A single-row DataFrame containing aggregate write metrics with
columns ``total_written_rows`` and ``total_written_bytes``.
Warning:
This features is early in development and will likely experience API changes.
Note:
Primitive columns still rely on pandas/SQLAlchemy (or ``column_types``) for concrete SQL types, while non-primitive columns are pre-normalized in Python according to ``non_primitive_handling`` before reaching the SQL driver.
Examples:
Write to a SQL table using a database URL and explicit SQLAlchemy dtypes:
>>> from sqlalchemy import DateTime, Integer, String
>>> import datetime
>>> import daft
>>> df = daft.from_pydict(
... {
... "id": [1, 2],
... "name": ["Alice", "Bob"],
... "created_at": [
... datetime.datetime(2024, 1, 1, 0, 0, 0),
... datetime.datetime(2024, 1, 2, 0, 0, 0),
... ],
... }
... )
>>> column_types = {
... "id": Integer(),
... "name": String(length=255),
... "created_at": DateTime(timezone=True),
... }
>>> metrics_df = df.write_sql("users", "sqlite:///my_database.db", column_types=column_types)
Write to a SQL table using a SQLAlchemy connection factory and dtypes:
>>> import sqlalchemy
>>> def create_conn():
... return sqlalchemy.create_engine("sqlite:///my_database.db").connect()
>>> metrics_df = df.write_sql("users", create_conn, column_types=column_types)
Write to a SQL table using a database URL with column_types=None to rely on inferred types:
>>> df = daft.from_pydict({"id": [1], "name": ["Alice"]})
>>> metrics_df = df.write_sql("users", "sqlite:///my_database.db", column_types=None)
"""
from daft.io._sql import SQLDataSink
sink = SQLDataSink(
table_name=table_name,
conn=conn,
write_mode=write_mode,
column_types=column_types,
df_schema=self.schema(),
non_primitive_handling=non_primitive_handling,
)
if non_primitive_handling is None:
# Check for non-primitive types in the schema and warn if found
non_primitive_cols = [
field.name
for field in self.schema()
if field.dtype.is_python()
or field.dtype.is_list()
or field.dtype.is_struct()
or field.dtype.is_map()
or field.dtype.is_tensor()
or field.dtype.is_image()
or field.dtype.is_embedding()
]
if non_primitive_cols:
warnings.warn(
f"Detected non-primitive columns: {non_primitive_cols}. Writing as text (default). Set `non_primitive_handling` to control or suppress.",
UserWarning,
stacklevel=2,
)
return self.write_sink(sink)
@DataframePublicAPI
def write_parquet(
self,
root_dir: str | pathlib.Path,
compression: str = "snappy",
write_mode: Literal["append", "overwrite", "overwrite-partitions"] = "append",
partition_cols: list[ColumnInputType] | None = None,
io_config: IOConfig | None = None,
) -> "DataFrame":
"""Writes the DataFrame as parquet files, returning a new DataFrame with paths to the files that were written.
Files will be written to `<root_dir>/*` with randomly generated UUIDs as the file names.
Args:
root_dir (str): root file path to write parquet files to.
compression (str, optional): compression algorithm. Defaults to "snappy".
write_mode (str, optional): Operation mode of the write. `append` will add new data, `overwrite` will replace the contents of the root directory with new data. `overwrite-partitions` will replace only the contents in the partitions that are being written to. Defaults to "append".
partition_cols (Optional[List[ColumnInputType]], optional): How to subpartition each partition further. Defaults to None.
io_config (Optional[IOConfig], optional): configurations to use when interacting with remote storage.
Returns:
DataFrame: The filenames that were written out as strings.
Note:
This call is **blocking** and will execute the DataFrame when called
Examples:
>>> import daft
>>> df = daft.from_pydict({"x": [1, 2, 3], "y": ["a", "b", "c"]})
>>> df.write_parquet("output_dir", write_mode="overwrite") # doctest: +SKIP
Tip:
See also [`df.write_csv()`][daft.DataFrame.write_csv] and [`df.write_json()`][daft.DataFrame.write_json]
Other formats for writing DataFrames
"""
if write_mode not in ["append", "overwrite", "overwrite-partitions"]:
raise ValueError(
f"Only support `append`, `overwrite`, or `overwrite-partitions` mode. {write_mode} is unsupported"
)
if write_mode == "overwrite-partitions" and partition_cols is None:
raise ValueError("Partition columns must be specified to use `overwrite-partitions` mode.")
io_config = get_context().daft_planning_config.default_io_config if io_config is None else io_config
cols: list[Expression] | None = None
if partition_cols is not None:
cols = column_inputs_to_expressions(tuple(partition_cols))
builder = self._builder.write_tabular(
root_dir=root_dir,
partition_cols=cols,
write_mode=WriteMode.from_str(write_mode),
file_format=FileFormat.Parquet,
compression=compression,
io_config=io_config,
)
# Block and write, then retrieve data
write_df = DataFrame(builder)
write_df.collect()
assert write_df._result is not None
# Populate and return a new disconnected DataFrame
result_df = DataFrame(write_df._builder)
result_df._result_cache = write_df._result_cache
result_df._preview = write_df._preview
result_df._metadata = write_df._metadata
return result_df
@DataframePublicAPI
def write_csv(
self,
root_dir: str | pathlib.Path,
write_mode: Literal["append", "overwrite", "overwrite-partitions"] = "append",
partition_cols: list[ColumnInputType] | None = None,
io_config: IOConfig | None = None,
delimiter: str | None = None,
quote: str | None = None,
escape: str | None = None,
header: bool | None = True,
date_format: str | None = None,
timestamp_format: str | None = None,
) -> "DataFrame":
r"""Writes the DataFrame as CSV files, returning a new DataFrame with paths to the files that were written.
Files will be written to `<root_dir>/*` with randomly generated UUIDs as the file names.
Args:
root_dir (str): root file path to write CSV files to.