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.
Files changed (63) hide show
  1. cudf_polars/VERSION +1 -1
  2. cudf_polars/callback.py +82 -65
  3. cudf_polars/containers/column.py +138 -7
  4. cudf_polars/containers/dataframe.py +26 -39
  5. cudf_polars/dsl/expr.py +3 -1
  6. cudf_polars/dsl/expressions/aggregation.py +27 -63
  7. cudf_polars/dsl/expressions/base.py +40 -72
  8. cudf_polars/dsl/expressions/binaryop.py +5 -41
  9. cudf_polars/dsl/expressions/boolean.py +25 -53
  10. cudf_polars/dsl/expressions/datetime.py +97 -17
  11. cudf_polars/dsl/expressions/literal.py +27 -33
  12. cudf_polars/dsl/expressions/rolling.py +110 -9
  13. cudf_polars/dsl/expressions/selection.py +8 -26
  14. cudf_polars/dsl/expressions/slicing.py +47 -0
  15. cudf_polars/dsl/expressions/sorting.py +5 -18
  16. cudf_polars/dsl/expressions/string.py +33 -36
  17. cudf_polars/dsl/expressions/ternary.py +3 -10
  18. cudf_polars/dsl/expressions/unary.py +35 -75
  19. cudf_polars/dsl/ir.py +749 -212
  20. cudf_polars/dsl/nodebase.py +8 -1
  21. cudf_polars/dsl/to_ast.py +5 -3
  22. cudf_polars/dsl/translate.py +319 -171
  23. cudf_polars/dsl/utils/__init__.py +8 -0
  24. cudf_polars/dsl/utils/aggregations.py +292 -0
  25. cudf_polars/dsl/utils/groupby.py +97 -0
  26. cudf_polars/dsl/utils/naming.py +34 -0
  27. cudf_polars/dsl/utils/replace.py +46 -0
  28. cudf_polars/dsl/utils/rolling.py +113 -0
  29. cudf_polars/dsl/utils/windows.py +186 -0
  30. cudf_polars/experimental/base.py +17 -19
  31. cudf_polars/experimental/benchmarks/__init__.py +4 -0
  32. cudf_polars/experimental/benchmarks/pdsh.py +1279 -0
  33. cudf_polars/experimental/dask_registers.py +196 -0
  34. cudf_polars/experimental/distinct.py +174 -0
  35. cudf_polars/experimental/explain.py +127 -0
  36. cudf_polars/experimental/expressions.py +521 -0
  37. cudf_polars/experimental/groupby.py +288 -0
  38. cudf_polars/experimental/io.py +58 -29
  39. cudf_polars/experimental/join.py +353 -0
  40. cudf_polars/experimental/parallel.py +166 -93
  41. cudf_polars/experimental/repartition.py +69 -0
  42. cudf_polars/experimental/scheduler.py +155 -0
  43. cudf_polars/experimental/select.py +92 -7
  44. cudf_polars/experimental/shuffle.py +294 -0
  45. cudf_polars/experimental/sort.py +45 -0
  46. cudf_polars/experimental/spilling.py +151 -0
  47. cudf_polars/experimental/utils.py +100 -0
  48. cudf_polars/testing/asserts.py +146 -6
  49. cudf_polars/testing/io.py +72 -0
  50. cudf_polars/testing/plugin.py +78 -76
  51. cudf_polars/typing/__init__.py +59 -6
  52. cudf_polars/utils/config.py +353 -0
  53. cudf_polars/utils/conversion.py +40 -0
  54. cudf_polars/utils/dtypes.py +22 -5
  55. cudf_polars/utils/timer.py +39 -0
  56. cudf_polars/utils/versions.py +5 -4
  57. {cudf_polars_cu12-25.2.2.dist-info → cudf_polars_cu12-25.6.0.dist-info}/METADATA +10 -7
  58. cudf_polars_cu12-25.6.0.dist-info/RECORD +73 -0
  59. {cudf_polars_cu12-25.2.2.dist-info → cudf_polars_cu12-25.6.0.dist-info}/WHEEL +1 -1
  60. cudf_polars/experimental/dask_serialize.py +0 -59
  61. cudf_polars_cu12-25.2.2.dist-info/RECORD +0 -48
  62. {cudf_polars_cu12-25.2.2.dist-info → cudf_polars_cu12-25.6.0.dist-info/licenses}/LICENSE +0 -0
  63. {cudf_polars_cu12-25.2.2.dist-info → cudf_polars_cu12-25.6.0.dist-info}/top_level.txt +0 -0
