Skip to content
Merged
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
11 changes: 11 additions & 0 deletions python/ray/data/_internal/arrow_block.py
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@
U,
)
from ray.data.context import DEFAULT_TARGET_MAX_BLOCK_SIZE, DataContext
from ray.data.expressions import Expr

try:
import pyarrow
Expand Down Expand Up @@ -460,6 +461,16 @@ def iter_rows(
for i in range(self.num_rows()):
yield self._get_row(i)

def filter(self, predicate_expr: "Expr") -> "pyarrow.Table":
"""Filter rows based on a predicate expression."""
from ray.data._expression_evaluator import eval_expr

# Evaluate the expression to get a boolean mask
mask = eval_expr(predicate_expr, self._table)

# Use PyArrow's built-in filter method
return self._table.filter(mask)


class ArrowBlockColumnAccessor(BlockColumnAccessor):
def __init__(self, col: Union["pyarrow.Array", "pyarrow.ChunkedArray"]):
Expand Down
42 changes: 32 additions & 10 deletions python/ray/data/_internal/logical/operators/map_operator.py
Original file line number Diff line number Diff line change
Expand Up @@ -224,20 +224,42 @@ class Filter(AbstractUDFMap):
def __init__(
self,
input_op: LogicalOperator,
fn: Optional[UserDefinedFunction] = None,
fn_args: Optional[Iterable[Any]] = None,
fn_kwargs: Optional[Dict[str, Any]] = None,
fn_constructor_args: Optional[Iterable[Any]] = None,
fn_constructor_kwargs: Optional[Dict[str, Any]] = None,
filter_expr: Optional["pa.dataset.Expression"] = None,
fn: Optional[
UserDefinedFunction
] = None, # TODO: Deprecate this parameter in favor of predicate_expr
fn_args: Optional[
Iterable[Any]
] = None, # TODO: Deprecate this parameter in favor of predicate_expr
fn_kwargs: Optional[
Dict[str, Any]
] = None, # TODO: Deprecate this parameter in favor of predicate_expr
fn_constructor_args: Optional[
Iterable[Any]
] = None, # TODO: Deprecate this parameter in favor of predicate_expr
fn_constructor_kwargs: Optional[
Dict[str, Any]
] = None, # TODO: Deprecate this parameter in favor of predicate_expr
predicate_expr: Optional[Expr] = None,
filter_expr: Optional[
"pa.dataset.Expression"
] = None, # TODO: Deprecate this parameter in favor of predicate_expr
compute: Optional[ComputeStrategy] = None,
ray_remote_args_fn: Optional[Callable[[], Dict[str, Any]]] = None,
ray_remote_args: Optional[Dict[str, Any]] = None,
):
# Ensure exactly one of fn or filter_expr is provided
if not ((fn is None) ^ (filter_expr is None)):
raise ValueError("Exactly one of 'fn' or 'filter_expr' must be provided")
self._filter_expr = filter_expr
# Ensure exactly one of fn, filter_expr, or predicate_expr is provided
provided_params = sum(
[fn is not None, filter_expr is not None, predicate_expr is not None]
)
if provided_params != 1:
raise ValueError(
"Exactly one of 'fn', 'filter_expr', or 'predicate_expr' must be provided"
)

self._filter_expr = (
filter_expr # TODO: Deprecate this parameter in favor of predicate_expr
)
self._predicate_expr = predicate_expr

super().__init__(
"Filter",
Expand Down
11 changes: 11 additions & 0 deletions python/ray/data/_internal/pandas_block.py
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
U,
)
from ray.data.context import DataContext
from ray.data.expressions import Expr

if TYPE_CHECKING:
import pandas
Expand Down Expand Up @@ -609,3 +610,13 @@ def iter_rows(
yield row.as_pydict()
else:
yield row

def filter(self, predicate_expr: "Expr") -> "pandas.DataFrame":
"""Filter rows based on a predicate expression."""
from ray.data._expression_evaluator import eval_expr
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's move this to _internal

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'll make this a TODO just to keep the change cleaner

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, let's do in a follow-up. But let's do it right away


# Evaluate the expression to get a boolean mask
mask = eval_expr(predicate_expr, self._table)

# Use pandas boolean indexing
return self._table[mask]
21 changes: 21 additions & 0 deletions python/ray/data/_internal/planner/plan_udf_map_op.py
Original file line number Diff line number Diff line change
Expand Up @@ -215,6 +215,7 @@ def plan_filter_op(
)

expression = op._filter_expr
predicate_expr = op._predicate_expr
compute = get_compute(op._compute)
if expression is not None:

Expand All @@ -234,6 +235,26 @@ def filter_batch_fn(block: "pa.Table") -> "pa.Table":
output_block_size_option=output_block_size_option,
)

elif predicate_expr is not None:
# Ray Data expression path using BlockAccessor
def filter_block_fn(block: Block) -> Block:
try:
block_accessor = BlockAccessor.for_block(block)
if not block_accessor.num_rows():
return block
return block_accessor.filter(predicate_expr)

except Exception as e:
_try_wrap_udf_exception(e)

transform_fn = BatchMapTransformFn(
_generate_transform_fn_for_map_batches(filter_block_fn),
batch_size=None,
batch_format=BatchFormat.ARROW,
zero_copy_batch=True,
is_udf=True,
output_block_size_option=output_block_size_option,
)
Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Bug: Import Error and Inconsistent Exception Handling

The BlockMapTransformFn class is used without being imported, which causes a NameError. Additionally, the predicate expression filter path lacks _try_wrap_udf_exception, leading to inconsistent error reporting for user code exceptions compared to the string expression path.

Fix in Cursor Fix in Web

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

... Ofc it's imported...

else:
udf_is_callable_class = isinstance(op._fn, CallableClass)
filter_fn, init_fn = _get_udf(
Expand Down
82 changes: 62 additions & 20 deletions python/ray/data/dataset.py
Original file line number Diff line number Diff line change
Expand Up @@ -1390,8 +1390,13 @@ def duplicate_row(row: Dict[str, Any]) -> List[Dict[str, Any]]:
@PublicAPI(api_group=BT_API_GROUP)
def filter(
self,
fn: Optional[UserDefinedFunction[Dict[str, Any], bool]] = None,
expr: Optional[str] = None,
fn: Optional[
UserDefinedFunction[Dict[str, Any], bool]
] = None, # TODO: Deprecate this parameter in favor of predicate
expr: Optional[
str
] = None, # TODO: Deprecate this parameter in favor of predicate
predicate: Optional[Expr] = None,
*,
compute: Union[str, ComputeStrategy] = None,
fn_args: Optional[Iterable[Any]] = None,
Expand Down Expand Up @@ -1420,17 +1425,22 @@ def filter(
Examples:

>>> import ray
>>> from ray.data.expressions import col
>>> ds = ray.data.range(100)
>>> ds.filter(expr="id <= 4").take_all()
>>> ds.filter(expr="id <= 4").take_all() # Will be deprecated in the future
[{'id': 0}, {'id': 1}, {'id': 2}, {'id': 3}, {'id': 4}]
>>> # Using predicate expressions
>>> ds.filter(predicate=(col("id") > 10) & (col("id") < 20)).take_all()
[{'id': 11}, {'id': 12}, {'id': 13}, {'id': 14}, {'id': 15}, {'id': 16}, {'id': 17}, {'id': 18}, {'id': 19}]

Time complexity: O(dataset size / parallelism)

Args:
fn: The predicate to apply to each row, or a class type
that can be instantiated to create such a callable.
expr: An expression string needs to be a valid Python expression that
will be converted to ``pyarrow.dataset.Expression`` type.
will be converted to ``pyarrow.dataset.Expression`` type. (Soon to be deprecated)
predicate: An expression that represents a predicate (boolean condition) for filtering.
fn_args: Positional arguments to pass to ``fn`` after the first argument.
These arguments are top-level arguments to the underlying Ray task.
fn_kwargs: Keyword arguments to pass to ``fn``. These arguments are
Expand Down Expand Up @@ -1479,9 +1489,33 @@ def filter(
:func:`ray.remote` for details.
"""
# Ensure exactly one of fn or expr is provided
resolved_expr = None
if not ((fn is None) ^ (expr is None)):
raise ValueError("Exactly one of 'fn' or 'expr' must be provided.")
# Ensure exactly one of fn, expr, or predicate is provided
provided_params = sum([fn is not None, expr is not None, predicate is not None])
if provided_params != 1:
raise ValueError(
"Exactly one of 'fn', 'expr', or 'predicate' must be provided."
)
if predicate is not None:
if (
fn_args is not None
or fn_kwargs is not None
or fn_constructor_args is not None
or fn_constructor_kwargs is not None
):
raise ValueError(
"when 'predicate' is used, 'fn_args/fn_kwargs' or 'fn_constructor_args/fn_constructor_kwargs' cannot be used."
)
from ray.data._internal.compute import TaskPoolStrategy

compute = TaskPoolStrategy(size=concurrency)
# Create Filter operator with predicate expression
filter_op = Filter(
input_op=self._logical_plan.dag,
predicate_expr=predicate,
compute=compute,
ray_remote_args_fn=ray_remote_args_fn,
ray_remote_args=ray_remote_args,
)
elif expr is not None:
if (
fn_args is not None
Expand All @@ -1503,6 +1537,14 @@ def filter(
resolved_expr = ExpressionEvaluator.get_filters(expression=expr)

compute = TaskPoolStrategy(size=concurrency)
# Create Filter operator with string expression
filter_op = Filter(
input_op=self._logical_plan.dag,
filter_expr=resolved_expr,
compute=compute,
ray_remote_args_fn=ray_remote_args_fn,
ray_remote_args=ray_remote_args,
)
else:
warnings.warn(
"Use 'expr' instead of 'fn' when possible for performant filters."
Expand All @@ -1520,6 +1562,18 @@ def filter(
f"fn must be a UserDefinedFunction, but got "
f"{type(fn).__name__} instead."
)
# Create Filter operator with function
filter_op = Filter(
input_op=self._logical_plan.dag,
fn=fn,
fn_args=fn_args,
fn_kwargs=fn_kwargs,
fn_constructor_args=fn_constructor_args,
fn_constructor_kwargs=fn_constructor_kwargs,
compute=compute,
ray_remote_args_fn=ray_remote_args_fn,
ray_remote_args=ray_remote_args,
)

ray_remote_args = merge_resources_to_ray_remote_args(
num_cpus,
Expand All @@ -1528,19 +1582,7 @@ def filter(
ray_remote_args,
)
plan = self._plan.copy()
op = Filter(
input_op=self._logical_plan.dag,
fn=fn,
fn_args=fn_args,
fn_kwargs=fn_kwargs,
fn_constructor_args=fn_constructor_args,
fn_constructor_kwargs=fn_constructor_kwargs,
filter_expr=resolved_expr,
compute=compute,
ray_remote_args_fn=ray_remote_args_fn,
ray_remote_args=ray_remote_args,
)
logical_plan = LogicalPlan(op, self.context)
logical_plan = LogicalPlan(filter_op, self.context)
return Dataset(plan, logical_plan)

@PublicAPI(api_group=SSR_API_GROUP)
Expand Down
Loading