-
Notifications
You must be signed in to change notification settings - Fork 6.9k
[Data] - Add Predicate Pushdown Rule #58150
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 8 commits
3830608
136a2c1
cf2b4a5
02c83e9
8012a50
00e2b5a
f73f834
e69d031
b08a6e9
775f1f5
82d516a
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -54,6 +54,7 @@ | |
| from ray.data.datasource.path_util import ( | ||
| _resolve_paths_and_filesystem, | ||
| ) | ||
| from ray.data.expressions import Expr | ||
| from ray.util.debug import log_once | ||
|
|
||
| if TYPE_CHECKING: | ||
|
|
@@ -284,7 +285,7 @@ def __init__( | |
| self._file_metadata_shuffler = None | ||
| self._include_paths = include_paths | ||
| self._partitioning = partitioning | ||
|
|
||
| self._predicate_expr: Optional[Expr] = None | ||
| if shuffle == "files": | ||
| self._file_metadata_shuffler = np.random.default_rng() | ||
| elif isinstance(shuffle, FileShuffleConfig): | ||
|
|
@@ -352,6 +353,12 @@ def get_read_tasks( | |
| ) | ||
|
|
||
| read_tasks = [] | ||
| filter_expr = ( | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. How much work is actually push our expr all the way into the reader itself? If not a lot let's do the right thing right away (otherwise do it in stacked PR)
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'm already pushing this into the reader.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I meant, threading of our expressions instead of PA ones
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'm not sure I follow. How will pyarrow accept Ray Data's Expressions? At some point we have to convert before calling
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Discussed offline. As part of the next PR, I'll refactor the remaining 2 functions that are not managed by Pyarrow to only pass in Ray Data's Expr |
||
| self._predicate_expr.to_pyarrow() | ||
| if self._predicate_expr is not None | ||
| else None | ||
| ) | ||
|
|
||
| for fragments, paths in zip( | ||
| np.array_split(pq_fragments, parallelism), | ||
| np.array_split(pq_paths, parallelism), | ||
|
|
@@ -401,6 +408,7 @@ def get_read_tasks( | |
| f, | ||
| include_paths, | ||
| partitioning, | ||
| filter_expr, | ||
| ), | ||
| meta, | ||
| schema=target_schema, | ||
|
|
@@ -424,6 +432,9 @@ def supports_distributed_reads(self) -> bool: | |
| def supports_projection_pushdown(self) -> bool: | ||
| return True | ||
|
|
||
| def supports_predicate_pushdown(self) -> bool: | ||
| return True | ||
|
|
||
| def get_current_projection(self) -> Optional[List[str]]: | ||
| # NOTE: In case there's no projection both file and partition columns | ||
| # will be none | ||
|
|
@@ -432,6 +443,9 @@ def get_current_projection(self) -> Optional[List[str]]: | |
|
|
||
| return (self._data_columns or []) + (self._partition_columns or []) | ||
|
|
||
| def get_current_predicate(self) -> Optional[Expr]: | ||
| return self._predicate_expr | ||
|
|
||
| def apply_projection( | ||
| self, | ||
| columns: Optional[List[str]], | ||
|
|
@@ -446,6 +460,37 @@ def apply_projection( | |
|
|
||
| return clone | ||
|
|
||
| # TODO: This should be moved to the Datasource class | ||
| def apply_predicate( | ||
| self, | ||
| predicate_expr: Expr, | ||
| ) -> "ParquetDatasource": | ||
| from ray.data._internal.planner.plan_expression.expression_visitors import ( | ||
| _ColumnSubstitutionVisitor, | ||
| ) | ||
| from ray.data.expressions import col | ||
|
|
||
| clone = copy.copy(self) | ||
| # Handle column renaming for Ray Data expressions | ||
| if self._data_columns_rename_map: | ||
| # Create mapping from new column names to old column names | ||
| # It's new to old mapping because we need to visit the predicate expression (which has all the new cols) | ||
| # and map them to the old columns so that the filtering can be pushed into the read tasks. | ||
| column_mapping = { | ||
| new_col: col(old_col) | ||
|
||
| for old_col, new_col in self._data_columns_rename_map.items() | ||
| } | ||
| visitor = _ColumnSubstitutionVisitor(column_mapping) | ||
| predicate_expr = visitor.visit(predicate_expr) | ||
|
|
||
| # Combine with existing predicate using AND | ||
| clone._predicate_expr = ( | ||
| predicate_expr | ||
| if clone._predicate_expr is None | ||
| else clone._predicate_expr & predicate_expr | ||
| ) | ||
| return clone | ||
goutamvenkat-anyscale marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
|
|
||
| def _estimate_in_mem_size(self, fragments: List[_ParquetFragment]) -> int: | ||
| in_mem_size = sum([f.file_size for f in fragments]) * self._encoding_ratio | ||
|
|
||
|
|
@@ -463,6 +508,7 @@ def read_fragments( | |
| fragments: List[_ParquetFragment], | ||
| include_paths: bool, | ||
| partitioning: Partitioning, | ||
| filter_expr: Optional["pyarrow.dataset.Expression"] = None, | ||
| ) -> Iterator["pyarrow.Table"]: | ||
| # This import is necessary to load the tensor extension type. | ||
| from ray.data.extensions.tensor_extension import ArrowTensorType # noqa | ||
|
|
@@ -484,6 +530,7 @@ def read_fragments( | |
| partition_columns=partition_columns, | ||
| partitioning=partitioning, | ||
| include_path=include_paths, | ||
| filter_expr=filter_expr, | ||
| batch_size=default_read_batch_size_rows, | ||
| to_batches_kwargs=to_batches_kwargs, | ||
| ), | ||
|
|
@@ -522,7 +569,14 @@ def _read_batches_from( | |
| # NOTE: Passed in kwargs overrides always take precedence | ||
| # TODO deprecate to_batches_kwargs | ||
| use_threads = to_batches_kwargs.pop("use_threads", use_threads) | ||
| filter_expr = to_batches_kwargs.pop("filter", filter_expr) | ||
| # TODO: We should deprecate filter through the read_parquet API and only allow through dataset.filter() | ||
| filter_from_kwargs = to_batches_kwargs.pop("filter", None) | ||
goutamvenkat-anyscale marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| if filter_from_kwargs is not None: | ||
| filter_expr = ( | ||
| filter_from_kwargs | ||
| if filter_expr is None | ||
| else filter_expr & filter_from_kwargs | ||
| ) | ||
| # NOTE: Arrow's ``to_batches`` expects ``batch_size`` as an int | ||
| if batch_size is not None: | ||
| to_batches_kwargs.setdefault("batch_size", batch_size) | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.