@@ -0,0 +1,196 @@
1
+ # SPDX-FileCopyrightText: Copyright (c) 2024-2025, NVIDIA CORPORATION & AFFILIATES.
2
+ # SPDX-License-Identifier: Apache-2.0
3
+
4
+ """Dask function registrations such as serializers and dispatch implementations."""
5
+
6
+ from __future__ import annotations
7
+
8
+ from typing import TYPE_CHECKING, Any, ClassVar, overload
9
+
10
+ from dask.sizeof import sizeof as sizeof_dispatch
11
+ from distributed.protocol import dask_deserialize, dask_serialize
12
+ from distributed.protocol.cuda import cuda_deserialize, cuda_serialize
13
+ from distributed.utils import log_errors
14
+
15
+ import pylibcudf as plc
16
+ import rmm
17
+
18
+ from cudf_polars.containers import Column, DataFrame
19
+
20
+ if TYPE_CHECKING:
21
+ from collections.abc import Mapping
22
+
23
+ from distributed import Client
24
+
25
+ from rmm.pylibrmm.memory_resource import DeviceMemoryResource
26
+ from rmm.pylibrmm.stream import Stream
27
+
28
+ from cudf_polars.typing import ColumnHeader, ColumnOptions, DataFrameHeader
29
+
30
+
31
+ __all__ = ["DaskRegisterManager", "register"]
32
+
33
+
34
+ class DaskRegisterManager: # pragma: no cover; Only used with Distributed scheduler
35
+ """Manager to ensure ensure serializer is only registered once."""
36
+
37
+ _registered: bool = False
38
+ _client_run_executed: ClassVar[set[str]] = set()
39
+
40
+ @classmethod
41
+ def register_once(cls) -> None:
42
+ """Register Dask/cudf-polars serializers in calling process."""
43
+ if not cls._registered:
44
+ from cudf_polars.experimental.dask_registers import register
45
+
46
+ register()
47
+ cls._registered = True
48
+
49
+ @classmethod
50
+ def run_on_cluster(cls, client: Client) -> None:
51
+ """Run register on the workers and scheduler once."""
52
+ if client.id not in cls._client_run_executed:
53
+ client.run(cls.register_once)
54
+ client.run_on_scheduler(cls.register_once)
55
+ cls._client_run_executed.add(client.id)
56
+
57
+
58
+ def register() -> None:
59
+ """Register dask serialization and dispatch functions."""
60
+
61
+ @overload
62
+ def serialize_column_or_frame(
63
+ x: DataFrame,
64
+ ) -> tuple[DataFrameHeader, list[memoryview]]: ...
65
+
66
+ @overload
67
+ def serialize_column_or_frame(
68
+ x: Column,
69
+ ) -> tuple[ColumnHeader, list[memoryview]]: ...
70
+
71
+ @cuda_serialize.register((Column, DataFrame))
72
+ def serialize_column_or_frame(
73
+ x: DataFrame | Column,
74
+ ) -> tuple[DataFrameHeader | ColumnHeader, list[memoryview]]:
75
+ with log_errors():
76
+ header, frames = x.serialize()
77
+ return header, list(frames) # Dask expect a list of frames
78
+
79
+ @cuda_deserialize.register(DataFrame)
80
+ def _(
81
+ header: DataFrameHeader, frames: tuple[memoryview, plc.gpumemoryview]
82
+ ) -> DataFrame:
83
+ with log_errors():
84
+ metadata, gpudata = frames # TODO: check if this is a length-2 list...
85
+ return DataFrame.deserialize(header, (metadata, plc.gpumemoryview(gpudata)))
86
+
87
+ @cuda_deserialize.register(Column)
88
+ def _(header: ColumnHeader, frames: tuple[memoryview, plc.gpumemoryview]) -> Column:
89
+ with log_errors():
90
+ metadata, gpudata = frames
91
+ return Column.deserialize(header, (metadata, plc.gpumemoryview(gpudata)))
92
+
93
+ @overload
94
+ def dask_serialize_column_or_frame(
95
+ x: DataFrame,
96
+ ) -> tuple[DataFrameHeader, tuple[memoryview, memoryview]]: ...
97
+
98
+ @overload
99
+ def dask_serialize_column_or_frame(
100
+ x: Column,
101
+ ) -> tuple[ColumnHeader, tuple[memoryview, memoryview]]: ...
102
+
103
+ @dask_serialize.register(Column)
104
+ def dask_serialize_column_or_frame(
105
+ x: DataFrame | Column,
106
+ ) -> tuple[DataFrameHeader | ColumnHeader, tuple[memoryview, memoryview]]:
107
+ with log_errors():
108
+ header, (metadata, gpudata) = x.serialize()
109
+
110
+ # For robustness, we check that the gpu data is contiguous
111
+ cai = gpudata.__cuda_array_interface__
112
+ assert len(cai["shape"]) == 1
113
+ assert cai["strides"] is None or cai["strides"] == (1,)
114
+ assert cai["typestr"] == "|u1"
115
+ nbytes = cai["shape"][0]
116
+
117
+ # Copy the gpudata to host memory
118
+ gpudata_on_host = memoryview(
119
+ rmm.DeviceBuffer(ptr=gpudata.ptr, size=nbytes).copy_to_host()
120
+ )
121
+ return header, (metadata, gpudata_on_host)
122
+
123
+ @dask_deserialize.register(Column)
124
+ def _(header: ColumnHeader, frames: tuple[memoryview, memoryview]) -> Column:
125
+ with log_errors():
126
+ assert len(frames) == 2
127
+ # Copy the second frame (the gpudata in host memory) back to the gpu
128
+ frames = frames[0], plc.gpumemoryview(rmm.DeviceBuffer.to_device(frames[1]))
129
+ return Column.deserialize(header, frames)
130
+
131
+ @dask_serialize.register(DataFrame)
132
+ def _(
133
+ x: DataFrame, context: Mapping[str, Any] | None = None
134
+ ) -> tuple[DataFrameHeader, tuple[memoryview, memoryview]]:
135
+ # Do regular serialization if no staging buffer is provided.
136
+ if context is None or "staging_device_buffer" not in context:
137
+ return dask_serialize_column_or_frame(x)
138
+
139
+ # If a staging buffer is provided, we use `ChunkedPack` to
140
+ # serialize the dataframe using the provided staging buffer.
141
+ with log_errors():
142
+ # Keyword arguments for `Column.__init__`.
143
+ columns_kwargs: list[ColumnOptions] = [
144
+ {
145
+ "is_sorted": col.is_sorted,
146
+ "order": col.order,
147
+ "null_order": col.null_order,
148
+ "name": col.name,
149
+ }
150
+ for col in x.columns
151
+ ]
152
+ header: DataFrameHeader = {
153
+ "columns_kwargs": columns_kwargs,
154
+ "frame_count": 2,
155
+ }
156
+ if "stream" not in context:
157
+ raise ValueError(
158
+ "context: stream must be given when staging_device_buffer is"
159
+ )
160
+ if "device_mr" not in context:
161
+ raise ValueError(
162
+ "context: device_mr must be given when staging_device_buffer is"
163
+ )
164
+ stream: Stream = context["stream"]
165
+ device_mr: DeviceMemoryResource = context["device_mr"]
166
+ buf: rmm.DeviceBuffer = context["staging_device_buffer"]
167
+ frame = plc.contiguous_split.ChunkedPack.create(
168
+ x.table, buf.nbytes, stream, device_mr
169
+ ).pack_to_host(buf)
170
+ return header, frame
171
+
172
+ @dask_deserialize.register(DataFrame)
173
+ def _(header: DataFrameHeader, frames: tuple[memoryview, memoryview]) -> DataFrame:
174
+ with log_errors():
175
+ assert len(frames) == 2
176
+ # Copy the second frame (the gpudata in host memory) back to the gpu
177
+ frames = frames[0], plc.gpumemoryview(rmm.DeviceBuffer.to_device(frames[1]))
178
+ return DataFrame.deserialize(header, frames)
179
+
180
+ @sizeof_dispatch.register(Column)
181
+ def _(x: Column) -> int:
182
+ """The total size of the device buffers used by the DataFrame or Column."""
183
+ return x.obj.device_buffer_size()
184
+
185
+ @sizeof_dispatch.register(DataFrame)
186
+ def _(x: DataFrame) -> int:
187
+ """The total size of the device buffers used by the DataFrame or Column."""
188
+ return sum(c.obj.device_buffer_size() for c in x.columns)
189
+
190
+ # Register rapidsmpf serializer if it's installed.
191
+ try:
192
+ from rapidsmpf.integrations.dask.spilling import register_dask_serialize
193
+
194
+ register_dask_serialize() # pragma: no cover; rapidsmpf dependency not included yet
195
+ except ImportError:
196
+ pass
@@ -0,0 +1,174 @@
1
+ # SPDX-FileCopyrightText: Copyright (c) 2024-2025, NVIDIA CORPORATION & AFFILIATES.
2
+ # SPDX-License-Identifier: Apache-2.0
3
+ """Multi-partition Distinct logic."""
4
+
5
+ from __future__ import annotations
6
+
7
+ import math
8
+ from typing import TYPE_CHECKING
9
+
10
+ import pylibcudf as plc
11
+
12
+ from cudf_polars.dsl.expressions.base import Col, NamedExpr
13
+ from cudf_polars.dsl.ir import Distinct
14
+ from cudf_polars.experimental.base import PartitionInfo
15
+ from cudf_polars.experimental.dispatch import lower_ir_node
16
+ from cudf_polars.experimental.utils import _fallback_inform, _lower_ir_fallback
17
+
18
+ if TYPE_CHECKING:
19
+ from collections.abc import MutableMapping
20
+
21
+ from cudf_polars.dsl.ir import IR
22
+ from cudf_polars.experimental.dispatch import LowerIRTransformer
23
+ from cudf_polars.utils.config import ConfigOptions
24
+
25
+
26
+ def lower_distinct(
27
+ ir: Distinct,
28
+ child: IR,
29
+ partition_info: MutableMapping[IR, PartitionInfo],
30
+ config_options: ConfigOptions,
31
+ *,
32
+ cardinality: float | None = None,
33
+ ) -> tuple[IR, MutableMapping[IR, PartitionInfo]]:
34
+ """
35
+ Lower a Distinct IR into partition-wise stages.
36
+
37
+ Parameters
38
+ ----------
39
+ ir
40
+ The Distinct IR node to lower.
41
+ child
42
+ The reconstructed child of ``ir``. May differ
43
+ from ``ir.children[0]``.
44
+ partition_info
45
+ A mapping from all unique IR nodes to the
46
+ associated partitioning information.
47
+ config_options
48
+ GPUEngine configuration options.
49
+ cardinality
50
+ Cardinality factor to use for algorithm selection.
51
+
52
+ Returns
53
+ -------
54
+ new_node
55
+ The lowered Distinct node.
56
+ partition_info
57
+ A mapping from unique nodes in the new graph to associated
58
+ partitioning information.
59
+ """
60
+ from cudf_polars.experimental.repartition import Repartition
61
+ from cudf_polars.experimental.shuffle import Shuffle
62
+
63
+ # Extract child partitioning
64
+ child_count = partition_info[child].count
65
+
66
+ # Assume shuffle is not stable for now. Therefore, we
67
+ # require a tree reduction if row order matters.
68
+ require_tree_reduction = ir.stable or ir.keep in (
69
+ plc.stream_compaction.DuplicateKeepOption.KEEP_FIRST,
70
+ plc.stream_compaction.DuplicateKeepOption.KEEP_LAST,
71
+ )
72
+
73
+ subset: frozenset = ir.subset or frozenset(ir.schema)
74
+ shuffle_keys = tuple(NamedExpr(name, Col(ir.schema[name], name)) for name in subset)
75
+ shuffled = partition_info[child].partitioned_on == shuffle_keys
76
+ if ir.keep == plc.stream_compaction.DuplicateKeepOption.KEEP_NONE:
77
+ # Need to shuffle the original data for keep == "none"
78
+ if require_tree_reduction:
79
+ # TODO: We cannot drop all duplicates without
80
+ # shuffling the data up front, and we assume
81
+ # shuffling is unstable for now. Note that the
82
+ # task-based shuffle should be stable, but it
83
+ # its performance is very poor.
84
+ raise NotImplementedError(
85
+ "Unsupported unique options for multiple partitions."
86
+ )
87
+ if not shuffled:
88
+ child = Shuffle(child.schema, shuffle_keys, config_options, child)
89
+ partition_info[child] = PartitionInfo(
90
+ count=child_count,
91
+ partitioned_on=shuffle_keys,
92
+ )
93
+ shuffled = True
94
+
95
+ output_count = 1
96
+ n_ary = 32 # Arbitrary default (for now)
97
+ if ir.zlice is not None:
98
+ # Head/tail slice operation has been pushed into Distinct
99
+ if ir.zlice[0] < 1 and ir.zlice[1] is not None:
100
+ # Use rough 1m-row heuristic to set n_ary
101
+ n_ary = max(int(1_000_000 / ir.zlice[1]), 2)
102
+ else: # pragma: no cover
103
+ # TODO: General slicing is not supported for multiple
104
+ # partitions. For now, we raise an error to fall back
105
+ # to one partition.
106
+ raise NotImplementedError("Unsupported slice for multiple partitions.")
107
+ elif cardinality is not None:
108
+ # Use cardinality to determine partitioningcardinality
109
+ n_ary = min(max(int(1.0 / cardinality), 2), child_count)
110
+ output_count = max(int(cardinality * child_count), 1)
111
+
112
+ if output_count > 1 and require_tree_reduction:
113
+ # Need to reduce down to a single partition even
114
+ # if the cardinality is large.
115
+ output_count = 1
116
+ _fallback_inform(
117
+ "Unsupported unique options for multiple partitions.",
118
+ config_options,
119
+ )
120
+
121
+ # Partition-wise unique
122
+ count = child_count
123
+ new_node: IR = ir.reconstruct([child])
124
+ partition_info[new_node] = PartitionInfo(count=count)
125
+
126
+ if shuffled or output_count == 1:
127
+ # Tree reduction
128
+ while count > output_count:
129
+ new_node = Repartition(new_node.schema, new_node)
130
+ count = max(math.ceil(count / n_ary), output_count)
131
+ partition_info[new_node] = PartitionInfo(count=count)
132
+ new_node = ir.reconstruct([new_node])
133
+ partition_info[new_node] = PartitionInfo(count=count)
134
+ else:
135
+ # Shuffle
136
+ new_node = Shuffle(new_node.schema, shuffle_keys, config_options, new_node)
137
+ partition_info[new_node] = PartitionInfo(count=output_count)
138
+ new_node = ir.reconstruct([new_node])
139
+ partition_info[new_node] = PartitionInfo(
140
+ count=output_count,
141
+ partitioned_on=shuffle_keys,
142
+ )
143
+
144
+ return new_node, partition_info
145
+
146
+
147
+ @lower_ir_node.register(Distinct)
148
+ def _(
149
+ ir: Distinct, rec: LowerIRTransformer
150
+ ) -> tuple[IR, MutableMapping[IR, PartitionInfo]]:
151
+ # Extract child partitioning
152
+ child, partition_info = rec(ir.children[0])
153
+ config_options = rec.state["config_options"]
154
+ assert config_options.executor.name == "streaming", (
155
+ "'in-memory' executor not supported in 'lower_ir_node'"
156
+ )
157
+
158
+ subset: frozenset = ir.subset or frozenset(ir.schema)
159
+ cardinality_factor = {
160
+ c: max(min(f, 1.0), 0.00001)
161
+ for c, f in config_options.executor.cardinality_factor.items()
162
+ if c in subset
163
+ }
164
+ cardinality = max(cardinality_factor.values()) if cardinality_factor else None
165
+ try:
166
+ return lower_distinct(
167
+ ir,
168
+ child,
169
+ partition_info,
170
+ config_options,
171
+ cardinality=cardinality,
172
+ )
173
+ except NotImplementedError as err:
174
+ return _lower_ir_fallback(ir, rec, msg=str(err))
@@ -0,0 +1,127 @@
1
+ # SPDX-FileCopyrightText: Copyright (c) 2025 NVIDIA CORPORATION & AFFILIATES.
2
+ # SPDX-License-Identifier: Apache-2.0
3
+ # SPDX-FileCopyrightText: Copyright (c) 2025 NVIDIA CORPORATION & AFFILIATES.
4
+ # SPDX-License-Identifier: Apache-2.0
5
+ """Explain logical and physical plans."""
6
+
7
+ from __future__ import annotations
8
+
9
+ import functools
10
+ from itertools import groupby
11
+ from typing import TYPE_CHECKING
12
+
13
+ from cudf_polars.dsl.ir import (
14
+ GroupBy,
15
+ Join,
16
+ Scan,
17
+ Sort,
18
+ )
19
+ from cudf_polars.dsl.translate import Translator
20
+ from cudf_polars.experimental.parallel import lower_ir_graph
21
+ from cudf_polars.utils.config import ConfigOptions
22
+
23
+ if TYPE_CHECKING:
24
+ from collections.abc import MutableMapping
25
+
26
+ import polars as pl
27
+
28
+ from cudf_polars.dsl.ir import IR
29
+ from cudf_polars.experimental.base import PartitionInfo
30
+
31
+
32
+ def explain_query(
33
+ q: pl.LazyFrame, engine: pl.GPUEngine, *, physical: bool = True
34
+ ) -> str:
35
+ """
36
+ Return a formatted string representation of the IR plan.
37
+
38
+ Parameters
39
+ ----------
40
+ q : pl.LazyFrame
41
+ The LazyFrame to explain.
42
+ engine : pl.GPUEngine
43
+ The configured GPU engine to use.
44
+ physical : bool, default True
45
+ If True, show the physical (lowered) plan.
46
+ If False, show the logical (pre-lowering) plan.
47
+
48
+ Returns
49
+ -------
50
+ str
51
+ A string representation of the IR plan.
52
+ """
53
+ config = ConfigOptions.from_polars_engine(engine)
54
+ ir = Translator(q._ldf.visit(), engine).translate_ir()
55
+
56
+ if physical:
57
+ lowered_ir, partition_info = lower_ir_graph(ir, config)
58
+ return _repr_ir_tree(lowered_ir, partition_info)
59
+ else:
60
+ return _repr_ir_tree(ir)
61
+
62
+
63
+ def _repr_ir_tree(
64
+ ir: IR,
65
+ partition_info: MutableMapping[IR, PartitionInfo] | None = None,
66
+ *,
67
+ offset: str = "",
68
+ ) -> str:
69
+ header = _repr_ir(ir, offset=offset)
70
+ count = partition_info[ir].count if partition_info else None
71
+ if count is not None:
72
+ header = header.rstrip("\n") + f" [{count}]\n"
73
+
74
+ children_strs = [
75
+ _repr_ir_tree(child, partition_info, offset=offset + " ")
76
+ for child in ir.children
77
+ ]
78
+
79
+ return header + "".join(
80
+ f"{line}{offset} (repeated {count} times)\n"
81
+ if (count := sum(1 for _ in group)) > 1
82
+ else line
83
+ for line, group in groupby(children_strs)
84
+ )
85
+
86
+
87
+ def _repr_schema(schema: tuple | None) -> str:
88
+ if schema is None:
89
+ return "" # pragma: no cover; no test yet
90
+ names = tuple(schema)
91
+ if len(names) > 6:
92
+ names = names[:3] + ("...",) + names[-2:]
93
+ return f" {names}"
94
+
95
+
96
+ def _repr_header(offset: str, label: str, schema: tuple | dict | None) -> str:
97
+ return f"{offset}{label}{_repr_schema(tuple(schema) if schema is not None else None)}\n"
98
+
99
+
100
+ @functools.singledispatch
101
+ def _repr_ir(ir: IR, *, offset: str = "") -> str:
102
+ return _repr_header(offset, type(ir).__name__.upper(), ir.schema)
103
+
104
+
105
+ @_repr_ir.register
106
+ def _(ir: GroupBy, *, offset: str = "") -> str:
107
+ keys = tuple(ne.name for ne in ir.keys)
108
+ return _repr_header(offset, f"GROUPBY {keys}", ir.schema)
109
+
110
+
111
+ @_repr_ir.register
112
+ def _(ir: Join, *, offset: str = "") -> str:
113
+ left_on = tuple(ne.name for ne in ir.left_on)
114
+ right_on = tuple(ne.name for ne in ir.right_on)
115
+ return _repr_header(offset, f"JOIN {ir.options[0]} {left_on} {right_on}", ir.schema)
116
+
117
+
118
+ @_repr_ir.register
119
+ def _(ir: Sort, *, offset: str = "") -> str:
120
+ by = tuple(ne.name for ne in ir.by)
121
+ return _repr_header(offset, f"SORT {by}", ir.schema)
122
+
123
+
124
+ @_repr_ir.register
125
+ def _(ir: Scan, *, offset: str = "") -> str:
126
+ label = f"SCAN {ir.typ.upper()}"
127
+ return _repr_header(offset, label, ir.schema)