cudf-polars-cu12 25.2.2__py3-none-any.whl → 25.6.0__py3-none-any.whl
This diff represents the content of publicly available package versions that have been released to one of the supported registries. The information contained in this diff is provided for informational purposes only and reflects changes between package versions as they appear in their respective public registries.
- cudf_polars/VERSION +1 -1
- cudf_polars/callback.py +82 -65
- cudf_polars/containers/column.py +138 -7
- cudf_polars/containers/dataframe.py +26 -39
- cudf_polars/dsl/expr.py +3 -1
- cudf_polars/dsl/expressions/aggregation.py +27 -63
- cudf_polars/dsl/expressions/base.py +40 -72
- cudf_polars/dsl/expressions/binaryop.py +5 -41
- cudf_polars/dsl/expressions/boolean.py +25 -53
- cudf_polars/dsl/expressions/datetime.py +97 -17
- cudf_polars/dsl/expressions/literal.py +27 -33
- cudf_polars/dsl/expressions/rolling.py +110 -9
- cudf_polars/dsl/expressions/selection.py +8 -26
- cudf_polars/dsl/expressions/slicing.py +47 -0
- cudf_polars/dsl/expressions/sorting.py +5 -18
- cudf_polars/dsl/expressions/string.py +33 -36
- cudf_polars/dsl/expressions/ternary.py +3 -10
- cudf_polars/dsl/expressions/unary.py +35 -75
- cudf_polars/dsl/ir.py +749 -212
- cudf_polars/dsl/nodebase.py +8 -1
- cudf_polars/dsl/to_ast.py +5 -3
- cudf_polars/dsl/translate.py +319 -171
- cudf_polars/dsl/utils/__init__.py +8 -0
- cudf_polars/dsl/utils/aggregations.py +292 -0
- cudf_polars/dsl/utils/groupby.py +97 -0
- cudf_polars/dsl/utils/naming.py +34 -0
- cudf_polars/dsl/utils/replace.py +46 -0
- cudf_polars/dsl/utils/rolling.py +113 -0
- cudf_polars/dsl/utils/windows.py +186 -0
- cudf_polars/experimental/base.py +17 -19
- cudf_polars/experimental/benchmarks/__init__.py +4 -0
- cudf_polars/experimental/benchmarks/pdsh.py +1279 -0
- cudf_polars/experimental/dask_registers.py +196 -0
- cudf_polars/experimental/distinct.py +174 -0
- cudf_polars/experimental/explain.py +127 -0
- cudf_polars/experimental/expressions.py +521 -0
- cudf_polars/experimental/groupby.py +288 -0
- cudf_polars/experimental/io.py +58 -29
- cudf_polars/experimental/join.py +353 -0
- cudf_polars/experimental/parallel.py +166 -93
- cudf_polars/experimental/repartition.py +69 -0
- cudf_polars/experimental/scheduler.py +155 -0
- cudf_polars/experimental/select.py +92 -7
- cudf_polars/experimental/shuffle.py +294 -0
- cudf_polars/experimental/sort.py +45 -0
- cudf_polars/experimental/spilling.py +151 -0
- cudf_polars/experimental/utils.py +100 -0
- cudf_polars/testing/asserts.py +146 -6
- cudf_polars/testing/io.py +72 -0
- cudf_polars/testing/plugin.py +78 -76
- cudf_polars/typing/__init__.py +59 -6
- cudf_polars/utils/config.py +353 -0
- cudf_polars/utils/conversion.py +40 -0
- cudf_polars/utils/dtypes.py +22 -5
- cudf_polars/utils/timer.py +39 -0
- cudf_polars/utils/versions.py +5 -4
- {cudf_polars_cu12-25.2.2.dist-info → cudf_polars_cu12-25.6.0.dist-info}/METADATA +10 -7
- cudf_polars_cu12-25.6.0.dist-info/RECORD +73 -0
- {cudf_polars_cu12-25.2.2.dist-info → cudf_polars_cu12-25.6.0.dist-info}/WHEEL +1 -1
- cudf_polars/experimental/dask_serialize.py +0 -59
- cudf_polars_cu12-25.2.2.dist-info/RECORD +0 -48
- {cudf_polars_cu12-25.2.2.dist-info → cudf_polars_cu12-25.6.0.dist-info/licenses}/LICENSE +0 -0
- {cudf_polars_cu12-25.2.2.dist-info → cudf_polars_cu12-25.6.0.dist-info}/top_level.txt +0 -0
|
@@ -0,0 +1,288 @@
|
|
|
1
|
+
# SPDX-FileCopyrightText: Copyright (c) 2025 NVIDIA CORPORATION & AFFILIATES.
|
|
2
|
+
# SPDX-License-Identifier: Apache-2.0
|
|
3
|
+
"""Parallel GroupBy Logic."""
|
|
4
|
+
|
|
5
|
+
from __future__ import annotations
|
|
6
|
+
|
|
7
|
+
import itertools
|
|
8
|
+
import math
|
|
9
|
+
from typing import TYPE_CHECKING
|
|
10
|
+
|
|
11
|
+
import pylibcudf as plc
|
|
12
|
+
|
|
13
|
+
from cudf_polars.dsl.expr import Agg, BinOp, Col, Len, NamedExpr
|
|
14
|
+
from cudf_polars.dsl.ir import GroupBy, Select
|
|
15
|
+
from cudf_polars.dsl.traversal import traversal
|
|
16
|
+
from cudf_polars.dsl.utils.naming import unique_names
|
|
17
|
+
from cudf_polars.experimental.base import PartitionInfo
|
|
18
|
+
from cudf_polars.experimental.dispatch import lower_ir_node
|
|
19
|
+
from cudf_polars.experimental.repartition import Repartition
|
|
20
|
+
from cudf_polars.experimental.shuffle import Shuffle
|
|
21
|
+
from cudf_polars.experimental.utils import _lower_ir_fallback
|
|
22
|
+
|
|
23
|
+
if TYPE_CHECKING:
|
|
24
|
+
from collections.abc import Generator, MutableMapping
|
|
25
|
+
|
|
26
|
+
from cudf_polars.dsl.expr import Expr
|
|
27
|
+
from cudf_polars.dsl.ir import IR
|
|
28
|
+
from cudf_polars.experimental.parallel import LowerIRTransformer
|
|
29
|
+
|
|
30
|
+
|
|
31
|
+
# Supported multi-partition aggregations
|
|
32
|
+
_GB_AGG_SUPPORTED = ("sum", "count", "mean", "min", "max", "n_unique")
|
|
33
|
+
|
|
34
|
+
|
|
35
|
+
def combine(
|
|
36
|
+
*decompositions: tuple[NamedExpr, list[NamedExpr], list[NamedExpr]],
|
|
37
|
+
) -> tuple[list[NamedExpr], list[NamedExpr], list[NamedExpr]]:
|
|
38
|
+
"""
|
|
39
|
+
Combine multiple groupby-aggregation decompositions.
|
|
40
|
+
|
|
41
|
+
Parameters
|
|
42
|
+
----------
|
|
43
|
+
decompositions
|
|
44
|
+
Packed sequence of `decompose` results.
|
|
45
|
+
|
|
46
|
+
Returns
|
|
47
|
+
-------
|
|
48
|
+
Unified groupby-aggregation decomposition.
|
|
49
|
+
"""
|
|
50
|
+
if len(decompositions) == 0:
|
|
51
|
+
return [], [], []
|
|
52
|
+
selections, aggregations, reductions = zip(*decompositions, strict=True)
|
|
53
|
+
assert all(isinstance(ne, NamedExpr) for ne in selections)
|
|
54
|
+
return (
|
|
55
|
+
list(selections),
|
|
56
|
+
list(itertools.chain.from_iterable(aggregations)),
|
|
57
|
+
list(itertools.chain.from_iterable(reductions)),
|
|
58
|
+
)
|
|
59
|
+
|
|
60
|
+
|
|
61
|
+
def decompose(
|
|
62
|
+
name: str, expr: Expr, *, names: Generator[str, None, None]
|
|
63
|
+
) -> tuple[NamedExpr, list[NamedExpr], list[NamedExpr]]:
|
|
64
|
+
"""
|
|
65
|
+
Decompose a groupby-aggregation expression.
|
|
66
|
+
|
|
67
|
+
Parameters
|
|
68
|
+
----------
|
|
69
|
+
name
|
|
70
|
+
Output schema name.
|
|
71
|
+
expr
|
|
72
|
+
The aggregation expression for a single column.
|
|
73
|
+
names
|
|
74
|
+
Generator of unique names for temporaries.
|
|
75
|
+
|
|
76
|
+
Returns
|
|
77
|
+
-------
|
|
78
|
+
NamedExpr
|
|
79
|
+
The expression selecting the *output* column or columns.
|
|
80
|
+
list[NamedExpr]
|
|
81
|
+
The initial aggregation expressions.
|
|
82
|
+
list[NamedExpr]
|
|
83
|
+
The reduction expressions.
|
|
84
|
+
"""
|
|
85
|
+
dtype = expr.dtype
|
|
86
|
+
|
|
87
|
+
if isinstance(expr, Len):
|
|
88
|
+
selection = NamedExpr(name, Col(dtype, name))
|
|
89
|
+
aggregation = [NamedExpr(name, expr)]
|
|
90
|
+
reduction = [NamedExpr(name, Agg(dtype, "sum", None, Col(dtype, name)))]
|
|
91
|
+
return selection, aggregation, reduction
|
|
92
|
+
if isinstance(expr, Agg):
|
|
93
|
+
if expr.name in ("sum", "count", "min", "max", "n_unique"):
|
|
94
|
+
if expr.name in ("sum", "count", "n_unique"):
|
|
95
|
+
aggfunc = "sum"
|
|
96
|
+
else:
|
|
97
|
+
aggfunc = expr.name
|
|
98
|
+
selection = NamedExpr(name, Col(dtype, name))
|
|
99
|
+
aggregation = [NamedExpr(name, expr)]
|
|
100
|
+
reduction = [NamedExpr(name, Agg(dtype, aggfunc, None, Col(dtype, name)))]
|
|
101
|
+
return selection, aggregation, reduction
|
|
102
|
+
elif expr.name == "mean":
|
|
103
|
+
(child,) = expr.children
|
|
104
|
+
(sum, count), aggregations, reductions = combine(
|
|
105
|
+
decompose(
|
|
106
|
+
f"{next(names)}__mean_sum",
|
|
107
|
+
Agg(dtype, "sum", None, child),
|
|
108
|
+
names=names,
|
|
109
|
+
),
|
|
110
|
+
decompose(f"{next(names)}__mean_count", Len(dtype), names=names),
|
|
111
|
+
)
|
|
112
|
+
selection = NamedExpr(
|
|
113
|
+
name,
|
|
114
|
+
BinOp(dtype, plc.binaryop.BinaryOperator.DIV, sum.value, count.value),
|
|
115
|
+
)
|
|
116
|
+
return selection, aggregations, reductions
|
|
117
|
+
else:
|
|
118
|
+
raise NotImplementedError(
|
|
119
|
+
"group_by does not support multiple partitions "
|
|
120
|
+
f"for this aggregation type:\n{type(expr)}\n"
|
|
121
|
+
f"Only {_GB_AGG_SUPPORTED} are supported."
|
|
122
|
+
)
|
|
123
|
+
else: # pragma: no cover
|
|
124
|
+
# Unsupported expression
|
|
125
|
+
raise NotImplementedError(
|
|
126
|
+
f"GroupBy does not support multiple partitions for this expression:\n{expr}"
|
|
127
|
+
)
|
|
128
|
+
|
|
129
|
+
|
|
130
|
+
@lower_ir_node.register(GroupBy)
|
|
131
|
+
def _(
|
|
132
|
+
ir: GroupBy, rec: LowerIRTransformer
|
|
133
|
+
) -> tuple[IR, MutableMapping[IR, PartitionInfo]]:
|
|
134
|
+
# Extract child partitioning
|
|
135
|
+
child, partition_info = rec(ir.children[0])
|
|
136
|
+
|
|
137
|
+
# Handle single-partition case
|
|
138
|
+
if partition_info[child].count == 1:
|
|
139
|
+
single_part_node = ir.reconstruct([child])
|
|
140
|
+
partition_info[single_part_node] = partition_info[child]
|
|
141
|
+
return single_part_node, partition_info
|
|
142
|
+
|
|
143
|
+
# Check group-by keys
|
|
144
|
+
if not all(
|
|
145
|
+
expr.is_pointwise for expr in traversal([e.value for e in ir.keys])
|
|
146
|
+
): # pragma: no cover
|
|
147
|
+
return _lower_ir_fallback(
|
|
148
|
+
ir,
|
|
149
|
+
rec,
|
|
150
|
+
msg="group_by does not support multiple partitions for non-pointwise keys.",
|
|
151
|
+
)
|
|
152
|
+
|
|
153
|
+
# Check if we are dealing with any high-cardinality columns
|
|
154
|
+
post_aggregation_count = 1 # Default tree reduction
|
|
155
|
+
groupby_key_columns = [ne.name for ne in ir.keys]
|
|
156
|
+
shuffled = partition_info[child].partitioned_on == ir.keys
|
|
157
|
+
|
|
158
|
+
assert ir.config_options.executor.name == "streaming", (
|
|
159
|
+
"'in-memory' executor not supported in 'generate_ir_tasks'"
|
|
160
|
+
)
|
|
161
|
+
|
|
162
|
+
cardinality_factor = {
|
|
163
|
+
c: min(f, 1.0)
|
|
164
|
+
for c, f in ir.config_options.executor.cardinality_factor.items()
|
|
165
|
+
if c in groupby_key_columns
|
|
166
|
+
}
|
|
167
|
+
if cardinality_factor:
|
|
168
|
+
# The `cardinality_factor` dictionary can be used
|
|
169
|
+
# to specify a mapping between column names and
|
|
170
|
+
# cardinality "factors". Each factor estimates the
|
|
171
|
+
# fractional number of unique values in the column.
|
|
172
|
+
# Each value should be in the range (0, 1].
|
|
173
|
+
child_count = partition_info[child].count
|
|
174
|
+
post_aggregation_count = max(
|
|
175
|
+
int(max(cardinality_factor.values()) * child_count),
|
|
176
|
+
1,
|
|
177
|
+
)
|
|
178
|
+
|
|
179
|
+
new_node: IR
|
|
180
|
+
name_generator = unique_names(ir.schema.keys())
|
|
181
|
+
# Decompose the aggregation requests into three distinct phases
|
|
182
|
+
try:
|
|
183
|
+
selection_exprs, piecewise_exprs, reduction_exprs = combine(
|
|
184
|
+
*(
|
|
185
|
+
decompose(agg.name, agg.value, names=name_generator)
|
|
186
|
+
for agg in ir.agg_requests
|
|
187
|
+
)
|
|
188
|
+
)
|
|
189
|
+
except NotImplementedError:
|
|
190
|
+
if shuffled: # pragma: no cover
|
|
191
|
+
# Don't fallback if we are already shuffled.
|
|
192
|
+
# We can just preserve the child's partitioning
|
|
193
|
+
new_node = ir.reconstruct([child])
|
|
194
|
+
partition_info[new_node] = partition_info[child]
|
|
195
|
+
return new_node, partition_info
|
|
196
|
+
return _lower_ir_fallback(
|
|
197
|
+
ir, rec, msg="Failed to decompose groupby aggs for multiple partitions."
|
|
198
|
+
)
|
|
199
|
+
|
|
200
|
+
# Partition-wise groupby operation
|
|
201
|
+
pwise_schema = {k.name: k.value.dtype for k in ir.keys} | {
|
|
202
|
+
k.name: k.value.dtype for k in piecewise_exprs
|
|
203
|
+
}
|
|
204
|
+
gb_pwise = GroupBy(
|
|
205
|
+
pwise_schema,
|
|
206
|
+
ir.keys,
|
|
207
|
+
piecewise_exprs,
|
|
208
|
+
ir.maintain_order,
|
|
209
|
+
None,
|
|
210
|
+
ir.config_options,
|
|
211
|
+
child,
|
|
212
|
+
)
|
|
213
|
+
child_count = partition_info[child].count
|
|
214
|
+
partition_info[gb_pwise] = PartitionInfo(count=child_count)
|
|
215
|
+
|
|
216
|
+
# Reduction
|
|
217
|
+
gb_inter: GroupBy | Repartition | Shuffle
|
|
218
|
+
reduction_schema = {k.name: k.value.dtype for k in ir.keys} | {
|
|
219
|
+
k.name: k.value.dtype for k in reduction_exprs
|
|
220
|
+
}
|
|
221
|
+
if not shuffled and post_aggregation_count > 1:
|
|
222
|
+
# Shuffle reduction
|
|
223
|
+
if ir.maintain_order: # pragma: no cover
|
|
224
|
+
return _lower_ir_fallback(
|
|
225
|
+
ir,
|
|
226
|
+
rec,
|
|
227
|
+
msg="maintain_order not supported for multiple output partitions.",
|
|
228
|
+
)
|
|
229
|
+
|
|
230
|
+
gb_inter = Shuffle(
|
|
231
|
+
gb_pwise.schema,
|
|
232
|
+
ir.keys,
|
|
233
|
+
ir.config_options,
|
|
234
|
+
gb_pwise,
|
|
235
|
+
)
|
|
236
|
+
partition_info[gb_inter] = PartitionInfo(count=post_aggregation_count)
|
|
237
|
+
else:
|
|
238
|
+
# N-ary tree reduction
|
|
239
|
+
assert ir.config_options.executor.name == "streaming", (
|
|
240
|
+
"'in-memory' executor not supported in 'generate_ir_tasks'"
|
|
241
|
+
)
|
|
242
|
+
|
|
243
|
+
n_ary = ir.config_options.executor.groupby_n_ary
|
|
244
|
+
count = child_count
|
|
245
|
+
gb_inter = gb_pwise
|
|
246
|
+
while count > post_aggregation_count:
|
|
247
|
+
gb_inter = Repartition(gb_inter.schema, gb_inter)
|
|
248
|
+
count = max(math.ceil(count / n_ary), post_aggregation_count)
|
|
249
|
+
partition_info[gb_inter] = PartitionInfo(count=count)
|
|
250
|
+
if count > post_aggregation_count:
|
|
251
|
+
gb_inter = GroupBy(
|
|
252
|
+
reduction_schema,
|
|
253
|
+
ir.keys,
|
|
254
|
+
reduction_exprs,
|
|
255
|
+
ir.maintain_order,
|
|
256
|
+
None,
|
|
257
|
+
ir.config_options,
|
|
258
|
+
gb_inter,
|
|
259
|
+
)
|
|
260
|
+
partition_info[gb_inter] = PartitionInfo(count=count)
|
|
261
|
+
|
|
262
|
+
# Final aggregation
|
|
263
|
+
gb_reduce = GroupBy(
|
|
264
|
+
reduction_schema,
|
|
265
|
+
ir.keys,
|
|
266
|
+
reduction_exprs,
|
|
267
|
+
ir.maintain_order,
|
|
268
|
+
ir.zlice,
|
|
269
|
+
ir.config_options,
|
|
270
|
+
gb_inter,
|
|
271
|
+
)
|
|
272
|
+
partition_info[gb_reduce] = PartitionInfo(count=post_aggregation_count)
|
|
273
|
+
|
|
274
|
+
# Final Select phase
|
|
275
|
+
new_node = Select(
|
|
276
|
+
ir.schema,
|
|
277
|
+
[
|
|
278
|
+
*(NamedExpr(k.name, Col(k.value.dtype, k.name)) for k in ir.keys),
|
|
279
|
+
*selection_exprs,
|
|
280
|
+
],
|
|
281
|
+
False, # noqa: FBT003
|
|
282
|
+
gb_reduce,
|
|
283
|
+
)
|
|
284
|
+
partition_info[new_node] = PartitionInfo(
|
|
285
|
+
count=post_aggregation_count,
|
|
286
|
+
partitioned_on=ir.keys,
|
|
287
|
+
)
|
|
288
|
+
return new_node, partition_info
|
cudf_polars/experimental/io.py
CHANGED
|
@@ -4,11 +4,12 @@
|
|
|
4
4
|
|
|
5
5
|
from __future__ import annotations
|
|
6
6
|
|
|
7
|
+
import dataclasses
|
|
7
8
|
import enum
|
|
8
9
|
import math
|
|
9
10
|
import random
|
|
10
11
|
from enum import IntEnum
|
|
11
|
-
from typing import TYPE_CHECKING, Any
|
|
12
|
+
from typing import TYPE_CHECKING, Any, TypeVar
|
|
12
13
|
|
|
13
14
|
import pylibcudf as plc
|
|
14
15
|
|
|
@@ -19,19 +20,28 @@ from cudf_polars.experimental.dispatch import lower_ir_node
|
|
|
19
20
|
if TYPE_CHECKING:
|
|
20
21
|
from collections.abc import MutableMapping
|
|
21
22
|
|
|
23
|
+
import numpy as np
|
|
24
|
+
import numpy.typing as npt
|
|
25
|
+
|
|
26
|
+
from cudf_polars.containers import DataFrame
|
|
22
27
|
from cudf_polars.dsl.expr import NamedExpr
|
|
23
28
|
from cudf_polars.experimental.dispatch import LowerIRTransformer
|
|
24
29
|
from cudf_polars.typing import Schema
|
|
30
|
+
from cudf_polars.utils.config import ConfigOptions
|
|
31
|
+
|
|
32
|
+
T = TypeVar("T", bound=npt.NBitBase)
|
|
25
33
|
|
|
26
34
|
|
|
27
35
|
@lower_ir_node.register(DataFrameScan)
|
|
28
36
|
def _(
|
|
29
37
|
ir: DataFrameScan, rec: LowerIRTransformer
|
|
30
38
|
) -> tuple[IR, MutableMapping[IR, PartitionInfo]]:
|
|
31
|
-
|
|
32
|
-
"
|
|
39
|
+
assert ir.config_options.executor.name == "streaming", (
|
|
40
|
+
"'in-memory' executor not supported in 'generate_ir_tasks'"
|
|
33
41
|
)
|
|
34
42
|
|
|
43
|
+
rows_per_partition = ir.config_options.executor.max_rows_per_partition
|
|
44
|
+
|
|
35
45
|
nrows = max(ir.df.shape()[0], 1)
|
|
36
46
|
count = math.ceil(nrows / rows_per_partition)
|
|
37
47
|
|
|
@@ -91,10 +101,18 @@ class ScanPartitionPlan:
|
|
|
91
101
|
"""Extract the partitioning plan of a Scan operation."""
|
|
92
102
|
if ir.typ == "parquet":
|
|
93
103
|
# TODO: Use system info to set default blocksize
|
|
94
|
-
|
|
95
|
-
|
|
96
|
-
|
|
97
|
-
|
|
104
|
+
assert ir.config_options.executor.name == "streaming", (
|
|
105
|
+
"'in-memory' executor not supported in 'generate_ir_tasks'"
|
|
106
|
+
)
|
|
107
|
+
|
|
108
|
+
blocksize: int = ir.config_options.executor.target_partition_size
|
|
109
|
+
# _sample_pq_statistics is generic over the bit-width of the array
|
|
110
|
+
# We don't care about that here, so we ignore it.
|
|
111
|
+
stats = _sample_pq_statistics(ir) # type: ignore[var-annotated]
|
|
112
|
+
# Some columns (e.g., "include_file_paths") may be present in the schema
|
|
113
|
+
# but not in the Parquet statistics dict. We use stats.get(column, 0)
|
|
114
|
+
# to safely fall back to 0 in those cases.
|
|
115
|
+
file_size = sum(float(stats.get(column, 0)) for column in ir.schema)
|
|
98
116
|
if file_size > 0:
|
|
99
117
|
if file_size > blocksize:
|
|
100
118
|
# Split large files
|
|
@@ -168,14 +186,15 @@ class SplitScan(IR):
|
|
|
168
186
|
schema: Schema,
|
|
169
187
|
typ: str,
|
|
170
188
|
reader_options: dict[str, Any],
|
|
171
|
-
config_options:
|
|
189
|
+
config_options: ConfigOptions,
|
|
172
190
|
paths: list[str],
|
|
173
191
|
with_columns: list[str] | None,
|
|
174
192
|
skip_rows: int,
|
|
175
193
|
n_rows: int,
|
|
176
194
|
row_index: tuple[str, int] | None,
|
|
195
|
+
include_file_paths: str | None,
|
|
177
196
|
predicate: NamedExpr | None,
|
|
178
|
-
):
|
|
197
|
+
) -> DataFrame:
|
|
179
198
|
"""Evaluate and return a dataframe."""
|
|
180
199
|
if typ not in ("parquet",): # pragma: no cover
|
|
181
200
|
raise NotImplementedError(f"Unhandled Scan type for file splitting: {typ}")
|
|
@@ -233,30 +252,37 @@ class SplitScan(IR):
|
|
|
233
252
|
skip_rows,
|
|
234
253
|
n_rows,
|
|
235
254
|
row_index,
|
|
255
|
+
include_file_paths,
|
|
236
256
|
predicate,
|
|
237
257
|
)
|
|
238
258
|
|
|
239
259
|
|
|
240
|
-
def _sample_pq_statistics(ir: Scan) -> dict[str,
|
|
260
|
+
def _sample_pq_statistics(ir: Scan) -> dict[str, np.floating[T]]:
|
|
261
|
+
import itertools
|
|
262
|
+
|
|
241
263
|
import numpy as np
|
|
242
|
-
import pyarrow.dataset as pa_ds
|
|
243
264
|
|
|
244
265
|
# Use average total_uncompressed_size of three files
|
|
245
|
-
|
|
246
|
-
|
|
266
|
+
n_sample = min(3, len(ir.paths))
|
|
267
|
+
metadata = plc.io.parquet_metadata.read_parquet_metadata(
|
|
268
|
+
plc.io.SourceInfo(random.sample(ir.paths, n_sample))
|
|
269
|
+
)
|
|
247
270
|
column_sizes = {}
|
|
248
|
-
|
|
249
|
-
|
|
250
|
-
|
|
251
|
-
for rg in range(md.num_row_groups):
|
|
252
|
-
row_group = md.row_group(rg)
|
|
253
|
-
for col in range(row_group.num_columns):
|
|
254
|
-
column = row_group.column(col)
|
|
255
|
-
name = column.path_in_schema
|
|
256
|
-
if name not in column_sizes:
|
|
257
|
-
column_sizes[name] = np.zeros(n_sample, dtype="int64")
|
|
258
|
-
column_sizes[name][i] += column.total_uncompressed_size
|
|
271
|
+
rowgroup_offsets_per_file = np.insert(
|
|
272
|
+
np.cumsum(metadata.num_rowgroups_per_file()), 0, 0
|
|
273
|
+
)
|
|
259
274
|
|
|
275
|
+
# For each column, calculate the `total_uncompressed_size` for each file
|
|
276
|
+
for name, uncompressed_sizes in metadata.columnchunk_metadata().items():
|
|
277
|
+
column_sizes[name] = np.array(
|
|
278
|
+
[
|
|
279
|
+
np.sum(uncompressed_sizes[start:end])
|
|
280
|
+
for (start, end) in itertools.pairwise(rowgroup_offsets_per_file)
|
|
281
|
+
],
|
|
282
|
+
dtype="int64",
|
|
283
|
+
)
|
|
284
|
+
|
|
285
|
+
# Return the mean per-file `total_uncompressed_size` for each column
|
|
260
286
|
return {name: np.mean(sizes) for name, sizes in column_sizes.items()}
|
|
261
287
|
|
|
262
288
|
|
|
@@ -270,11 +296,12 @@ def _(
|
|
|
270
296
|
paths = list(ir.paths)
|
|
271
297
|
if plan.flavor == ScanPartitionFlavor.SPLIT_FILES:
|
|
272
298
|
# Disable chunked reader when splitting files
|
|
273
|
-
config_options =
|
|
274
|
-
|
|
275
|
-
|
|
276
|
-
|
|
277
|
-
|
|
299
|
+
config_options = dataclasses.replace(
|
|
300
|
+
ir.config_options,
|
|
301
|
+
parquet_options=dataclasses.replace(
|
|
302
|
+
ir.config_options.parquet_options, chunked=False
|
|
303
|
+
),
|
|
304
|
+
)
|
|
278
305
|
|
|
279
306
|
slices: list[SplitScan] = []
|
|
280
307
|
for path in paths:
|
|
@@ -289,6 +316,7 @@ def _(
|
|
|
289
316
|
ir.skip_rows,
|
|
290
317
|
ir.n_rows,
|
|
291
318
|
ir.row_index,
|
|
319
|
+
ir.include_file_paths,
|
|
292
320
|
ir.predicate,
|
|
293
321
|
)
|
|
294
322
|
slices.extend(
|
|
@@ -312,6 +340,7 @@ def _(
|
|
|
312
340
|
ir.skip_rows,
|
|
313
341
|
ir.n_rows,
|
|
314
342
|
ir.row_index,
|
|
343
|
+
ir.include_file_paths,
|
|
315
344
|
ir.predicate,
|
|
316
345
|
)
|
|
317
346
|
for i in range(0, len(paths), plan.factor)
|