From a590076c37351965c4465ab1145b50ff57c31b44 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 6 Nov 2024 14:58:33 -0800 Subject: [PATCH 01/40] cleanup --- python/cudf_polars/cudf_polars/callback.py | 6 + .../cudf_polars/dsl/expressions/sorting.py | 2 +- python/cudf_polars/cudf_polars/dsl/ir.py | 26 ++- .../cudf_polars/experimental/__init__.py | 8 + .../cudf_polars/experimental/partitioned.py | 73 ++++++++ .../cudf_polars/experimental/single.py | 166 ++++++++++++++++++ 6 files changed, 272 insertions(+), 9 deletions(-) create mode 100644 python/cudf_polars/cudf_polars/experimental/__init__.py create mode 100644 python/cudf_polars/cudf_polars/experimental/partitioned.py create mode 100644 python/cudf_polars/cudf_polars/experimental/single.py diff --git a/python/cudf_polars/cudf_polars/callback.py b/python/cudf_polars/cudf_polars/callback.py index 76816ee0a61..4121a774f26 100644 --- a/python/cudf_polars/cudf_polars/callback.py +++ b/python/cudf_polars/cudf_polars/callback.py @@ -145,6 +145,12 @@ def _callback( set_device(device), set_memory_resource(memory_resource), ): + if os.environ.get("CUDF_POLARS_DASK", "OFF").upper() == "ON": + # Use experimental Dask executor + from cudf_polars.experimental.partitioned import evaluate + + return evaluate(ir).to_polars() + return ir.evaluate(cache={}).to_polars() diff --git a/python/cudf_polars/cudf_polars/dsl/expressions/sorting.py b/python/cudf_polars/cudf_polars/dsl/expressions/sorting.py index 99512e2ef52..f8079234df4 100644 --- a/python/cudf_polars/cudf_polars/dsl/expressions/sorting.py +++ b/python/cudf_polars/cudf_polars/dsl/expressions/sorting.py @@ -69,7 +69,7 @@ def __init__( *by: Expr, ) -> None: self.dtype = dtype - self.options = options + self.options = (options[0], tuple(options[1]), tuple(options[2])) self.children = (column, *by) def do_evaluate( diff --git a/python/cudf_polars/cudf_polars/dsl/ir.py b/python/cudf_polars/cudf_polars/dsl/ir.py index a242ff9300f..ef7d33d122b 100644 --- a/python/cudf_polars/cudf_polars/dsl/ir.py +++ b/python/cudf_polars/cudf_polars/dsl/ir.py @@ -1468,13 +1468,20 @@ def __init__(self, schema: Schema, name: str, options: Any, df: IR): self.options = ( tuple(indices), tuple(pivotees), - (variable_name, schema[variable_name]), - (value_name, schema[value_name]), + variable_name, + value_name, ) - self._non_child_args = (name, self.options) + self._non_child_args = (schema, name, self.options) + + def get_hashable(self) -> Hashable: + """Hashable representation of the node.""" + schema_hash = tuple(self.schema.items()) + return (type(self), schema_hash, self.name, str(self.options)) @classmethod - def do_evaluate(cls, name: str, options: Any, df: DataFrame) -> DataFrame: + def do_evaluate( + cls, schema: Schema, name: str, options: Any, df: DataFrame + ) -> DataFrame: """Evaluate and return a dataframe.""" if name == "rechunk": # No-op in our data model @@ -1496,8 +1503,8 @@ def do_evaluate(cls, name: str, options: Any, df: DataFrame) -> DataFrame: ( indices, pivotees, - (variable_name, variable_dtype), - (value_name, value_dtype), + variable_name, + value_name, ) = options npiv = len(pivotees) index_columns = [ @@ -1514,7 +1521,7 @@ def do_evaluate(cls, name: str, options: Any, df: DataFrame) -> DataFrame: plc.interop.from_arrow( pa.array( pivotees, - type=plc.interop.to_arrow(variable_dtype), + type=plc.interop.to_arrow(schema[variable_name]), ), ) ] @@ -1522,7 +1529,10 @@ def do_evaluate(cls, name: str, options: Any, df: DataFrame) -> DataFrame: df.num_rows, ).columns() value_column = plc.concatenate.concatenate( - [df.column_map[pivotee].astype(value_dtype).obj for pivotee in pivotees] + [ + df.column_map[pivotee].astype(schema[value_name]).obj + for pivotee in pivotees + ] ) return DataFrame( [ diff --git a/python/cudf_polars/cudf_polars/experimental/__init__.py b/python/cudf_polars/cudf_polars/experimental/__init__.py new file mode 100644 index 00000000000..c3ad73f3b1e --- /dev/null +++ b/python/cudf_polars/cudf_polars/experimental/__init__.py @@ -0,0 +1,8 @@ +# SPDX-FileCopyrightText: Copyright (c) 2024 NVIDIA CORPORATION & AFFILIATES. +# SPDX-License-Identifier: Apache-2.0 + +"""Experimental features, which can change without any depreciation period.""" + +from __future__ import annotations + +__all__: list[str] = [] diff --git a/python/cudf_polars/cudf_polars/experimental/partitioned.py b/python/cudf_polars/cudf_polars/experimental/partitioned.py new file mode 100644 index 00000000000..5bf43e95ebd --- /dev/null +++ b/python/cudf_polars/cudf_polars/experimental/partitioned.py @@ -0,0 +1,73 @@ +# SPDX-FileCopyrightText: Copyright (c) 2024 NVIDIA CORPORATION & AFFILIATES. +# SPDX-License-Identifier: Apache-2.0 +"""Partitioned LogicalPlan nodes.""" + +from __future__ import annotations + +from typing import TYPE_CHECKING, Any, Protocol, runtime_checkable + +if TYPE_CHECKING: + from collections.abc import MutableMapping + + from cudf_polars.containers import DataFrame + from cudf_polars.dsl.ir import IR + + +class PartitionInfo: + """ + Partitioning information. + + This class only tracks the partition count (for now). + """ + + __slots__ = ("npartitions",) + + def __init__(self, npartitions: int): + self.npartitions = npartitions + + +@runtime_checkable +class PartitionedIR(Protocol): + """ + Partitioned IR Protocol. + + IR nodes must satistfy this protocol to generate a valid task graph. + """ + + _key: str + _parts: PartitionInfo + + def _tasks(self) -> MutableMapping: + raise NotImplementedError() + + +def task_graph(_ir: IR) -> tuple[MutableMapping[str, Any], str]: + """Construct a Dask-compatible task graph.""" + from cudf_polars.dsl.traversal import traversal + from cudf_polars.experimental.single import lower_ir_graph + + # Rewrite IR graph into a ParIR graph + ir: PartitionedIR = lower_ir_graph(_ir) + + dsk = { + k: v for layer in [n._tasks() for n in traversal(ir)] for k, v in layer.items() + } + + # Add task to reduce output partitions + npartitions = ir._parts.npartitions + key_name = ir._key + if npartitions == 1: + dsk[key_name] = (key_name, 0) + else: + # Need DataFrame.concat support + raise NotImplementedError() + + return dsk, key_name + + +def evaluate(ir: IR) -> DataFrame: + """Evaluate an IR graph with Dask.""" + from dask import get + + dsk, key = task_graph(ir) + return get(dsk, key) diff --git a/python/cudf_polars/cudf_polars/experimental/single.py b/python/cudf_polars/cudf_polars/experimental/single.py new file mode 100644 index 00000000000..b2350e7ef94 --- /dev/null +++ b/python/cudf_polars/cudf_polars/experimental/single.py @@ -0,0 +1,166 @@ +# SPDX-FileCopyrightText: Copyright (c) 2024 NVIDIA CORPORATION & AFFILIATES. +# SPDX-License-Identifier: Apache-2.0 +"""Single-partition LogicalPlan nodes.""" + +from __future__ import annotations + +from functools import cached_property +from typing import TYPE_CHECKING + +from cudf_polars.dsl.ir import ( + Cache, + DataFrameScan, + Distinct, + Filter, + GroupBy, + HConcat, + HStack, + Join, + MapFunction, + Projection, + PythonScan, + Reduce, + Scan, + Select, + Slice, + Sort, + Union, +) +from cudf_polars.dsl.traversal import CachingVisitor +from cudf_polars.experimental.partitioned import PartitionInfo + +if TYPE_CHECKING: + from cudf_polars.dsl.ir import IR + + +class SPartitionwise: + """Single partition-wise PartitionedIR.""" + + @cached_property + def _key(self): + return f"{type(self).__name__.lower()}-{hash(self)}" + + def _tasks(self): + return { + (self._key, 0): ( + self.do_evaluate, + *self._non_child_args, + *((child._key, 0) for child in self.children), + ) + } + + @cached_property + def _parts(self) -> PartitionInfo: + return PartitionInfo(npartitions=1) + + +class SParPythonScan(PythonScan, SPartitionwise): + """Single-partition demo class.""" + + +class SParScan(Scan, SPartitionwise): + """Single-partition demo class.""" + + +class SParCache(Cache, SPartitionwise): + """Single-partition demo class.""" + + +class SParDataFrameScan(DataFrameScan, SPartitionwise): + """Single-partition demo class.""" + + +class SParSelect(Select, SPartitionwise): + """Single-partition demo class.""" + + +class SParReduce(Reduce, SPartitionwise): + """Single-partition demo class.""" + + +class SParGroupBy(GroupBy, SPartitionwise): + """Single-partition demo class.""" + + +class SParJoin(Join, SPartitionwise): + """Single-partition demo class.""" + + +class SParHStack(HStack, SPartitionwise): + """Single-partition demo class.""" + + +class SParDistinct(Distinct, SPartitionwise): + """Single-partition demo class.""" + + +class SParSort(Sort, SPartitionwise): + """Single-partition demo class.""" + + +class SParSlice(Slice, SPartitionwise): + """Single-partition demo class.""" + + +class SParFilter(Filter, SPartitionwise): + """Single-partition demo class.""" + + +class SParProjection(Projection, SPartitionwise): + """Single-partition demo class.""" + + +class SParMapFunction(MapFunction, SPartitionwise): + """Single-partition demo class.""" + + +class SParUnion(Union, SPartitionwise): + """Single-partition demo class.""" + + +class SParHConcat(HConcat, SPartitionwise): + """Single-partition demo class.""" + + +def _single_partition_node(node: IR, rec) -> SPartitionwise: + children = [rec(child) for child in node.children] + + if isinstance(node, PythonScan): + return SParPythonScan(*node._ctor_arguments(children)) + elif isinstance(node, Scan): + return SParScan(*node._ctor_arguments(children)) + elif isinstance(node, Cache): + return SParCache(*node._ctor_arguments(children)) + elif isinstance(node, DataFrameScan): + return SParDataFrameScan(*node._ctor_arguments(children)) + elif isinstance(node, Select): + return SParSelect(*node._ctor_arguments(children)) + elif isinstance(node, Reduce): + return SParReduce(*node._ctor_arguments(children)) + elif isinstance(node, GroupBy): + return SParGroupBy(*node._ctor_arguments(children)) + elif isinstance(node, Join): + return SParJoin(*node._ctor_arguments(children)) + elif isinstance(node, HStack): + return SParHStack(*node._ctor_arguments(children)) + elif isinstance(node, Distinct): + return SParDistinct(*node._ctor_arguments(children)) + elif isinstance(node, Sort): + return SParSort(*node._ctor_arguments(children)) + elif isinstance(node, Slice): + return SParSlice(*node._ctor_arguments(children)) + elif isinstance(node, Filter): + return SParFilter(*node._ctor_arguments(children)) + elif isinstance(node, Projection): + return SParProjection(*node._ctor_arguments(children)) + elif isinstance(node, MapFunction): + return SParMapFunction(*node._ctor_arguments(children)) + elif isinstance(node, Union): + return SParUnion(*node._ctor_arguments(children)) + elif isinstance(node, HConcat): + return SParHConcat(*node._ctor_arguments(children)) + else: + raise NotImplementedError(f"Cannot convert {type(node)} to PartitionedIR.") + + +lower_ir_graph = CachingVisitor(_single_partition_node) From 7f1bec78f07fe9b47d2e759f97c5a6b8989e8aee Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 6 Nov 2024 17:31:56 -0800 Subject: [PATCH 02/40] rename to parallel --- python/cudf_polars/cudf_polars/callback.py | 4 +- .../{partitioned.py => parallel.py} | 2 +- .../cudf_polars/experimental/single.py | 130 ++++++++++++------ 3 files changed, 93 insertions(+), 43 deletions(-) rename python/cudf_polars/cudf_polars/experimental/{partitioned.py => parallel.py} (97%) diff --git a/python/cudf_polars/cudf_polars/callback.py b/python/cudf_polars/cudf_polars/callback.py index 4121a774f26..29e8509febd 100644 --- a/python/cudf_polars/cudf_polars/callback.py +++ b/python/cudf_polars/cudf_polars/callback.py @@ -147,9 +147,9 @@ def _callback( ): if os.environ.get("CUDF_POLARS_DASK", "OFF").upper() == "ON": # Use experimental Dask executor - from cudf_polars.experimental.partitioned import evaluate + from cudf_polars.experimental.parallel import evaluate_dask - return evaluate(ir).to_polars() + return evaluate_dask(ir).to_polars() return ir.evaluate(cache={}).to_polars() diff --git a/python/cudf_polars/cudf_polars/experimental/partitioned.py b/python/cudf_polars/cudf_polars/experimental/parallel.py similarity index 97% rename from python/cudf_polars/cudf_polars/experimental/partitioned.py rename to python/cudf_polars/cudf_polars/experimental/parallel.py index 5bf43e95ebd..c0c9690be2f 100644 --- a/python/cudf_polars/cudf_polars/experimental/partitioned.py +++ b/python/cudf_polars/cudf_polars/experimental/parallel.py @@ -65,7 +65,7 @@ def task_graph(_ir: IR) -> tuple[MutableMapping[str, Any], str]: return dsk, key_name -def evaluate(ir: IR) -> DataFrame: +def evaluate_dask(ir: IR) -> DataFrame: """Evaluate an IR graph with Dask.""" from dask import get diff --git a/python/cudf_polars/cudf_polars/experimental/single.py b/python/cudf_polars/cudf_polars/experimental/single.py index b2350e7ef94..c9c8e564245 100644 --- a/python/cudf_polars/cudf_polars/experimental/single.py +++ b/python/cudf_polars/cudf_polars/experimental/single.py @@ -4,10 +4,11 @@ from __future__ import annotations -from functools import cached_property +from functools import cached_property, singledispatch from typing import TYPE_CHECKING from cudf_polars.dsl.ir import ( + IR, Cache, DataFrameScan, Distinct, @@ -27,7 +28,7 @@ Union, ) from cudf_polars.dsl.traversal import CachingVisitor -from cudf_polars.experimental.partitioned import PartitionInfo +from cudf_polars.experimental.parallel import PartitionInfo if TYPE_CHECKING: from cudf_polars.dsl.ir import IR @@ -122,45 +123,94 @@ class SParHConcat(HConcat, SPartitionwise): """Single-partition demo class.""" +@singledispatch def _single_partition_node(node: IR, rec) -> SPartitionwise: - children = [rec(child) for child in node.children] - - if isinstance(node, PythonScan): - return SParPythonScan(*node._ctor_arguments(children)) - elif isinstance(node, Scan): - return SParScan(*node._ctor_arguments(children)) - elif isinstance(node, Cache): - return SParCache(*node._ctor_arguments(children)) - elif isinstance(node, DataFrameScan): - return SParDataFrameScan(*node._ctor_arguments(children)) - elif isinstance(node, Select): - return SParSelect(*node._ctor_arguments(children)) - elif isinstance(node, Reduce): - return SParReduce(*node._ctor_arguments(children)) - elif isinstance(node, GroupBy): - return SParGroupBy(*node._ctor_arguments(children)) - elif isinstance(node, Join): - return SParJoin(*node._ctor_arguments(children)) - elif isinstance(node, HStack): - return SParHStack(*node._ctor_arguments(children)) - elif isinstance(node, Distinct): - return SParDistinct(*node._ctor_arguments(children)) - elif isinstance(node, Sort): - return SParSort(*node._ctor_arguments(children)) - elif isinstance(node, Slice): - return SParSlice(*node._ctor_arguments(children)) - elif isinstance(node, Filter): - return SParFilter(*node._ctor_arguments(children)) - elif isinstance(node, Projection): - return SParProjection(*node._ctor_arguments(children)) - elif isinstance(node, MapFunction): - return SParMapFunction(*node._ctor_arguments(children)) - elif isinstance(node, Union): - return SParUnion(*node._ctor_arguments(children)) - elif isinstance(node, HConcat): - return SParHConcat(*node._ctor_arguments(children)) - else: - raise NotImplementedError(f"Cannot convert {type(node)} to PartitionedIR.") + raise NotImplementedError(f"Cannot convert {type(node)} to PartitionedIR.") + + +@_single_partition_node.register +def _(node: PythonScan, rec) -> SParPythonScan: + return SParPythonScan(*node._ctor_arguments(map(rec, node.children))) + + +@_single_partition_node.register +def _(node: Scan, rec) -> SParScan: + return SParScan(*node._ctor_arguments(map(rec, node.children))) + + +@_single_partition_node.register +def _(node: DataFrameScan, rec) -> SParDataFrameScan: + return SParDataFrameScan(*node._ctor_arguments(map(rec, node.children))) + + +@_single_partition_node.register +def _(node: Cache, rec) -> SParCache: + return SParCache(*node._ctor_arguments(map(rec, node.children))) + + +@_single_partition_node.register +def _(node: Reduce, rec) -> SParReduce: + return SParReduce(*node._ctor_arguments(map(rec, node.children))) + + +@_single_partition_node.register +def _(node: Select, rec) -> SParSelect: + return SParSelect(*node._ctor_arguments(map(rec, node.children))) + + +@_single_partition_node.register +def _(node: GroupBy, rec) -> SParGroupBy: + return SParGroupBy(*node._ctor_arguments(map(rec, node.children))) + + +@_single_partition_node.register +def _(node: Join, rec) -> SParJoin: + return SParJoin(*node._ctor_arguments(map(rec, node.children))) + + +@_single_partition_node.register +def _(node: HStack, rec) -> SParHStack: + return SParHStack(*node._ctor_arguments(map(rec, node.children))) + + +@_single_partition_node.register +def _(node: Distinct, rec) -> SParDistinct: + return SParDistinct(*node._ctor_arguments(map(rec, node.children))) + + +@_single_partition_node.register +def _(node: Sort, rec) -> SParSort: + return SParSort(*node._ctor_arguments(map(rec, node.children))) + + +@_single_partition_node.register +def _(node: Slice, rec) -> SParSlice: + return SParSlice(*node._ctor_arguments(map(rec, node.children))) + + +@_single_partition_node.register +def _(node: Filter, rec) -> SParFilter: + return SParFilter(*node._ctor_arguments(map(rec, node.children))) + + +@_single_partition_node.register +def _(node: Projection, rec) -> SParProjection: + return SParProjection(*node._ctor_arguments(map(rec, node.children))) + + +@_single_partition_node.register +def _(node: MapFunction, rec) -> SParMapFunction: + return SParMapFunction(*node._ctor_arguments(map(rec, node.children))) + + +@_single_partition_node.register +def _(node: Union, rec) -> SParUnion: + return SParUnion(*node._ctor_arguments(map(rec, node.children))) + + +@_single_partition_node.register +def _(node: HConcat, rec) -> SParHConcat: + return SParHConcat(*node._ctor_arguments(map(rec, node.children))) lower_ir_graph = CachingVisitor(_single_partition_node) From 69a33741962eff173f76e1ed56c2ab8f2ebfeac4 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 7 Nov 2024 13:28:05 -0800 Subject: [PATCH 03/40] simplify solution --- python/cudf_polars/cudf_polars/dsl/ir.py | 2 +- .../cudf_polars/cudf_polars/dsl/nodebase.py | 30 ++- .../cudf_polars/experimental/parallel.py | 93 ++++---- .../cudf_polars/experimental/single.py | 216 ------------------ 4 files changed, 81 insertions(+), 260 deletions(-) delete mode 100644 python/cudf_polars/cudf_polars/experimental/single.py diff --git a/python/cudf_polars/cudf_polars/dsl/ir.py b/python/cudf_polars/cudf_polars/dsl/ir.py index ef7d33d122b..540661c1246 100644 --- a/python/cudf_polars/cudf_polars/dsl/ir.py +++ b/python/cudf_polars/cudf_polars/dsl/ir.py @@ -1476,7 +1476,7 @@ def __init__(self, schema: Schema, name: str, options: Any, df: IR): def get_hashable(self) -> Hashable: """Hashable representation of the node.""" schema_hash = tuple(self.schema.items()) - return (type(self), schema_hash, self.name, str(self.options)) + return (type(self), schema_hash, self.name, str(self.options), *self.children) @classmethod def do_evaluate( diff --git a/python/cudf_polars/cudf_polars/dsl/nodebase.py b/python/cudf_polars/cudf_polars/dsl/nodebase.py index 228d300f467..b0ea50f3d8d 100644 --- a/python/cudf_polars/cudf_polars/dsl/nodebase.py +++ b/python/cudf_polars/cudf_polars/dsl/nodebase.py @@ -18,6 +18,19 @@ T = TypeVar("T", bound="Node[Any]") +class PartitionInfo: + """ + Partitioning information. + + This class only tracks the partition count (for now). + """ + + __slots__ = ("count",) + + def __init__(self, count: int | None): + self.count = count + + class Node(Generic[T]): """ An abstract node type. @@ -34,9 +47,10 @@ class Node(Generic[T]): *children).`` """ - __slots__ = ("_hash_value", "_repr_value", "children") + __slots__ = ("_hash_value", "_repr_value", "_parts_info", "children") _hash_value: int _repr_value: str + _parts_info: PartitionInfo children: tuple[T, ...] _non_child: ClassVar[tuple[str, ...]] = () @@ -150,3 +164,17 @@ def __repr__(self) -> str: args = ", ".join(f"{arg!r}" for arg in self._ctor_arguments(self.children)) self._repr_value = f"{type(self).__name__}({args})" return self._repr_value + + @property + def parts(self) -> PartitionInfo: + """Return Partitioning information for this node.""" + try: + return self._parts_info + except AttributeError: + self._parts_info = self._get_partition_info() + return self._parts_info + + def _get_partition_info(self) -> PartitionInfo: + from cudf_polars.experimental.parallel import partition_info_dispatch + + return partition_info_dispatch(self) diff --git a/python/cudf_polars/cudf_polars/experimental/parallel.py b/python/cudf_polars/cudf_polars/experimental/parallel.py index c0c9690be2f..8e19c2ed0d5 100644 --- a/python/cudf_polars/cudf_polars/experimental/parallel.py +++ b/python/cudf_polars/cudf_polars/experimental/parallel.py @@ -4,63 +4,72 @@ from __future__ import annotations -from typing import TYPE_CHECKING, Any, Protocol, runtime_checkable +from functools import singledispatch +from typing import TYPE_CHECKING, Any + +from cudf_polars.dsl.nodebase import PartitionInfo +from cudf_polars.dsl.traversal import traversal if TYPE_CHECKING: from collections.abc import MutableMapping from cudf_polars.containers import DataFrame from cudf_polars.dsl.ir import IR - - -class PartitionInfo: - """ - Partitioning information. - - This class only tracks the partition count (for now). - """ - - __slots__ = ("npartitions",) - - def __init__(self, npartitions: int): - self.npartitions = npartitions - - -@runtime_checkable -class PartitionedIR(Protocol): - """ - Partitioned IR Protocol. - - IR nodes must satistfy this protocol to generate a valid task graph. - """ - - _key: str - _parts: PartitionInfo - - def _tasks(self) -> MutableMapping: - raise NotImplementedError() - - -def task_graph(_ir: IR) -> tuple[MutableMapping[str, Any], str]: + from cudf_polars.dsl.nodebase import Node + + +def get_key_name(node: Node) -> str: + """Generate the key name for a Node.""" + return f"{type(node).__name__.lower()}{hash(node)}" + + +@singledispatch +def partition_info_dispatch(node: Node) -> PartitionInfo: + """Return partitioning information for a given node.""" + # Assume the partition count is preserved by default. + count = 1 + if node.children: + count = max(child.parts.count for child in node.children) + if count > 1: + raise NotImplementedError( + f"Multi-partition support is not implemented for {type(node)}." + ) + return PartitionInfo(count=count) + + +@singledispatch +def generate_tasks(ir: IR) -> MutableMapping[Any, Any]: + """Generate tasks for an IR node.""" + if ir.parts.count == 1: + key_name = get_key_name(ir) + return { + (key_name, 0): ( + ir.do_evaluate, + *ir._non_child_args, + *((get_key_name(child), 0) for child in ir.children), + ) + } + raise NotImplementedError(f"Cannot generate tasks for {ir}.") + + +def task_graph(ir: IR) -> tuple[MutableMapping[str, Any], str]: """Construct a Dask-compatible task graph.""" - from cudf_polars.dsl.traversal import traversal - from cudf_polars.experimental.single import lower_ir_graph - - # Rewrite IR graph into a ParIR graph - ir: PartitionedIR = lower_ir_graph(_ir) + # NOTE: It may be necessary to add an optimization + # pass here to "rewrite" the single-partition IR graph. dsk = { - k: v for layer in [n._tasks() for n in traversal(ir)] for k, v in layer.items() + k: v + for layer in [generate_tasks(n) for n in traversal(ir)] + for k, v in layer.items() } # Add task to reduce output partitions - npartitions = ir._parts.npartitions - key_name = ir._key - if npartitions == 1: + key_name = get_key_name(ir) + if ir.parts.count == 1: dsk[key_name] = (key_name, 0) else: # Need DataFrame.concat support - raise NotImplementedError() + raise NotImplementedError("Multi-partition output is not supported.") return dsk, key_name diff --git a/python/cudf_polars/cudf_polars/experimental/single.py b/python/cudf_polars/cudf_polars/experimental/single.py deleted file mode 100644 index c9c8e564245..00000000000 --- a/python/cudf_polars/cudf_polars/experimental/single.py +++ /dev/null @@ -1,216 +0,0 @@ -# SPDX-FileCopyrightText: Copyright (c) 2024 NVIDIA CORPORATION & AFFILIATES. -# SPDX-License-Identifier: Apache-2.0 -"""Single-partition LogicalPlan nodes.""" - -from __future__ import annotations - -from functools import cached_property, singledispatch -from typing import TYPE_CHECKING - -from cudf_polars.dsl.ir import ( - IR, - Cache, - DataFrameScan, - Distinct, - Filter, - GroupBy, - HConcat, - HStack, - Join, - MapFunction, - Projection, - PythonScan, - Reduce, - Scan, - Select, - Slice, - Sort, - Union, -) -from cudf_polars.dsl.traversal import CachingVisitor -from cudf_polars.experimental.parallel import PartitionInfo - -if TYPE_CHECKING: - from cudf_polars.dsl.ir import IR - - -class SPartitionwise: - """Single partition-wise PartitionedIR.""" - - @cached_property - def _key(self): - return f"{type(self).__name__.lower()}-{hash(self)}" - - def _tasks(self): - return { - (self._key, 0): ( - self.do_evaluate, - *self._non_child_args, - *((child._key, 0) for child in self.children), - ) - } - - @cached_property - def _parts(self) -> PartitionInfo: - return PartitionInfo(npartitions=1) - - -class SParPythonScan(PythonScan, SPartitionwise): - """Single-partition demo class.""" - - -class SParScan(Scan, SPartitionwise): - """Single-partition demo class.""" - - -class SParCache(Cache, SPartitionwise): - """Single-partition demo class.""" - - -class SParDataFrameScan(DataFrameScan, SPartitionwise): - """Single-partition demo class.""" - - -class SParSelect(Select, SPartitionwise): - """Single-partition demo class.""" - - -class SParReduce(Reduce, SPartitionwise): - """Single-partition demo class.""" - - -class SParGroupBy(GroupBy, SPartitionwise): - """Single-partition demo class.""" - - -class SParJoin(Join, SPartitionwise): - """Single-partition demo class.""" - - -class SParHStack(HStack, SPartitionwise): - """Single-partition demo class.""" - - -class SParDistinct(Distinct, SPartitionwise): - """Single-partition demo class.""" - - -class SParSort(Sort, SPartitionwise): - """Single-partition demo class.""" - - -class SParSlice(Slice, SPartitionwise): - """Single-partition demo class.""" - - -class SParFilter(Filter, SPartitionwise): - """Single-partition demo class.""" - - -class SParProjection(Projection, SPartitionwise): - """Single-partition demo class.""" - - -class SParMapFunction(MapFunction, SPartitionwise): - """Single-partition demo class.""" - - -class SParUnion(Union, SPartitionwise): - """Single-partition demo class.""" - - -class SParHConcat(HConcat, SPartitionwise): - """Single-partition demo class.""" - - -@singledispatch -def _single_partition_node(node: IR, rec) -> SPartitionwise: - raise NotImplementedError(f"Cannot convert {type(node)} to PartitionedIR.") - - -@_single_partition_node.register -def _(node: PythonScan, rec) -> SParPythonScan: - return SParPythonScan(*node._ctor_arguments(map(rec, node.children))) - - -@_single_partition_node.register -def _(node: Scan, rec) -> SParScan: - return SParScan(*node._ctor_arguments(map(rec, node.children))) - - -@_single_partition_node.register -def _(node: DataFrameScan, rec) -> SParDataFrameScan: - return SParDataFrameScan(*node._ctor_arguments(map(rec, node.children))) - - -@_single_partition_node.register -def _(node: Cache, rec) -> SParCache: - return SParCache(*node._ctor_arguments(map(rec, node.children))) - - -@_single_partition_node.register -def _(node: Reduce, rec) -> SParReduce: - return SParReduce(*node._ctor_arguments(map(rec, node.children))) - - -@_single_partition_node.register -def _(node: Select, rec) -> SParSelect: - return SParSelect(*node._ctor_arguments(map(rec, node.children))) - - -@_single_partition_node.register -def _(node: GroupBy, rec) -> SParGroupBy: - return SParGroupBy(*node._ctor_arguments(map(rec, node.children))) - - -@_single_partition_node.register -def _(node: Join, rec) -> SParJoin: - return SParJoin(*node._ctor_arguments(map(rec, node.children))) - - -@_single_partition_node.register -def _(node: HStack, rec) -> SParHStack: - return SParHStack(*node._ctor_arguments(map(rec, node.children))) - - -@_single_partition_node.register -def _(node: Distinct, rec) -> SParDistinct: - return SParDistinct(*node._ctor_arguments(map(rec, node.children))) - - -@_single_partition_node.register -def _(node: Sort, rec) -> SParSort: - return SParSort(*node._ctor_arguments(map(rec, node.children))) - - -@_single_partition_node.register -def _(node: Slice, rec) -> SParSlice: - return SParSlice(*node._ctor_arguments(map(rec, node.children))) - - -@_single_partition_node.register -def _(node: Filter, rec) -> SParFilter: - return SParFilter(*node._ctor_arguments(map(rec, node.children))) - - -@_single_partition_node.register -def _(node: Projection, rec) -> SParProjection: - return SParProjection(*node._ctor_arguments(map(rec, node.children))) - - -@_single_partition_node.register -def _(node: MapFunction, rec) -> SParMapFunction: - return SParMapFunction(*node._ctor_arguments(map(rec, node.children))) - - -@_single_partition_node.register -def _(node: Union, rec) -> SParUnion: - return SParUnion(*node._ctor_arguments(map(rec, node.children))) - - -@_single_partition_node.register -def _(node: HConcat, rec) -> SParHConcat: - return SParHConcat(*node._ctor_arguments(map(rec, node.children))) - - -lower_ir_graph = CachingVisitor(_single_partition_node) From 915a779ef1e4e2d1d63d6fb9e826b6b2ad804af4 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Fri, 8 Nov 2024 13:09:21 -0800 Subject: [PATCH 04/40] deeper dive --- .../cudf_polars/containers/dataframe.py | 9 + .../cudf_polars/cudf_polars/dsl/nodebase.py | 6 +- .../cudf_polars/experimental/parallel.py | 365 ++++++++++++++++-- 3 files changed, 349 insertions(+), 31 deletions(-) diff --git a/python/cudf_polars/cudf_polars/containers/dataframe.py b/python/cudf_polars/cudf_polars/containers/dataframe.py index 08bc9d0ea3f..f8ee6911931 100644 --- a/python/cudf_polars/cudf_polars/containers/dataframe.py +++ b/python/cudf_polars/cudf_polars/containers/dataframe.py @@ -226,6 +226,15 @@ def filter(self, mask: Column) -> Self: table = plc.stream_compaction.apply_boolean_mask(self.table, mask.obj) return type(self).from_table(table, self.column_names).sorted_like(self) + @classmethod + def concat(cls, dfs: list[Self]) -> Self: + """Concatenate a list of dataframes.""" + assert len(dfs), "Cannot concatenate an empty list" + if len(dfs) == 1: + return dfs[0] + table = plc.concatenate.concatenate([df.table for df in dfs]) + return cls.from_table(table, dfs[0].column_names) + def slice(self, zlice: tuple[int, int] | None) -> Self: """ Slice a dataframe. diff --git a/python/cudf_polars/cudf_polars/dsl/nodebase.py b/python/cudf_polars/cudf_polars/dsl/nodebase.py index b0ea50f3d8d..319f247e25a 100644 --- a/python/cudf_polars/cudf_polars/dsl/nodebase.py +++ b/python/cudf_polars/cudf_polars/dsl/nodebase.py @@ -27,7 +27,7 @@ class PartitionInfo: __slots__ = ("count",) - def __init__(self, count: int | None): + def __init__(self, count: int): self.count = count @@ -175,6 +175,6 @@ def parts(self) -> PartitionInfo: return self._parts_info def _get_partition_info(self) -> PartitionInfo: - from cudf_polars.experimental.parallel import partition_info_dispatch + from cudf_polars.experimental.parallel import _ir_partition_info_dispatch - return partition_info_dispatch(self) + return _ir_partition_info_dispatch(self) diff --git a/python/cudf_polars/cudf_polars/experimental/parallel.py b/python/cudf_polars/cudf_polars/experimental/parallel.py index 8e19c2ed0d5..e895d95b5a1 100644 --- a/python/cudf_polars/cudf_polars/experimental/parallel.py +++ b/python/cudf_polars/cudf_polars/experimental/parallel.py @@ -4,79 +4,388 @@ from __future__ import annotations +import warnings from functools import singledispatch from typing import TYPE_CHECKING, Any +import pylibcudf as plc + +from cudf_polars.containers import Column, DataFrame +from cudf_polars.dsl.expr import Agg, Col, NamedExpr +from cudf_polars.dsl.ir import Scan, Select, broadcast from cudf_polars.dsl.nodebase import PartitionInfo from cudf_polars.dsl.traversal import traversal if TYPE_CHECKING: - from collections.abc import MutableMapping + from collections.abc import MutableMapping, Sequence - from cudf_polars.containers import DataFrame + from cudf_polars.dsl.expr import Expr from cudf_polars.dsl.ir import IR from cudf_polars.dsl.nodebase import Node -def get_key_name(node: Node) -> str: +def get_key_name(node: Node | NamedExpr) -> str: """Generate the key name for a Node.""" - return f"{type(node).__name__.lower()}{hash(node)}" + if isinstance(node, NamedExpr): + return f"named-{get_key_name(node.value)}" + return f"{type(node).__name__.lower()}-{hash(node)}" @singledispatch -def partition_info_dispatch(node: Node) -> PartitionInfo: - """Return partitioning information for a given node.""" - # Assume the partition count is preserved by default. +def _ir_partition_info_dispatch(node: Node) -> PartitionInfo: count = 1 if node.children: count = max(child.parts.count for child in node.children) if count > 1: - raise NotImplementedError( - f"Multi-partition support is not implemented for {type(node)}." + warnings.warn( + f"Multi-partition support is not implemented for {type(node)}. " + f"Partitions will be concatenated. Expect poor performance.", + stacklevel=1, ) - return PartitionInfo(count=count) + return PartitionInfo(count=1) @singledispatch -def generate_tasks(ir: IR) -> MutableMapping[Any, Any]: - """Generate tasks for an IR node.""" +def _expr_partition_info_dispatch( + expr: Expr | NamedExpr, child_ir: IR +) -> PartitionInfo: + # The default partitioning of an Expr node depends + # on the child Expr nodes or the child IR node it + # is operating on (for leaf Exprs). + if isinstance(expr, NamedExpr): + expr = expr.value + if expr.children: + count = max([child.parts.count for child in expr.children]) + else: + count = child_ir.parts.count + if count > 1: + warnings.warn( + f"Multi-partition support is not implemented for {type(expr)}. " + f"Partitions will be concatenated. Expect poor performance.", + stacklevel=1, + ) + return PartitionInfo(count=1) + + +def _default_generate_ir_tasks(ir: IR) -> MutableMapping[Any, Any]: + # Single-partition default behavior if ir.parts.count == 1: + # Start by concatenating multi-partition + # children (if necessary) + graph = {} + child_names = [] key_name = get_key_name(ir) - return { - (key_name, 0): ( - ir.do_evaluate, - *ir._non_child_args, - *((get_key_name(child), 0) for child in ir.children), - ) - } + for child in ir.children: + child_name_in = get_key_name(child) + if child.parts.count > 1: + child_names.append("concat-" + child_name_in) + graph[(child_names[-1], 0)] = ( + DataFrame.concat, + [(child_name_in, i) for i in range(child.parts.count)], + ) + else: + child_names.append(child_name_in) + + # Add single-partition task + graph[(key_name, 0)] = ( + ir.do_evaluate, + *ir._non_child_args, + *((child_name, 0) for child_name in child_names), + ) + return graph + raise NotImplementedError(f"Cannot generate tasks for {ir}.") +@singledispatch +def generate_ir_tasks(ir: IR) -> MutableMapping[Any, Any]: + """ + Generate tasks for an IR node. + + An IR node only needs to generate the graph for + the current IR logic (not including child IRs). + """ + return _default_generate_ir_tasks(ir) + + +@singledispatch +def generate_expr_tasks( + expr: Expr | NamedExpr, child_ir: IR +) -> MutableMapping[Any, Any]: + """ + Generate tasks for an Expr node. + + An Expr node is responsible for constructing the full + expression graph recursively (allowing fusion). + """ + raise NotImplementedError(f"Cannot generate tasks for {expr}.") + + def task_graph(ir: IR) -> tuple[MutableMapping[str, Any], str]: """Construct a Dask-compatible task graph.""" # NOTE: It may be necessary to add an optimization # pass here to "rewrite" the single-partition IR graph. - dsk = { + graph = { k: v - for layer in [generate_tasks(n) for n in traversal(ir)] + for layer in [generate_ir_tasks(n) for n in traversal(ir)] for k, v in layer.items() } # Add task to reduce output partitions key_name = get_key_name(ir) - if ir.parts.count == 1: - dsk[key_name] = (key_name, 0) + if ir.parts.count > 1: + graph[key_name] = ( + DataFrame.concat, + [(key_name, i) for i in range(ir.parts.count)], + ) else: - # Need DataFrame.concat support - raise NotImplementedError("Multi-partition output is not supported.") + graph[key_name] = (key_name, 0) - return dsk, key_name + return graph, key_name def evaluate_dask(ir: IR) -> DataFrame: """Evaluate an IR graph with Dask.""" from dask import get - dsk, key = task_graph(ir) - return get(dsk, key) + graph, key = task_graph(ir) + return get(graph, key) + + +## +## Scan +## + +_SCAN_SUPPORTED = ("parquet",) + + +@_ir_partition_info_dispatch.register(Scan) +def _(ir: Scan) -> PartitionInfo: + if ir.typ in _SCAN_SUPPORTED: + return PartitionInfo(count=len(ir.paths)) + return PartitionInfo(count=1) + + +@generate_ir_tasks.register(Scan) +def _(ir: Scan) -> MutableMapping[Any, Any]: + key_name = get_key_name(ir) + if ir.parts.count == 1: + return { + (key_name, 0): ( + ir.do_evaluate, + *ir._non_child_args, + ) + } + else: + # Only support 1:1 mapping between + # paths and partitions for now + assert len(ir.paths) == ir.parts.count + return { + (key_name, i): ( + ir.do_evaluate, + ir.schema, + ir.typ, + ir.reader_options, + [path], + ir.with_columns, + ir.skip_rows, + ir.n_rows, + ir.row_index, + ir.predicate, + ) + for i, path in enumerate(ir.paths) + } + + +## +## Select +## + + +@_ir_partition_info_dispatch.register(Select) +def _(ir: Select) -> PartitionInfo: + # Partitioning depends on the expression + df = ir.children[0] + column_partition_counts = [ + _expr_partition_info_dispatch(expr, df).count for expr in ir.exprs + ] + count = max(column_partition_counts) + if count > 1: + warnings.warn( + f"Multi-partition support is not implemented for {type(ir)}. " + f"Partitions will be concatenated. Expect poor performance.", + stacklevel=1, + ) + return PartitionInfo(count=1) + + +def _select(columns: list[Column], should_broadcast: bool): # noqa: FBT001 + if should_broadcast: + columns = broadcast(*columns) + return DataFrame(columns) + + +@generate_ir_tasks.register(Select) +def _(ir: Select) -> MutableMapping[Any, Any]: + try: + expr_graphs = [generate_expr_tasks(e, ir.children[0]) for e in ir.exprs] + key_name = get_key_name(ir) + expr_keys = [get_key_name(e) for e in ir.exprs] + graph = { + (key_name, i): ( + _select, + [(c_key, i) for c_key in expr_keys], + ir.should_broadcast, + ) + for i in range(ir.parts.count) + } + for expr_graph in expr_graphs: + graph.update(expr_graph) + except NotImplementedError as err: + if ir.parts.count == 1: + return _default_generate_ir_tasks(ir) + raise NotImplementedError("Not supported.") from err + else: + return graph + + +## +## NamedExpr +## + + +@_expr_partition_info_dispatch.register(NamedExpr) +def _(expr: NamedExpr, child_ir: IR) -> PartitionInfo: + return _expr_partition_info_dispatch(expr.value, child_ir) + + +def _rename_column(column: Column, name: str): + return column.rename(name) + + +@generate_expr_tasks.register(NamedExpr) +def _(expr: NamedExpr, child_ir: IR) -> MutableMapping[Any, Any]: + graph = generate_expr_tasks(expr.value, child_ir) + named_expr_key_name = get_key_name(expr) + expr_key_name = get_key_name(expr.value) + for i in range(expr.value.parts.count): + graph[(named_expr_key_name, i)] = ( + _rename_column, + graph.pop((expr_key_name, i)), + expr.name, + ) + return graph + + +## +## Col +## + + +@_expr_partition_info_dispatch.register(Col) +def _(expr: Col, child_ir: IR) -> PartitionInfo: + assert not expr.children + count = child_ir.parts.count + return PartitionInfo(count=count) + + +def _get_col(df: DataFrame, name: str) -> Column: + return df.column_map[name].rename(None) + + +@generate_expr_tasks.register(Col) +def _(expr: Col, child_ir: IR) -> MutableMapping[Any, Any]: + key_name = get_key_name(expr) + child_name = get_key_name(child_ir) + return { + (key_name, i): (_get_col, (child_name, i), expr.name) + for i in range(child_ir.parts.count) + } + + +## +## Agg +## + +_AGG_SUPPORTED = ("sum",) + + +@_expr_partition_info_dispatch.register(Agg) +def _(expr: Agg, child_ir: IR) -> PartitionInfo: + if expr.children: + count = max([child.parts.count for child in expr.children]) + else: + count = child_ir.parts.count + if count > 1 and expr.name not in _AGG_SUPPORTED: + # Only support sum reductions for now. + warnings.warn( + f"Multi-partition support is not implemented for {type(expr)}. " + f"Partitions will be concatenated. Expect poor performance.", + stacklevel=1, + ) + return PartitionInfo(count=1) + + +def _agg_chunk( + column: Column, request: plc.aggregation.Aggregation, dtype: plc.DataType +) -> Column: + # TODO: This logic should be different than `request` in many cases + return Column( + plc.Column.from_scalar( + plc.reduce.reduce(column.obj, request, dtype), + 1, + ) + ) + + +def _agg_combine(columns: Sequence[Column]) -> Column: + return Column(plc.concatenate.concatenate([col.obj for col in columns])) + + +def _agg_finalize( + column: Column, + request: plc.aggregation.Aggregation, + dtype: plc.DataType, +) -> Column: + # TODO: This logic should be different than `request` in many cases + return Column( + plc.Column.from_scalar( + plc.reduce.reduce(column.obj, request, dtype), + 1, + ) + ) + + +@generate_expr_tasks.register(Agg) +def _(expr: Agg, child_ir: IR) -> MutableMapping[Any, Any]: + if expr.name not in _AGG_SUPPORTED: + raise NotImplementedError(f"Cannot generate tasks for {expr}.") + + child = expr.children[0] + npartitions_in = child.parts.count + key = get_key_name(expr) + child_key = get_key_name(child) + child_dsk = generate_expr_tasks(child, child_ir) + + # Simple all-to-one reduction + chunk_key = f"chunk-{key}" + combine_key = f"concat-{key}" + graph: MutableMapping[tuple[str, int], Any] = { + (chunk_key, i): ( + _agg_chunk, + # Fuse with child-expr task + child_dsk[(child_key, i)], + expr.request, + expr.dtype, + ) + for i in range(npartitions_in) + } + graph[(combine_key, 0)] = (_agg_combine, list(graph.keys())) + graph[(key, 0)] = ( + _agg_finalize, + (combine_key, 0), + expr.request, + expr.dtype, + ) + return graph From bd9d783ead60cb7636809d55b606c65e2f65e973 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Fri, 8 Nov 2024 15:03:10 -0800 Subject: [PATCH 05/40] improve simple agg reduction --- .../cudf_polars/experimental/parallel.py | 64 +++++++++---------- 1 file changed, 32 insertions(+), 32 deletions(-) diff --git a/python/cudf_polars/cudf_polars/experimental/parallel.py b/python/cudf_polars/cudf_polars/experimental/parallel.py index e895d95b5a1..9a1883e9d35 100644 --- a/python/cudf_polars/cudf_polars/experimental/parallel.py +++ b/python/cudf_polars/cudf_polars/experimental/parallel.py @@ -17,7 +17,7 @@ from cudf_polars.dsl.traversal import traversal if TYPE_CHECKING: - from collections.abc import MutableMapping, Sequence + from collections.abc import Callable, MutableMapping, Sequence from cudf_polars.dsl.expr import Expr from cudf_polars.dsl.ir import IR @@ -308,7 +308,14 @@ def _(expr: Col, child_ir: IR) -> MutableMapping[Any, Any]: ## Agg ## -_AGG_SUPPORTED = ("sum",) +_AGG_SUPPORTED = ( + "min", + "max", + "first", + "last", + "sum", + "count", +) @_expr_partition_info_dispatch.register(Agg) @@ -327,36 +334,14 @@ def _(expr: Agg, child_ir: IR) -> PartitionInfo: return PartitionInfo(count=1) -def _agg_chunk( - column: Column, request: plc.aggregation.Aggregation, dtype: plc.DataType -) -> Column: - # TODO: This logic should be different than `request` in many cases - return Column( - plc.Column.from_scalar( - plc.reduce.reduce(column.obj, request, dtype), - 1, - ) - ) +def _apply_op(op: Callable, column: Column): + return op(column) def _agg_combine(columns: Sequence[Column]) -> Column: return Column(plc.concatenate.concatenate([col.obj for col in columns])) -def _agg_finalize( - column: Column, - request: plc.aggregation.Aggregation, - dtype: plc.DataType, -) -> Column: - # TODO: This logic should be different than `request` in many cases - return Column( - plc.Column.from_scalar( - plc.reduce.reduce(column.obj, request, dtype), - 1, - ) - ) - - @generate_expr_tasks.register(Agg) def _(expr: Agg, child_ir: IR) -> MutableMapping[Any, Any]: if expr.name not in _AGG_SUPPORTED: @@ -368,24 +353,39 @@ def _(expr: Agg, child_ir: IR) -> MutableMapping[Any, Any]: child_key = get_key_name(child) child_dsk = generate_expr_tasks(child, child_ir) + # Check for simple case + # TODO: Avoid generating entire child_dsk graph? + if expr.name in ("first", "last"): + if expr.name == "last": + index = npartitions_in - 1 + else: + index = 0 + return { + (key, 0): ( + _apply_op, + expr.op, + # Fuse with child-expr task + child_dsk[(child_key, index)], + ) + } + # Simple all-to-one reduction + # TODO: Proper tree reduction chunk_key = f"chunk-{key}" combine_key = f"concat-{key}" graph: MutableMapping[tuple[str, int], Any] = { (chunk_key, i): ( - _agg_chunk, + _apply_op, + expr.op, # Fuse with child-expr task child_dsk[(child_key, i)], - expr.request, - expr.dtype, ) for i in range(npartitions_in) } graph[(combine_key, 0)] = (_agg_combine, list(graph.keys())) graph[(key, 0)] = ( - _agg_finalize, + _apply_op, + expr.op, (combine_key, 0), - expr.request, - expr.dtype, ) return graph From 7363d917c5be3f89e33a0989605f1308e5d94dde Mon Sep 17 00:00:00 2001 From: rjzamora Date: Sat, 9 Nov 2024 19:22:34 -0800 Subject: [PATCH 06/40] cleanup fundamental bugs --- .../cudf_polars/containers/dataframe.py | 2 +- python/cudf_polars/cudf_polars/dsl/ir.py | 15 +- .../cudf_polars/cudf_polars/dsl/nodebase.py | 17 +- .../cudf_polars/experimental/parallel.py | 280 +++++++++++++----- 4 files changed, 223 insertions(+), 91 deletions(-) diff --git a/python/cudf_polars/cudf_polars/containers/dataframe.py b/python/cudf_polars/cudf_polars/containers/dataframe.py index f8ee6911931..00b40af5d49 100644 --- a/python/cudf_polars/cudf_polars/containers/dataframe.py +++ b/python/cudf_polars/cudf_polars/containers/dataframe.py @@ -227,7 +227,7 @@ def filter(self, mask: Column) -> Self: return type(self).from_table(table, self.column_names).sorted_like(self) @classmethod - def concat(cls, dfs: list[Self]) -> Self: + def concat(cls, dfs: Sequence[Self]) -> Self: """Concatenate a list of dataframes.""" assert len(dfs), "Cannot concatenate an empty list" if len(dfs) == 1: diff --git a/python/cudf_polars/cudf_polars/dsl/ir.py b/python/cudf_polars/cudf_polars/dsl/ir.py index 540661c1246..1dedf3e0c1d 100644 --- a/python/cudf_polars/cudf_polars/dsl/ir.py +++ b/python/cudf_polars/cudf_polars/dsl/ir.py @@ -36,6 +36,7 @@ from collections.abc import Callable, Hashable, MutableMapping, Sequence from typing import Literal + from cudf_polars.dsl.nodebase import PartitionInfo from cudf_polars.typing import Schema @@ -127,12 +128,13 @@ def broadcast(*columns: Column, target_length: int | None = None) -> list[Column class IR(Node["IR"]): """Abstract plan node, representing an unevaluated dataframe.""" - __slots__ = ("schema", "_non_child_args") + __slots__ = ("schema", "_non_child_args", "_parts_info") # This annotation is needed because of https://github.com/python/mypy/issues/17981 _non_child: ClassVar[tuple[str, ...]] = ("schema",) # Concrete classes should set this up with the arguments that will # be passed to do_evaluate. _non_child_args: tuple[Any, ...] + _parts_info: PartitionInfo schema: Schema """Mapping from column names to their data types.""" @@ -209,6 +211,17 @@ def evaluate(self, *, cache: MutableMapping[int, DataFrame]) -> DataFrame: *(child.evaluate(cache=cache) for child in self.children), ) + @property + def parts(self) -> PartitionInfo: + """Return Partitioning information for this IR node.""" + try: + return self._parts_info + except AttributeError: + from cudf_polars.experimental.parallel import ir_parts_info + + self._parts_info = ir_parts_info(self) + return self._parts_info + class PythonScan(IR): """Representation of input from a python function.""" diff --git a/python/cudf_polars/cudf_polars/dsl/nodebase.py b/python/cudf_polars/cudf_polars/dsl/nodebase.py index 319f247e25a..c734af916e0 100644 --- a/python/cudf_polars/cudf_polars/dsl/nodebase.py +++ b/python/cudf_polars/cudf_polars/dsl/nodebase.py @@ -47,10 +47,9 @@ class Node(Generic[T]): *children).`` """ - __slots__ = ("_hash_value", "_repr_value", "_parts_info", "children") + __slots__ = ("_hash_value", "_repr_value", "children") _hash_value: int _repr_value: str - _parts_info: PartitionInfo children: tuple[T, ...] _non_child: ClassVar[tuple[str, ...]] = () @@ -164,17 +163,3 @@ def __repr__(self) -> str: args = ", ".join(f"{arg!r}" for arg in self._ctor_arguments(self.children)) self._repr_value = f"{type(self).__name__}({args})" return self._repr_value - - @property - def parts(self) -> PartitionInfo: - """Return Partitioning information for this node.""" - try: - return self._parts_info - except AttributeError: - self._parts_info = self._get_partition_info() - return self._parts_info - - def _get_partition_info(self) -> PartitionInfo: - from cudf_polars.experimental.parallel import _ir_partition_info_dispatch - - return _ir_partition_info_dispatch(self) diff --git a/python/cudf_polars/cudf_polars/experimental/parallel.py b/python/cudf_polars/cudf_polars/experimental/parallel.py index 9a1883e9d35..e37caed2108 100644 --- a/python/cudf_polars/cudf_polars/experimental/parallel.py +++ b/python/cudf_polars/cudf_polars/experimental/parallel.py @@ -5,19 +5,19 @@ from __future__ import annotations import warnings -from functools import singledispatch +from functools import partial, singledispatch from typing import TYPE_CHECKING, Any import pylibcudf as plc from cudf_polars.containers import Column, DataFrame -from cudf_polars.dsl.expr import Agg, Col, NamedExpr +from cudf_polars.dsl.expr import Agg, BinOp, Col, NamedExpr from cudf_polars.dsl.ir import Scan, Select, broadcast from cudf_polars.dsl.nodebase import PartitionInfo from cudf_polars.dsl.traversal import traversal if TYPE_CHECKING: - from collections.abc import Callable, MutableMapping, Sequence + from collections.abc import Callable, Mapping, MutableMapping, Sequence from cudf_polars.dsl.expr import Expr from cudf_polars.dsl.ir import IR @@ -32,37 +32,40 @@ def get_key_name(node: Node | NamedExpr) -> str: @singledispatch -def _ir_partition_info_dispatch(node: Node) -> PartitionInfo: +def ir_parts_info(ir: IR) -> PartitionInfo: + """Return the partitioning info for an IR node.""" count = 1 - if node.children: - count = max(child.parts.count for child in node.children) + if ir.children: + count = max(child.parts.count for child in ir.children) if count > 1: warnings.warn( - f"Multi-partition support is not implemented for {type(node)}. " + f"Multi-partition support is not implemented for {type(ir)}. " f"Partitions will be concatenated. Expect poor performance.", - stacklevel=1, + stacklevel=2, ) return PartitionInfo(count=1) @singledispatch -def _expr_partition_info_dispatch( - expr: Expr | NamedExpr, child_ir: IR -) -> PartitionInfo: - # The default partitioning of an Expr node depends - # on the child Expr nodes or the child IR node it - # is operating on (for leaf Exprs). +def expr_parts_info(expr: Expr | NamedExpr, child_ir: IR) -> PartitionInfo: + """ + Return the partitioning info for an Expr. + + Since the partitioning of a leaf Expr depends on + the child IR node, a `child_ir` positional argument + is also required. + """ if isinstance(expr, NamedExpr): - expr = expr.value + return expr_parts_info(expr.value, child_ir) if expr.children: - count = max([child.parts.count for child in expr.children]) + count = max([expr_parts_info(child, child_ir).count for child in expr.children]) else: count = child_ir.parts.count if count > 1: warnings.warn( f"Multi-partition support is not implemented for {type(expr)}. " f"Partitions will be concatenated. Expect poor performance.", - stacklevel=1, + stacklevel=2, ) return PartitionInfo(count=1) @@ -153,6 +156,37 @@ def evaluate_dask(ir: IR) -> DataFrame: return get(graph, key) +## +## Node-specific Dispatch Logic +## + + +## +## NamedExpr +## + + +def _rename_column(column: Column, name: str): + return column.rename(name) + + +@generate_expr_tasks.register(NamedExpr) +def _(expr: NamedExpr, child_ir: IR) -> MutableMapping[Any, Any]: + # Special case: The graph of a NamedExpr + # will simply rename the column returned + # by its value attribute. + graph = generate_expr_tasks(expr.value, child_ir) + named_expr_key_name = get_key_name(expr) + expr_key_name = get_key_name(expr.value) + for i in range(expr_parts_info(expr.value, child_ir).count): + graph[(named_expr_key_name, i)] = ( + _rename_column, + graph.pop((expr_key_name, i)), + expr.name, + ) + return graph + + ## ## Scan ## @@ -160,7 +194,7 @@ def evaluate_dask(ir: IR) -> DataFrame: _SCAN_SUPPORTED = ("parquet",) -@_ir_partition_info_dispatch.register(Scan) +@ir_parts_info.register(Scan) def _(ir: Scan) -> PartitionInfo: if ir.typ in _SCAN_SUPPORTED: return PartitionInfo(count=len(ir.paths)) @@ -203,19 +237,17 @@ def _(ir: Scan) -> MutableMapping[Any, Any]: ## -@_ir_partition_info_dispatch.register(Select) +@ir_parts_info.register(Select) def _(ir: Select) -> PartitionInfo: # Partitioning depends on the expression df = ir.children[0] - column_partition_counts = [ - _expr_partition_info_dispatch(expr, df).count for expr in ir.exprs - ] + column_partition_counts = [expr_parts_info(expr, df).count for expr in ir.exprs] count = max(column_partition_counts) if count > 1: warnings.warn( f"Multi-partition support is not implemented for {type(ir)}. " f"Partitions will be concatenated. Expect poor performance.", - stacklevel=1, + stacklevel=2, ) return PartitionInfo(count=1) @@ -250,40 +282,12 @@ def _(ir: Select) -> MutableMapping[Any, Any]: return graph -## -## NamedExpr -## - - -@_expr_partition_info_dispatch.register(NamedExpr) -def _(expr: NamedExpr, child_ir: IR) -> PartitionInfo: - return _expr_partition_info_dispatch(expr.value, child_ir) - - -def _rename_column(column: Column, name: str): - return column.rename(name) - - -@generate_expr_tasks.register(NamedExpr) -def _(expr: NamedExpr, child_ir: IR) -> MutableMapping[Any, Any]: - graph = generate_expr_tasks(expr.value, child_ir) - named_expr_key_name = get_key_name(expr) - expr_key_name = get_key_name(expr.value) - for i in range(expr.value.parts.count): - graph[(named_expr_key_name, i)] = ( - _rename_column, - graph.pop((expr_key_name, i)), - expr.name, - ) - return graph - - ## ## Col ## -@_expr_partition_info_dispatch.register(Col) +@expr_parts_info.register(Col) def _(expr: Col, child_ir: IR) -> PartitionInfo: assert not expr.children count = child_ir.parts.count @@ -304,6 +308,57 @@ def _(expr: Col, child_ir: IR) -> MutableMapping[Any, Any]: } +## +## BinOp +## + + +@expr_parts_info.register(BinOp) +def _(expr: BinOp, child_ir: IR) -> PartitionInfo: + counts = {expr_parts_info(child, child_ir).count for child in expr.children} + if len(counts) != 1: + raise NotImplementedError("Mismatched partition counts.") + count = counts.pop() + return PartitionInfo(count=count) + + +def _binop(left: Column, right: Column, op: Callable, dtype: plc.DataType) -> Column: + lop = left.obj + rop = right.obj + if left.obj.size() != right.obj.size(): + if left.is_scalar: + lop = left.obj_scalar + elif right.is_scalar: + rop = right.obj_scalar + return Column( + plc.binaryop.binary_operation(lop, rop, op, dtype), + ) + + +@generate_expr_tasks.register(BinOp) +def _(expr: BinOp, child_ir: IR) -> MutableMapping[Any, Any]: + name = get_key_name(expr) + left = expr.children[0] + left_name = get_key_name(left) + left_graph = generate_expr_tasks(left, child_ir) + right = expr.children[1] + right_name = get_key_name(right) + right_graph = generate_expr_tasks(right, child_ir) + graph = { + (name, i): ( + _binop, + left_graph.pop((left_name, i)), + right_graph.pop((right_name, i)), + expr.op, + expr.dtype, + ) + for i in range(expr_parts_info(left, child_ir).count) + } + graph.update(left_graph) + graph.update(right_graph) + return graph + + ## ## Agg ## @@ -315,31 +370,65 @@ def _(expr: Col, child_ir: IR) -> MutableMapping[Any, Any]: "last", "sum", "count", + "mean", ) -@_expr_partition_info_dispatch.register(Agg) +@expr_parts_info.register(Agg) def _(expr: Agg, child_ir: IR) -> PartitionInfo: if expr.children: - count = max([child.parts.count for child in expr.children]) + count = max([expr_parts_info(child, child_ir).count for child in expr.children]) else: count = child_ir.parts.count if count > 1 and expr.name not in _AGG_SUPPORTED: + assert 0 == 1 # Only support sum reductions for now. warnings.warn( f"Multi-partition support is not implemented for {type(expr)}. " f"Partitions will be concatenated. Expect poor performance.", - stacklevel=1, + stacklevel=2, ) return PartitionInfo(count=1) -def _apply_op(op: Callable, column: Column): +def _tree_agg( + op: Callable, + input: Column | Sequence[Column], +) -> Column: + if isinstance(input, Column): + column = input + elif len(input) == 1: + column = input[0] + else: + column = Column(plc.concatenate.concatenate([col.obj for col in input])) return op(column) -def _agg_combine(columns: Sequence[Column]) -> Column: - return Column(plc.concatenate.concatenate([col.obj for col in columns])) +def _tree_agg_multi( + ops: Mapping[str, Callable], + input: Column | Sequence[DataFrame], +) -> DataFrame: + if isinstance(input, Column): + columns = [op(input).rename(name) for name, op in ops.items()] + else: + df = DataFrame.concat(input) + columns = [ + op(df.select_columns({name})[0]).rename(name) for name, op in ops.items() + ] + return DataFrame(columns) + + +def _finalize_mean(df: DataFrame, dtype: plc.DataType) -> Column: + _sum = df.select_columns({"sum"})[0] + _count = df.select_columns({"count"})[0] + return Column( + plc.binaryop.binary_operation( + _sum.obj, + _count.obj, + plc.binaryop.BinaryOperator.DIV, + dtype, + ) + ) @generate_expr_tasks.register(Agg) @@ -348,11 +437,20 @@ def _(expr: Agg, child_ir: IR) -> MutableMapping[Any, Any]: raise NotImplementedError(f"Cannot generate tasks for {expr}.") child = expr.children[0] - npartitions_in = child.parts.count + npartitions_in = expr_parts_info(child, child_ir).count key = get_key_name(expr) child_key = get_key_name(child) child_dsk = generate_expr_tasks(child, child_ir) + # Single input-partition shortcut + if npartitions_in == 1: + return { + (key, 0): ( + expr.op, + child_dsk.pop((child_key, 0)), + ) + } + # Check for simple case # TODO: Avoid generating entire child_dsk graph? if expr.name in ("first", "last"): @@ -362,30 +460,66 @@ def _(expr: Agg, child_ir: IR) -> MutableMapping[Any, Any]: index = 0 return { (key, 0): ( - _apply_op, + _tree_agg, expr.op, # Fuse with child-expr task - child_dsk[(child_key, index)], + child_dsk.pop((child_key, index)), ) } + # Tree func is different for "complex" aggs + # (Probably a better way to generalize this) + chunk_func: Callable + tree_func: Callable + finalize: Callable | None = None + chunk_op: Callable | MutableMapping[str, Callable] + tree_op: Callable | MutableMapping[str, Callable] + if expr.name == "mean": + chunk_func = tree_func = _tree_agg_multi + finalize = _finalize_mean + chunk_op = { + "sum": partial(expr._reduce, request=plc.aggregation.sum()), + "count": expr._count, + } + tree_op = { + "sum": partial(expr._reduce, request=plc.aggregation.sum()), + "count": partial(expr._reduce, request=plc.aggregation.sum()), + } + else: + chunk_func = tree_func = _tree_agg + if expr.name == "count": + # After the initial count operations, + # we just want to apply a sum aggregation + chunk_op = expr.op + tree_op = partial(expr._reduce, request=plc.aggregation.sum()) + else: + chunk_op = expr.op + tree_op = expr.op + # Simple all-to-one reduction - # TODO: Proper tree reduction - chunk_key = f"chunk-{key}" - combine_key = f"concat-{key}" + # TODO: Add proper tree reduction + tree_key: str = f"tree-{key}" + combine_key: str = f"combine-{key}" graph: MutableMapping[tuple[str, int], Any] = { - (chunk_key, i): ( - _apply_op, - expr.op, + (tree_key, i): ( + chunk_func, + chunk_op, # Fuse with child-expr task - child_dsk[(child_key, i)], + child_dsk.pop((child_key, i)), ) for i in range(npartitions_in) } - graph[(combine_key, 0)] = (_agg_combine, list(graph.keys())) - graph[(key, 0)] = ( - _apply_op, - expr.op, - (combine_key, 0), + graph[(combine_key, 0)] = ( + tree_func, + tree_op, + list(graph.keys()), ) + if finalize: + graph[(key, 0)] = ( + finalize, + graph.pop((combine_key, 0)), + expr.dtype, + ) + else: + graph[(key, 0)] = graph.pop((combine_key, 0)) return graph From 58ee5f412cb766fd8bc2e1d073ff6578ac04cac4 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Sat, 9 Nov 2024 20:22:26 -0800 Subject: [PATCH 07/40] move PartitionInfo --- python/cudf_polars/cudf_polars/dsl/ir.py | 14 +++++++++- .../cudf_polars/cudf_polars/dsl/nodebase.py | 13 --------- .../cudf_polars/experimental/parallel.py | 27 +++++++------------ 3 files changed, 23 insertions(+), 31 deletions(-) diff --git a/python/cudf_polars/cudf_polars/dsl/ir.py b/python/cudf_polars/cudf_polars/dsl/ir.py index 1dedf3e0c1d..0b00550ecb4 100644 --- a/python/cudf_polars/cudf_polars/dsl/ir.py +++ b/python/cudf_polars/cudf_polars/dsl/ir.py @@ -36,7 +36,6 @@ from collections.abc import Callable, Hashable, MutableMapping, Sequence from typing import Literal - from cudf_polars.dsl.nodebase import PartitionInfo from cudf_polars.typing import Schema @@ -125,6 +124,19 @@ def broadcast(*columns: Column, target_length: int | None = None) -> list[Column ] +class PartitionInfo: + """ + Partitioning information. + + This class only tracks the partition count (for now). + """ + + __slots__ = ("count",) + + def __init__(self, count: int): + self.count = count + + class IR(Node["IR"]): """Abstract plan node, representing an unevaluated dataframe.""" diff --git a/python/cudf_polars/cudf_polars/dsl/nodebase.py b/python/cudf_polars/cudf_polars/dsl/nodebase.py index c734af916e0..228d300f467 100644 --- a/python/cudf_polars/cudf_polars/dsl/nodebase.py +++ b/python/cudf_polars/cudf_polars/dsl/nodebase.py @@ -18,19 +18,6 @@ T = TypeVar("T", bound="Node[Any]") -class PartitionInfo: - """ - Partitioning information. - - This class only tracks the partition count (for now). - """ - - __slots__ = ("count",) - - def __init__(self, count: int): - self.count = count - - class Node(Generic[T]): """ An abstract node type. diff --git a/python/cudf_polars/cudf_polars/experimental/parallel.py b/python/cudf_polars/cudf_polars/experimental/parallel.py index e37caed2108..3445c434778 100644 --- a/python/cudf_polars/cudf_polars/experimental/parallel.py +++ b/python/cudf_polars/cudf_polars/experimental/parallel.py @@ -12,8 +12,7 @@ from cudf_polars.containers import Column, DataFrame from cudf_polars.dsl.expr import Agg, BinOp, Col, NamedExpr -from cudf_polars.dsl.ir import Scan, Select, broadcast -from cudf_polars.dsl.nodebase import PartitionInfo +from cudf_polars.dsl.ir import PartitionInfo, Scan, Select, broadcast from cudf_polars.dsl.traversal import traversal if TYPE_CHECKING: @@ -41,7 +40,7 @@ def ir_parts_info(ir: IR) -> PartitionInfo: warnings.warn( f"Multi-partition support is not implemented for {type(ir)}. " f"Partitions will be concatenated. Expect poor performance.", - stacklevel=2, + stacklevel=1, ) return PartitionInfo(count=1) @@ -63,9 +62,9 @@ def expr_parts_info(expr: Expr | NamedExpr, child_ir: IR) -> PartitionInfo: count = child_ir.parts.count if count > 1: warnings.warn( - f"Multi-partition support is not implemented for {type(expr)}. " - f"Partitions will be concatenated. Expect poor performance.", - stacklevel=2, + f"Multi-partition support is not implemented for {type(expr)} " + "Partitions will be concatenated. Expect poor performance.", + stacklevel=1, ) return PartitionInfo(count=1) @@ -243,13 +242,7 @@ def _(ir: Select) -> PartitionInfo: df = ir.children[0] column_partition_counts = [expr_parts_info(expr, df).count for expr in ir.exprs] count = max(column_partition_counts) - if count > 1: - warnings.warn( - f"Multi-partition support is not implemented for {type(ir)}. " - f"Partitions will be concatenated. Expect poor performance.", - stacklevel=2, - ) - return PartitionInfo(count=1) + return PartitionInfo(count=count) def _select(columns: list[Column], should_broadcast: bool): # noqa: FBT001 @@ -381,12 +374,12 @@ def _(expr: Agg, child_ir: IR) -> PartitionInfo: else: count = child_ir.parts.count if count > 1 and expr.name not in _AGG_SUPPORTED: - assert 0 == 1 # Only support sum reductions for now. warnings.warn( - f"Multi-partition support is not implemented for {type(expr)}. " - f"Partitions will be concatenated. Expect poor performance.", - stacklevel=2, + f"Multi-partition support is not implemented for {type(expr)} " + f"with expr.name={expr.name}. Partitions will be concatenated. " + f"Expect poor performance.", + stacklevel=1, ) return PartitionInfo(count=1) From ecc51ef588b18f82f380fde4f985801d1dc9fb2f Mon Sep 17 00:00:00 2001 From: rjzamora Date: Sat, 9 Nov 2024 20:34:25 -0800 Subject: [PATCH 08/40] add Literal --- .../cudf_polars/experimental/parallel.py | 18 +++++++++++++++++- 1 file changed, 17 insertions(+), 1 deletion(-) diff --git a/python/cudf_polars/cudf_polars/experimental/parallel.py b/python/cudf_polars/cudf_polars/experimental/parallel.py index 3445c434778..2b053464894 100644 --- a/python/cudf_polars/cudf_polars/experimental/parallel.py +++ b/python/cudf_polars/cudf_polars/experimental/parallel.py @@ -11,7 +11,7 @@ import pylibcudf as plc from cudf_polars.containers import Column, DataFrame -from cudf_polars.dsl.expr import Agg, BinOp, Col, NamedExpr +from cudf_polars.dsl.expr import Agg, BinOp, Col, Literal, NamedExpr from cudf_polars.dsl.ir import PartitionInfo, Scan, Select, broadcast from cudf_polars.dsl.traversal import traversal @@ -352,6 +352,22 @@ def _(expr: BinOp, child_ir: IR) -> MutableMapping[Any, Any]: return graph +## +## Literal +## + + +@expr_parts_info.register(Literal) +def _(expr: Literal, child_ir: IR) -> PartitionInfo: + return PartitionInfo(count=1) + + +@generate_expr_tasks.register(Literal) +def _(expr: Literal, child_ir: IR) -> MutableMapping[Any, Any]: + value = Column(plc.Column.from_scalar(plc.interop.from_arrow(expr.value), 1)) + return {(get_key_name(expr), 0): value} + + ## ## Agg ## From fb2d6bf7ca0797573bece58c2af52c6bf9fb404e Mon Sep 17 00:00:00 2001 From: rjzamora Date: Tue, 12 Nov 2024 07:51:40 -0800 Subject: [PATCH 09/40] add lower_ir_graph --- .../cudf_polars/containers/dataframe.py | 9 ----- .../cudf_polars/experimental/parallel.py | 33 +++++++++++++++---- 2 files changed, 26 insertions(+), 16 deletions(-) diff --git a/python/cudf_polars/cudf_polars/containers/dataframe.py b/python/cudf_polars/cudf_polars/containers/dataframe.py index 00b40af5d49..08bc9d0ea3f 100644 --- a/python/cudf_polars/cudf_polars/containers/dataframe.py +++ b/python/cudf_polars/cudf_polars/containers/dataframe.py @@ -226,15 +226,6 @@ def filter(self, mask: Column) -> Self: table = plc.stream_compaction.apply_boolean_mask(self.table, mask.obj) return type(self).from_table(table, self.column_names).sorted_like(self) - @classmethod - def concat(cls, dfs: Sequence[Self]) -> Self: - """Concatenate a list of dataframes.""" - assert len(dfs), "Cannot concatenate an empty list" - if len(dfs) == 1: - return dfs[0] - table = plc.concatenate.concatenate([df.table for df in dfs]) - return cls.from_table(table, dfs[0].column_names) - def slice(self, zlice: tuple[int, int] | None) -> Self: """ Slice a dataframe. diff --git a/python/cudf_polars/cudf_polars/experimental/parallel.py b/python/cudf_polars/cudf_polars/experimental/parallel.py index 2b053464894..11558579596 100644 --- a/python/cudf_polars/cudf_polars/experimental/parallel.py +++ b/python/cudf_polars/cudf_polars/experimental/parallel.py @@ -12,7 +12,7 @@ from cudf_polars.containers import Column, DataFrame from cudf_polars.dsl.expr import Agg, BinOp, Col, Literal, NamedExpr -from cudf_polars.dsl.ir import PartitionInfo, Scan, Select, broadcast +from cudf_polars.dsl.ir import PartitionInfo, Scan, Select, Union, broadcast from cudf_polars.dsl.traversal import traversal if TYPE_CHECKING: @@ -23,6 +23,11 @@ from cudf_polars.dsl.nodebase import Node +def _concat(dfs: Sequence[DataFrame]) -> DataFrame: + # Concatenate a sequence of DataFrames vertically + return Union.do_evaluate(None, *dfs) + + def get_key_name(node: Node | NamedExpr) -> str: """Generate the key name for a Node.""" if isinstance(node, NamedExpr): @@ -30,6 +35,21 @@ def get_key_name(node: Node | NamedExpr) -> str: return f"{type(node).__name__.lower()}-{hash(node)}" +@singledispatch +def lower_ir_node(ir: IR, rec) -> IR: + """Rewrite an IR node with proper partitioning.""" + # Return same node by default + return ir + + +def lower_ir_graph(ir: IR) -> IR: + """Rewrite an IR graph with proper partitioning.""" + from cudf_polars.dsl.traversal import CachingVisitor + + mapper = CachingVisitor(lower_ir_node) + return mapper(ir) + + @singledispatch def ir_parts_info(ir: IR) -> PartitionInfo: """Return the partitioning info for an IR node.""" @@ -82,7 +102,7 @@ def _default_generate_ir_tasks(ir: IR) -> MutableMapping[Any, Any]: if child.parts.count > 1: child_names.append("concat-" + child_name_in) graph[(child_names[-1], 0)] = ( - DataFrame.concat, + _concat, [(child_name_in, i) for i in range(child.parts.count)], ) else: @@ -123,10 +143,9 @@ def generate_expr_tasks( raise NotImplementedError(f"Cannot generate tasks for {expr}.") -def task_graph(ir: IR) -> tuple[MutableMapping[str, Any], str]: +def task_graph(_ir: IR) -> tuple[MutableMapping[str, Any], str]: """Construct a Dask-compatible task graph.""" - # NOTE: It may be necessary to add an optimization - # pass here to "rewrite" the single-partition IR graph. + ir: IR = lower_ir_graph(_ir) graph = { k: v @@ -138,7 +157,7 @@ def task_graph(ir: IR) -> tuple[MutableMapping[str, Any], str]: key_name = get_key_name(ir) if ir.parts.count > 1: graph[key_name] = ( - DataFrame.concat, + _concat, [(key_name, i) for i in range(ir.parts.count)], ) else: @@ -420,7 +439,7 @@ def _tree_agg_multi( if isinstance(input, Column): columns = [op(input).rename(name) for name, op in ops.items()] else: - df = DataFrame.concat(input) + df = _concat(input) columns = [ op(df.select_columns({name})[0]).rename(name) for name, op in ops.items() ] From 6e66998d6f281a5c197e445ffc8f7d9e98f89464 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Tue, 12 Nov 2024 09:14:41 -0800 Subject: [PATCH 10/40] strip out most exploratory logic --- .../cudf_polars/experimental/parallel.py | 503 ++---------------- 1 file changed, 35 insertions(+), 468 deletions(-) diff --git a/python/cudf_polars/cudf_polars/experimental/parallel.py b/python/cudf_polars/cudf_polars/experimental/parallel.py index 11558579596..540f53e1a27 100644 --- a/python/cudf_polars/cudf_polars/experimental/parallel.py +++ b/python/cudf_polars/cudf_polars/experimental/parallel.py @@ -4,30 +4,21 @@ from __future__ import annotations -import warnings -from functools import partial, singledispatch +from functools import singledispatch from typing import TYPE_CHECKING, Any -import pylibcudf as plc - -from cudf_polars.containers import Column, DataFrame -from cudf_polars.dsl.expr import Agg, BinOp, Col, Literal, NamedExpr -from cudf_polars.dsl.ir import PartitionInfo, Scan, Select, Union, broadcast +from cudf_polars.dsl.expr import NamedExpr +from cudf_polars.dsl.ir import PartitionInfo from cudf_polars.dsl.traversal import traversal if TYPE_CHECKING: - from collections.abc import Callable, Mapping, MutableMapping, Sequence + from collections.abc import MutableMapping - from cudf_polars.dsl.expr import Expr + from cudf_polars.containers import DataFrame from cudf_polars.dsl.ir import IR from cudf_polars.dsl.nodebase import Node -def _concat(dfs: Sequence[DataFrame]) -> DataFrame: - # Concatenate a sequence of DataFrames vertically - return Union.do_evaluate(None, *dfs) - - def get_key_name(node: Node | NamedExpr) -> str: """Generate the key name for a Node.""" if isinstance(node, NamedExpr): @@ -50,73 +41,49 @@ def lower_ir_graph(ir: IR) -> IR: return mapper(ir) -@singledispatch -def ir_parts_info(ir: IR) -> PartitionInfo: - """Return the partitioning info for an IR node.""" +def _default_ir_parts_info(ir: IR) -> PartitionInfo: + # Single-partition default behavior. + # This is used by `ir_parts_info` for + # all unregistered IR sub-types. count = 1 if ir.children: count = max(child.parts.count for child in ir.children) if count > 1: - warnings.warn( - f"Multi-partition support is not implemented for {type(ir)}. " - f"Partitions will be concatenated. Expect poor performance.", - stacklevel=1, + raise NotImplementedError( + f"Class {type(ir)} does not support multiple partitions." ) - return PartitionInfo(count=1) + return PartitionInfo(count=count) @singledispatch -def expr_parts_info(expr: Expr | NamedExpr, child_ir: IR) -> PartitionInfo: - """ - Return the partitioning info for an Expr. - - Since the partitioning of a leaf Expr depends on - the child IR node, a `child_ir` positional argument - is also required. - """ - if isinstance(expr, NamedExpr): - return expr_parts_info(expr.value, child_ir) - if expr.children: - count = max([expr_parts_info(child, child_ir).count for child in expr.children]) - else: - count = child_ir.parts.count - if count > 1: - warnings.warn( - f"Multi-partition support is not implemented for {type(expr)} " - "Partitions will be concatenated. Expect poor performance.", - stacklevel=1, - ) - return PartitionInfo(count=1) +def ir_parts_info(ir: IR) -> PartitionInfo: + """Return the partitioning info for an IR node.""" + return _default_ir_parts_info(ir) -def _default_generate_ir_tasks(ir: IR) -> MutableMapping[Any, Any]: - # Single-partition default behavior - if ir.parts.count == 1: - # Start by concatenating multi-partition - # children (if necessary) - graph = {} - child_names = [] - key_name = get_key_name(ir) - for child in ir.children: - child_name_in = get_key_name(child) - if child.parts.count > 1: - child_names.append("concat-" + child_name_in) - graph[(child_names[-1], 0)] = ( - _concat, - [(child_name_in, i) for i in range(child.parts.count)], - ) - else: - child_names.append(child_name_in) +def _default_ir_tasks(ir: IR) -> MutableMapping[Any, Any]: + # Single-partition default behavior. + # This is used by `generate_ir_tasks` for + # all unregistered IR sub-types. + if ir.parts.count > 1: + raise NotImplementedError(f"Failed to generate tasks for {ir}.") + + child_names = [] + for child in ir.children: + child_names.append(get_key_name(child)) + if child.parts.count > 1: + raise NotImplementedError( + f"Failed to generate tasks for {ir} with child {child}." + ) - # Add single-partition task - graph[(key_name, 0)] = ( + key_name = get_key_name(ir) + return { + (key_name, 0): ( ir.do_evaluate, *ir._non_child_args, *((child_name, 0) for child_name in child_names), ) - return graph - - raise NotImplementedError(f"Cannot generate tasks for {ir}.") + } @singledispatch @@ -127,20 +94,7 @@ def generate_ir_tasks(ir: IR) -> MutableMapping[Any, Any]: An IR node only needs to generate the graph for the current IR logic (not including child IRs). """ - return _default_generate_ir_tasks(ir) - - -@singledispatch -def generate_expr_tasks( - expr: Expr | NamedExpr, child_ir: IR -) -> MutableMapping[Any, Any]: - """ - Generate tasks for an Expr node. - - An Expr node is responsible for constructing the full - expression graph recursively (allowing fusion). - """ - raise NotImplementedError(f"Cannot generate tasks for {expr}.") + return _default_ir_tasks(ir) def task_graph(_ir: IR) -> tuple[MutableMapping[str, Any], str]: @@ -152,16 +106,8 @@ def task_graph(_ir: IR) -> tuple[MutableMapping[str, Any], str]: for layer in [generate_ir_tasks(n) for n in traversal(ir)] for k, v in layer.items() } - - # Add task to reduce output partitions key_name = get_key_name(ir) - if ir.parts.count > 1: - graph[key_name] = ( - _concat, - [(key_name, i) for i in range(ir.parts.count)], - ) - else: - graph[key_name] = (key_name, 0) + graph[key_name] = (key_name, 0) return graph, key_name @@ -172,382 +118,3 @@ def evaluate_dask(ir: IR) -> DataFrame: graph, key = task_graph(ir) return get(graph, key) - - -## -## Node-specific Dispatch Logic -## - - -## -## NamedExpr -## - - -def _rename_column(column: Column, name: str): - return column.rename(name) - - -@generate_expr_tasks.register(NamedExpr) -def _(expr: NamedExpr, child_ir: IR) -> MutableMapping[Any, Any]: - # Special case: The graph of a NamedExpr - # will simply rename the column returned - # by its value attribute. - graph = generate_expr_tasks(expr.value, child_ir) - named_expr_key_name = get_key_name(expr) - expr_key_name = get_key_name(expr.value) - for i in range(expr_parts_info(expr.value, child_ir).count): - graph[(named_expr_key_name, i)] = ( - _rename_column, - graph.pop((expr_key_name, i)), - expr.name, - ) - return graph - - -## -## Scan -## - -_SCAN_SUPPORTED = ("parquet",) - - -@ir_parts_info.register(Scan) -def _(ir: Scan) -> PartitionInfo: - if ir.typ in _SCAN_SUPPORTED: - return PartitionInfo(count=len(ir.paths)) - return PartitionInfo(count=1) - - -@generate_ir_tasks.register(Scan) -def _(ir: Scan) -> MutableMapping[Any, Any]: - key_name = get_key_name(ir) - if ir.parts.count == 1: - return { - (key_name, 0): ( - ir.do_evaluate, - *ir._non_child_args, - ) - } - else: - # Only support 1:1 mapping between - # paths and partitions for now - assert len(ir.paths) == ir.parts.count - return { - (key_name, i): ( - ir.do_evaluate, - ir.schema, - ir.typ, - ir.reader_options, - [path], - ir.with_columns, - ir.skip_rows, - ir.n_rows, - ir.row_index, - ir.predicate, - ) - for i, path in enumerate(ir.paths) - } - - -## -## Select -## - - -@ir_parts_info.register(Select) -def _(ir: Select) -> PartitionInfo: - # Partitioning depends on the expression - df = ir.children[0] - column_partition_counts = [expr_parts_info(expr, df).count for expr in ir.exprs] - count = max(column_partition_counts) - return PartitionInfo(count=count) - - -def _select(columns: list[Column], should_broadcast: bool): # noqa: FBT001 - if should_broadcast: - columns = broadcast(*columns) - return DataFrame(columns) - - -@generate_ir_tasks.register(Select) -def _(ir: Select) -> MutableMapping[Any, Any]: - try: - expr_graphs = [generate_expr_tasks(e, ir.children[0]) for e in ir.exprs] - key_name = get_key_name(ir) - expr_keys = [get_key_name(e) for e in ir.exprs] - graph = { - (key_name, i): ( - _select, - [(c_key, i) for c_key in expr_keys], - ir.should_broadcast, - ) - for i in range(ir.parts.count) - } - for expr_graph in expr_graphs: - graph.update(expr_graph) - except NotImplementedError as err: - if ir.parts.count == 1: - return _default_generate_ir_tasks(ir) - raise NotImplementedError("Not supported.") from err - else: - return graph - - -## -## Col -## - - -@expr_parts_info.register(Col) -def _(expr: Col, child_ir: IR) -> PartitionInfo: - assert not expr.children - count = child_ir.parts.count - return PartitionInfo(count=count) - - -def _get_col(df: DataFrame, name: str) -> Column: - return df.column_map[name].rename(None) - - -@generate_expr_tasks.register(Col) -def _(expr: Col, child_ir: IR) -> MutableMapping[Any, Any]: - key_name = get_key_name(expr) - child_name = get_key_name(child_ir) - return { - (key_name, i): (_get_col, (child_name, i), expr.name) - for i in range(child_ir.parts.count) - } - - -## -## BinOp -## - - -@expr_parts_info.register(BinOp) -def _(expr: BinOp, child_ir: IR) -> PartitionInfo: - counts = {expr_parts_info(child, child_ir).count for child in expr.children} - if len(counts) != 1: - raise NotImplementedError("Mismatched partition counts.") - count = counts.pop() - return PartitionInfo(count=count) - - -def _binop(left: Column, right: Column, op: Callable, dtype: plc.DataType) -> Column: - lop = left.obj - rop = right.obj - if left.obj.size() != right.obj.size(): - if left.is_scalar: - lop = left.obj_scalar - elif right.is_scalar: - rop = right.obj_scalar - return Column( - plc.binaryop.binary_operation(lop, rop, op, dtype), - ) - - -@generate_expr_tasks.register(BinOp) -def _(expr: BinOp, child_ir: IR) -> MutableMapping[Any, Any]: - name = get_key_name(expr) - left = expr.children[0] - left_name = get_key_name(left) - left_graph = generate_expr_tasks(left, child_ir) - right = expr.children[1] - right_name = get_key_name(right) - right_graph = generate_expr_tasks(right, child_ir) - graph = { - (name, i): ( - _binop, - left_graph.pop((left_name, i)), - right_graph.pop((right_name, i)), - expr.op, - expr.dtype, - ) - for i in range(expr_parts_info(left, child_ir).count) - } - graph.update(left_graph) - graph.update(right_graph) - return graph - - -## -## Literal -## - - -@expr_parts_info.register(Literal) -def _(expr: Literal, child_ir: IR) -> PartitionInfo: - return PartitionInfo(count=1) - - -@generate_expr_tasks.register(Literal) -def _(expr: Literal, child_ir: IR) -> MutableMapping[Any, Any]: - value = Column(plc.Column.from_scalar(plc.interop.from_arrow(expr.value), 1)) - return {(get_key_name(expr), 0): value} - - -## -## Agg -## - -_AGG_SUPPORTED = ( - "min", - "max", - "first", - "last", - "sum", - "count", - "mean", -) - - -@expr_parts_info.register(Agg) -def _(expr: Agg, child_ir: IR) -> PartitionInfo: - if expr.children: - count = max([expr_parts_info(child, child_ir).count for child in expr.children]) - else: - count = child_ir.parts.count - if count > 1 and expr.name not in _AGG_SUPPORTED: - # Only support sum reductions for now. - warnings.warn( - f"Multi-partition support is not implemented for {type(expr)} " - f"with expr.name={expr.name}. Partitions will be concatenated. " - f"Expect poor performance.", - stacklevel=1, - ) - return PartitionInfo(count=1) - - -def _tree_agg( - op: Callable, - input: Column | Sequence[Column], -) -> Column: - if isinstance(input, Column): - column = input - elif len(input) == 1: - column = input[0] - else: - column = Column(plc.concatenate.concatenate([col.obj for col in input])) - return op(column) - - -def _tree_agg_multi( - ops: Mapping[str, Callable], - input: Column | Sequence[DataFrame], -) -> DataFrame: - if isinstance(input, Column): - columns = [op(input).rename(name) for name, op in ops.items()] - else: - df = _concat(input) - columns = [ - op(df.select_columns({name})[0]).rename(name) for name, op in ops.items() - ] - return DataFrame(columns) - - -def _finalize_mean(df: DataFrame, dtype: plc.DataType) -> Column: - _sum = df.select_columns({"sum"})[0] - _count = df.select_columns({"count"})[0] - return Column( - plc.binaryop.binary_operation( - _sum.obj, - _count.obj, - plc.binaryop.BinaryOperator.DIV, - dtype, - ) - ) - - -@generate_expr_tasks.register(Agg) -def _(expr: Agg, child_ir: IR) -> MutableMapping[Any, Any]: - if expr.name not in _AGG_SUPPORTED: - raise NotImplementedError(f"Cannot generate tasks for {expr}.") - - child = expr.children[0] - npartitions_in = expr_parts_info(child, child_ir).count - key = get_key_name(expr) - child_key = get_key_name(child) - child_dsk = generate_expr_tasks(child, child_ir) - - # Single input-partition shortcut - if npartitions_in == 1: - return { - (key, 0): ( - expr.op, - child_dsk.pop((child_key, 0)), - ) - } - - # Check for simple case - # TODO: Avoid generating entire child_dsk graph? - if expr.name in ("first", "last"): - if expr.name == "last": - index = npartitions_in - 1 - else: - index = 0 - return { - (key, 0): ( - _tree_agg, - expr.op, - # Fuse with child-expr task - child_dsk.pop((child_key, index)), - ) - } - - # Tree func is different for "complex" aggs - # (Probably a better way to generalize this) - chunk_func: Callable - tree_func: Callable - finalize: Callable | None = None - chunk_op: Callable | MutableMapping[str, Callable] - tree_op: Callable | MutableMapping[str, Callable] - if expr.name == "mean": - chunk_func = tree_func = _tree_agg_multi - finalize = _finalize_mean - chunk_op = { - "sum": partial(expr._reduce, request=plc.aggregation.sum()), - "count": expr._count, - } - tree_op = { - "sum": partial(expr._reduce, request=plc.aggregation.sum()), - "count": partial(expr._reduce, request=plc.aggregation.sum()), - } - else: - chunk_func = tree_func = _tree_agg - if expr.name == "count": - # After the initial count operations, - # we just want to apply a sum aggregation - chunk_op = expr.op - tree_op = partial(expr._reduce, request=plc.aggregation.sum()) - else: - chunk_op = expr.op - tree_op = expr.op - - # Simple all-to-one reduction - # TODO: Add proper tree reduction - tree_key: str = f"tree-{key}" - combine_key: str = f"combine-{key}" - graph: MutableMapping[tuple[str, int], Any] = { - (tree_key, i): ( - chunk_func, - chunk_op, - # Fuse with child-expr task - child_dsk.pop((child_key, i)), - ) - for i in range(npartitions_in) - } - graph[(combine_key, 0)] = ( - tree_func, - tree_op, - list(graph.keys()), - ) - if finalize: - graph[(key, 0)] = ( - finalize, - graph.pop((combine_key, 0)), - expr.dtype, - ) - else: - graph[(key, 0)] = graph.pop((combine_key, 0)) - return graph From 6886f8deb99fc3231fbbba34f812f9448cdb10ec Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Wed, 13 Nov 2024 05:49:00 -0800 Subject: [PATCH 11/40] Add basic Dask evaluate test --- .../tests/experimental/__init__.py | 6 +++++ .../tests/experimental/test_parallel.py | 24 +++++++++++++++++++ 2 files changed, 30 insertions(+) create mode 100644 python/cudf_polars/tests/experimental/__init__.py create mode 100644 python/cudf_polars/tests/experimental/test_parallel.py diff --git a/python/cudf_polars/tests/experimental/__init__.py b/python/cudf_polars/tests/experimental/__init__.py new file mode 100644 index 00000000000..4611d642f14 --- /dev/null +++ b/python/cudf_polars/tests/experimental/__init__.py @@ -0,0 +1,6 @@ +# SPDX-FileCopyrightText: Copyright (c) 2024 NVIDIA CORPORATION & AFFILIATES. +# SPDX-License-Identifier: Apache-2.0 + +from __future__ import annotations + +__all__: list[str] = [] diff --git a/python/cudf_polars/tests/experimental/test_parallel.py b/python/cudf_polars/tests/experimental/test_parallel.py new file mode 100644 index 00000000000..9ac10b21b83 --- /dev/null +++ b/python/cudf_polars/tests/experimental/test_parallel.py @@ -0,0 +1,24 @@ +# SPDX-FileCopyrightText: Copyright (c) 2024 NVIDIA CORPORATION & AFFILIATES. +# SPDX-License-Identifier: Apache-2.0 + +from __future__ import annotations + +import polars as pl +from polars.testing import assert_frame_equal + +from cudf_polars import Translator +from cudf_polars.experimental.parallel import evaluate_dask + + +def test_evaluate_dask(): + df = pl.LazyFrame({"a": [1, 2, 3], "b": [3, 4, 5], "c": [5, 6, 7], "d": [7, 9, 8]}) + + q = df.select(pl.col("a") - (pl.col("b") + pl.col("c") * 2), pl.col("d")).sort("d") + + qir = Translator(q._ldf.visit()).translate_ir() + + expected = qir.evaluate(cache={}).to_polars() + + got = evaluate_dask(qir).to_polars() + + assert_frame_equal(expected, got) From 29b2d7b174e3d1b6c361b78d50c8ddc9142c58b3 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Wed, 13 Nov 2024 06:17:24 -0800 Subject: [PATCH 12/40] Replace environment variable with new `"executor"` config --- python/cudf_polars/cudf_polars/callback.py | 19 +++++++++++++------ 1 file changed, 13 insertions(+), 6 deletions(-) diff --git a/python/cudf_polars/cudf_polars/callback.py b/python/cudf_polars/cudf_polars/callback.py index c6d4ca56de4..b013af47082 100644 --- a/python/cudf_polars/cudf_polars/callback.py +++ b/python/cudf_polars/cudf_polars/callback.py @@ -135,6 +135,7 @@ def _callback( *, device: int | None, memory_resource: int | None, + executor: str | None, ) -> pl.DataFrame: assert with_columns is None assert pyarrow_predicate is None @@ -145,13 +146,14 @@ def _callback( set_device(device), set_memory_resource(memory_resource), ): - if os.environ.get("CUDF_POLARS_DASK", "OFF").upper() == "ON": - # Use experimental Dask executor + if executor is None or executor == "cudf": + return ir.evaluate(cache={}).to_polars() + elif executor == "dask": from cudf_polars.experimental.parallel import evaluate_dask return evaluate_dask(ir).to_polars() - - return ir.evaluate(cache={}).to_polars() + else: + raise ValueError(f"Unknown executor '{executor}'") def execute_with_cudf( @@ -180,7 +182,8 @@ def execute_with_cudf( device = config.device memory_resource = config.memory_resource raise_on_fail = config.config.get("raise_on_fail", False) - if unsupported := (config.config.keys() - {"raise_on_fail"}): + executor = config.config.get("executor", None) + if unsupported := (config.config.keys() - {"raise_on_fail", "executor"}): raise ValueError( f"Engine configuration contains unsupported settings {unsupported}" ) @@ -207,7 +210,11 @@ def execute_with_cudf( else: nt.set_udf( partial( - _callback, ir, device=device, memory_resource=memory_resource + _callback, + ir, + device=device, + memory_resource=memory_resource, + executor=executor, ) ) except exception as e: From 3a68a6df165afcd1027aebf6b3890730bcb27e62 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Wed, 13 Nov 2024 06:53:45 -0800 Subject: [PATCH 13/40] Add kwarg to specify executor in `assert_gpu_result_equal` --- python/cudf_polars/cudf_polars/testing/asserts.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/python/cudf_polars/cudf_polars/testing/asserts.py b/python/cudf_polars/cudf_polars/testing/asserts.py index 2207545aa60..e975c32cf6b 100644 --- a/python/cudf_polars/cudf_polars/testing/asserts.py +++ b/python/cudf_polars/cudf_polars/testing/asserts.py @@ -33,6 +33,7 @@ def assert_gpu_result_equal( rtol: float = 1e-05, atol: float = 1e-08, categorical_as_str: bool = False, + executor: str | None = None, ) -> None: """ Assert that collection of a lazyframe on GPU produces correct results. @@ -68,6 +69,8 @@ def assert_gpu_result_equal( Absolute tolerance for float comparisons categorical_as_str Decat categoricals to strings before comparing + executor + The executor configuration to pass to `GPUEngine` Raises ------ From 8079ac02c1dec693e509fa8fa229eb0c1f77195e Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Wed, 13 Nov 2024 06:55:40 -0800 Subject: [PATCH 14/40] Add couple of Dask executor tests --- .../tests/experimental/test_parallel.py | 29 +++++++++++++++++++ 1 file changed, 29 insertions(+) diff --git a/python/cudf_polars/tests/experimental/test_parallel.py b/python/cudf_polars/tests/experimental/test_parallel.py index 9ac10b21b83..058c09f0067 100644 --- a/python/cudf_polars/tests/experimental/test_parallel.py +++ b/python/cudf_polars/tests/experimental/test_parallel.py @@ -8,6 +8,7 @@ from cudf_polars import Translator from cudf_polars.experimental.parallel import evaluate_dask +from cudf_polars.testing.asserts import assert_gpu_result_equal def test_evaluate_dask(): @@ -22,3 +23,31 @@ def test_evaluate_dask(): got = evaluate_dask(qir).to_polars() assert_frame_equal(expected, got) + + +def test_can_convert_lists(): + df = pl.LazyFrame( + { + "a": pl.Series([[1, 2], [3]], dtype=pl.List(pl.Int8())), + "b": pl.Series([[1], [2]], dtype=pl.List(pl.UInt16())), + "c": pl.Series( + [ + [["1", "2", "3"], ["4", "567"]], + [["8", "9"], []], + ], + dtype=pl.List(pl.List(pl.String())), + ), + "d": pl.Series([[[1, 2]], []], dtype=pl.List(pl.List(pl.UInt16()))), + } + ) + + assert_gpu_result_equal(df, executor="dask") + + +def test_scan_csv_comment_char(tmp_path): + with (tmp_path / "test.csv").open("w") as f: + f.write("""foo,bar,baz\n# 1,2,3\n3,4,5""") + + q = pl.scan_csv(tmp_path / "test.csv", comment_prefix="#") + + assert_gpu_result_equal(q, executor="dask") From 8aed94fad44ac24bbe47bc084fedaa0406e665db Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Wed, 13 Nov 2024 16:08:30 +0100 Subject: [PATCH 15/40] Improve `count` code Co-authored-by: Lawrence Mitchell --- python/cudf_polars/cudf_polars/experimental/parallel.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/python/cudf_polars/cudf_polars/experimental/parallel.py b/python/cudf_polars/cudf_polars/experimental/parallel.py index 540f53e1a27..769bc70e25b 100644 --- a/python/cudf_polars/cudf_polars/experimental/parallel.py +++ b/python/cudf_polars/cudf_polars/experimental/parallel.py @@ -45,9 +45,7 @@ def _default_ir_parts_info(ir: IR) -> PartitionInfo: # Single-partition default behavior. # This is used by `ir_parts_info` for # all unregistered IR sub-types. - count = 1 - if ir.children: - count = max(child.parts.count for child in ir.children) + count = max((child.parts.count for child in ir.children), default=1) if count > 1: raise NotImplementedError( f"Class {type(ir)} does not support multiple partitions." From aadaf100d967f8dc22da041ce19db1cdf95da73c Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Wed, 13 Nov 2024 08:07:28 -0800 Subject: [PATCH 16/40] Pass `executor` to `GPUEngine` in `assert_gpu_result_equal` --- python/cudf_polars/cudf_polars/testing/asserts.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/cudf_polars/cudf_polars/testing/asserts.py b/python/cudf_polars/cudf_polars/testing/asserts.py index e975c32cf6b..a06a78d9948 100644 --- a/python/cudf_polars/cudf_polars/testing/asserts.py +++ b/python/cudf_polars/cudf_polars/testing/asserts.py @@ -84,7 +84,7 @@ def assert_gpu_result_equal( ) expect = lazydf.collect(**final_polars_collect_kwargs) - engine = GPUEngine(raise_on_fail=True) + engine = GPUEngine(raise_on_fail=True, executor=executor) got = lazydf.collect(**final_cudf_collect_kwargs, engine=engine) assert_frame_equal( expect, From c8ca09e41222e3a66a680d7ead21f406a57c9280 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Thu, 14 Nov 2024 08:26:13 -0800 Subject: [PATCH 17/40] Clarify intent renaming executor to "dask-experimental" --- python/cudf_polars/cudf_polars/callback.py | 2 +- python/cudf_polars/tests/experimental/test_dask_serialize.py | 2 +- python/cudf_polars/tests/experimental/test_parallel.py | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/python/cudf_polars/cudf_polars/callback.py b/python/cudf_polars/cudf_polars/callback.py index 8d5cf1f5f9e..5a761627569 100644 --- a/python/cudf_polars/cudf_polars/callback.py +++ b/python/cudf_polars/cudf_polars/callback.py @@ -148,7 +148,7 @@ def _callback( ): if executor is None or executor == "cudf": return ir.evaluate(cache={}).to_polars() - elif executor == "dask": + elif executor == "dask-experimental": from cudf_polars.experimental.parallel import evaluate_dask return evaluate_dask(ir).to_polars() diff --git a/python/cudf_polars/tests/experimental/test_dask_serialize.py b/python/cudf_polars/tests/experimental/test_dask_serialize.py index e556b7e4445..6712ac3e8f6 100644 --- a/python/cudf_polars/tests/experimental/test_dask_serialize.py +++ b/python/cudf_polars/tests/experimental/test_dask_serialize.py @@ -29,7 +29,7 @@ pa.table({"a": [1, 2, None], "b": [None, 3, 4]}), ], ) -@pytest.mark.parametrize("protocol", ["cuda", "dask"]) +@pytest.mark.parametrize("protocol", ["cuda", "dask-experimental"]) def test_dask_serialization_roundtrip(arrow_tbl, protocol): plc_tbl = plc.interop.from_arrow(arrow_tbl) df = DataFrame.from_table(plc_tbl, names=arrow_tbl.column_names) diff --git a/python/cudf_polars/tests/experimental/test_parallel.py b/python/cudf_polars/tests/experimental/test_parallel.py index 058c09f0067..aa3689f95bb 100644 --- a/python/cudf_polars/tests/experimental/test_parallel.py +++ b/python/cudf_polars/tests/experimental/test_parallel.py @@ -41,7 +41,7 @@ def test_can_convert_lists(): } ) - assert_gpu_result_equal(df, executor="dask") + assert_gpu_result_equal(df, executor="dask-experimental") def test_scan_csv_comment_char(tmp_path): @@ -50,4 +50,4 @@ def test_scan_csv_comment_char(tmp_path): q = pl.scan_csv(tmp_path / "test.csv", comment_prefix="#") - assert_gpu_result_equal(q, executor="dask") + assert_gpu_result_equal(q, executor="dask-experimental") From 3fd51bb0a47927b36998118c71d2f1949ee786b8 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 14 Nov 2024 09:45:11 -0800 Subject: [PATCH 18/40] move PartitionInfo out of ir module --- python/cudf_polars/cudf_polars/dsl/ir.py | 27 +--------- .../cudf_polars/experimental/parallel.py | 54 +++++++++++++++---- 2 files changed, 44 insertions(+), 37 deletions(-) diff --git a/python/cudf_polars/cudf_polars/dsl/ir.py b/python/cudf_polars/cudf_polars/dsl/ir.py index 5f9cdec91c1..f89ef4d1eae 100644 --- a/python/cudf_polars/cudf_polars/dsl/ir.py +++ b/python/cudf_polars/cudf_polars/dsl/ir.py @@ -127,29 +127,15 @@ def broadcast(*columns: Column, target_length: int | None = None) -> list[Column ] -class PartitionInfo: - """ - Partitioning information. - - This class only tracks the partition count (for now). - """ - - __slots__ = ("count",) - - def __init__(self, count: int): - self.count = count - - class IR(Node["IR"]): """Abstract plan node, representing an unevaluated dataframe.""" - __slots__ = ("schema", "_non_child_args", "_parts_info") + __slots__ = ("schema", "_non_child_args") # This annotation is needed because of https://github.com/python/mypy/issues/17981 _non_child: ClassVar[tuple[str, ...]] = ("schema",) # Concrete classes should set this up with the arguments that will # be passed to do_evaluate. _non_child_args: tuple[Any, ...] - _parts_info: PartitionInfo schema: Schema """Mapping from column names to their data types.""" @@ -226,17 +212,6 @@ def evaluate(self, *, cache: MutableMapping[int, DataFrame]) -> DataFrame: *(child.evaluate(cache=cache) for child in self.children), ) - @property - def parts(self) -> PartitionInfo: - """Return Partitioning information for this IR node.""" - try: - return self._parts_info - except AttributeError: - from cudf_polars.experimental.parallel import ir_parts_info - - self._parts_info = ir_parts_info(self) - return self._parts_info - class ErrorNode(IR): """Represents an error translating the IR.""" diff --git a/python/cudf_polars/cudf_polars/experimental/parallel.py b/python/cudf_polars/cudf_polars/experimental/parallel.py index 769bc70e25b..cff11787f85 100644 --- a/python/cudf_polars/cudf_polars/experimental/parallel.py +++ b/python/cudf_polars/cudf_polars/experimental/parallel.py @@ -8,8 +8,7 @@ from typing import TYPE_CHECKING, Any from cudf_polars.dsl.expr import NamedExpr -from cudf_polars.dsl.ir import PartitionInfo -from cudf_polars.dsl.traversal import traversal +from cudf_polars.dsl.traversal import reuse_if_unchanged, traversal if TYPE_CHECKING: from collections.abc import MutableMapping @@ -19,6 +18,30 @@ from cudf_polars.dsl.nodebase import Node +class PartitionInfo: + """ + Partitioning information. + + This class only tracks the partition count (for now). + """ + + __slots__ = ("count",) + + def __init__(self, count: int): + self.count = count + + +# The hash of an IR object must always map to a +# unique PartitionInfo object, and we can cache +# this mapping until evaluation is complete. +_IR_PARTS_CACHE: MutableMapping[int, PartitionInfo] = {} + + +def _clear_parts_info_cache() -> None: + """Clear cached partitioning information.""" + _IR_PARTS_CACHE.clear() + + def get_key_name(node: Node | NamedExpr) -> str: """Generate the key name for a Node.""" if isinstance(node, NamedExpr): @@ -30,7 +53,7 @@ def get_key_name(node: Node | NamedExpr) -> str: def lower_ir_node(ir: IR, rec) -> IR: """Rewrite an IR node with proper partitioning.""" # Return same node by default - return ir + return reuse_if_unchanged(ir, rec) def lower_ir_graph(ir: IR) -> IR: @@ -43,9 +66,8 @@ def lower_ir_graph(ir: IR) -> IR: def _default_ir_parts_info(ir: IR) -> PartitionInfo: # Single-partition default behavior. - # This is used by `ir_parts_info` for - # all unregistered IR sub-types. - count = max((child.parts.count for child in ir.children), default=1) + # This is used by `_ir_parts_info` for all unregistered IR sub-types. + count = max((ir_parts_info(child).count for child in ir.children), default=1) if count > 1: raise NotImplementedError( f"Class {type(ir)} does not support multiple partitions." @@ -54,22 +76,31 @@ def _default_ir_parts_info(ir: IR) -> PartitionInfo: @singledispatch +def _ir_parts_info(ir: IR) -> PartitionInfo: + """IR partitioning-info dispatch.""" + return _default_ir_parts_info(ir) + + def ir_parts_info(ir: IR) -> PartitionInfo: """Return the partitioning info for an IR node.""" - return _default_ir_parts_info(ir) + key = hash(ir) + try: + return _IR_PARTS_CACHE[key] + except KeyError: + _IR_PARTS_CACHE[key] = _ir_parts_info(ir) + return _IR_PARTS_CACHE[key] def _default_ir_tasks(ir: IR) -> MutableMapping[Any, Any]: # Single-partition default behavior. - # This is used by `generate_ir_tasks` for - # all unregistered IR sub-types. - if ir.parts.count > 1: + # This is used by `generate_ir_tasks` for all unregistered IR sub-types. + if ir_parts_info(ir).count > 1: raise NotImplementedError(f"Failed to generate tasks for {ir}.") child_names = [] for child in ir.children: child_names.append(get_key_name(child)) - if child.parts.count > 1: + if ir_parts_info(child).count > 1: raise NotImplementedError( f"Failed to generate tasks for {ir} with child {child}." ) @@ -107,6 +138,7 @@ def task_graph(_ir: IR) -> tuple[MutableMapping[str, Any], str]: key_name = get_key_name(ir) graph[key_name] = (key_name, 0) + _clear_parts_info_cache() return graph, key_name From 453e274467b78a831a7bea658036114a965ffd69 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 14 Nov 2024 09:54:06 -0800 Subject: [PATCH 19/40] skip coverage on sanity-check errors --- .../cudf_polars/cudf_polars/experimental/parallel.py | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/python/cudf_polars/cudf_polars/experimental/parallel.py b/python/cudf_polars/cudf_polars/experimental/parallel.py index cff11787f85..108d85a6f10 100644 --- a/python/cudf_polars/cudf_polars/experimental/parallel.py +++ b/python/cudf_polars/cudf_polars/experimental/parallel.py @@ -45,7 +45,7 @@ def _clear_parts_info_cache() -> None: def get_key_name(node: Node | NamedExpr) -> str: """Generate the key name for a Node.""" if isinstance(node, NamedExpr): - return f"named-{get_key_name(node.value)}" + return f"named-{get_key_name(node.value)}" # pragma: no cover return f"{type(node).__name__.lower()}-{hash(node)}" @@ -71,7 +71,7 @@ def _default_ir_parts_info(ir: IR) -> PartitionInfo: if count > 1: raise NotImplementedError( f"Class {type(ir)} does not support multiple partitions." - ) + ) # pragma: no cover return PartitionInfo(count=count) @@ -95,7 +95,9 @@ def _default_ir_tasks(ir: IR) -> MutableMapping[Any, Any]: # Single-partition default behavior. # This is used by `generate_ir_tasks` for all unregistered IR sub-types. if ir_parts_info(ir).count > 1: - raise NotImplementedError(f"Failed to generate tasks for {ir}.") + raise NotImplementedError( + f"Failed to generate multiple output tasks for {ir}." + ) # pragma: no cover child_names = [] for child in ir.children: @@ -103,7 +105,7 @@ def _default_ir_tasks(ir: IR) -> MutableMapping[Any, Any]: if ir_parts_info(child).count > 1: raise NotImplementedError( f"Failed to generate tasks for {ir} with child {child}." - ) + ) # pragma: no cover key_name = get_key_name(ir) return { From 2b74f283c74ea5eb0d438facb0e828e841def973 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Thu, 14 Nov 2024 11:32:19 -0800 Subject: [PATCH 20/40] Add `--executor` to pytest --- .../cudf_polars/cudf_polars/testing/asserts.py | 7 ++++++- python/cudf_polars/tests/conftest.py | 16 ++++++++++++++++ 2 files changed, 22 insertions(+), 1 deletion(-) diff --git a/python/cudf_polars/cudf_polars/testing/asserts.py b/python/cudf_polars/cudf_polars/testing/asserts.py index a06a78d9948..2061f87b39e 100644 --- a/python/cudf_polars/cudf_polars/testing/asserts.py +++ b/python/cudf_polars/cudf_polars/testing/asserts.py @@ -20,6 +20,11 @@ __all__: list[str] = ["assert_gpu_result_equal", "assert_ir_translation_raises"] +# Will be overriden by `conftest.py` with the value from the `--executor` +# command-line argument +Executor = None + + def assert_gpu_result_equal( lazydf: pl.LazyFrame, *, @@ -84,7 +89,7 @@ def assert_gpu_result_equal( ) expect = lazydf.collect(**final_polars_collect_kwargs) - engine = GPUEngine(raise_on_fail=True, executor=executor) + engine = GPUEngine(raise_on_fail=True, executor=Executor) got = lazydf.collect(**final_cudf_collect_kwargs, engine=engine) assert_frame_equal( expect, diff --git a/python/cudf_polars/tests/conftest.py b/python/cudf_polars/tests/conftest.py index 9bbce6bc080..c57e6f733cc 100644 --- a/python/cudf_polars/tests/conftest.py +++ b/python/cudf_polars/tests/conftest.py @@ -8,3 +8,19 @@ @pytest.fixture(params=[False, True], ids=["no_nulls", "nulls"], scope="session") def with_nulls(request): return request.param + + +def pytest_addoption(parser): + parser.addoption( + "--executor", + action="store", + default="cudf", + choices=("cudf", "dask-experimental"), + help="Executor to use for GPUEngine.", + ) + + +def pytest_configure(config): + import cudf_polars.testing.asserts + + cudf_polars.testing.asserts.Executor = config.getoption("--executor") From 2398a2e65dd7c65349235053b2f3bcca5a929cd2 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Thu, 14 Nov 2024 11:35:00 -0800 Subject: [PATCH 21/40] Enable dask-experimental tests in CI, remove duplicates --- ci/run_cudf_polars_pytests.sh | 4 +++ .../tests/experimental/test_parallel.py | 29 ------------------- 2 files changed, 4 insertions(+), 29 deletions(-) diff --git a/ci/run_cudf_polars_pytests.sh b/ci/run_cudf_polars_pytests.sh index c10612a065a..bf5a3ccee8e 100755 --- a/ci/run_cudf_polars_pytests.sh +++ b/ci/run_cudf_polars_pytests.sh @@ -8,4 +8,8 @@ set -euo pipefail # Support invoking run_cudf_polars_pytests.sh outside the script directory cd "$(dirname "$(realpath "${BASH_SOURCE[0]}")")"/../python/cudf_polars/ +# Test the default "cudf" executor python -m pytest --cache-clear "$@" tests + +# Test the "dask-experimental" executor +python -m pytest --cache-clear "$@" tests --executor dask-experimental diff --git a/python/cudf_polars/tests/experimental/test_parallel.py b/python/cudf_polars/tests/experimental/test_parallel.py index aa3689f95bb..9ac10b21b83 100644 --- a/python/cudf_polars/tests/experimental/test_parallel.py +++ b/python/cudf_polars/tests/experimental/test_parallel.py @@ -8,7 +8,6 @@ from cudf_polars import Translator from cudf_polars.experimental.parallel import evaluate_dask -from cudf_polars.testing.asserts import assert_gpu_result_equal def test_evaluate_dask(): @@ -23,31 +22,3 @@ def test_evaluate_dask(): got = evaluate_dask(qir).to_polars() assert_frame_equal(expected, got) - - -def test_can_convert_lists(): - df = pl.LazyFrame( - { - "a": pl.Series([[1, 2], [3]], dtype=pl.List(pl.Int8())), - "b": pl.Series([[1], [2]], dtype=pl.List(pl.UInt16())), - "c": pl.Series( - [ - [["1", "2", "3"], ["4", "567"]], - [["8", "9"], []], - ], - dtype=pl.List(pl.List(pl.String())), - ), - "d": pl.Series([[[1, 2]], []], dtype=pl.List(pl.List(pl.UInt16()))), - } - ) - - assert_gpu_result_equal(df, executor="dask-experimental") - - -def test_scan_csv_comment_char(tmp_path): - with (tmp_path / "test.csv").open("w") as f: - f.write("""foo,bar,baz\n# 1,2,3\n3,4,5""") - - q = pl.scan_csv(tmp_path / "test.csv", comment_prefix="#") - - assert_gpu_result_equal(q, executor="dask-experimental") From 9aa479a0030217cadca9697d5394774cd00d2a2b Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Thu, 14 Nov 2024 11:45:47 -0800 Subject: [PATCH 22/40] Fix wrong protocol name in deserialization test --- python/cudf_polars/tests/experimental/test_dask_serialize.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/cudf_polars/tests/experimental/test_dask_serialize.py b/python/cudf_polars/tests/experimental/test_dask_serialize.py index 6712ac3e8f6..e556b7e4445 100644 --- a/python/cudf_polars/tests/experimental/test_dask_serialize.py +++ b/python/cudf_polars/tests/experimental/test_dask_serialize.py @@ -29,7 +29,7 @@ pa.table({"a": [1, 2, None], "b": [None, 3, 4]}), ], ) -@pytest.mark.parametrize("protocol", ["cuda", "dask-experimental"]) +@pytest.mark.parametrize("protocol", ["cuda", "dask"]) def test_dask_serialization_roundtrip(arrow_tbl, protocol): plc_tbl = plc.interop.from_arrow(arrow_tbl) df = DataFrame.from_table(plc_tbl, names=arrow_tbl.column_names) From 22678a51dc6c1e1900e1de34d738aa850545139b Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Thu, 14 Nov 2024 12:27:23 -0800 Subject: [PATCH 23/40] Remove `executor` kwarg from `assert_gpu_result_equal` --- python/cudf_polars/cudf_polars/testing/asserts.py | 3 --- 1 file changed, 3 deletions(-) diff --git a/python/cudf_polars/cudf_polars/testing/asserts.py b/python/cudf_polars/cudf_polars/testing/asserts.py index 2061f87b39e..0155cb2a9f7 100644 --- a/python/cudf_polars/cudf_polars/testing/asserts.py +++ b/python/cudf_polars/cudf_polars/testing/asserts.py @@ -38,7 +38,6 @@ def assert_gpu_result_equal( rtol: float = 1e-05, atol: float = 1e-08, categorical_as_str: bool = False, - executor: str | None = None, ) -> None: """ Assert that collection of a lazyframe on GPU produces correct results. @@ -74,8 +73,6 @@ def assert_gpu_result_equal( Absolute tolerance for float comparisons categorical_as_str Decat categoricals to strings before comparing - executor - The executor configuration to pass to `GPUEngine` Raises ------ From efadb78601581544cda97c470bc8fed3629ffe83 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Fri, 15 Nov 2024 07:48:29 -0800 Subject: [PATCH 24/40] Reintroduce `executor` kwarg in `assert_gpu_result_equal` --- python/cudf_polars/cudf_polars/testing/asserts.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/python/cudf_polars/cudf_polars/testing/asserts.py b/python/cudf_polars/cudf_polars/testing/asserts.py index 0155cb2a9f7..b6e4277ecbc 100644 --- a/python/cudf_polars/cudf_polars/testing/asserts.py +++ b/python/cudf_polars/cudf_polars/testing/asserts.py @@ -38,6 +38,7 @@ def assert_gpu_result_equal( rtol: float = 1e-05, atol: float = 1e-08, categorical_as_str: bool = False, + executor: str | None = None, ) -> None: """ Assert that collection of a lazyframe on GPU produces correct results. @@ -73,6 +74,9 @@ def assert_gpu_result_equal( Absolute tolerance for float comparisons categorical_as_str Decat categoricals to strings before comparing + executor + The executor configuration to pass to `GPUEngine`. If not specified + uses the module level `Executor` attribute. Raises ------ @@ -86,7 +90,7 @@ def assert_gpu_result_equal( ) expect = lazydf.collect(**final_polars_collect_kwargs) - engine = GPUEngine(raise_on_fail=True, executor=Executor) + engine = GPUEngine(raise_on_fail=True, executor=executor or Executor) got = lazydf.collect(**final_cudf_collect_kwargs, engine=engine) assert_frame_equal( expect, From 9b78d8ff2ea0e465875e8c0b1ccd193ad6e9986b Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Fri, 15 Nov 2024 07:56:09 -0800 Subject: [PATCH 25/40] Add basic tests for all executors to ensure 100% coverage --- python/cudf_polars/tests/test_executors.py | 72 ++++++++++++++++++++++ 1 file changed, 72 insertions(+) create mode 100644 python/cudf_polars/tests/test_executors.py diff --git a/python/cudf_polars/tests/test_executors.py b/python/cudf_polars/tests/test_executors.py new file mode 100644 index 00000000000..2de06e47fa1 --- /dev/null +++ b/python/cudf_polars/tests/test_executors.py @@ -0,0 +1,72 @@ +# SPDX-FileCopyrightText: Copyright (c) 2024 NVIDIA CORPORATION & AFFILIATES. +# SPDX-License-Identifier: Apache-2.0 + +from __future__ import annotations + +import pytest + +import polars as pl + +from cudf_polars.testing.asserts import assert_gpu_result_equal + + +@pytest.mark.parametrize("executor", [None, "cudf", "dask-experimental"]) +def test_executor_basics(executor): + """Test basics of each executor.""" + if executor == "dask-experimental": + pytest.importorskip("dask") + + df = pl.LazyFrame( + { + "a": pl.Series([[1, 2], [3]], dtype=pl.List(pl.Int8())), + "b": pl.Series([[1], [2]], dtype=pl.List(pl.UInt16())), + "c": pl.Series( + [ + [["1", "2", "3"], ["4", "567"]], + [["8", "9"], []], + ], + dtype=pl.List(pl.List(pl.String())), + ), + "d": pl.Series([[[1, 2]], []], dtype=pl.List(pl.List(pl.UInt16()))), + } + ) + + assert_gpu_result_equal(df, executor=executor) + + +def test_cudf_cache_evaluate(): + """Tests `cudf_polars.dsl.ir.Cache.evaluate()`.""" + ldf = pl.DataFrame( + { + "a": [1, 2, 3, 4, 5, 6, 7], + "b": [1, 1, 1, 1, 1, 1, 1], + } + ).lazy() + ldf2 = ldf.select((pl.col("a") + pl.col("b")).alias("c"), pl.col("a")) + query = pl.concat([ldf, ldf2], how="diagonal") + assert_gpu_result_equal(query, executor="cudf") + + +def test_dask_experimental_map_function_get_hashable(): + """Tests `cudf_polars.dsl.ir.MapFunction.get_hashable()`.""" + df = pl.LazyFrame( + { + "a": pl.Series([11, 12, 13], dtype=pl.UInt16), + "b": pl.Series([1, 3, 5], dtype=pl.Int16), + "c": pl.Series([2, 4, 6], dtype=pl.Float32), + "d": ["a", "b", "c"], + } + ) + q = df.unpivot(index="d") + assert_gpu_result_equal(q, executor="dask-experimental") + + +def test_unknown_executor(): + """Test invalid executor.""" + df = pl.LazyFrame({}) + + with pytest.raises( + pl.exceptions.ComputeError, + match="ValueError: Unknown executor 'unknown-executor'", + ): + assert_gpu_result_equal(df, executor="unknown-executor") From 3aeb1e442d1c172635d79fc56378b59b643174fa Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Mon, 18 Nov 2024 05:25:06 -0800 Subject: [PATCH 26/40] Fix `executor` in `assert_gpu_result_equal` --- python/cudf_polars/cudf_polars/testing/asserts.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/python/cudf_polars/cudf_polars/testing/asserts.py b/python/cudf_polars/cudf_polars/testing/asserts.py index 37700302d88..12e7e007d46 100644 --- a/python/cudf_polars/cudf_polars/testing/asserts.py +++ b/python/cudf_polars/cudf_polars/testing/asserts.py @@ -89,14 +89,13 @@ def assert_gpu_result_equal( If GPU collection failed in some way. """ if engine is None: - engine = GPUEngine(raise_on_fail=True, executor=Executor) + engine = GPUEngine(raise_on_fail=True, executor=executor or Executor) final_polars_collect_kwargs, final_cudf_collect_kwargs = _process_kwargs( collect_kwargs, polars_collect_kwargs, cudf_collect_kwargs ) expect = lazydf.collect(**final_polars_collect_kwargs) - engine = GPUEngine(raise_on_fail=True, executor=executor or Executor) got = lazydf.collect(**final_cudf_collect_kwargs, engine=engine) assert_frame_equal( expect, From dbf37d3bbcc3d8734804383a4d7960e354b8aea0 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 21 Nov 2024 06:58:33 -0800 Subject: [PATCH 27/40] address code review - round 1 --- .../cudf_polars/experimental/parallel.py | 57 ++++++++++--------- 1 file changed, 30 insertions(+), 27 deletions(-) diff --git a/python/cudf_polars/cudf_polars/experimental/parallel.py b/python/cudf_polars/cudf_polars/experimental/parallel.py index 108d85a6f10..02964b73000 100644 --- a/python/cudf_polars/cudf_polars/experimental/parallel.py +++ b/python/cudf_polars/cudf_polars/experimental/parallel.py @@ -4,18 +4,20 @@ from __future__ import annotations -from functools import singledispatch +import operator +from functools import reduce, singledispatch from typing import TYPE_CHECKING, Any from cudf_polars.dsl.expr import NamedExpr +from cudf_polars.dsl.ir import IR from cudf_polars.dsl.traversal import reuse_if_unchanged, traversal if TYPE_CHECKING: from collections.abc import MutableMapping from cudf_polars.containers import DataFrame - from cudf_polars.dsl.ir import IR from cudf_polars.dsl.nodebase import Node + from cudf_polars.typing import IRTransformer class PartitionInfo: @@ -50,10 +52,13 @@ def get_key_name(node: Node | NamedExpr) -> str: @singledispatch -def lower_ir_node(ir: IR, rec) -> IR: +def lower_ir_node(ir: IR, rec: IRTransformer) -> IR: """Rewrite an IR node with proper partitioning.""" - # Return same node by default - return reuse_if_unchanged(ir, rec) + raise AssertionError(f"Unhandled type {type(ir)}") + + +# Return same node by default +lower_ir_node.register(IR)(reuse_if_unchanged) def lower_ir_graph(ir: IR) -> IR: @@ -64,6 +69,13 @@ def lower_ir_graph(ir: IR) -> IR: return mapper(ir) +@singledispatch +def _ir_parts_info(ir: IR) -> PartitionInfo: + """IR partitioning-info dispatch.""" + raise AssertionError(f"Unhandled type {type(ir)}") + + +@_ir_parts_info.register(IR) def _default_ir_parts_info(ir: IR) -> PartitionInfo: # Single-partition default behavior. # This is used by `_ir_parts_info` for all unregistered IR sub-types. @@ -75,12 +87,6 @@ def _default_ir_parts_info(ir: IR) -> PartitionInfo: return PartitionInfo(count=count) -@singledispatch -def _ir_parts_info(ir: IR) -> PartitionInfo: - """IR partitioning-info dispatch.""" - return _default_ir_parts_info(ir) - - def ir_parts_info(ir: IR) -> PartitionInfo: """Return the partitioning info for an IR node.""" key = hash(ir) @@ -91,6 +97,18 @@ def ir_parts_info(ir: IR) -> PartitionInfo: return _IR_PARTS_CACHE[key] +@singledispatch +def generate_ir_tasks(ir: IR) -> MutableMapping[Any, Any]: + """ + Generate tasks for an IR node. + + An IR node only needs to generate the graph for + the current IR logic (not including child IRs). + """ + raise AssertionError(f"Unhandled type {type(ir)}") + + +@generate_ir_tasks.register(IR) def _default_ir_tasks(ir: IR) -> MutableMapping[Any, Any]: # Single-partition default behavior. # This is used by `generate_ir_tasks` for all unregistered IR sub-types. @@ -117,26 +135,11 @@ def _default_ir_tasks(ir: IR) -> MutableMapping[Any, Any]: } -@singledispatch -def generate_ir_tasks(ir: IR) -> MutableMapping[Any, Any]: - """ - Generate tasks for an IR node. - - An IR node only needs to generate the graph for - the current IR logic (not including child IRs). - """ - return _default_ir_tasks(ir) - - def task_graph(_ir: IR) -> tuple[MutableMapping[str, Any], str]: """Construct a Dask-compatible task graph.""" ir: IR = lower_ir_graph(_ir) - graph = { - k: v - for layer in [generate_ir_tasks(n) for n in traversal(ir)] - for k, v in layer.items() - } + graph = reduce(operator.or_, map(generate_ir_tasks, traversal(ir))) key_name = get_key_name(ir) graph[key_name] = (key_name, 0) From 4d21f7c9d9ef0987a73d440c145afcbbf81dff99 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 21 Nov 2024 07:06:01 -0800 Subject: [PATCH 28/40] move sort tupling --- python/cudf_polars/cudf_polars/dsl/expressions/sorting.py | 2 +- python/cudf_polars/cudf_polars/dsl/translate.py | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/python/cudf_polars/cudf_polars/dsl/expressions/sorting.py b/python/cudf_polars/cudf_polars/dsl/expressions/sorting.py index f8079234df4..99512e2ef52 100644 --- a/python/cudf_polars/cudf_polars/dsl/expressions/sorting.py +++ b/python/cudf_polars/cudf_polars/dsl/expressions/sorting.py @@ -69,7 +69,7 @@ def __init__( *by: Expr, ) -> None: self.dtype = dtype - self.options = (options[0], tuple(options[1]), tuple(options[2])) + self.options = options self.children = (column, *by) def do_evaluate( diff --git a/python/cudf_polars/cudf_polars/dsl/translate.py b/python/cudf_polars/cudf_polars/dsl/translate.py index 12fc2a196cd..9480ce6e535 100644 --- a/python/cudf_polars/cudf_polars/dsl/translate.py +++ b/python/cudf_polars/cudf_polars/dsl/translate.py @@ -633,9 +633,10 @@ def _(node: pl_expr.Sort, translator: Translator, dtype: plc.DataType) -> expr.E @_translate_expr.register def _(node: pl_expr.SortBy, translator: Translator, dtype: plc.DataType) -> expr.Expr: + options = node.sort_options return expr.SortBy( dtype, - node.sort_options, + (options[0], tuple(options[1]), tuple(options[2])), translator.translate_expr(n=node.expr), *(translator.translate_expr(n=n) for n in node.by), ) From e241af3f1a35d4c95f2977faf7e40b80354d31db Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 21 Nov 2024 07:16:39 -0800 Subject: [PATCH 29/40] remove need for stringify --- python/cudf_polars/cudf_polars/dsl/ir.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/python/cudf_polars/cudf_polars/dsl/ir.py b/python/cudf_polars/cudf_polars/dsl/ir.py index a83130666b6..6899747f439 100644 --- a/python/cudf_polars/cudf_polars/dsl/ir.py +++ b/python/cudf_polars/cudf_polars/dsl/ir.py @@ -1599,13 +1599,15 @@ def __init__(self, schema: Schema, name: str, options: Any, df: IR): # polars requires that all to-explode columns have the # same sub-shapes raise NotImplementedError("Explode with more than one column") + self.options = (tuple(to_explode),) elif self.name == "rename": - old, new, _ = self.options + old, new, strict = self.options # TODO: perhaps polars should validate renaming in the IR? if len(new) != len(set(new)) or ( set(new) & (set(df.schema.keys()) - set(old)) ): raise NotImplementedError("Duplicate new names in rename.") + self.options = (tuple(old), tuple(new), strict) elif self.name == "unpivot": indices, pivotees, variable_name, value_name = self.options value_name = "value" if value_name is None else value_name @@ -1631,7 +1633,7 @@ def __init__(self, schema: Schema, name: str, options: Any, df: IR): def get_hashable(self) -> Hashable: # pragma: no cover; Needed by experimental """Hashable representation of the node.""" schema_hash = tuple(self.schema.items()) - return (type(self), schema_hash, self.name, str(self.options), *self.children) + return (type(self), schema_hash, self.name, self.options, *self.children) @classmethod def do_evaluate( From 1064fcb35a0e4b151bc1006b802dd6cc3b910826 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 21 Nov 2024 07:22:11 -0800 Subject: [PATCH 30/40] address code review - round 2 --- .../cudf_polars/experimental/parallel.py | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/python/cudf_polars/cudf_polars/experimental/parallel.py b/python/cudf_polars/cudf_polars/experimental/parallel.py index 02964b73000..a260a8ab5fa 100644 --- a/python/cudf_polars/cudf_polars/experimental/parallel.py +++ b/python/cudf_polars/cudf_polars/experimental/parallel.py @@ -33,10 +33,10 @@ def __init__(self, count: int): self.count = count -# The hash of an IR object must always map to a -# unique PartitionInfo object, and we can cache +# An IR object must always map to a unique +# PartitionInfo object, and we can cache # this mapping until evaluation is complete. -_IR_PARTS_CACHE: MutableMapping[int, PartitionInfo] = {} +_IR_PARTS_CACHE: MutableMapping[IR, PartitionInfo] = {} def _clear_parts_info_cache() -> None: @@ -89,12 +89,11 @@ def _default_ir_parts_info(ir: IR) -> PartitionInfo: def ir_parts_info(ir: IR) -> PartitionInfo: """Return the partitioning info for an IR node.""" - key = hash(ir) try: - return _IR_PARTS_CACHE[key] + return _IR_PARTS_CACHE[ir] except KeyError: - _IR_PARTS_CACHE[key] = _ir_parts_info(ir) - return _IR_PARTS_CACHE[key] + _IR_PARTS_CACHE[ir] = _ir_parts_info(ir) + return _IR_PARTS_CACHE[ir] @singledispatch From aeecd4dcf7ff1228a06b87ba6646fbb26a9cbf7c Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 21 Nov 2024 09:48:38 -0800 Subject: [PATCH 31/40] remove global caching --- .../cudf_polars/experimental/parallel.py | 108 +++++++++--------- 1 file changed, 53 insertions(+), 55 deletions(-) diff --git a/python/cudf_polars/cudf_polars/experimental/parallel.py b/python/cudf_polars/cudf_polars/experimental/parallel.py index a260a8ab5fa..e8237c1eb4d 100644 --- a/python/cudf_polars/cudf_polars/experimental/parallel.py +++ b/python/cudf_polars/cudf_polars/experimental/parallel.py @@ -8,16 +8,16 @@ from functools import reduce, singledispatch from typing import TYPE_CHECKING, Any -from cudf_polars.dsl.expr import NamedExpr from cudf_polars.dsl.ir import IR -from cudf_polars.dsl.traversal import reuse_if_unchanged, traversal +from cudf_polars.dsl.traversal import traversal if TYPE_CHECKING: from collections.abc import MutableMapping + from typing import TypeAlias from cudf_polars.containers import DataFrame from cudf_polars.dsl.nodebase import Node - from cudf_polars.typing import IRTransformer + from cudf_polars.typing import GenericTransformer class PartitionInfo: @@ -33,71 +33,63 @@ def __init__(self, count: int): self.count = count -# An IR object must always map to a unique -# PartitionInfo object, and we can cache -# this mapping until evaluation is complete. -_IR_PARTS_CACHE: MutableMapping[IR, PartitionInfo] = {} +LowerIRTransformer: TypeAlias = ( + "GenericTransformer[IR, MutableMapping[IR, PartitionInfo]]" +) +"""Protocol for Lowering IR nodes.""" -def _clear_parts_info_cache() -> None: - """Clear cached partitioning information.""" - _IR_PARTS_CACHE.clear() - - -def get_key_name(node: Node | NamedExpr) -> str: +def get_key_name(node: Node) -> str: """Generate the key name for a Node.""" - if isinstance(node, NamedExpr): - return f"named-{get_key_name(node.value)}" # pragma: no cover return f"{type(node).__name__.lower()}-{hash(node)}" @singledispatch -def lower_ir_node(ir: IR, rec: IRTransformer) -> IR: +def lower_ir_node( + ir: IR, rec: LowerIRTransformer +) -> tuple[IR, MutableMapping[IR, PartitionInfo]]: """Rewrite an IR node with proper partitioning.""" raise AssertionError(f"Unhandled type {type(ir)}") -# Return same node by default -lower_ir_node.register(IR)(reuse_if_unchanged) - - -def lower_ir_graph(ir: IR) -> IR: - """Rewrite an IR graph with proper partitioning.""" - from cudf_polars.dsl.traversal import CachingVisitor - - mapper = CachingVisitor(lower_ir_node) - return mapper(ir) - - -@singledispatch -def _ir_parts_info(ir: IR) -> PartitionInfo: - """IR partitioning-info dispatch.""" - raise AssertionError(f"Unhandled type {type(ir)}") +@lower_ir_node.register(IR) +def _default_lower_ir_node( + ir: IR, rec: LowerIRTransformer +) -> tuple[IR, MutableMapping[IR, PartitionInfo]]: + if len(ir.children) == 0: + # Default leaf node has single partition + return ir, {ir: PartitionInfo(count=1)} + # Lower children + children, _partition_info = zip(*(rec(c) for c in ir.children), strict=False) + partition_info = reduce(operator.or_, _partition_info) -@_ir_parts_info.register(IR) -def _default_ir_parts_info(ir: IR) -> PartitionInfo: - # Single-partition default behavior. - # This is used by `_ir_parts_info` for all unregistered IR sub-types. - count = max((ir_parts_info(child).count for child in ir.children), default=1) + # Check that child partitioning is supported + count = max(partition_info[c].count for c in children) if count > 1: raise NotImplementedError( f"Class {type(ir)} does not support multiple partitions." ) # pragma: no cover - return PartitionInfo(count=count) + # Return reconstructed node and + partition = PartitionInfo(count=1) + new_node = ir.reconstruct(children) + partition_info[new_node] = partition + return new_node, partition_info -def ir_parts_info(ir: IR) -> PartitionInfo: - """Return the partitioning info for an IR node.""" - try: - return _IR_PARTS_CACHE[ir] - except KeyError: - _IR_PARTS_CACHE[ir] = _ir_parts_info(ir) - return _IR_PARTS_CACHE[ir] + +def lower_ir_graph(ir: IR) -> tuple[IR, MutableMapping[IR, PartitionInfo]]: + """Rewrite an IR graph with proper partitioning.""" + from cudf_polars.dsl.traversal import CachingVisitor + + mapper = CachingVisitor(lower_ir_node) + return mapper(ir) @singledispatch -def generate_ir_tasks(ir: IR) -> MutableMapping[Any, Any]: +def generate_ir_tasks( + ir: IR, partition_info: MutableMapping[IR, PartitionInfo] +) -> MutableMapping[Any, Any]: """ Generate tasks for an IR node. @@ -108,10 +100,12 @@ def generate_ir_tasks(ir: IR) -> MutableMapping[Any, Any]: @generate_ir_tasks.register(IR) -def _default_ir_tasks(ir: IR) -> MutableMapping[Any, Any]: +def _default_ir_tasks( + ir: IR, partition_info: MutableMapping[IR, PartitionInfo] +) -> MutableMapping[Any, Any]: # Single-partition default behavior. # This is used by `generate_ir_tasks` for all unregistered IR sub-types. - if ir_parts_info(ir).count > 1: + if partition_info[ir].count > 1: raise NotImplementedError( f"Failed to generate multiple output tasks for {ir}." ) # pragma: no cover @@ -119,7 +113,7 @@ def _default_ir_tasks(ir: IR) -> MutableMapping[Any, Any]: child_names = [] for child in ir.children: child_names.append(get_key_name(child)) - if ir_parts_info(child).count > 1: + if partition_info[child].count > 1: raise NotImplementedError( f"Failed to generate tasks for {ir} with child {child}." ) # pragma: no cover @@ -134,15 +128,17 @@ def _default_ir_tasks(ir: IR) -> MutableMapping[Any, Any]: } -def task_graph(_ir: IR) -> tuple[MutableMapping[str, Any], str]: +def task_graph( + ir: IR, partition_info: MutableMapping[IR, PartitionInfo] +) -> tuple[MutableMapping[str, Any], str]: """Construct a Dask-compatible task graph.""" - ir: IR = lower_ir_graph(_ir) - - graph = reduce(operator.or_, map(generate_ir_tasks, traversal(ir))) + graph = reduce( + operator.or_, + [generate_ir_tasks(node, partition_info) for node in traversal(ir)], + ) key_name = get_key_name(ir) graph[key_name] = (key_name, 0) - _clear_parts_info_cache() return graph, key_name @@ -150,5 +146,7 @@ def evaluate_dask(ir: IR) -> DataFrame: """Evaluate an IR graph with Dask.""" from dask import get - graph, key = task_graph(ir) + ir, partition_info = lower_ir_graph(ir) + + graph, key = task_graph(ir, partition_info) return get(graph, key) From 09c5217a4a54d2ba4cf0b320f0a8d3a84b22606a Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 21 Nov 2024 10:12:03 -0800 Subject: [PATCH 32/40] use general StateInfo --- .../cudf_polars/experimental/parallel.py | 33 +++++++++++-------- 1 file changed, 19 insertions(+), 14 deletions(-) diff --git a/python/cudf_polars/cudf_polars/experimental/parallel.py b/python/cudf_polars/cudf_polars/experimental/parallel.py index e8237c1eb4d..1131fb2acf6 100644 --- a/python/cudf_polars/cudf_polars/experimental/parallel.py +++ b/python/cudf_polars/cudf_polars/experimental/parallel.py @@ -39,6 +39,17 @@ def __init__(self, count: int): """Protocol for Lowering IR nodes.""" +class StateInfo: + """Bag of arbitrary state information.""" + + def __init__(self, *, parts_info: MutableMapping[IR, PartitionInfo] | None = None): + self.__parts_info = parts_info or {} + + def parts(self, ir: IR) -> PartitionInfo: + """Return partitioning information for an IR node.""" + return self.__parts_info[ir] + + def get_key_name(node: Node) -> str: """Generate the key name for a Node.""" return f"{type(node).__name__.lower()}-{hash(node)}" @@ -87,9 +98,7 @@ def lower_ir_graph(ir: IR) -> tuple[IR, MutableMapping[IR, PartitionInfo]]: @singledispatch -def generate_ir_tasks( - ir: IR, partition_info: MutableMapping[IR, PartitionInfo] -) -> MutableMapping[Any, Any]: +def generate_ir_tasks(ir: IR, state: StateInfo) -> MutableMapping[Any, Any]: """ Generate tasks for an IR node. @@ -100,12 +109,10 @@ def generate_ir_tasks( @generate_ir_tasks.register(IR) -def _default_ir_tasks( - ir: IR, partition_info: MutableMapping[IR, PartitionInfo] -) -> MutableMapping[Any, Any]: +def _default_ir_tasks(ir: IR, state: StateInfo) -> MutableMapping[Any, Any]: # Single-partition default behavior. # This is used by `generate_ir_tasks` for all unregistered IR sub-types. - if partition_info[ir].count > 1: + if state.parts(ir).count > 1: raise NotImplementedError( f"Failed to generate multiple output tasks for {ir}." ) # pragma: no cover @@ -113,7 +120,7 @@ def _default_ir_tasks( child_names = [] for child in ir.children: child_names.append(get_key_name(child)) - if partition_info[child].count > 1: + if state.parts(child).count > 1: raise NotImplementedError( f"Failed to generate tasks for {ir} with child {child}." ) # pragma: no cover @@ -128,13 +135,11 @@ def _default_ir_tasks( } -def task_graph( - ir: IR, partition_info: MutableMapping[IR, PartitionInfo] -) -> tuple[MutableMapping[str, Any], str]: +def task_graph(ir: IR, state: StateInfo) -> tuple[MutableMapping[str, Any], str]: """Construct a Dask-compatible task graph.""" graph = reduce( operator.or_, - [generate_ir_tasks(node, partition_info) for node in traversal(ir)], + [generate_ir_tasks(node, state) for node in traversal(ir)], ) key_name = get_key_name(ir) graph[key_name] = (key_name, 0) @@ -146,7 +151,7 @@ def evaluate_dask(ir: IR) -> DataFrame: """Evaluate an IR graph with Dask.""" from dask import get - ir, partition_info = lower_ir_graph(ir) + ir, parts_info = lower_ir_graph(ir) - graph, key = task_graph(ir, partition_info) + graph, key = task_graph(ir, StateInfo(parts_info=parts_info)) return get(graph, key) From 62f10bc90058fc1dc0b361dd860137f2fbcf00df Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 21 Nov 2024 10:30:26 -0800 Subject: [PATCH 33/40] revert (for now) --- .../cudf_polars/experimental/parallel.py | 33 ++++++++----------- 1 file changed, 14 insertions(+), 19 deletions(-) diff --git a/python/cudf_polars/cudf_polars/experimental/parallel.py b/python/cudf_polars/cudf_polars/experimental/parallel.py index 1131fb2acf6..e8237c1eb4d 100644 --- a/python/cudf_polars/cudf_polars/experimental/parallel.py +++ b/python/cudf_polars/cudf_polars/experimental/parallel.py @@ -39,17 +39,6 @@ def __init__(self, count: int): """Protocol for Lowering IR nodes.""" -class StateInfo: - """Bag of arbitrary state information.""" - - def __init__(self, *, parts_info: MutableMapping[IR, PartitionInfo] | None = None): - self.__parts_info = parts_info or {} - - def parts(self, ir: IR) -> PartitionInfo: - """Return partitioning information for an IR node.""" - return self.__parts_info[ir] - - def get_key_name(node: Node) -> str: """Generate the key name for a Node.""" return f"{type(node).__name__.lower()}-{hash(node)}" @@ -98,7 +87,9 @@ def lower_ir_graph(ir: IR) -> tuple[IR, MutableMapping[IR, PartitionInfo]]: @singledispatch -def generate_ir_tasks(ir: IR, state: StateInfo) -> MutableMapping[Any, Any]: +def generate_ir_tasks( + ir: IR, partition_info: MutableMapping[IR, PartitionInfo] +) -> MutableMapping[Any, Any]: """ Generate tasks for an IR node. @@ -109,10 +100,12 @@ def generate_ir_tasks(ir: IR, state: StateInfo) -> MutableMapping[Any, Any]: @generate_ir_tasks.register(IR) -def _default_ir_tasks(ir: IR, state: StateInfo) -> MutableMapping[Any, Any]: +def _default_ir_tasks( + ir: IR, partition_info: MutableMapping[IR, PartitionInfo] +) -> MutableMapping[Any, Any]: # Single-partition default behavior. # This is used by `generate_ir_tasks` for all unregistered IR sub-types. - if state.parts(ir).count > 1: + if partition_info[ir].count > 1: raise NotImplementedError( f"Failed to generate multiple output tasks for {ir}." ) # pragma: no cover @@ -120,7 +113,7 @@ def _default_ir_tasks(ir: IR, state: StateInfo) -> MutableMapping[Any, Any]: child_names = [] for child in ir.children: child_names.append(get_key_name(child)) - if state.parts(child).count > 1: + if partition_info[child].count > 1: raise NotImplementedError( f"Failed to generate tasks for {ir} with child {child}." ) # pragma: no cover @@ -135,11 +128,13 @@ def _default_ir_tasks(ir: IR, state: StateInfo) -> MutableMapping[Any, Any]: } -def task_graph(ir: IR, state: StateInfo) -> tuple[MutableMapping[str, Any], str]: +def task_graph( + ir: IR, partition_info: MutableMapping[IR, PartitionInfo] +) -> tuple[MutableMapping[str, Any], str]: """Construct a Dask-compatible task graph.""" graph = reduce( operator.or_, - [generate_ir_tasks(node, state) for node in traversal(ir)], + [generate_ir_tasks(node, partition_info) for node in traversal(ir)], ) key_name = get_key_name(ir) graph[key_name] = (key_name, 0) @@ -151,7 +146,7 @@ def evaluate_dask(ir: IR) -> DataFrame: """Evaluate an IR graph with Dask.""" from dask import get - ir, parts_info = lower_ir_graph(ir) + ir, partition_info = lower_ir_graph(ir) - graph, key = task_graph(ir, StateInfo(parts_info=parts_info)) + graph, key = task_graph(ir, partition_info) return get(graph, key) From 9967cfb44742077c9b68bcd57ec6f389723fad20 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Fri, 22 Nov 2024 06:05:09 -0800 Subject: [PATCH 34/40] skip coverage on singledispatch miss --- python/cudf_polars/cudf_polars/experimental/parallel.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/cudf_polars/cudf_polars/experimental/parallel.py b/python/cudf_polars/cudf_polars/experimental/parallel.py index e8237c1eb4d..e82ead8d874 100644 --- a/python/cudf_polars/cudf_polars/experimental/parallel.py +++ b/python/cudf_polars/cudf_polars/experimental/parallel.py @@ -49,7 +49,7 @@ def lower_ir_node( ir: IR, rec: LowerIRTransformer ) -> tuple[IR, MutableMapping[IR, PartitionInfo]]: """Rewrite an IR node with proper partitioning.""" - raise AssertionError(f"Unhandled type {type(ir)}") + raise AssertionError(f"Unhandled type {type(ir)}") # pragma: no cover @lower_ir_node.register(IR) @@ -96,7 +96,7 @@ def generate_ir_tasks( An IR node only needs to generate the graph for the current IR logic (not including child IRs). """ - raise AssertionError(f"Unhandled type {type(ir)}") + raise AssertionError(f"Unhandled type {type(ir)}") # pragma: no cover @generate_ir_tasks.register(IR) From 075d41e8f929c6af597085d988df369050906651 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Fri, 22 Nov 2024 06:06:15 -0800 Subject: [PATCH 35/40] typo --- python/cudf_polars/cudf_polars/experimental/parallel.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/cudf_polars/cudf_polars/experimental/parallel.py b/python/cudf_polars/cudf_polars/experimental/parallel.py index e82ead8d874..58fbff7dca4 100644 --- a/python/cudf_polars/cudf_polars/experimental/parallel.py +++ b/python/cudf_polars/cudf_polars/experimental/parallel.py @@ -71,7 +71,7 @@ def _default_lower_ir_node( f"Class {type(ir)} does not support multiple partitions." ) # pragma: no cover - # Return reconstructed node and + # Return reconstructed node and partition-info dict partition = PartitionInfo(count=1) new_node = ir.reconstruct(children) partition_info[new_node] = partition From 21e598af17a8755c09f8e14aa18b858830ba329e Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Mon, 25 Nov 2024 05:33:08 -0800 Subject: [PATCH 36/40] Rename `"cudf"` executor to `"pylibcudf"` --- python/cudf_polars/cudf_polars/callback.py | 2 +- python/cudf_polars/tests/conftest.py | 4 ++-- python/cudf_polars/tests/test_executors.py | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/python/cudf_polars/cudf_polars/callback.py b/python/cudf_polars/cudf_polars/callback.py index f99951dc6f0..0c1b887afdf 100644 --- a/python/cudf_polars/cudf_polars/callback.py +++ b/python/cudf_polars/cudf_polars/callback.py @@ -146,7 +146,7 @@ def _callback( set_device(device), set_memory_resource(memory_resource), ): - if executor is None or executor == "cudf": + if executor is None or executor == "pylibcudf": return ir.evaluate(cache={}).to_polars() elif executor == "dask-experimental": from cudf_polars.experimental.parallel import evaluate_dask diff --git a/python/cudf_polars/tests/conftest.py b/python/cudf_polars/tests/conftest.py index c57e6f733cc..6338bf0cae1 100644 --- a/python/cudf_polars/tests/conftest.py +++ b/python/cudf_polars/tests/conftest.py @@ -14,8 +14,8 @@ def pytest_addoption(parser): parser.addoption( "--executor", action="store", - default="cudf", - choices=("cudf", "dask-experimental"), + default="pylibcudf", + choices=("pylibcudf", "dask-experimental"), help="Executor to use for GPUEngine.", ) diff --git a/python/cudf_polars/tests/test_executors.py b/python/cudf_polars/tests/test_executors.py index 2de06e47fa1..efb1fdfd2e2 100644 --- a/python/cudf_polars/tests/test_executors.py +++ b/python/cudf_polars/tests/test_executors.py @@ -10,7 +10,7 @@ from cudf_polars.testing.asserts import assert_gpu_result_equal -@pytest.mark.parametrize("executor", [None, "cudf", "dask-experimental"]) +@pytest.mark.parametrize("executor", [None, "pylibcudf", "dask-experimental"]) def test_executor_basics(executor): """Test basics of each executor.""" if executor == "dask-experimental": @@ -44,7 +44,7 @@ def test_cudf_cache_evaluate(): ).lazy() ldf2 = ldf.select((pl.col("a") + pl.col("b")).alias("c"), pl.col("a")) query = pl.concat([ldf, ldf2], how="diagonal") - assert_gpu_result_equal(query, executor="cudf") + assert_gpu_result_equal(query, executor="pylibcudf") def test_dask_experimental_map_function_get_hashable(): From 7febe212a7fdd7435c2e4ddf03229036c7ce7782 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Mon, 25 Nov 2024 05:39:55 -0800 Subject: [PATCH 37/40] Update `test_evaluate_dask()` with `collect()` --- .../tests/experimental/test_parallel.py | 17 +++++++---------- 1 file changed, 7 insertions(+), 10 deletions(-) diff --git a/python/cudf_polars/tests/experimental/test_parallel.py b/python/cudf_polars/tests/experimental/test_parallel.py index 99d212f47d3..d46ab88eebf 100644 --- a/python/cudf_polars/tests/experimental/test_parallel.py +++ b/python/cudf_polars/tests/experimental/test_parallel.py @@ -7,18 +7,15 @@ from polars import GPUEngine from polars.testing import assert_frame_equal -from cudf_polars import Translator -from cudf_polars.experimental.parallel import evaluate_dask -from cudf_polars.testing.asserts import Executor - def test_evaluate_dask(): df = pl.LazyFrame({"a": [1, 2, 3], "b": [3, 4, 5], "c": [5, 6, 7], "d": [7, 9, 8]}) q = df.select(pl.col("a") - (pl.col("b") + pl.col("c") * 2), pl.col("d")).sort("d") - config = GPUEngine(raise_on_fail=True, executor=Executor) - qir = Translator(q._ldf.visit(), config).translate_ir() - - expected = qir.evaluate(cache={}).to_polars() - got = evaluate_dask(qir).to_polars() - assert_frame_equal(expected, got) + expected = q.collect(engine="cpu") + got_gpu = q.collect(engine=GPUEngine(raise_on_fail=True)) + got_dask = q.collect( + engine=GPUEngine(raise_on_fail=True, executor="dask-experimental") + ) + assert_frame_equal(expected, got_gpu) + assert_frame_equal(expected, got_dask) From ac4d2da00dbfb2a4afe3dd6ba87f45b4618a6f68 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Mon, 25 Nov 2024 05:40:37 -0800 Subject: [PATCH 38/40] Remove tests docstrings --- python/cudf_polars/tests/test_executors.py | 4 ---- 1 file changed, 4 deletions(-) diff --git a/python/cudf_polars/tests/test_executors.py b/python/cudf_polars/tests/test_executors.py index efb1fdfd2e2..3eaea2ec9ea 100644 --- a/python/cudf_polars/tests/test_executors.py +++ b/python/cudf_polars/tests/test_executors.py @@ -12,7 +12,6 @@ @pytest.mark.parametrize("executor", [None, "pylibcudf", "dask-experimental"]) def test_executor_basics(executor): - """Test basics of each executor.""" if executor == "dask-experimental": pytest.importorskip("dask") @@ -35,7 +34,6 @@ def test_executor_basics(executor): def test_cudf_cache_evaluate(): - """Tests `cudf_polars.dsl.ir.Cache.evaluate()`.""" ldf = pl.DataFrame( { "a": [1, 2, 3, 4, 5, 6, 7], @@ -48,7 +46,6 @@ def test_cudf_cache_evaluate(): def test_dask_experimental_map_function_get_hashable(): - """Tests `cudf_polars.dsl.ir.MapFunction.get_hashable()`.""" df = pl.LazyFrame( { "a": pl.Series([11, 12, 13], dtype=pl.UInt16), @@ -62,7 +59,6 @@ def test_dask_experimental_map_function_get_hashable(): def test_unknown_executor(): - """Test invalid executor.""" df = pl.LazyFrame({}) with pytest.raises( From 2ced4a0a74a4f9c7b58ce79ea13908c686aa0f6a Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Mon, 25 Nov 2024 05:43:38 -0800 Subject: [PATCH 39/40] Improve `executor` typing --- python/cudf_polars/cudf_polars/callback.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/cudf_polars/cudf_polars/callback.py b/python/cudf_polars/cudf_polars/callback.py index 0c1b887afdf..7e4cc1430e6 100644 --- a/python/cudf_polars/cudf_polars/callback.py +++ b/python/cudf_polars/cudf_polars/callback.py @@ -9,7 +9,7 @@ import os import warnings from functools import cache, partial -from typing import TYPE_CHECKING +from typing import TYPE_CHECKING, Literal import nvtx @@ -135,7 +135,7 @@ def _callback( *, device: int | None, memory_resource: int | None, - executor: str | None, + executor: Literal["pylibcudf", "dask-experimental"] | None, ) -> pl.DataFrame: assert with_columns is None assert pyarrow_predicate is None From f6f7eda46faeae9f28c9b16c84a977e95b1c5a4d Mon Sep 17 00:00:00 2001 From: rjzamora Date: Mon, 25 Nov 2024 07:29:32 -0800 Subject: [PATCH 40/40] address code review --- python/cudf_polars/cudf_polars/dsl/ir.py | 5 - .../cudf_polars/experimental/parallel.py | 118 +++++++++++++++--- 2 files changed, 101 insertions(+), 22 deletions(-) diff --git a/python/cudf_polars/cudf_polars/dsl/ir.py b/python/cudf_polars/cudf_polars/dsl/ir.py index 6899747f439..6617b71be81 100644 --- a/python/cudf_polars/cudf_polars/dsl/ir.py +++ b/python/cudf_polars/cudf_polars/dsl/ir.py @@ -1630,11 +1630,6 @@ def __init__(self, schema: Schema, name: str, options: Any, df: IR): ) self._non_child_args = (schema, name, self.options) - def get_hashable(self) -> Hashable: # pragma: no cover; Needed by experimental - """Hashable representation of the node.""" - schema_hash = tuple(self.schema.items()) - return (type(self), schema_hash, self.name, self.options, *self.children) - @classmethod def do_evaluate( cls, schema: Schema, name: str, options: Any, df: DataFrame diff --git a/python/cudf_polars/cudf_polars/experimental/parallel.py b/python/cudf_polars/cudf_polars/experimental/parallel.py index 58fbff7dca4..6518dd60c7d 100644 --- a/python/cudf_polars/cudf_polars/experimental/parallel.py +++ b/python/cudf_polars/cudf_polars/experimental/parallel.py @@ -48,14 +48,35 @@ def get_key_name(node: Node) -> str: def lower_ir_node( ir: IR, rec: LowerIRTransformer ) -> tuple[IR, MutableMapping[IR, PartitionInfo]]: - """Rewrite an IR node with proper partitioning.""" + """ + Rewrite an IR node and extract partitioning information. + + Parameters + ---------- + ir + IR node to rewrite. + rec + Recursive LowerIRTransformer callable. + + Returns + ------- + new_ir, partition_info + The rewritten node, and a mapping from unique nodes in + the full IR graph to associated partitioning information. + + Notes + ----- + This function is used by `lower_ir_graph`. + + See Also + -------- + lower_ir_graph + """ raise AssertionError(f"Unhandled type {type(ir)}") # pragma: no cover @lower_ir_node.register(IR) -def _default_lower_ir_node( - ir: IR, rec: LowerIRTransformer -) -> tuple[IR, MutableMapping[IR, PartitionInfo]]: +def _(ir: IR, rec: LowerIRTransformer) -> tuple[IR, MutableMapping[IR, PartitionInfo]]: if len(ir.children) == 0: # Default leaf node has single partition return ir, {ir: PartitionInfo(count=1)} @@ -79,7 +100,29 @@ def _default_lower_ir_node( def lower_ir_graph(ir: IR) -> tuple[IR, MutableMapping[IR, PartitionInfo]]: - """Rewrite an IR graph with proper partitioning.""" + """ + Rewrite an IR graph and extract partitioning information. + + Parameters + ---------- + ir + Root of the graph to rewrite. + + Returns + ------- + new_ir, partition_info + The rewritten graph, and a mapping from unique nodes + in the new graph to associated partitioning information. + + Notes + ----- + This function traverses the unique nodes of the graph with + root `ir`, and applies :func:`lower_ir_node` to each node. + + See Also + -------- + lower_ir_node + """ from cudf_polars.dsl.traversal import CachingVisitor mapper = CachingVisitor(lower_ir_node) @@ -91,16 +134,34 @@ def generate_ir_tasks( ir: IR, partition_info: MutableMapping[IR, PartitionInfo] ) -> MutableMapping[Any, Any]: """ - Generate tasks for an IR node. - - An IR node only needs to generate the graph for - the current IR logic (not including child IRs). + Generate a task graph for evaluation of an IR node. + + Parameters + ---------- + ir + IR node to generate tasks for. + partition_info + Partitioning information, obtained from :func:`lower_ir_graph`. + + Returns + ------- + mapping + A (partial) dask task graph for the evaluation of an ir node. + + Notes + ----- + Task generation should only produce the tasks for the current node, + referring to child tasks by name. + + See Also + -------- + task_graph """ raise AssertionError(f"Unhandled type {type(ir)}") # pragma: no cover @generate_ir_tasks.register(IR) -def _default_ir_tasks( +def _( ir: IR, partition_info: MutableMapping[IR, PartitionInfo] ) -> MutableMapping[Any, Any]: # Single-partition default behavior. @@ -130,16 +191,39 @@ def _default_ir_tasks( def task_graph( ir: IR, partition_info: MutableMapping[IR, PartitionInfo] -) -> tuple[MutableMapping[str, Any], str]: - """Construct a Dask-compatible task graph.""" +) -> tuple[MutableMapping[Any, Any], str | tuple[str, int]]: + """ + Construct a task graph for evaluation of an IR graph. + + Parameters + ---------- + ir + Root of the graph to rewrite. + partition_info + A mapping from all unique IR nodes to the + associated partitioning information. + + Returns + ------- + graph + A Dask-compatible task graph for the entire + IR graph with root `ir`. + + Notes + ----- + This function traverses the unique nodes of the + graph with root `ir`, and extracts the tasks for + each node with :func:`generate_ir_tasks`. + + See Also + -------- + generate_ir_tasks + """ graph = reduce( operator.or_, - [generate_ir_tasks(node, partition_info) for node in traversal(ir)], + (generate_ir_tasks(node, partition_info) for node in traversal(ir)), ) - key_name = get_key_name(ir) - graph[key_name] = (key_name, 0) - - return graph, key_name + return graph, (get_key_name(ir), 0) def evaluate_dask(ir: IR) -> DataFrame: