From dc7978e1d37ecdc76637eb33f9453b2da34b5d91 Mon Sep 17 00:00:00 2001 From: clarkzinzow Date: Fri, 23 Feb 2024 14:48:07 -0800 Subject: [PATCH] Remove old non-MicroPartition/non-scan operator path; refactor optimization tests to be based on structural equality. --- .github/workflows/python-package.yml | 5 - daft/daft.pyi | 11 +- daft/execution/execution_step.py | 137 +--- daft/execution/physical_plan.py | 74 +- daft/execution/rust_physical_plan_shim.py | 34 - daft/io/_csv.py | 4 +- daft/io/_iceberg.py | 2 +- daft/io/_json.py | 4 +- daft/io/_parquet.py | 4 +- daft/io/common.py | 74 +- daft/logical/builder.py | 27 +- daft/logical/schema.py | 6 - .../plan_scheduler/physical_plan_scheduler.py | 6 +- daft/runners/pyrunner.py | 22 +- daft/runners/ray_runner.py | 32 +- daft/runners/runner_io.py | 57 +- daft/table/__init__.py | 15 +- src/daft-core/src/python/schema.rs | 5 - src/daft-plan/src/builder.rs | 100 +-- src/daft-plan/src/logical_ops/project.rs | 18 +- src/daft-plan/src/logical_ops/source.rs | 13 +- .../src/optimization/logical_plan_tracker.rs | 42 +- src/daft-plan/src/optimization/mod.rs | 2 + src/daft-plan/src/optimization/optimizer.rs | 129 ++-- .../optimization/rules/drop_repartition.rs | 77 +- .../optimization/rules/push_down_filter.rs | 675 ++++++++++++------ .../src/optimization/rules/push_down_limit.rs | 259 +++---- .../rules/push_down_projection.rs | 223 +++--- src/daft-plan/src/optimization/test/mod.rs | 44 ++ src/daft-plan/src/physical_ops/csv.rs | 47 +- src/daft-plan/src/physical_ops/explode.rs | 14 +- src/daft-plan/src/physical_ops/json.rs | 47 +- src/daft-plan/src/physical_ops/mod.rs | 6 +- src/daft-plan/src/physical_ops/parquet.rs | 47 +- src/daft-plan/src/physical_ops/project.rs | 17 +- src/daft-plan/src/physical_plan.rs | 214 +----- src/daft-plan/src/planner.rs | 72 +- src/daft-plan/src/source_info/mod.rs | 91 +-- src/daft-plan/src/test/mod.rs | 48 +- src/daft-scan/src/anonymous.rs | 13 + src/daft-scan/src/glob.rs | 11 +- src/daft-scan/src/lib.rs | 18 +- src/daft-scan/src/python.rs | 4 + tests/dataframe/test_creation.py | 15 +- .../io/parquet/test_reads_s3_minio.py | 9 +- tests/io/test_merge_scan_tasks.py | 5 - 46 files changed, 1058 insertions(+), 1721 deletions(-) create mode 100644 src/daft-plan/src/optimization/test/mod.rs diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index 69e42b4378..72299123ce 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -92,7 +92,6 @@ jobs: # cargo llvm-cov --no-run --lcov --output-path report-output/rust-coverage-${{ runner.os }}-${{ matrix.python-version }}-${{ matrix.daft-runner }}-${{ matrix.pyarrow-version }}.lcov env: DAFT_RUNNER: ${{ matrix.daft-runner }} - DAFT_MICROPARTITIONS: ${{ matrix.micropartitions }} - name: Build library and Test with pytest (windows) if: ${{ (matrix.os == 'windows') }} run: | @@ -222,7 +221,6 @@ jobs: pytest tests/integration/test_tpch.py --durations=50 env: DAFT_RUNNER: ${{ matrix.daft-runner }} - DAFT_MICROPARTITIONS: ${{ matrix.micropartitions }} - name: Send Slack notification on failure uses: slackapi/slack-github-action@v1.24.0 if: ${{ failure() && (github.ref == 'refs/heads/main') }} @@ -298,7 +296,6 @@ jobs: pytest tests/integration/io -m 'integration' --durations=50 env: DAFT_RUNNER: ${{ matrix.daft-runner }} - DAFT_MICROPARTITIONS: ${{ matrix.micropartitions }} - name: Send Slack notification on failure uses: slackapi/slack-github-action@v1.24.0 if: ${{ failure() && (github.ref == 'refs/heads/main') }} @@ -395,7 +392,6 @@ jobs: pytest tests/integration/io -m 'integration' --credentials --durations=50 env: DAFT_RUNNER: ${{ matrix.daft-runner }} - DAFT_MICROPARTITIONS: ${{ matrix.micropartitions }} - name: Send Slack notification on failure uses: slackapi/slack-github-action@v1.24.0 if: ${{ failure() }} @@ -475,7 +471,6 @@ jobs: pytest tests/integration/iceberg -m 'integration' --durations=50 env: DAFT_RUNNER: ${{ matrix.daft-runner }} - DAFT_MICROPARTITIONS: ${{ matrix.micropartitions }} - name: Send Slack notification on failure uses: slackapi/slack-github-action@v1.24.0 if: ${{ failure() && (github.ref == 'refs/heads/main') }} diff --git a/daft/daft.pyi b/daft/daft.pyi index fc2ddbd8ac..fd53e76d89 100644 --- a/daft/daft.pyi +++ b/daft/daft.pyi @@ -845,7 +845,6 @@ class PySchema: def __getitem__(self, name: str) -> PyField: ... def names(self) -> list[str]: ... def union(self, other: PySchema) -> PySchema: ... - def apply_hints(self, other: PySchema) -> PySchema: ... def eq(self, other: PySchema) -> bool: ... @staticmethod def from_field_name_and_types(names_and_types: list[tuple[str, PyDataType]]) -> PySchema: ... @@ -1157,9 +1156,7 @@ class PhysicalPlanScheduler: def num_partitions(self) -> int: ... def partition_spec(self) -> PartitionSpec: ... def repr_ascii(self, simple: bool) -> str: ... - def to_partition_tasks( - self, psets: dict[str, list[PartitionT]], is_ray_runner: bool - ) -> physical_plan.InProgressPhysicalPlan: ... + def to_partition_tasks(self, psets: dict[str, list[PartitionT]]) -> physical_plan.InProgressPhysicalPlan: ... class LogicalPlanBuilder: """ @@ -1175,11 +1172,7 @@ class LogicalPlanBuilder: partition_key: str, cache_entry: PartitionCacheEntry, schema: PySchema, num_partitions: int, size_bytes: int ) -> LogicalPlanBuilder: ... @staticmethod - def table_scan_with_scan_operator(scan_operator: ScanOperatorHandle) -> LogicalPlanBuilder: ... - @staticmethod - def table_scan( - file_infos: FileInfos, schema: PySchema, file_format_config: FileFormatConfig, storage_config: StorageConfig - ) -> LogicalPlanBuilder: ... + def table_scan(scan_operator: ScanOperatorHandle) -> LogicalPlanBuilder: ... def project(self, projection: list[PyExpr], resource_request: ResourceRequest) -> LogicalPlanBuilder: ... def filter(self, predicate: PyExpr) -> LogicalPlanBuilder: ... def limit(self, limit: int, eager: bool) -> LogicalPlanBuilder: ... diff --git a/daft/execution/execution_step.py b/daft/execution/execution_step.py index 73857cc4bb..5bd5e483df 100644 --- a/daft/execution/execution_step.py +++ b/daft/execution/execution_step.py @@ -11,18 +11,7 @@ else: from typing import Protocol -from daft.daft import ( - CsvSourceConfig, - FileFormat, - FileFormatConfig, - IOConfig, - JoinType, - JsonReadOptions, - JsonSourceConfig, - ParquetSourceConfig, - ResourceRequest, - StorageConfig, -) +from daft.daft import FileFormat, IOConfig, JoinType, ResourceRequest from daft.expressions import Expression, ExpressionsProjection, col from daft.logical.map_partition_ops import MapPartitionOp from daft.logical.schema import Schema @@ -32,8 +21,6 @@ PartialPartitionMetadata, PartitionMetadata, PartitionT, - TableParseCSVOptions, - TableReadOptions, ) from daft.table import MicroPartition, table_io @@ -303,128 +290,6 @@ def num_outputs(self) -> int: return 1 -@dataclass(frozen=True) -class ReadFile(SingleOutputInstruction): - index: int | None - # Known number of rows. - file_rows: int | None - # Max number of rows to read. - limit_rows: int | None - schema: Schema - storage_config: StorageConfig - columns_to_read: list[str] | None - file_format_config: FileFormatConfig - - def run(self, inputs: list[MicroPartition]) -> list[MicroPartition]: - return self._read_file(inputs) - - def _read_file(self, inputs: list[MicroPartition]) -> list[MicroPartition]: - assert len(inputs) == 1 - [filepaths_partition] = inputs - partition = self._handle_tabular_files_scan( - filepaths_partition=filepaths_partition, - ) - return [partition] - - def run_partial_metadata(self, input_metadatas: list[PartialPartitionMetadata]) -> list[PartialPartitionMetadata]: - assert len(input_metadatas) == 1 - - num_rows = self.file_rows - # Only take the file read limit into account if we know how big the file is to begin with. - if num_rows is not None and self.limit_rows is not None: - num_rows = min(num_rows, self.limit_rows) - - return [ - PartialPartitionMetadata( - num_rows=num_rows, - size_bytes=None, - ) - ] - - def _handle_tabular_files_scan( - self, - filepaths_partition: MicroPartition, - ) -> MicroPartition: - data = filepaths_partition.to_pydict() - filepaths = data["path"] - - if self.index is not None: - filepaths = [filepaths[self.index]] - - # Common options for reading vPartition - read_options = TableReadOptions( - num_rows=self.limit_rows, - column_names=self.columns_to_read, # read only specified columns - ) - - file_format = self.file_format_config.file_format() - format_config = self.file_format_config.config - if file_format == FileFormat.Csv: - assert isinstance(format_config, CsvSourceConfig) - table = MicroPartition.concat( - [ - table_io.read_csv( - file=fp, - schema=self.schema, - storage_config=self.storage_config, - csv_options=TableParseCSVOptions( - delimiter=format_config.delimiter, - header_index=0 if format_config.has_headers else None, - double_quote=format_config.double_quote, - quote=format_config.quote, - escape_char=format_config.escape_char, - comment=format_config.comment, - buffer_size=format_config.buffer_size, - chunk_size=format_config.chunk_size, - ), - read_options=read_options, - ) - for fp in filepaths - ] - ) - elif file_format == FileFormat.Json: - assert isinstance(format_config, JsonSourceConfig) - table = MicroPartition.concat( - [ - table_io.read_json( - file=fp, - schema=self.schema, - storage_config=self.storage_config, - json_read_options=JsonReadOptions( - buffer_size=format_config.buffer_size, chunk_size=format_config.chunk_size - ), - read_options=read_options, - ) - for fp in filepaths - ] - ) - elif file_format == FileFormat.Parquet: - assert isinstance(format_config, ParquetSourceConfig) - table = MicroPartition.concat( - [ - table_io.read_parquet( - file=fp, - schema=self.schema, - storage_config=self.storage_config, - read_options=read_options, - ) - for fp in filepaths - ] - ) - else: - raise NotImplementedError(f"PyRunner has not implemented scan: {file_format}") - - expected_schema = ( - Schema._from_fields([self.schema[name] for name in read_options.column_names]) - if read_options.column_names is not None - else self.schema - ) - assert ( - table.schema() == expected_schema - ), f"Expected table to have schema:\n{expected_schema}\n\nReceived instead:\n{table.schema()}" - return table - - @dataclass(frozen=True) class WriteFile(SingleOutputInstruction): file_format: FileFormat diff --git a/daft/execution/physical_plan.py b/daft/execution/physical_plan.py index f7509493b3..6989915e77 100644 --- a/daft/execution/physical_plan.py +++ b/daft/execution/physical_plan.py @@ -22,14 +22,7 @@ from typing import Generator, Generic, Iterable, Iterator, TypeVar, Union from daft.context import get_context -from daft.daft import ( - FileFormat, - FileFormatConfig, - IOConfig, - JoinType, - ResourceRequest, - StorageConfig, -) +from daft.daft import FileFormat, IOConfig, JoinType, ResourceRequest from daft.execution import execution_step from daft.execution.execution_step import ( Instruction, @@ -84,71 +77,6 @@ def partition_read( ) -def file_read( - child_plan: InProgressPhysicalPlan[PartitionT], - # Max number of rows to read. - limit_rows: int | None, - schema: Schema, - storage_config: StorageConfig, - columns_to_read: list[str] | None, - file_format_config: FileFormatConfig, -) -> InProgressPhysicalPlan[PartitionT]: - """child_plan represents partitions with filenames. - - Yield a plan to read those filenames. - """ - materializations: deque[SingleOutputPartitionTask[PartitionT]] = deque() - stage_id = next(stage_id_counter) - output_partition_index = 0 - - while True: - # Check if any inputs finished executing. - while len(materializations) > 0 and materializations[0].done(): - done_task = materializations.popleft() - - vpartition = done_task.vpartition() - file_infos = vpartition.to_pydict() - file_sizes_bytes = file_infos["size"] - file_rows = file_infos["num_rows"] - - # Emit one partition for each file (NOTE: hardcoded for now). - for i in range(len(vpartition)): - file_read_step = PartitionTaskBuilder[PartitionT]( - inputs=[done_task.partition()], - partial_metadatas=None, # Child's metadata doesn't really matter for a file read - ).add_instruction( - instruction=execution_step.ReadFile( - index=i, - file_rows=file_rows[i], - limit_rows=limit_rows, - schema=schema, - storage_config=storage_config, - columns_to_read=columns_to_read, - file_format_config=file_format_config, - ), - # Set the filesize as the memory request. - # (Note: this is very conservative; file readers empirically use much more peak memory than 1x file size.) - resource_request=ResourceRequest(memory_bytes=file_sizes_bytes[i]), - ) - yield file_read_step - output_partition_index += 1 - - # Materialize a single dependency. - try: - child_step = next(child_plan) - if isinstance(child_step, PartitionTaskBuilder): - child_step = child_step.finalize_partition_task_single_output(stage_id=stage_id) - materializations.append(child_step) - yield child_step - - except StopIteration: - if len(materializations) > 0: - logger.debug("file_read blocked on completion of first source in: %s", materializations) - yield None - else: - return - - def file_write( child_plan: InProgressPhysicalPlan[PartitionT], file_format: FileFormat, diff --git a/daft/execution/rust_physical_plan_shim.py b/daft/execution/rust_physical_plan_shim.py index 25db7bec7a..9b3c94b9f8 100644 --- a/daft/execution/rust_physical_plan_shim.py +++ b/daft/execution/rust_physical_plan_shim.py @@ -1,19 +1,15 @@ from __future__ import annotations from dataclasses import dataclass -from typing import Iterator, cast from daft.daft import ( FileFormat, - FileFormatConfig, IOConfig, JoinType, PyExpr, PySchema, - PyTable, ResourceRequest, ScanTask, - StorageConfig, ) from daft.execution import execution_step, physical_plan from daft.expressions import Expression, ExpressionsProjection @@ -94,36 +90,6 @@ def run_partial_metadata(self, input_metadatas: list[PartialPartitionMetadata]) ] -def tabular_scan( - schema: PySchema, - columns_to_read: list[str] | None, - file_info_table: PyTable, - file_format_config: FileFormatConfig, - storage_config: StorageConfig, - limit: int, - is_ray_runner: bool, -) -> physical_plan.InProgressPhysicalPlan[PartitionT]: - # TODO(Clark): Fix this Ray runner hack. - part = MicroPartition._from_pytable(file_info_table) - if is_ray_runner: - import ray - - parts = [ray.put(part)] - else: - parts = [part] - parts_t = cast(Iterator[PartitionT], parts) - - file_info_iter = physical_plan.partition_read(iter(parts_t)) - return physical_plan.file_read( - child_plan=file_info_iter, - limit_rows=limit, - schema=Schema._from_pyschema(schema), - storage_config=storage_config, - columns_to_read=columns_to_read, - file_format_config=file_format_config, - ) - - def project( input: physical_plan.InProgressPhysicalPlan[PartitionT], projection: list[PyExpr], resource_request: ResourceRequest ) -> physical_plan.InProgressPhysicalPlan[PartitionT]: diff --git a/daft/io/_csv.py b/daft/io/_csv.py index ec3b1f1e7f..aa30959dad 100644 --- a/daft/io/_csv.py +++ b/daft/io/_csv.py @@ -14,7 +14,7 @@ ) from daft.dataframe import DataFrame from daft.datatype import DataType -from daft.io.common import _get_tabular_files_scan +from daft.io.common import get_tabular_files_scan @PublicAPI @@ -84,5 +84,5 @@ def read_csv( storage_config = StorageConfig.native(NativeStorageConfig(True, io_config)) else: storage_config = StorageConfig.python(PythonStorageConfig(io_config=io_config)) - builder = _get_tabular_files_scan(path, schema_hints, file_format_config, storage_config=storage_config) + builder = get_tabular_files_scan(path, schema_hints, file_format_config, storage_config=storage_config) return DataFrame(builder) diff --git a/daft/io/_iceberg.py b/daft/io/_iceberg.py index c936779d3d..fa14dca710 100644 --- a/daft/io/_iceberg.py +++ b/daft/io/_iceberg.py @@ -109,5 +109,5 @@ def read_iceberg( iceberg_operator = IcebergScanOperator(pyiceberg_table, storage_config=storage_config) handle = ScanOperatorHandle.from_python_scan_operator(iceberg_operator) - builder = LogicalPlanBuilder.from_tabular_scan_with_scan_operator(scan_operator=handle) + builder = LogicalPlanBuilder.from_tabular_scan(scan_operator=handle) return DataFrame(builder) diff --git a/daft/io/_json.py b/daft/io/_json.py index c052fb2c52..95a00fb449 100644 --- a/daft/io/_json.py +++ b/daft/io/_json.py @@ -14,7 +14,7 @@ ) from daft.dataframe import DataFrame from daft.datatype import DataType -from daft.io.common import _get_tabular_files_scan +from daft.io.common import get_tabular_files_scan @PublicAPI @@ -56,5 +56,5 @@ def read_json( storage_config = StorageConfig.native(NativeStorageConfig(True, io_config)) else: storage_config = StorageConfig.python(PythonStorageConfig(io_config=io_config)) - builder = _get_tabular_files_scan(path, schema_hints, file_format_config, storage_config=storage_config) + builder = get_tabular_files_scan(path, schema_hints, file_format_config, storage_config=storage_config) return DataFrame(builder) diff --git a/daft/io/_parquet.py b/daft/io/_parquet.py index a41d82ec64..9f77f28bb0 100644 --- a/daft/io/_parquet.py +++ b/daft/io/_parquet.py @@ -14,7 +14,7 @@ ) from daft.dataframe import DataFrame from daft.datatype import DataType -from daft.io.common import _get_tabular_files_scan +from daft.io.common import get_tabular_files_scan @PublicAPI @@ -61,5 +61,5 @@ def read_parquet( else: storage_config = StorageConfig.python(PythonStorageConfig(io_config=io_config)) - builder = _get_tabular_files_scan(path, schema_hints, file_format_config, storage_config=storage_config) + builder = get_tabular_files_scan(path, schema_hints, file_format_config, storage_config=storage_config) return DataFrame(builder) diff --git a/daft/io/common.py b/daft/io/common.py index a55ab92992..3668963c3d 100644 --- a/daft/io/common.py +++ b/daft/io/common.py @@ -1,16 +1,8 @@ from __future__ import annotations -import os from typing import TYPE_CHECKING -from daft.context import get_context -from daft.daft import ( - FileFormatConfig, - NativeStorageConfig, - PythonStorageConfig, - ScanOperatorHandle, - StorageConfig, -) +from daft.daft import FileFormatConfig, ScanOperatorHandle, StorageConfig from daft.datatype import DataType from daft.logical.builder import LogicalPlanBuilder from daft.logical.schema import Schema @@ -26,7 +18,7 @@ def _get_schema_from_hints(hints: dict[str, DataType]) -> Schema: raise NotImplementedError(f"Unsupported schema hints: {type(hints)}") -def _get_tabular_files_scan( +def get_tabular_files_scan( path: str | list[str], schema_hints: dict[str, DataType] | None, file_format_config: FileFormatConfig, @@ -34,59 +26,23 @@ def _get_tabular_files_scan( ) -> LogicalPlanBuilder: """Returns a TabularFilesScan LogicalPlan for a given glob filepath.""" # Glob the path using the Runner - # NOTE: Globbing will always need the IOConfig, regardless of whether "native reads" are used - io_config = None - if isinstance(storage_config.config, NativeStorageConfig): - io_config = storage_config.config.io_config - elif isinstance(storage_config.config, PythonStorageConfig): - io_config = storage_config.config.io_config - else: - raise NotImplementedError(f"Tabular scan with config not implemented: {storage_config.config}") - schema_hint = _get_schema_from_hints(schema_hints) if schema_hints is not None else None - ### FEATURE_FLAG: $DAFT_MICROPARTITIONS - # - # This environment variable will make Daft use the new "v2 scans" and MicroPartitions when building Daft logical plans - if os.getenv("DAFT_MICROPARTITIONS", "1") == "1": - scan_op: ScanOperatorHandle - if isinstance(path, list): - scan_op = ScanOperatorHandle.glob_scan( - path, - file_format_config, - storage_config, - schema_hint=schema_hint._schema if schema_hint is not None else None, - ) - elif isinstance(path, str): - scan_op = ScanOperatorHandle.glob_scan( - [path], - file_format_config, - storage_config, - schema_hint=schema_hint._schema if schema_hint is not None else None, - ) - else: - raise NotImplementedError(f"_get_tabular_files_scan cannot construct ScanOperatorHandle for input: {path}") - - builder = LogicalPlanBuilder.from_tabular_scan_with_scan_operator( - scan_operator=scan_op, - ) - return builder - - paths = path if isinstance(path, list) else [str(path)] - runner_io = get_context().runner().runner_io() - file_infos = runner_io.glob_paths_details(paths, file_format_config=file_format_config, io_config=io_config) + if isinstance(path, list): + paths = path + elif isinstance(path, str): + paths = [path] + else: + raise NotImplementedError(f"get_tabular_files_scan cannot construct ScanOperatorHandle for input: {path}") - # Infer schema - schema = runner_io.get_schema_from_first_filepath(file_infos, file_format_config, storage_config) + scan_op = ScanOperatorHandle.glob_scan( + paths, + file_format_config, + storage_config, + schema_hint=schema_hint._schema if schema_hint is not None else None, + ) - # Apply hints from schema_hints if provided - if schema_hint is not None: - schema = schema.apply_hints(schema_hint) - # Construct plan builder = LogicalPlanBuilder.from_tabular_scan( - file_infos=file_infos, - schema=schema, - file_format_config=file_format_config, - storage_config=storage_config, + scan_operator=scan_op, ) return builder diff --git a/daft/logical/builder.py b/daft/logical/builder.py index f5ad0451aa..996c52ec64 100644 --- a/daft/logical/builder.py +++ b/daft/logical/builder.py @@ -3,22 +3,13 @@ import pathlib from typing import TYPE_CHECKING -from daft.daft import ( - CountMode, - FileFormat, - FileFormatConfig, - FileInfos, - IOConfig, - JoinStrategy, - JoinType, -) +from daft.daft import CountMode, FileFormat, IOConfig, JoinStrategy, JoinType from daft.daft import LogicalPlanBuilder as _LogicalPlanBuilder from daft.daft import ( PartitionScheme, PyDaftExecutionConfig, ResourceRequest, ScanOperatorHandle, - StorageConfig, ) from daft.expressions import Expression, col from daft.logical.schema import Schema @@ -82,25 +73,13 @@ def from_in_memory_scan( ) return cls(builder) - @classmethod - def from_tabular_scan_with_scan_operator( - cls, - *, - scan_operator: ScanOperatorHandle, - ) -> LogicalPlanBuilder: - builder = _LogicalPlanBuilder.table_scan_with_scan_operator(scan_operator) - return cls(builder) - @classmethod def from_tabular_scan( cls, *, - file_infos: FileInfos, - schema: Schema, - file_format_config: FileFormatConfig, - storage_config: StorageConfig, + scan_operator: ScanOperatorHandle, ) -> LogicalPlanBuilder: - builder = _LogicalPlanBuilder.table_scan(file_infos, schema._schema, file_format_config, storage_config) + builder = _LogicalPlanBuilder.table_scan(scan_operator) return cls(builder) def project( diff --git a/daft/logical/schema.py b/daft/logical/schema.py index 9a1950db14..2655885bb9 100644 --- a/daft/logical/schema.py +++ b/daft/logical/schema.py @@ -145,12 +145,6 @@ def union(self, other: Schema) -> Schema: return Schema._from_pyschema(self._schema.union(other._schema)) - def apply_hints(self, other: Schema) -> Schema: - if not isinstance(other, Schema): - raise ValueError(f"Expected Schema, got other: {type(other)}") - - return Schema._from_pyschema(self._schema.apply_hints(other._schema)) - def __reduce__(self) -> tuple: return Schema._from_pyschema, (self._schema,) diff --git a/daft/plan_scheduler/physical_plan_scheduler.py b/daft/plan_scheduler/physical_plan_scheduler.py index 110287de4d..459da8ed3a 100644 --- a/daft/plan_scheduler/physical_plan_scheduler.py +++ b/daft/plan_scheduler/physical_plan_scheduler.py @@ -32,7 +32,5 @@ def pretty_print(self, simple: bool = False) -> str: def __repr__(self) -> str: return self._scheduler.repr_ascii(simple=False) - def to_partition_tasks( - self, psets: dict[str, list[PartitionT]], is_ray_runner: bool - ) -> physical_plan.MaterializedPhysicalPlan: - return physical_plan.materialize(self._scheduler.to_partition_tasks(psets, is_ray_runner)) + def to_partition_tasks(self, psets: dict[str, list[PartitionT]]) -> physical_plan.MaterializedPhysicalPlan: + return physical_plan.materialize(self._scheduler.to_partition_tasks(psets)) diff --git a/daft/runners/pyrunner.py b/daft/runners/pyrunner.py index 41b435ff24..8bbd6fd03d 100644 --- a/daft/runners/pyrunner.py +++ b/daft/runners/pyrunner.py @@ -9,19 +9,12 @@ import psutil from daft.context import get_context -from daft.daft import ( - FileFormatConfig, - FileInfos, - IOConfig, - ResourceRequest, - StorageConfig, -) +from daft.daft import FileFormatConfig, FileInfos, IOConfig, ResourceRequest from daft.execution import physical_plan from daft.execution.execution_step import Instruction, PartitionTask from daft.filesystem import glob_path_with_stats from daft.internal.gpu import cuda_device_count from daft.logical.builder import LogicalPlanBuilder -from daft.logical.schema import Schema from daft.runners import runner_io from daft.runners.partitioning import ( MaterializedResult, @@ -113,17 +106,6 @@ def glob_paths_details( return file_infos - def get_schema_from_first_filepath( - self, - file_infos: FileInfos, - file_format_config: FileFormatConfig, - storage_config: StorageConfig, - ) -> Schema: - if len(file_infos) == 0: - raise ValueError("No files to get schema from") - # Naively retrieve the first filepath in the PartitionSet - return runner_io.sample_schema(file_infos[0].file_path, file_format_config, storage_config) - class PyRunner(Runner[MicroPartition]): def __init__(self, use_thread_pool: bool | None) -> None: @@ -163,7 +145,7 @@ def run_iter( plan_scheduler = builder.to_physical_plan_scheduler(daft_execution_config) psets = {k: v.values() for k, v in self._part_set_cache.get_all_partition_sets().items()} # Get executable tasks from planner. - tasks = plan_scheduler.to_partition_tasks(psets, is_ray_runner=False) + tasks = plan_scheduler.to_partition_tasks(psets) with profiler("profile_PyRunner.run_{datetime.now().isoformat()}.json"): results_gen = self._physical_plan_to_partitions(tasks) yield from results_gen diff --git a/daft/runners/ray_runner.py b/daft/runners/ray_runner.py index e2da71b62b..33cd2cf568 100644 --- a/daft/runners/ray_runner.py +++ b/daft/runners/ray_runner.py @@ -32,7 +32,6 @@ IOConfig, PyDaftExecutionConfig, ResourceRequest, - StorageConfig, ) from daft.datatype import DataType from daft.execution.execution_step import ( @@ -126,17 +125,6 @@ def _to_pandas_ref(df: pd.DataFrame | ray.ObjectRef[pd.DataFrame]) -> ray.Object raise ValueError("Expected a Ray object ref or a Pandas DataFrame, " f"got {type(df)}") -@ray.remote -def sample_schema_from_filepath( - first_file_path: str, - file_format_config: FileFormatConfig, - storage_config: StorageConfig, -) -> Schema: - """Ray remote function to run schema sampling on top of a MicroPartition containing a single filepath""" - # Currently just samples the Schema from the first file - return runner_io.sample_schema(first_file_path, file_format_config, storage_config) - - @dataclass class RayPartitionSet(PartitionSet[ray.ObjectRef]): _results: dict[PartID, RayMaterializedResult] @@ -244,24 +232,6 @@ def glob_paths_details( ._table ) - def get_schema_from_first_filepath( - self, - file_infos: FileInfos, - file_format_config: FileFormatConfig, - storage_config: StorageConfig, - ) -> Schema: - if len(file_infos) == 0: - raise ValueError("No files to get schema from") - # Naively retrieve the first filepath in the file info table. - first_path = file_infos[0].file_path - return ray.get( - sample_schema_from_filepath.remote( - first_path, - file_format_config, - storage_config, - ) - ) - def partition_set_from_ray_dataset( self, ds: RayDataset, @@ -507,7 +477,7 @@ def _run_plan( result_uuid: str, ) -> None: # Get executable tasks from plan scheduler. - tasks = plan_scheduler.to_partition_tasks(psets, is_ray_runner=True) + tasks = plan_scheduler.to_partition_tasks(psets) daft_execution_config = self.execution_configs_objref_by_df[result_uuid] inflight_tasks: dict[str, PartitionTask[ray.ObjectRef]] = dict() diff --git a/daft/runners/runner_io.py b/daft/runners/runner_io.py index c03c7fcac8..980a855875 100644 --- a/daft/runners/runner_io.py +++ b/daft/runners/runner_io.py @@ -3,19 +3,7 @@ from abc import abstractmethod from typing import TYPE_CHECKING -from daft.daft import ( - CsvSourceConfig, - FileFormat, - FileFormatConfig, - FileInfos, - IOConfig, - JsonSourceConfig, - ParquetSourceConfig, - StorageConfig, -) -from daft.logical.schema import Schema -from daft.runners.partitioning import TableParseCSVOptions -from daft.table import schema_inference +from daft.daft import FileFormatConfig, FileInfos, IOConfig if TYPE_CHECKING: pass @@ -43,46 +31,3 @@ def glob_paths_details( FileInfo: The file infos for the globbed paths. """ raise NotImplementedError() - - @abstractmethod - def get_schema_from_first_filepath( - self, - file_infos: FileInfos, - file_format_config: FileFormatConfig, - storage_config: StorageConfig, - ) -> Schema: - raise NotImplementedError() - - -def sample_schema( - filepath: str, - file_format_config: FileFormatConfig, - storage_config: StorageConfig, -) -> Schema: - """Helper method that samples a schema from the specified source""" - file_format = file_format_config.file_format() - config = file_format_config.config - if file_format == FileFormat.Csv: - assert isinstance(config, CsvSourceConfig) - return schema_inference.from_csv( - file=filepath, - storage_config=storage_config, - csv_options=TableParseCSVOptions( - delimiter=config.delimiter, - header_index=0 if config.has_headers else None, - ), - ) - elif file_format == FileFormat.Json: - assert isinstance(config, JsonSourceConfig) - return schema_inference.from_json( - file=filepath, - storage_config=storage_config, - ) - elif file_format == FileFormat.Parquet: - assert isinstance(config, ParquetSourceConfig) - return schema_inference.from_parquet( - file=filepath, - storage_config=storage_config, - ) - else: - raise NotImplementedError(f"Schema inference for {file_format} not implemented") diff --git a/daft/table/__init__.py b/daft/table/__init__.py index 5b66e7b616..5ecab85779 100644 --- a/daft/table/__init__.py +++ b/daft/table/__init__.py @@ -1,18 +1,7 @@ from __future__ import annotations -import os - -from .table import Table, read_parquet_into_pyarrow, read_parquet_into_pyarrow_bulk - # Need to import after `.table` due to circular dep issue otherwise -from .micropartition import MicroPartition as _MicroPartition # isort:skip - - -MicroPartition = _MicroPartition - -# Use $DAFT_MICROPARTITIONS envvar as a feature flag to turn off MicroPartitions -if os.getenv("DAFT_MICROPARTITIONS", "1") != "1": - MicroPartition = Table # type: ignore - +from .micropartition import MicroPartition +from .table import Table, read_parquet_into_pyarrow, read_parquet_into_pyarrow_bulk __all__ = ["MicroPartition", "Table", "read_parquet_into_pyarrow", "read_parquet_into_pyarrow_bulk"] diff --git a/src/daft-core/src/python/schema.rs b/src/daft-core/src/python/schema.rs index f4f50bdadf..975aeffc5c 100644 --- a/src/daft-core/src/python/schema.rs +++ b/src/daft-core/src/python/schema.rs @@ -33,11 +33,6 @@ impl PySchema { Ok(new_schema.into()) } - pub fn apply_hints(&self, hints: &PySchema) -> PyResult { - let new_schema = Arc::new(self.schema.apply_hints(&hints.schema)?); - Ok(new_schema.into()) - } - pub fn eq(&self, other: &PySchema) -> PyResult { Ok(self.schema.fields.eq(&other.schema.fields)) } diff --git a/src/daft-plan/src/builder.rs b/src/daft-plan/src/builder.rs index 6c4b434c11..1fa4819413 100644 --- a/src/daft-plan/src/builder.rs +++ b/src/daft-plan/src/builder.rs @@ -7,10 +7,7 @@ use crate::{ partitioning::PartitionSchemeConfig, planner::plan, sink_info::{OutputFileInfo, SinkInfo}, - source_info::{ - ExternalInfo as ExternalSourceInfo, FileInfos as InputFileInfos, LegacyExternalInfo, - SourceInfo, - }, + source_info::SourceInfo, JoinStrategy, JoinType, PartitionScheme, PhysicalPlanScheduler, ResourceRequest, }; use common_error::{DaftError, DaftResult}; @@ -18,11 +15,7 @@ use common_io_config::IOConfig; use daft_core::schema::Schema; use daft_core::schema::SchemaRef; use daft_dsl::Expr; -use daft_scan::{ - file_format::{FileFormat, FileFormatConfig}, - storage_config::{PyStorageConfig, StorageConfig}, - Pushdowns, ScanExternalInfo, ScanOperatorRef, -}; +use daft_scan::{file_format::FileFormat, Pushdowns, ScanExternalInfo, ScanOperatorRef}; #[cfg(feature = "python")] use { @@ -30,7 +23,7 @@ use { common_daft_config::PyDaftExecutionConfig, daft_core::python::schema::PySchema, daft_dsl::python::PyExpr, - daft_scan::{file_format::PyFileFormatConfig, python::pylib::ScanOperatorHandle}, + daft_scan::python::pylib::ScanOperatorHandle, pyo3::prelude::*, }; @@ -39,7 +32,7 @@ use { /// /// This builder holds the current root (sink) of the logical plan, and the building methods return /// a brand new builder holding a new plan; i.e., this is an immutable builder. -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct LogicalPlanBuilder { // The current root of the logical plan in this builder. pub plan: Arc, @@ -72,56 +65,33 @@ impl LogicalPlanBuilder { Ok(logical_plan.into()) } - pub fn table_scan_with_scan_operator( + pub fn table_scan( scan_operator: ScanOperatorRef, pushdowns: Option, ) -> DaftResult { let schema = scan_operator.0.schema(); let partitioning_keys = scan_operator.0.partitioning_keys(); - let source_info = - SourceInfo::ExternalInfo(ExternalSourceInfo::Scan(ScanExternalInfo::new( - scan_operator.clone(), - schema.clone(), - partitioning_keys.into(), - pushdowns.unwrap_or_default(), - ))); - let logical_plan: LogicalPlan = - logical_ops::Source::new(schema.clone(), source_info.into()).into(); - Ok(logical_plan.into()) - } - - pub fn table_scan( - file_infos: InputFileInfos, - schema: Arc, - file_format_config: Arc, - storage_config: Arc, - ) -> DaftResult { - Self::table_scan_with_pushdowns( - file_infos, - schema, - file_format_config, - storage_config, - Default::default(), - ) - } - - pub fn table_scan_with_pushdowns( - file_infos: InputFileInfos, - schema: Arc, - file_format_config: Arc, - storage_config: Arc, - pushdowns: Pushdowns, - ) -> DaftResult { - let source_info = - SourceInfo::ExternalInfo(ExternalSourceInfo::Legacy(LegacyExternalInfo::new( - schema.clone(), - file_infos.into(), - file_format_config, - storage_config, - pushdowns, - ))); + let source_info = SourceInfo::ExternalInfo(ScanExternalInfo::new( + scan_operator.clone(), + schema.clone(), + partitioning_keys.into(), + pushdowns.clone().unwrap_or_default(), + )); + // If column selection (projection) pushdown is specified, prune unselected columns from the schema. + let output_schema = if let Some(Pushdowns { columns: Some(columns), .. }) = &pushdowns && columns.len() < schema.fields.len() { + let pruned_upstream_schema = schema + .fields + .iter() + .filter_map(|(name, field)| { + columns.contains(name).then(|| field.clone()) + }) + .collect::>(); + Arc::new(Schema::new(pruned_upstream_schema)?) + } else { + schema.clone() + }; let logical_plan: LogicalPlan = - logical_ops::Source::new(schema.clone(), source_info.into()).into(); + logical_ops::Source::new(output_schema, source_info.into()).into(); Ok(logical_plan.into()) } @@ -323,24 +293,8 @@ impl PyLogicalPlanBuilder { } #[staticmethod] - pub fn table_scan_with_scan_operator(scan_operator: ScanOperatorHandle) -> PyResult { - Ok(LogicalPlanBuilder::table_scan_with_scan_operator(scan_operator.into(), None)?.into()) - } - - #[staticmethod] - pub fn table_scan( - file_infos: InputFileInfos, - schema: PySchema, - file_format_config: PyFileFormatConfig, - storage_config: PyStorageConfig, - ) -> PyResult { - Ok(LogicalPlanBuilder::table_scan( - file_infos, - schema.into(), - file_format_config.into(), - storage_config.into(), - )? - .into()) + pub fn table_scan(scan_operator: ScanOperatorHandle) -> PyResult { + Ok(LogicalPlanBuilder::table_scan(scan_operator.into(), None)?.into()) } pub fn project( diff --git a/src/daft-plan/src/logical_ops/project.rs b/src/daft-plan/src/logical_ops/project.rs index db87148d11..50ffb2fe78 100644 --- a/src/daft-plan/src/logical_ops/project.rs +++ b/src/daft-plan/src/logical_ops/project.rs @@ -407,7 +407,11 @@ mod tests { use daft_core::{datatypes::Field, DataType}; use daft_dsl::{binary_op, col, lit, Operator}; - use crate::{logical_ops::Project, test::dummy_scan_node, LogicalPlan}; + use crate::{ + logical_ops::Project, + test::{dummy_scan_node, dummy_scan_operator}, + LogicalPlan, + }; /// Test that nested common subexpressions are correctly split /// into multiple levels of projections. @@ -419,10 +423,10 @@ mod tests { /// 3: a+a as aa #[test] fn test_nested_subexpression() -> DaftResult<()> { - let source = dummy_scan_node(vec![ + let source = dummy_scan_node(dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Int64), - ]) + ])) .build(); let a2 = binary_op(Operator::Plus, &col("a"), &col("a")); let a4 = binary_op(Operator::Plus, &a2, &a2); @@ -463,10 +467,10 @@ mod tests { /// 2. a+a as aa, a #[test] fn test_shared_subexpression() -> DaftResult<()> { - let source = dummy_scan_node(vec![ + let source = dummy_scan_node(dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Int64), - ]) + ])) .build(); let a2 = binary_op(Operator::Plus, &col("a"), &col("a")); let expressions = vec![ @@ -500,10 +504,10 @@ mod tests { /// (unchanged) #[test] fn test_vacuous_subexpression() -> DaftResult<()> { - let source = dummy_scan_node(vec![ + let source = dummy_scan_node(dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Int64), - ]) + ])) .build(); let expressions = vec![ lit(3).alias("x"), diff --git a/src/daft-plan/src/logical_ops/source.rs b/src/daft-plan/src/logical_ops/source.rs index c7fbfb9cad..f6cb37b4c1 100644 --- a/src/daft-plan/src/logical_ops/source.rs +++ b/src/daft-plan/src/logical_ops/source.rs @@ -3,7 +3,7 @@ use std::sync::Arc; use daft_core::schema::SchemaRef; use daft_scan::ScanExternalInfo; -use crate::source_info::{ExternalInfo, SourceInfo}; +use crate::source_info::SourceInfo; #[cfg(feature = "python")] use crate::source_info::InMemoryInfo; @@ -30,19 +30,12 @@ impl Source { let mut res = vec![]; match self.source_info.as_ref() { - SourceInfo::ExternalInfo(ExternalInfo::Legacy(legacy_external_info)) => { - res.push(format!( - "Source: {}", - legacy_external_info.file_format_config.var_name() - )); - res.extend(legacy_external_info.multiline_display()); - } - SourceInfo::ExternalInfo(ExternalInfo::Scan(ScanExternalInfo { + SourceInfo::ExternalInfo(ScanExternalInfo { source_schema, scan_op, partitioning_keys, pushdowns, - })) => { + }) => { use itertools::Itertools; res.extend(scan_op.0.multiline_display()); diff --git a/src/daft-plan/src/optimization/logical_plan_tracker.rs b/src/daft-plan/src/optimization/logical_plan_tracker.rs index 238e08fbf1..75efdb2f40 100644 --- a/src/daft-plan/src/optimization/logical_plan_tracker.rs +++ b/src/daft-plan/src/optimization/logical_plan_tracker.rs @@ -71,22 +71,26 @@ mod tests { use std::{ collections::hash_map::DefaultHasher, hash::{Hash, Hasher}, + sync::Arc, }; use common_error::DaftResult; use daft_core::{datatypes::Field, DataType}; use daft_dsl::{col, lit}; - use crate::{optimization::logical_plan_tracker::LogicalPlanDigest, test::dummy_scan_node}; + use crate::{ + optimization::logical_plan_tracker::LogicalPlanDigest, + test::{dummy_scan_node, dummy_scan_operator}, + }; #[test] fn node_count() -> DaftResult<()> { // plan is Filter -> Concat -> {Projection -> Source, Projection -> Source}, // and should have a node count of 6. - let builder1 = dummy_scan_node(vec![ + let builder1 = dummy_scan_node(dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Utf8), - ]); + ])); assert_eq!( LogicalPlanDigest::new(builder1.plan.as_ref(), &mut Default::default()).node_count, 1usize.try_into().unwrap() @@ -96,10 +100,10 @@ mod tests { LogicalPlanDigest::new(builder1.plan.as_ref(), &mut Default::default()).node_count, 2usize.try_into().unwrap() ); - let builder2 = dummy_scan_node(vec![ + let builder2 = dummy_scan_node(dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Utf8), - ]); + ])); assert_eq!( LogicalPlanDigest::new(builder2.plan.as_ref(), &mut Default::default()).node_count, 1usize.try_into().unwrap() @@ -125,20 +129,14 @@ mod tests { #[test] fn same_plans_eq() -> DaftResult<()> { // Both plan1 and plan2 are Filter -> Project -> Source - let plan1 = dummy_scan_node(vec![ - Field::new("a", DataType::Int64), - Field::new("b", DataType::Utf8), - ]) - .project(vec![col("a")], Default::default())? - .filter(col("a").lt(&lit(2)))? - .build(); - let plan2 = dummy_scan_node(vec![ + let plan1 = dummy_scan_node(dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Utf8), - ]) + ])) .project(vec![col("a")], Default::default())? .filter(col("a").lt(&lit(2)))? .build(); + let plan2 = Arc::new(plan1.as_ref().clone()); // Double-check that logical plans are equal. assert_eq!(plan1, plan2); @@ -157,17 +155,17 @@ mod tests { #[test] fn different_plans_not_eq_op_ordering() -> DaftResult<()> { // plan1 is Project -> Filter -> Source, while plan2 is Filter -> Project -> Source. - let plan1 = dummy_scan_node(vec![ + let plan1 = dummy_scan_node(dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Utf8), - ]) + ])) .filter(col("a").lt(&lit(2)))? .project(vec![col("a")], Default::default())? .build(); - let plan2 = dummy_scan_node(vec![ + let plan2 = dummy_scan_node(dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Utf8), - ]) + ])) .project(vec![col("a")], Default::default())? .filter(col("a").lt(&lit(2)))? .build(); @@ -189,17 +187,17 @@ mod tests { #[test] fn different_plans_not_eq_same_order_diff_config() -> DaftResult<()> { // Both plan1 and plan2 are Filter -> Project -> Source, but with different filter predicates. - let plan1 = dummy_scan_node(vec![ + let plan1 = dummy_scan_node(dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Utf8), - ]) + ])) .project(vec![col("a")], Default::default())? .filter(col("a").lt(&lit(2)))? .build(); - let plan2 = dummy_scan_node(vec![ + let plan2 = dummy_scan_node(dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Utf8), - ]) + ])) .project(vec![col("a")], Default::default())? .filter(col("a").lt(&lit(4)))? .build(); diff --git a/src/daft-plan/src/optimization/mod.rs b/src/daft-plan/src/optimization/mod.rs index f7158979ea..82f349e55f 100644 --- a/src/daft-plan/src/optimization/mod.rs +++ b/src/daft-plan/src/optimization/mod.rs @@ -1,6 +1,8 @@ mod logical_plan_tracker; mod optimizer; mod rules; +#[cfg(test)] +mod test; pub use optimizer::Optimizer; pub use rules::Transformed; diff --git a/src/daft-plan/src/optimization/optimizer.rs b/src/daft-plan/src/optimization/optimizer.rs index 4229faeb7d..a554632414 100644 --- a/src/daft-plan/src/optimization/optimizer.rs +++ b/src/daft-plan/src/optimization/optimizer.rs @@ -318,7 +318,7 @@ mod tests { use crate::{ logical_ops::{Filter, Project}, optimization::rules::{ApplyOrder, OptimizerRule, Transformed}, - test::dummy_scan_node, + test::{dummy_scan_node, dummy_scan_operator}, LogicalPlan, }; @@ -336,7 +336,7 @@ mod tests { OptimizerConfig::new(5), ); let plan: Arc = - dummy_scan_node(vec![Field::new("a", DataType::Int64)]).build(); + dummy_scan_node(dummy_scan_operator(vec![Field::new("a", DataType::Int64)])).build(); let mut pass_count = 0; let mut did_transform = false; optimizer.optimize(plan.clone(), |new_plan, _, _, transformed, _| { @@ -391,7 +391,7 @@ mod tests { (col("a") + lit(2)).alias("b"), (col("a") + lit(3)).alias("c"), ]; - let plan = dummy_scan_node(vec![Field::new("a", DataType::Int64)]) + let plan = dummy_scan_node(dummy_scan_operator(vec![Field::new("a", DataType::Int64)])) .project(proj_exprs, Default::default())? .build(); let mut pass_count = 0; @@ -426,7 +426,7 @@ mod tests { (col("a") + lit(2)).alias("b"), (col("a") + lit(3)).alias("c"), ]; - let plan = dummy_scan_node(vec![Field::new("a", DataType::Int64)]) + let plan = dummy_scan_node(dummy_scan_operator(vec![Field::new("a", DataType::Int64)])) .project(proj_exprs, Default::default())? .build(); let mut pass_count = 0; @@ -440,51 +440,6 @@ mod tests { Ok(()) } - #[derive(Debug)] - struct RotateProjection { - reverse_first: Mutex, - } - - impl RotateProjection { - pub fn new(reverse_first: bool) -> Self { - Self { - reverse_first: Mutex::new(reverse_first), - } - } - } - - impl OptimizerRule for RotateProjection { - fn apply_order(&self) -> ApplyOrder { - ApplyOrder::TopDown - } - - fn try_optimize( - &self, - plan: Arc, - ) -> DaftResult>> { - let project = match plan.as_ref() { - LogicalPlan::Project(project) => project.clone(), - _ => return Ok(Transformed::No(plan)), - }; - let mut exprs = project.projection.clone(); - let mut reverse = self.reverse_first.lock().unwrap(); - if *reverse { - exprs.reverse(); - *reverse = false; - } else { - exprs.rotate_left(1); - } - Ok(Transformed::Yes( - LogicalPlan::from(Project::try_new( - project.input.clone(), - exprs, - project.resource_request.clone(), - )?) - .into(), - )) - } - } - /// Tests that the optimizer applies multiple rule batches. /// /// This test creates a Filter -> Projection -> Source plan and has 3 rule batches: @@ -517,16 +472,16 @@ mod tests { ], OptimizerConfig::new(20), ); - let fields = vec![Field::new("a", DataType::Int64)]; let proj_exprs = vec![ col("a") + lit(1), (col("a") + lit(2)).alias("b"), (col("a") + lit(3)).alias("c"), ]; let filter_predicate = col("a").lt(&lit(2)); - let plan = dummy_scan_node(fields.clone()) - .project(proj_exprs, Default::default())? - .filter(filter_predicate)? + let scan_op = dummy_scan_operator(vec![Field::new("a", DataType::Int64)]); + let plan = dummy_scan_node(scan_op.clone()) + .project(proj_exprs.clone(), Default::default())? + .filter(filter_predicate.clone())? .build(); let mut pass_count = 0; let mut did_transform = false; @@ -537,11 +492,24 @@ mod tests { assert!(did_transform); // 3 + 2 + 1 = 6 assert_eq!(pass_count, 6); - let expected = "\ - Filter: [[[col(a) < lit(2)] | lit(false)] | lit(false)] & lit(true)\ - \n Project: col(a) + lit(3) AS c, col(a) + lit(1), col(a) + lit(2) AS b\ - \n Source: Json, File paths = [/foo], File schema = a#Int64, Native storage config = { Use multithreading = true }, Output schema = a#Int64"; - assert_eq!(opt_plan.repr_indent(), expected); + + let mut new_proj_exprs = proj_exprs.clone(); + new_proj_exprs.rotate_left(2); + let new_pred = filter_predicate + .or(&lit(false)) + .or(&lit(false)) + .and(&lit(true)); + let expected = dummy_scan_node(scan_op) + .project(new_proj_exprs, Default::default())? + .filter(new_pred)? + .build(); + assert_eq!( + opt_plan, + expected, + "\n\nOptimized plan not equal to expected.\n\nOptimized:\n{}\n\nExpected:\n{}", + opt_plan.repr_ascii(false), + expected.repr_ascii(false) + ); Ok(()) } @@ -602,4 +570,49 @@ mod tests { )) } } + + #[derive(Debug)] + struct RotateProjection { + reverse_first: Mutex, + } + + impl RotateProjection { + pub fn new(reverse_first: bool) -> Self { + Self { + reverse_first: Mutex::new(reverse_first), + } + } + } + + impl OptimizerRule for RotateProjection { + fn apply_order(&self) -> ApplyOrder { + ApplyOrder::TopDown + } + + fn try_optimize( + &self, + plan: Arc, + ) -> DaftResult>> { + let project = match plan.as_ref() { + LogicalPlan::Project(project) => project.clone(), + _ => return Ok(Transformed::No(plan)), + }; + let mut exprs = project.projection.clone(); + let mut reverse = self.reverse_first.lock().unwrap(); + if *reverse { + exprs.reverse(); + *reverse = false; + } else { + exprs.rotate_left(1); + } + Ok(Transformed::Yes( + LogicalPlan::from(Project::try_new( + project.input.clone(), + exprs, + project.resource_request.clone(), + )?) + .into(), + )) + } + } } diff --git a/src/daft-plan/src/optimization/rules/drop_repartition.rs b/src/daft-plan/src/optimization/rules/drop_repartition.rs index f0449d6a8b..a820e67d0a 100644 --- a/src/daft-plan/src/optimization/rules/drop_repartition.rs +++ b/src/daft-plan/src/optimization/rules/drop_repartition.rs @@ -53,61 +53,54 @@ mod tests { use crate::{ optimization::{ - optimizer::{RuleBatch, RuleExecutionStrategy}, - rules::drop_repartition::DropRepartition, - Optimizer, + rules::drop_repartition::DropRepartition, test::assert_optimized_plan_with_rules_eq, }, - test::dummy_scan_node, + test::{dummy_scan_node, dummy_scan_operator}, LogicalPlan, PartitionSchemeConfig, }; /// Helper that creates an optimizer with the DropRepartition rule registered, optimizes - /// the provided plan with said optimizer, and compares the optimized plan's repr with - /// the provided expected repr. - fn assert_optimized_plan_eq(plan: Arc, expected: &str) -> DaftResult<()> { - let optimizer = Optimizer::with_rule_batches( - vec![RuleBatch::new( - vec![Box::new(DropRepartition::new())], - RuleExecutionStrategy::Once, - )], - Default::default(), - ); - let optimized_plan = optimizer - .optimize_with_rules( - optimizer.rule_batches[0].rules.as_slice(), - plan.clone(), - &optimizer.rule_batches[0].order, - )? - .unwrap() - .clone(); - assert_eq!(optimized_plan.repr_indent(), expected); - - Ok(()) + /// the provided plan with said optimizer, and compares the optimized plan with + /// the provided expected plan. + fn assert_optimized_plan_eq( + plan: Arc, + expected: Arc, + ) -> DaftResult<()> { + assert_optimized_plan_with_rules_eq(plan, expected, vec![Box::new(DropRepartition::new())]) } - /// Tests that DropRepartition does drops the upstream Repartition in back-to-back Repartitions if . + /// Tests that DropRepartition does drops the upstream Repartition in back-to-back Repartitions. /// /// Repartition1-Repartition2 -> Repartition1 #[test] fn repartition_dropped_in_back_to_back() -> DaftResult<()> { - let plan = dummy_scan_node(vec![ + let num_partitions1 = Some(10); + let num_partitions2 = Some(5); + let partition_by = vec![col("a")]; + let partition_scheme_config = PartitionSchemeConfig::Hash(Default::default()); + let scan_op = dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Utf8), - ]) - .repartition( - Some(10), - vec![col("a")], - PartitionSchemeConfig::Hash(Default::default()), - )? - .repartition( - Some(5), - vec![col("a")], - PartitionSchemeConfig::Hash(Default::default()), - )? - .build(); - let expected = "\ - Repartition: Scheme = Hash, Number of partitions = 5, Partition by = col(a)\ - \n Source: Json, File paths = [/foo], File schema = a#Int64, b#Utf8, Native storage config = { Use multithreading = true }, Output schema = a#Int64, b#Utf8"; + ]); + let plan = dummy_scan_node(scan_op.clone()) + .repartition( + num_partitions1, + partition_by.clone(), + partition_scheme_config.clone(), + )? + .repartition( + num_partitions2, + partition_by.clone(), + partition_scheme_config.clone(), + )? + .build(); + let expected = dummy_scan_node(scan_op) + .repartition( + num_partitions2, + partition_by.clone(), + partition_scheme_config.clone(), + )? + .build(); assert_optimized_plan_eq(plan, expected)?; Ok(()) } diff --git a/src/daft-plan/src/optimization/rules/push_down_filter.rs b/src/daft-plan/src/optimization/rules/push_down_filter.rs index c51de47691..130b73a623 100644 --- a/src/daft-plan/src/optimization/rules/push_down_filter.rs +++ b/src/daft-plan/src/optimization/rules/push_down_filter.rs @@ -12,11 +12,11 @@ use daft_dsl::{ }, Expr, }; -use daft_scan::{rewrite_predicate_for_partitioning, ScanExternalInfo}; +use daft_scan::rewrite_predicate_for_partitioning; use crate::{ logical_ops::{Concat, Filter, Project, Source}, - source_info::{ExternalInfo, SourceInfo}, + source_info::SourceInfo, LogicalPlan, }; @@ -77,16 +77,10 @@ impl OptimizerRule for PushDownFilter { // Filter pushdown is not supported for in-memory sources. #[cfg(feature = "python")] SourceInfo::InMemoryInfo(_) => return Ok(Transformed::No(plan)), - // Do not pushdown if Source node is already has a limit + // Do not pushdown if Source node already has a limit SourceInfo::ExternalInfo(external_info) if let Some(existing_limit) = - external_info.pushdowns().limit => - { - return Ok(Transformed::No(plan)) - } - // Do not pushdown if we are using python legacy scan info - SourceInfo::ExternalInfo(external_info) - if let ExternalInfo::Legacy(..) = external_info => + external_info.pushdowns.limit => { return Ok(Transformed::No(plan)) } @@ -115,14 +109,10 @@ impl OptimizerRule for PushDownFilter { if has_udf { return Ok(Transformed::No(plan)); } - let new_predicate = external_info.pushdowns().filters.as_ref().map(|f| predicate.and(f)).unwrap_or(predicate.clone()); - let partition_filter = if let ExternalInfo::Scan(ScanExternalInfo {scan_op, ..}) = &external_info { - rewrite_predicate_for_partitioning(new_predicate.clone(), scan_op.0.partitioning_keys())? - } else { - None - }; + let new_predicate = external_info.pushdowns.filters.as_ref().map(|f| predicate.and(f)).unwrap_or(predicate.clone()); + let partition_filter = rewrite_predicate_for_partitioning(new_predicate.clone(), external_info.scan_op.0.partitioning_keys())?; let new_pushdowns = - external_info.pushdowns().with_filters(Some(Arc::new(new_predicate))); + external_info.pushdowns.with_filters(Some(Arc::new(new_predicate))); let new_pushdowns = if let Some(pfilter) = partition_filter { new_pushdowns.with_partition_filters(Some(Arc::new(pfilter))) @@ -285,141 +275,151 @@ mod tests { use common_error::DaftResult; use daft_core::{datatypes::Field, DataType}; use daft_dsl::{col, lit}; + use daft_scan::Pushdowns; + use rstest::rstest; use crate::{ - optimization::{ - optimizer::{RuleBatch, RuleExecutionStrategy}, - rules::PushDownFilter, - Optimizer, - }, - test::{dummy_scan_node, dummy_scan_operator_node}, + optimization::{rules::PushDownFilter, test::assert_optimized_plan_with_rules_eq}, + test::{dummy_scan_node, dummy_scan_node_with_pushdowns, dummy_scan_operator}, JoinType, LogicalPlan, PartitionSchemeConfig, }; /// Helper that creates an optimizer with the PushDownFilter rule registered, optimizes - /// the provided plan with said optimizer, and compares the optimized plan's repr with - /// the provided expected repr. - fn assert_optimized_plan_eq(plan: Arc, expected: &str) -> DaftResult<()> { - let optimizer = Optimizer::with_rule_batches( - vec![RuleBatch::new( - vec![Box::new(PushDownFilter::new())], - RuleExecutionStrategy::Once, - )], - Default::default(), - ); - let optimized_plan = optimizer - .optimize_with_rules( - optimizer.rule_batches[0].rules.as_slice(), - plan.clone(), - &optimizer.rule_batches[0].order, - )? - .unwrap() - .clone(); - assert_eq!(optimized_plan.repr_indent(), expected); - - Ok(()) - } - - /// Tests combining of two Filters by merging their predicates. - #[test] - fn filter_combine_with_filter() -> DaftResult<()> { - let plan = dummy_scan_node(vec![ - Field::new("a", DataType::Int64), - Field::new("b", DataType::Utf8), - ]) - .filter(col("a").lt(&lit(2)))? - .filter(col("b").eq(&lit("foo")))? - .build(); - let expected = "\ - Filter: [col(b) == lit(\"foo\")] & [col(a) < lit(2)]\ - \n Source: Json, File paths = [/foo], File schema = a#Int64, b#Utf8, Native storage config = { Use multithreading = true }, Output schema = a#Int64, b#Utf8"; - assert_optimized_plan_eq(plan, expected)?; - Ok(()) + /// the provided plan with said optimizer, and compares the optimized plan with + /// the provided expected plan. + fn assert_optimized_plan_eq( + plan: Arc, + expected: Arc, + ) -> DaftResult<()> { + assert_optimized_plan_with_rules_eq(plan, expected, vec![Box::new(PushDownFilter::new())]) } - /// Tests combining of two Filters into a ScanOperator + /// Tests that we can't pushdown a filter into a ScanOperator that has a limit. #[test] - fn pushdown_filter_into_scan_operator() -> DaftResult<()> { - let plan = dummy_scan_operator_node(vec![ + fn filter_not_pushed_down_into_scan_with_limit() -> DaftResult<()> { + let scan_op = dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Utf8), - ]) - .filter(col("a").lt(&lit(2)))? - .filter(col("b").eq(&lit("foo")))? - .build(); - let expected = "\ - AnonymousScanOperator, File paths = [/foo], Use multithreading = true, File schema = a#Int64, b#Utf8, Partitioning keys = [], Filter pushdown = [col(b) == lit(\"foo\")] & [col(a) < lit(2)], Output schema = a#Int64, b#Utf8"; + ]); + let plan = + dummy_scan_node_with_pushdowns(scan_op, Pushdowns::default().with_limit(Some(1))) + .filter(col("a").lt(&lit(2)))? + .build(); + // Plan should be unchanged after optimization. + let expected = plan.clone(); assert_optimized_plan_eq(plan, expected)?; Ok(()) } - /// Tests that we cant pushdown a filter into a ScanOperator with a limit - #[test] - fn pushdown_filter_into_scan_operator_with_limit() -> DaftResult<()> { - let plan = dummy_scan_operator_node(vec![ + /// Tests combining of two Filters by merging their predicates. + #[rstest] + fn filter_combine_with_filter(#[values(false, true)] push_into_scan: bool) -> DaftResult<()> { + let scan_op = dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Utf8), - ]) - .limit(1, false)? - .filter(col("a").lt(&lit(2)))? - .build(); - let expected = "\ - Filter: col(a) < lit(2)\ - \n Limit: 1\ - \n AnonymousScanOperator, File paths = [/foo], Use multithreading = true, File schema = a#Int64, b#Utf8, Partitioning keys = [], Output schema = a#Int64, b#Utf8"; + ]); + let scan_plan = dummy_scan_node_with_pushdowns( + scan_op.clone(), + Pushdowns::default().with_limit(if push_into_scan { None } else { Some(1) }), + ); + let p1 = col("a").lt(&lit(2)); + let p2 = col("b").eq(&lit("foo")); + let plan = scan_plan.filter(p1.clone())?.filter(p2.clone())?.build(); + let merged_filter = p2.and(&p1); + let expected = if push_into_scan { + // Merged filter should be pushed into scan. + dummy_scan_node_with_pushdowns( + scan_op, + Pushdowns::default().with_filters(Some(merged_filter.into())), + ) + .build() + } else { + // Merged filter should not be pushed into scan. + scan_plan.filter(merged_filter)?.build() + }; assert_optimized_plan_eq(plan, expected)?; Ok(()) } - /// Tests that we cant pushdown a filter into a ScanOperator with an udf-ish expression + /// Tests that we can't pushdown a filter into a ScanOperator if it has an udf-ish expression. #[test] - fn pushdown_filter_into_scan_operator_with_udf() -> DaftResult<()> { + fn filter_with_udf_not_pushed_down_into_scan() -> DaftResult<()> { let pred = daft_dsl::functions::uri::download(&col("a"), 1, true, true, None); - let plan = dummy_scan_operator_node(vec![ + let plan = dummy_scan_node(dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Utf8), - ]) + ])) .filter(pred.is_null())? .build(); - let expected = "\ - Filter: is_null(download(col(a)))\ - \n AnonymousScanOperator, File paths = [/foo], Use multithreading = true, File schema = a#Int64, b#Utf8, Partitioning keys = [], Output schema = a#Int64, b#Utf8"; + let expected = plan.clone(); assert_optimized_plan_eq(plan, expected)?; Ok(()) } /// Tests that Filter commutes with Projections. - #[test] - fn filter_commutes_with_projection() -> DaftResult<()> { - let plan = dummy_scan_node(vec![ + #[rstest] + fn filter_commutes_with_projection( + #[values(false, true)] push_into_scan: bool, + ) -> DaftResult<()> { + let scan_op = dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Utf8), - ]) - .project(vec![col("a")], Default::default())? - .filter(col("a").lt(&lit(2)))? - .build(); - let expected = "\ - Project: col(a)\ - \n Filter: col(a) < lit(2)\ - \n Source: Json, File paths = [/foo], File schema = a#Int64, b#Utf8, Native storage config = { Use multithreading = true }, Output schema = a#Int64, b#Utf8"; + ]); + let scan_plan = dummy_scan_node_with_pushdowns( + scan_op.clone(), + Pushdowns::default().with_limit(if push_into_scan { None } else { Some(1) }), + ); + let pred = col("a").lt(&lit(2)); + let proj = vec![col("a")]; + let plan = scan_plan + .project(proj.clone(), Default::default())? + .filter(pred.clone())? + .build(); + let expected_scan_filter = if push_into_scan { + dummy_scan_node_with_pushdowns( + scan_op, + Pushdowns::default().with_filters(Some(pred.into())), + ) + } else { + scan_plan.filter(pred)? + }; + let expected = expected_scan_filter + .project(proj, Default::default())? + .build(); assert_optimized_plan_eq(plan, expected)?; Ok(()) } /// Tests that a Filter with multiple columns in its predicate commutes with a Projection on both of those columns. - #[test] - fn filter_commutes_with_projection_multi() -> DaftResult<()> { - let plan = dummy_scan_node(vec![ + #[rstest] + fn filter_commutes_with_projection_multi( + #[values(false, true)] push_into_scan: bool, + ) -> DaftResult<()> { + let scan_op = dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Utf8), - ]) - .project(vec![col("a"), col("b")], Default::default())? - .filter(col("a").lt(&lit(2)).and(&col("b").eq(&lit("foo"))))? - .build(); - let expected = "\ - Project: col(a), col(b)\ - \n Filter: [col(a) < lit(2)] & [col(b) == lit(\"foo\")]\ - \n Source: Json, File paths = [/foo], File schema = a#Int64, b#Utf8, Native storage config = { Use multithreading = true }, Output schema = a#Int64, b#Utf8"; + ]); + let scan_plan = dummy_scan_node_with_pushdowns( + scan_op.clone(), + Pushdowns::default().with_limit(if push_into_scan { None } else { Some(1) }), + ); + let pred = col("a").lt(&lit(2)).and(&col("b").eq(&lit("foo"))); + let proj = vec![col("a"), col("b")]; + let plan = scan_plan + .project(proj.clone(), Default::default())? + .filter(pred.clone())? + .build(); + let expected_scan_filter = if push_into_scan { + dummy_scan_node_with_pushdowns( + scan_op, + Pushdowns::default().with_filters(Some(pred.into())), + ) + } else { + scan_plan.filter(pred)? + }; + let expected = expected_scan_filter + .project(proj, Default::default())? + .build(); assert_optimized_plan_eq(plan, expected)?; Ok(()) } @@ -427,19 +427,16 @@ mod tests { /// Tests that Filter does not commute with a Projection if the projection expression involves compute. #[test] fn filter_does_not_commute_with_projection_if_compute() -> DaftResult<()> { - let plan = dummy_scan_node(vec![ + let plan = dummy_scan_node(dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Utf8), - ]) + ])) // Projection involves compute on filtered column "a". .project(vec![col("a") + lit(1)], Default::default())? .filter(col("a").lt(&lit(2)))? .build(); // Filter should NOT commute with Project, since this would involve redundant computation. - let expected = "\ - Filter: col(a) < lit(2)\ - \n Project: col(a) + lit(1)\ - \n Source: Json, File paths = [/foo], File schema = a#Int64, b#Utf8, Native storage config = { Use multithreading = true }, Output schema = a#Int64, b#Utf8"; + let expected = plan.clone(); assert_optimized_plan_eq(plan, expected)?; Ok(()) } @@ -447,166 +444,378 @@ mod tests { /// Tests that Filter commutes with Projection if projection expression involves deterministic compute. // REASON - No expression attribute indicating whether deterministic && (pure || idempotent). #[ignore] - #[test] - fn filter_commutes_with_projection_deterministic_compute() -> DaftResult<()> { - let plan = dummy_scan_node(vec![ + #[rstest] + fn filter_commutes_with_projection_deterministic_compute( + #[values(false, true)] push_into_scan: bool, + ) -> DaftResult<()> { + let scan_op = dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Utf8), - ]) - // Projection involves compute on filtered column "a". - .project(vec![col("a") + lit(1)], Default::default())? - .filter(col("a").lt(&lit(2)))? - .build(); - let expected = "\ - Project: col(a) + lit(1)\ - \n Filter: [col(a) + lit(1)] < lit(2)\ - \n Source: Json, File paths = [/foo], File schema = a#Int64, b#Utf8, Native storage config = { Use multithreading = true }, Output schema = a#Int64, b#Utf8"; + ]); + let scan_plan = dummy_scan_node_with_pushdowns( + scan_op.clone(), + Pushdowns::default().with_limit(if push_into_scan { None } else { Some(1) }), + ); + let pred = col("a").lt(&lit(2)); + let proj = vec![col("a") + lit(1)]; + let plan = scan_plan + // Projection involves compute on filtered column "a". + .project(proj.clone(), Default::default())? + .filter(pred.clone())? + .build(); + let expected_filter_scan = if push_into_scan { + dummy_scan_node_with_pushdowns( + scan_op, + Pushdowns::default().with_filters(Some(pred.into())), + ) + } else { + scan_plan.filter(pred)? + }; + let expected = expected_filter_scan + .project(proj, Default::default())? + .build(); assert_optimized_plan_eq(plan, expected)?; Ok(()) } /// Tests that Filter commutes with Sort. - #[test] - fn filter_commutes_with_sort() -> DaftResult<()> { - let plan = dummy_scan_node(vec![ + #[rstest] + fn filter_commutes_with_sort(#[values(false, true)] push_into_scan: bool) -> DaftResult<()> { + let scan_op = dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Utf8), - ]) - .sort(vec![col("a")], vec![true])? - .filter(col("a").lt(&lit(2)))? - .build(); - let expected = "\ - Sort: Sort by = (col(a), descending)\ - \n Filter: col(a) < lit(2)\ - \n Source: Json, File paths = [/foo], File schema = a#Int64, b#Utf8, Native storage config = { Use multithreading = true }, Output schema = a#Int64, b#Utf8"; - // TODO(Clark): For tests in which we only care about reordering of operators, maybe switch to a form that leverages the single-node display? - // let expected = format!("{sort}\n {filter}\n {source}"); + ]); + let scan_plan = dummy_scan_node_with_pushdowns( + scan_op.clone(), + Pushdowns::default().with_limit(if push_into_scan { None } else { Some(1) }), + ); + let pred = col("a").lt(&lit(2)); + let sort_by = vec![col("a")]; + let descending = vec![true]; + let plan = scan_plan + .sort(sort_by.clone(), descending.clone())? + .filter(pred.clone())? + .build(); + let expected_filter_scan = if push_into_scan { + dummy_scan_node_with_pushdowns( + scan_op, + Pushdowns::default().with_filters(Some(pred.into())), + ) + } else { + scan_plan.filter(pred)? + }; + let expected = expected_filter_scan.sort(sort_by, descending)?.build(); assert_optimized_plan_eq(plan, expected)?; Ok(()) } /// Tests that Filter commutes with Repartition. - #[test] - fn filter_commutes_with_repartition() -> DaftResult<()> { - let plan = dummy_scan_node(vec![ + #[rstest] + fn filter_commutes_with_repartition( + #[values(false, true)] push_into_scan: bool, + ) -> DaftResult<()> { + let scan_op = dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Utf8), - ]) - .repartition( - Some(1), - vec![col("a")], - PartitionSchemeConfig::Hash(Default::default()), - )? - .filter(col("a").lt(&lit(2)))? - .build(); - let expected = "\ - Repartition: Scheme = Hash, Number of partitions = 1, Partition by = col(a)\ - \n Filter: col(a) < lit(2)\ - \n Source: Json, File paths = [/foo], File schema = a#Int64, b#Utf8, Native storage config = { Use multithreading = true }, Output schema = a#Int64, b#Utf8"; + ]); + let scan_plan = dummy_scan_node_with_pushdowns( + scan_op.clone(), + Pushdowns::default().with_limit(if push_into_scan { None } else { Some(1) }), + ); + let pred = col("a").lt(&lit(2)); + let num_partitions = Some(1); + let repartition_by = vec![col("a")]; + let partition_scheme_config = PartitionSchemeConfig::Hash(Default::default()); + let plan = scan_plan + .repartition( + num_partitions, + repartition_by.clone(), + partition_scheme_config.clone(), + )? + .filter(pred.clone())? + .build(); + let expected_filter_scan = if push_into_scan { + dummy_scan_node_with_pushdowns( + scan_op, + Pushdowns::default().with_filters(Some(pred.into())), + ) + } else { + scan_plan.filter(pred)? + }; + let expected = expected_filter_scan + .repartition(num_partitions, repartition_by, partition_scheme_config)? + .build(); assert_optimized_plan_eq(plan, expected)?; Ok(()) } /// Tests that Filter commutes with Concat. - #[test] - fn filter_commutes_with_concat() -> DaftResult<()> { - let fields = vec![ + #[rstest] + fn filter_commutes_with_concat( + #[values(false, true)] push_into_left_scan: bool, + #[values(false, true)] push_into_right_scan: bool, + ) -> DaftResult<()> { + let scan_op = dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Utf8), - ]; - let plan = dummy_scan_node(fields.clone()) - .concat(&dummy_scan_node(fields))? - .filter(col("a").lt(&lit(2)))? + ]); + let left_scan_plan = dummy_scan_node_with_pushdowns( + scan_op.clone(), + Pushdowns::default().with_limit(if push_into_left_scan { None } else { Some(1) }), + ); + let right_scan_plan = dummy_scan_node_with_pushdowns( + scan_op.clone(), + Pushdowns::default().with_limit(if push_into_right_scan { None } else { Some(1) }), + ); + let pred = col("a").lt(&lit(2)); + let plan = left_scan_plan + .concat(&right_scan_plan)? + .filter(pred.clone())? + .build(); + let expected_left_filter_scan = if push_into_left_scan { + dummy_scan_node_with_pushdowns( + scan_op.clone(), + Pushdowns::default().with_filters(Some(pred.clone().into())), + ) + } else { + left_scan_plan.filter(pred.clone())? + }; + let expected_right_filter_scan = if push_into_right_scan { + dummy_scan_node_with_pushdowns( + scan_op, + Pushdowns::default().with_filters(Some(pred.into())), + ) + } else { + right_scan_plan.filter(pred)? + }; + let expected = expected_left_filter_scan + .concat(&expected_right_filter_scan)? + .build(); + assert_optimized_plan_eq(plan, expected)?; + Ok(()) + } + + /// Tests that Filter commutes with Join. + #[rstest] + fn filter_commutes_with_join( + #[values(false, true)] push_into_left_scan: bool, + #[values(false, true)] push_into_right_scan: bool, + ) -> DaftResult<()> { + let scan_op = dummy_scan_operator(vec![ + Field::new("a", DataType::Int64), + Field::new("b", DataType::Utf8), + ]); + let left_scan_plan = dummy_scan_node_with_pushdowns( + scan_op.clone(), + Pushdowns::default().with_limit(if push_into_left_scan { None } else { Some(1) }), + ); + let right_scan_plan = dummy_scan_node_with_pushdowns( + scan_op.clone(), + Pushdowns::default().with_limit(if push_into_right_scan { None } else { Some(1) }), + ); + let join_on = vec![col("b")]; + let pred = col("a").lt(&lit(2)); + let plan = left_scan_plan + .join( + &right_scan_plan, + join_on.clone(), + join_on.clone(), + JoinType::Inner, + None, + )? + .filter(pred.clone())? + .build(); + let expected_left_filter_scan = if push_into_left_scan { + dummy_scan_node_with_pushdowns( + scan_op.clone(), + Pushdowns::default().with_filters(Some(pred.clone().into())), + ) + } else { + left_scan_plan.filter(pred.clone())? + }; + let expected_right_filter_scan = if push_into_right_scan { + dummy_scan_node_with_pushdowns( + scan_op, + Pushdowns::default().with_filters(Some(pred.into())), + ) + } else { + right_scan_plan.filter(pred)? + }; + let expected = expected_left_filter_scan + .join( + &expected_right_filter_scan, + join_on.clone(), + join_on.clone(), + JoinType::Inner, + None, + )? .build(); - let expected = "\ - Concat\ - \n Filter: col(a) < lit(2)\ - \n Source: Json, File paths = [/foo], File schema = a#Int64, b#Utf8, Native storage config = { Use multithreading = true }, Output schema = a#Int64, b#Utf8\ - \n Filter: col(a) < lit(2)\ - \n Source: Json, File paths = [/foo], File schema = a#Int64, b#Utf8, Native storage config = { Use multithreading = true }, Output schema = a#Int64, b#Utf8"; assert_optimized_plan_eq(plan, expected)?; Ok(()) } /// Tests that Filter can be pushed into the left side of a Join. - #[test] - fn filter_commutes_with_join_left_side() -> DaftResult<()> { - let plan = dummy_scan_node(vec![ + #[rstest] + fn filter_commutes_with_join_left_side( + #[values(false, true)] push_into_left_scan: bool, + ) -> DaftResult<()> { + let left_scan_op = dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Utf8), - ]) - .join( - &dummy_scan_node(vec![ - Field::new("b", DataType::Utf8), - Field::new("c", DataType::Float64), - ]), - vec![col("b")], - vec![col("b")], - JoinType::Inner, - None, - )? - .filter(col("a").lt(&lit(2)))? - .build(); - let expected = "\ - Join: Type = Inner, Strategy = Auto, On = col(b), Output schema = a#Int64, b#Utf8, c#Float64\ - \n Filter: col(a) < lit(2)\ - \n Source: Json, File paths = [/foo], File schema = a#Int64, b#Utf8, Native storage config = { Use multithreading = true }, Output schema = a#Int64, b#Utf8\ - \n Source: Json, File paths = [/foo], File schema = b#Utf8, c#Float64, Native storage config = { Use multithreading = true }, Output schema = b#Utf8, c#Float64"; + ]); + let right_scan_op = dummy_scan_operator(vec![ + Field::new("b", DataType::Utf8), + Field::new("c", DataType::Float64), + ]); + let left_scan_plan = dummy_scan_node_with_pushdowns( + left_scan_op.clone(), + Pushdowns::default().with_limit(if push_into_left_scan { None } else { Some(1) }), + ); + let right_scan_plan = dummy_scan_node(right_scan_op.clone()); + let join_on = vec![col("b")]; + let pred = col("a").lt(&lit(2)); + let plan = left_scan_plan + .join( + &right_scan_plan, + join_on.clone(), + join_on.clone(), + JoinType::Inner, + None, + )? + .filter(pred.clone())? + .build(); + let expected_left_filter_scan = if push_into_left_scan { + dummy_scan_node_with_pushdowns( + left_scan_op.clone(), + Pushdowns::default().with_filters(Some(pred.clone().into())), + ) + } else { + left_scan_plan.filter(pred.clone())? + }; + let expected = expected_left_filter_scan + .join( + &right_scan_plan, + join_on.clone(), + join_on.clone(), + JoinType::Inner, + None, + )? + .build(); assert_optimized_plan_eq(plan, expected)?; Ok(()) } /// Tests that Filter can be pushed into the right side of a Join. - #[test] - fn filter_commutes_with_join_right_side() -> DaftResult<()> { - let plan = dummy_scan_node(vec![ + #[rstest] + fn filter_commutes_with_join_right_side( + #[values(false, true)] push_into_right_scan: bool, + ) -> DaftResult<()> { + let left_scan_op = dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Utf8), - ]) - .join( - &dummy_scan_node(vec![ - Field::new("b", DataType::Utf8), - Field::new("c", DataType::Float64), - ]), - vec![col("b")], - vec![col("b")], - JoinType::Inner, - None, - )? - .filter(col("c").lt(&lit(2.0)))? - .build(); - let expected = "\ - Join: Type = Inner, Strategy = Auto, On = col(b), Output schema = a#Int64, b#Utf8, c#Float64\ - \n Source: Json, File paths = [/foo], File schema = a#Int64, b#Utf8, Native storage config = { Use multithreading = true }, Output schema = a#Int64, b#Utf8\ - \n Filter: col(c) < lit(2.0)\ - \n Source: Json, File paths = [/foo], File schema = b#Utf8, c#Float64, Native storage config = { Use multithreading = true }, Output schema = b#Utf8, c#Float64"; + ]); + let right_scan_op = dummy_scan_operator(vec![ + Field::new("b", DataType::Utf8), + Field::new("c", DataType::Float64), + ]); + let left_scan_plan = dummy_scan_node(left_scan_op.clone()); + let right_scan_plan = dummy_scan_node_with_pushdowns( + right_scan_op.clone(), + Pushdowns::default().with_limit(if push_into_right_scan { None } else { Some(1) }), + ); + let join_on = vec![col("b")]; + let pred = col("c").lt(&lit(2.0)); + let plan = left_scan_plan + .join( + &right_scan_plan, + join_on.clone(), + join_on.clone(), + JoinType::Inner, + None, + )? + .filter(pred.clone())? + .build(); + let expected_right_filter_scan = if push_into_right_scan { + dummy_scan_node_with_pushdowns( + right_scan_op.clone(), + Pushdowns::default().with_filters(Some(pred.clone().into())), + ) + } else { + right_scan_plan.filter(pred.clone())? + }; + let expected = left_scan_plan + .join( + &expected_right_filter_scan, + join_on.clone(), + join_on.clone(), + JoinType::Inner, + None, + )? + .build(); assert_optimized_plan_eq(plan, expected)?; Ok(()) } - /// Tests that Filter can be pushed into both sides of a Join. - #[test] - fn filter_commutes_with_join_both_sides() -> DaftResult<()> { - let plan = dummy_scan_node(vec![ - Field::new("a", DataType::Int64), + /// Tests that Filter on join key commutes with Join. + #[rstest] + fn filter_commutes_with_join_on_join_key( + #[values(false, true)] push_into_left_scan: bool, + #[values(false, true)] push_into_right_scan: bool, + ) -> DaftResult<()> { + let left_scan_op = dummy_scan_operator(vec![ + Field::new("a", DataType::Utf8), Field::new("b", DataType::Int64), Field::new("c", DataType::Float64), - ]) - .join( - &dummy_scan_node(vec![Field::new("b", DataType::Int64)]), - vec![col("b")], - vec![col("b")], - JoinType::Inner, - None, - )? - .filter(col("b").lt(&lit(2)))? - .build(); - let expected = "\ - Join: Type = Inner, Strategy = Auto, On = col(b), Output schema = a#Int64, b#Int64, c#Float64\ - \n Filter: col(b) < lit(2)\ - \n Source: Json, File paths = [/foo], File schema = a#Int64, b#Int64, c#Float64, Native storage config = { Use multithreading = true }, Output schema = a#Int64, b#Int64, c#Float64\ - \n Filter: col(b) < lit(2)\ - \n Source: Json, File paths = [/foo], File schema = b#Int64, Native storage config = { Use multithreading = true }, Output schema = b#Int64"; + ]); + let right_scan_op = dummy_scan_operator(vec![ + Field::new("b", DataType::Int64), + Field::new("d", DataType::Boolean), + ]); + let left_scan_plan = dummy_scan_node_with_pushdowns( + left_scan_op.clone(), + Pushdowns::default().with_limit(if push_into_left_scan { None } else { Some(1) }), + ); + let right_scan_plan = dummy_scan_node_with_pushdowns( + right_scan_op.clone(), + Pushdowns::default().with_limit(if push_into_right_scan { None } else { Some(1) }), + ); + let join_on = vec![col("b")]; + let pred = col("b").lt(&lit(2)); + let plan = left_scan_plan + .join( + &right_scan_plan, + join_on.clone(), + join_on.clone(), + JoinType::Inner, + None, + )? + .filter(pred.clone())? + .build(); + let expected_left_filter_scan = if push_into_left_scan { + dummy_scan_node_with_pushdowns( + left_scan_op.clone(), + Pushdowns::default().with_filters(Some(pred.clone().into())), + ) + } else { + left_scan_plan.filter(pred.clone())? + }; + let expected_right_filter_scan = if push_into_right_scan { + dummy_scan_node_with_pushdowns( + right_scan_op, + Pushdowns::default().with_filters(Some(pred.into())), + ) + } else { + right_scan_plan.filter(pred)? + }; + let expected = expected_left_filter_scan + .join( + &expected_right_filter_scan, + join_on.clone(), + join_on.clone(), + JoinType::Inner, + None, + )? + .build(); assert_optimized_plan_eq(plan, expected)?; Ok(()) } diff --git a/src/daft-plan/src/optimization/rules/push_down_limit.rs b/src/daft-plan/src/optimization/rules/push_down_limit.rs index dab79c3607..dbafeee1d9 100644 --- a/src/daft-plan/src/optimization/rules/push_down_limit.rs +++ b/src/daft-plan/src/optimization/rules/push_down_limit.rs @@ -1,11 +1,10 @@ use std::sync::Arc; use common_error::DaftResult; -use daft_scan::ScanExternalInfo; use crate::{ logical_ops::{Limit as LogicalLimit, Source}, - source_info::{ExternalInfo, SourceInfo}, + source_info::SourceInfo, LogicalPlan, }; @@ -57,27 +56,21 @@ impl OptimizerRule for PushDownLimit { // Do not pushdown if Source node is already more limited than `limit` SourceInfo::ExternalInfo(external_info) if let Some(existing_limit) = - external_info.pushdowns().limit && existing_limit <= limit => + external_info.pushdowns.limit && existing_limit <= limit => { Ok(Transformed::No(plan)) } // Pushdown limit into the Source node as a "local" limit SourceInfo::ExternalInfo(external_info) => { let new_pushdowns = - external_info.pushdowns().with_limit(Some(limit)); + external_info.pushdowns.with_limit(Some(limit)); let new_external_info = external_info.with_pushdowns(new_pushdowns); let new_source = LogicalPlan::Source(Source::new( source.output_schema.clone(), SourceInfo::ExternalInfo(new_external_info).into(), )) .into(); - let out_plan = - match external_info { - ExternalInfo::Scan(ScanExternalInfo { - scan_op, .. - }) if scan_op.0.can_absorb_limit() => new_source, - _ => plan.with_new_children(&[new_source]).into(), - }; + let out_plan = if external_info.scan_op.0.can_absorb_limit() { new_source } else { plan.with_new_children(&[new_source]).into() }; Ok(Transformed::Yes(out_plan)) } } @@ -120,46 +113,26 @@ mod tests { use daft_core::{datatypes::Field, schema::Schema, DataType}; use daft_dsl::col; use daft_scan::Pushdowns; + use rstest::rstest; use std::sync::Arc; #[cfg(feature = "python")] use pyo3::Python; use crate::{ - optimization::{ - optimizer::{RuleBatch, RuleExecutionStrategy}, - rules::PushDownLimit, - Optimizer, - }, - test::{ - dummy_scan_node, dummy_scan_node_with_pushdowns, - dummy_scan_operator_node_with_pushdowns, - }, + optimization::{rules::PushDownLimit, test::assert_optimized_plan_with_rules_eq}, + test::{dummy_scan_node, dummy_scan_node_with_pushdowns, dummy_scan_operator}, LogicalPlan, PartitionSchemeConfig, }; /// Helper that creates an optimizer with the PushDownLimit rule registered, optimizes - /// the provided plan with said optimizer, and compares the optimized plan's repr with - /// the provided expected repr. - fn assert_optimized_plan_eq(plan: Arc, expected: &str) -> DaftResult<()> { - let optimizer = Optimizer::with_rule_batches( - vec![RuleBatch::new( - vec![Box::new(PushDownLimit::new())], - RuleExecutionStrategy::Once, - )], - Default::default(), - ); - let optimized_plan = optimizer - .optimize_with_rules( - optimizer.rule_batches[0].rules.as_slice(), - plan.clone(), - &optimizer.rule_batches[0].order, - )? - .unwrap() - .clone(); - assert_eq!(optimized_plan.repr_indent(), expected); - - Ok(()) + /// the provided plan with said optimizer, and compares the optimized plan with + /// the provided expected plan. + fn assert_optimized_plan_eq( + plan: Arc, + expected: Arc, + ) -> DaftResult<()> { + assert_optimized_plan_with_rules_eq(plan, expected, vec![Box::new(PushDownLimit::new())]) } /// Tests that Limit pushes into external Source. @@ -167,122 +140,101 @@ mod tests { /// Limit-Source -> Source[with_limit] #[test] fn limit_pushes_into_external_source() -> DaftResult<()> { - let plan = dummy_scan_node(vec![ + let limit = 5; + let scan_op = dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Utf8), - ]) - .limit(5, false)? + ]); + let plan = dummy_scan_node(scan_op.clone()) + .limit(limit, false)? + .build(); + let expected = dummy_scan_node_with_pushdowns( + scan_op, + Pushdowns::default().with_limit(Some(limit as usize)), + ) + .limit(limit, false)? .build(); - let expected = "\ - Limit: 5\ - \n Source: Json, File paths = [/foo], File schema = a#Int64, b#Utf8, Native storage config = { Use multithreading = true }, Limit pushdown = 5, Output schema = a#Int64, b#Utf8"; assert_optimized_plan_eq(plan, expected)?; Ok(()) } - /// Tests that Limit does not push into external Source with existing smaller limit. + /// Tests that Limit does not push into scan with existing smaller limit. /// /// Limit-Source[existing_limit] -> Source[existing_limit] #[test] - fn limit_does_not_push_into_external_source_if_smaller_limit() -> DaftResult<()> { + fn limit_does_not_push_into_scan_if_smaller_limit() -> DaftResult<()> { + let limit = 5; + let existing_limit = 3; + let scan_op = dummy_scan_operator(vec![ + Field::new("a", DataType::Int64), + Field::new("b", DataType::Utf8), + ]); let plan = dummy_scan_node_with_pushdowns( - vec![ - Field::new("a", DataType::Int64), - Field::new("b", DataType::Utf8), - ], - Pushdowns::default().with_limit(Some(3)), + scan_op.clone(), + Pushdowns::default().with_limit(Some(existing_limit)), ) - .limit(5, false)? + .limit(limit, false)? .build(); - let expected = "\ - Limit: 5\ - \n Source: Json, File paths = [/foo], File schema = a#Int64, b#Utf8, Native storage config = { Use multithreading = true }, Limit pushdown = 3, Output schema = a#Int64, b#Utf8"; - assert_optimized_plan_eq(plan, expected)?; - Ok(()) - } - - /// Tests that Limit does not push into external Source with existing smaller limit. - /// - /// Limit[x]-Limit[y] -> Limit[min(x,y)] - #[test] - fn limit_folds_with_smaller_limit() -> DaftResult<()> { - let plan = dummy_scan_node_with_pushdowns( - vec![ - Field::new("a", DataType::Int64), - Field::new("b", DataType::Utf8), - ], - Pushdowns::default(), + let expected = dummy_scan_node_with_pushdowns( + scan_op, + Pushdowns::default().with_limit(Some(existing_limit)), ) - .limit(5, false)? - .limit(10, false)? + .limit(limit, false)? .build(); - let expected = "\ - Limit: 5\ - \n Source: Json, File paths = [/foo], File schema = a#Int64, b#Utf8, Native storage config = { Use multithreading = true }, Limit pushdown = 5, Output schema = a#Int64, b#Utf8"; assert_optimized_plan_eq(plan, expected)?; Ok(()) } - /// Tests that Limit does not push into external Source with existing smaller limit. + /// Tests that Limit does push into scan with existing larger limit. /// - /// Limit[x]-Limit[y] -> Limit[min(x,y)] + /// Limit-Source[existing_limit] -> Source[new_limit] #[test] - fn limit_folds_with_large_limit() -> DaftResult<()> { + fn limit_does_push_into_scan_if_larger_limit() -> DaftResult<()> { + let limit = 5; + let existing_limit = 10; + let scan_op = dummy_scan_operator(vec![ + Field::new("a", DataType::Int64), + Field::new("b", DataType::Utf8), + ]); let plan = dummy_scan_node_with_pushdowns( - vec![ - Field::new("a", DataType::Int64), - Field::new("b", DataType::Utf8), - ], - Pushdowns::default().with_limit(Some(20)), + scan_op.clone(), + Pushdowns::default().with_limit(Some(existing_limit)), ) - .limit(10, false)? - .limit(5, false)? + .limit(limit, false)? .build(); - let expected = "\ - Limit: 5\ - \n Source: Json, File paths = [/foo], File schema = a#Int64, b#Utf8, Native storage config = { Use multithreading = true }, Limit pushdown = 5, Output schema = a#Int64, b#Utf8"; - assert_optimized_plan_eq(plan, expected)?; - Ok(()) - } - - /// Tests that Limit does push into external Source with existing larger limit. - /// - /// Limit-Source[existing_limit] -> Source[new_limit] - #[test] - fn limit_does_push_into_external_source_if_larger_limit() -> DaftResult<()> { - let plan = dummy_scan_node_with_pushdowns( - vec![ - Field::new("a", DataType::Int64), - Field::new("b", DataType::Utf8), - ], - Pushdowns::default().with_limit(Some(10)), + let expected = dummy_scan_node_with_pushdowns( + scan_op, + Pushdowns::default().with_limit(Some(limit as usize)), ) - .limit(5, false)? + .limit(limit, false)? .build(); - let expected = "\ - Limit: 5\ - \n Source: Json, File paths = [/foo], File schema = a#Int64, b#Utf8, Native storage config = { Use multithreading = true }, Limit pushdown = 5, Output schema = a#Int64, b#Utf8"; assert_optimized_plan_eq(plan, expected)?; Ok(()) } - /// Tests that Limit does push into external Source with existing larger limit. + /// Tests that multiple adjacent Limits fold into the smallest limit. /// - /// Limit-Source[existing_limit] -> Source[new_limit] - #[test] - fn limit_does_push_into_scan_operator_if_larger_limit() -> DaftResult<()> { - let plan = dummy_scan_operator_node_with_pushdowns( - vec![ - Field::new("a", DataType::Int64), - Field::new("b", DataType::Utf8), - ], - Pushdowns::default().with_limit(Some(10)), + /// Limit[x]-Limit[y] -> Limit[min(x,y)] + #[rstest] + fn limit_folds_with_smaller_limit( + #[values(false, true)] smaller_first: bool, + ) -> DaftResult<()> { + let smaller_limit = 5; + let limit = 10; + let scan_op = dummy_scan_operator(vec![ + Field::new("a", DataType::Int64), + Field::new("b", DataType::Utf8), + ]); + let plan = dummy_scan_node(scan_op.clone()) + .limit(if smaller_first { smaller_limit } else { limit }, false)? + .limit(if smaller_first { limit } else { smaller_limit }, false)? + .build(); + let expected = dummy_scan_node_with_pushdowns( + scan_op, + Pushdowns::default().with_limit(Some(smaller_limit as usize)), ) - .limit(5, false)? + .limit(smaller_limit, false)? .build(); - let expected = "\ - Limit: 5\ - \n AnonymousScanOperator, File paths = [/foo], Use multithreading = true, File schema = a#Int64, b#Utf8, Partitioning keys = [], Limit pushdown = 5, Output schema = a#Int64, b#Utf8"; assert_optimized_plan_eq(plan, expected)?; Ok(()) } @@ -299,10 +251,7 @@ mod tests { LogicalPlanBuilder::in_memory_scan("foo", py_obj, schema, Default::default(), 5)? .limit(5, false)? .build(); - let expected = "\ - Limit: 5\ - \n . Source: Json, File paths = [/foo], File schema = a#Int64, b#Utf8, Native storage config = { Use multithreading = true }, Output schema = a#Int64, b#Utf8"; - assert_optimized_plan_eq(plan, expected)?; + assert_optimized_plan_eq(plan.clone(), plan)?; Ok(()) } @@ -311,21 +260,29 @@ mod tests { /// Limit-Repartition-Source -> Repartition-Source[with_limit] #[test] fn limit_commutes_with_repartition() -> DaftResult<()> { - let plan = dummy_scan_node(vec![ + let limit = 5; + let num_partitions = Some(1); + let partition_by = vec![col("a")]; + let partition_scheme_config = PartitionSchemeConfig::Hash(Default::default()); + let scan_op = dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Utf8), - ]) - .repartition( - Some(1), - vec![col("a")], - PartitionSchemeConfig::Hash(Default::default()), - )? - .limit(5, false)? + ]); + let plan = dummy_scan_node(scan_op.clone()) + .repartition( + num_partitions, + partition_by.clone(), + partition_scheme_config.clone(), + )? + .limit(limit, false)? + .build(); + let expected = dummy_scan_node_with_pushdowns( + scan_op, + Pushdowns::default().with_limit(Some(limit as usize)), + ) + .limit(limit, false)? + .repartition(num_partitions, partition_by, partition_scheme_config)? .build(); - let expected = "\ - Repartition: Scheme = Hash, Number of partitions = 1, Partition by = col(a)\ - \n Limit: 5\ - \n Source: Json, File paths = [/foo], File schema = a#Int64, b#Utf8, Native storage config = { Use multithreading = true }, Limit pushdown = 5, Output schema = a#Int64, b#Utf8"; assert_optimized_plan_eq(plan, expected)?; Ok(()) } @@ -335,17 +292,23 @@ mod tests { /// Limit-Project-Source -> Project-Source[with_limit] #[test] fn limit_commutes_with_projection() -> DaftResult<()> { - let plan = dummy_scan_node(vec![ + let limit = 5; + let proj = vec![col("a")]; + let scan_op = dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Utf8), - ]) - .project(vec![col("a")], Default::default())? - .limit(5, false)? + ]); + let plan = dummy_scan_node(scan_op.clone()) + .project(proj.clone(), Default::default())? + .limit(limit, false)? + .build(); + let expected = dummy_scan_node_with_pushdowns( + scan_op, + Pushdowns::default().with_limit(Some(limit as usize)), + ) + .limit(limit, false)? + .project(proj, Default::default())? .build(); - let expected = "\ - Project: col(a)\ - \n Limit: 5\ - \n Source: Json, File paths = [/foo], File schema = a#Int64, b#Utf8, Native storage config = { Use multithreading = true }, Limit pushdown = 5, Output schema = a#Int64, b#Utf8"; assert_optimized_plan_eq(plan, expected)?; Ok(()) } diff --git a/src/daft-plan/src/optimization/rules/push_down_projection.rs b/src/daft-plan/src/optimization/rules/push_down_projection.rs index afcce3f2b3..3e6aad50b3 100644 --- a/src/daft-plan/src/optimization/rules/push_down_projection.rs +++ b/src/daft-plan/src/optimization/rules/push_down_projection.rs @@ -159,7 +159,7 @@ impl PushDownProjection { let new_source: LogicalPlan = Source::new( schema.into(), Arc::new(SourceInfo::ExternalInfo(external_info.with_pushdowns( - external_info.pushdowns().with_columns(Some(Arc::new( + external_info.pushdowns.with_columns(Some(Arc::new( required_columns.iter().cloned().collect(), ))), ))), @@ -504,39 +504,26 @@ mod tests { use common_error::DaftResult; use daft_core::{datatypes::Field, DataType}; use daft_dsl::{col, lit}; + use daft_scan::Pushdowns; use crate::{ - optimization::{ - optimizer::{RuleBatch, RuleExecutionStrategy}, - rules::PushDownProjection, - Optimizer, - }, - test::dummy_scan_node, + optimization::{rules::PushDownProjection, test::assert_optimized_plan_with_rules_eq}, + test::{dummy_scan_node, dummy_scan_node_with_pushdowns, dummy_scan_operator}, LogicalPlan, }; /// Helper that creates an optimizer with the PushDownProjection rule registered, optimizes - /// the provided plan with said optimizer, and compares the optimized plan's repr with - /// the provided expected repr. - fn assert_optimized_plan_eq(plan: Arc, expected: &str) -> DaftResult<()> { - let optimizer = Optimizer::with_rule_batches( - vec![RuleBatch::new( - vec![Box::new(PushDownProjection::new())], - RuleExecutionStrategy::Once, - )], - Default::default(), - ); - let optimized_plan = optimizer - .optimize_with_rules( - optimizer.rule_batches[0].rules.as_slice(), - plan.clone(), - &optimizer.rule_batches[0].order, - )? - .unwrap() - .clone(); - assert_eq!(optimized_plan.repr_indent(), expected); - - Ok(()) + /// the provided plan with said optimizer, and compares the optimized plan with + /// the provided expected plan. + fn assert_optimized_plan_eq( + plan: Arc, + expected: Arc, + ) -> DaftResult<()> { + assert_optimized_plan_with_rules_eq( + plan, + expected, + vec![Box::new(PushDownProjection::new())], + ) } /// Projection merging: Ensure factored projections do not get merged. @@ -546,12 +533,12 @@ mod tests { let a4 = &a2 + &a2; let a8 = &a4 + &a4; let expressions = vec![a8.alias("x")]; - let unoptimized = dummy_scan_node(vec![Field::new("a", DataType::Int64)]) + let scan_op = dummy_scan_operator(vec![Field::new("a", DataType::Int64)]); + let plan = dummy_scan_node(scan_op) .project(expressions, Default::default())? .build(); - let expected = unoptimized.repr_indent(); - assert_optimized_plan_eq(unoptimized, expected.as_str())?; + assert_optimized_plan_eq(plan.clone(), plan)?; Ok(()) } @@ -559,57 +546,65 @@ mod tests { /// in both the parent and the child. #[test] fn test_merge_projections() -> DaftResult<()> { - let unoptimized = dummy_scan_node(vec![ + let scan_op = dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Int64), - ]) - .project( - vec![col("a") + lit(1), col("b") + lit(2), col("a").alias("c")], - Default::default(), - )? - .project( - vec![col("a") + lit(3), col("b"), col("c") + lit(4)], - Default::default(), - )? - .build(); + ]); + let proj1 = vec![col("a") + lit(1), col("b") + lit(2), col("a").alias("c")]; + let proj2 = vec![col("a") + lit(3), col("b"), col("c") + lit(4)]; + let plan = dummy_scan_node(scan_op.clone()) + .project(proj1, Default::default())? + .project(proj2, Default::default())? + .build(); + + let merged_proj = vec![ + col("a") + lit(1) + lit(3), + col("b") + lit(2), + col("a").alias("c") + lit(4), + ]; + let expected = dummy_scan_node(scan_op) + .project(merged_proj, Default::default())? + .build(); - let expected = "\ - Project: [col(a) + lit(1)] + lit(3), col(b) + lit(2), col(a) + lit(4)\ - \n Source: Json, File paths = [/foo], File schema = a#Int64, b#Int64, Native storage config = { Use multithreading = true }, Output schema = a#Int64, b#Int64"; - assert_optimized_plan_eq(unoptimized, expected)?; + assert_optimized_plan_eq(plan, expected)?; Ok(()) } /// Projection dropping: Test that a no-op projection is dropped. #[test] fn test_drop_projection() -> DaftResult<()> { - let unoptimized = dummy_scan_node(vec![ + let scan_op = dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Int64), - ]) - .project(vec![col("a"), col("b")], Default::default())? - .build(); + ]); + let plan = dummy_scan_node(scan_op.clone()) + .project(vec![col("a"), col("b")], Default::default())? + .build(); + + let expected = dummy_scan_node(scan_op).build(); - let expected = "\ - Source: Json, File paths = [/foo], File schema = a#Int64, b#Int64, Native storage config = { Use multithreading = true }, Output schema = a#Int64, b#Int64"; - assert_optimized_plan_eq(unoptimized, expected)?; + assert_optimized_plan_eq(plan, expected)?; Ok(()) } + /// Projection dropping: Test that projections doing reordering are not dropped. #[test] fn test_dont_drop_projection() -> DaftResult<()> { - let unoptimized = dummy_scan_node(vec![ + let scan_op = dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Int64), - ]) - .project(vec![col("b"), col("a")], Default::default())? - .build(); + ]); + let proj = vec![col("b"), col("a")]; + let plan = dummy_scan_node(scan_op.clone()) + .project(proj.clone(), Default::default())? + .build(); - let expected = "\ - Project: col(b), col(a)\ - \n Source: Json, File paths = [/foo], File schema = a#Int64, b#Int64, Native storage config = { Use multithreading = true }, Output schema = a#Int64, b#Int64"; - assert_optimized_plan_eq(unoptimized, expected)?; + let expected = dummy_scan_node(scan_op) + .project(proj, Default::default())? + .build(); + + assert_optimized_plan_eq(plan, expected)?; Ok(()) } @@ -617,17 +612,24 @@ mod tests { /// Projection<-Source #[test] fn test_projection_source() -> DaftResult<()> { - let unoptimized = dummy_scan_node(vec![ + let scan_op = dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Int64), - ]) - .project(vec![col("b") + lit(3)], Default::default())? + ]); + let proj = vec![col("b") + lit(3)]; + let plan = dummy_scan_node(scan_op.clone()) + .project(proj.clone(), Default::default())? + .build(); + + let proj_pushdown = vec!["b".to_string()]; + let expected = dummy_scan_node_with_pushdowns( + scan_op, + Pushdowns::default().with_columns(Some(Arc::new(proj_pushdown))), + ) + .project(proj, Default::default())? .build(); - let expected = "\ - Project: col(b) + lit(3)\ - \n Source: Json, File paths = [/foo], File schema = a#Int64, b#Int64, Native storage config = { Use multithreading = true }, Projection pushdown = [b], Output schema = b#Int64"; - assert_optimized_plan_eq(unoptimized, expected)?; + assert_optimized_plan_eq(plan, expected)?; Ok(()) } @@ -635,25 +637,24 @@ mod tests { /// Projection<-Projection column pruning #[test] fn test_projection_projection() -> DaftResult<()> { - let unoptimized = dummy_scan_node(vec![ + let scan_op = dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Int64), - ]) - .project( - vec![col("b") + lit(3), col("a"), col("a").alias("x")], - Default::default(), - )? - .project( - vec![col("a"), col("b"), col("b").alias("c")], - Default::default(), - )? - .build(); + ]); + let proj1 = vec![col("b") + lit(3), col("a"), col("a").alias("x")]; + let proj2 = vec![col("a"), col("b"), col("b").alias("c")]; + let plan = dummy_scan_node(scan_op.clone()) + .project(proj1, Default::default())? + .project(proj2.clone(), Default::default())? + .build(); - let expected = "\ - Project: col(a), col(b), col(b) AS c\ - \n Project: col(b) + lit(3), col(a)\ - \n Source: Json, File paths = [/foo], File schema = a#Int64, b#Int64, Native storage config = { Use multithreading = true }, Output schema = a#Int64, b#Int64"; - assert_optimized_plan_eq(unoptimized, expected)?; + let new_proj1 = vec![col("b") + lit(3), col("a")]; + let expected = dummy_scan_node(scan_op) + .project(new_proj1, Default::default())? + .project(proj2, Default::default())? + .build(); + + assert_optimized_plan_eq(plan, expected)?; Ok(()) } @@ -661,20 +662,30 @@ mod tests { /// Projection<-Aggregation column pruning #[test] fn test_projection_aggregation() -> DaftResult<()> { - let unoptimized = dummy_scan_node(vec![ + let scan_op = dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Int64), Field::new("c", DataType::Int64), - ]) - .aggregate(vec![col("a").mean(), col("b").mean()], vec![col("c")])? - .project(vec![col("a")], Default::default())? + ]); + let agg = vec![col("a").mean(), col("b").mean()]; + let group_by = vec![col("c")]; + let proj = vec![col("a")]; + let plan = dummy_scan_node(scan_op.clone()) + .aggregate(agg, group_by.clone())? + .project(proj.clone(), Default::default())? + .build(); + + let proj_pushdown = vec!["a".to_string(), "c".to_string()]; + let new_agg = vec![col("a").mean()]; + let expected = dummy_scan_node_with_pushdowns( + scan_op, + Pushdowns::default().with_columns(Some(Arc::new(proj_pushdown))), + ) + .aggregate(new_agg, group_by)? + .project(proj, Default::default())? .build(); - let expected = "\ - Project: col(a)\ - \n Aggregation: mean(col(a)), Group by = col(c), Output schema = c#Int64, a#Float64\ - \n Source: Json, File paths = [/foo], File schema = a#Int64, b#Int64, c#Int64, Native storage config = { Use multithreading = true }, Projection pushdown = [a, c], Output schema = a#Int64, c#Int64"; - assert_optimized_plan_eq(unoptimized, expected)?; + assert_optimized_plan_eq(plan, expected)?; Ok(()) } @@ -682,20 +693,28 @@ mod tests { /// Projection<-X pushes down the combined required columns #[test] fn test_projection_pushdown() -> DaftResult<()> { - let unoptimized = dummy_scan_node(vec![ + let scan_op = dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Boolean), Field::new("c", DataType::Int64), - ]) - .filter(col("b"))? - .project(vec![col("a")], Default::default())? + ]); + let pred = col("b"); + let proj = vec![col("a")]; + let plan = dummy_scan_node(scan_op.clone()) + .filter(pred.clone())? + .project(proj.clone(), Default::default())? + .build(); + + let proj_pushdown = vec!["a".to_string(), "b".to_string()]; + let expected = dummy_scan_node_with_pushdowns( + scan_op, + Pushdowns::default().with_columns(Some(Arc::new(proj_pushdown))), + ) + .filter(pred)? + .project(proj, Default::default())? .build(); - let expected = "\ - Project: col(a)\ - \n Filter: col(b)\ - \n Source: Json, File paths = [/foo], File schema = a#Int64, b#Boolean, c#Int64, Native storage config = { Use multithreading = true }, Projection pushdown = [a, b], Output schema = a#Int64, b#Boolean"; - assert_optimized_plan_eq(unoptimized, expected)?; + assert_optimized_plan_eq(plan, expected)?; Ok(()) } diff --git a/src/daft-plan/src/optimization/test/mod.rs b/src/daft-plan/src/optimization/test/mod.rs new file mode 100644 index 0000000000..0e09c84eec --- /dev/null +++ b/src/daft-plan/src/optimization/test/mod.rs @@ -0,0 +1,44 @@ +use std::sync::Arc; + +use common_error::DaftResult; + +use crate::{ + optimization::{ + optimizer::{RuleBatch, RuleExecutionStrategy}, + Optimizer, + }, + LogicalPlan, +}; + +use super::optimizer::OptimizerRuleInBatch; + +/// Helper that creates an optimizer with the provided rules registered, optimizes +/// the provided plan with said optimizer, and compares the optimized plan with +/// the provided expected plan. +pub fn assert_optimized_plan_with_rules_eq( + plan: Arc, + expected: Arc, + rules: Vec>, +) -> DaftResult<()> { + let optimizer = Optimizer::with_rule_batches( + vec![RuleBatch::new(rules, RuleExecutionStrategy::Once)], + Default::default(), + ); + let optimized_plan = optimizer + .optimize_with_rules( + optimizer.rule_batches[0].rules.as_slice(), + plan.clone(), + &optimizer.rule_batches[0].order, + )? + .unwrap() + .clone(); + assert_eq!( + optimized_plan, + expected, + "\n\nOptimized plan not equal to expected.\n\nOptimized:\n{}\n\nExpected:\n{}", + optimized_plan.repr_ascii(false), + expected.repr_ascii(false) + ); + + Ok(()) +} diff --git a/src/daft-plan/src/physical_ops/csv.rs b/src/daft-plan/src/physical_ops/csv.rs index eed7ab7889..ccff3ff7c4 100644 --- a/src/daft-plan/src/physical_ops/csv.rs +++ b/src/daft-plan/src/physical_ops/csv.rs @@ -1,53 +1,8 @@ -use std::sync::Arc; - use daft_core::schema::SchemaRef; -use daft_scan::Pushdowns; -use crate::{ - physical_plan::PhysicalPlanRef, sink_info::OutputFileInfo, source_info::LegacyExternalInfo, - PartitionSpec, -}; +use crate::{physical_plan::PhysicalPlanRef, sink_info::OutputFileInfo}; use serde::{Deserialize, Serialize}; -#[derive(Clone, Debug, PartialEq, Deserialize, Serialize)] -pub struct TabularScanCsv { - pub projection_schema: SchemaRef, - pub external_info: LegacyExternalInfo, - pub partition_spec: Arc, - pub pushdowns: Pushdowns, -} - -impl TabularScanCsv { - pub(crate) fn new( - projection_schema: SchemaRef, - external_info: LegacyExternalInfo, - partition_spec: Arc, - pushdowns: Pushdowns, - ) -> Self { - Self { - projection_schema, - external_info, - partition_spec, - pushdowns, - } - } - - pub fn multiline_display(&self) -> Vec { - let mut res = vec![]; - res.push("TabularScanCsv:".to_string()); - res.push(format!( - "Projection schema = {}", - self.projection_schema.short_string() - )); - res.extend(self.external_info.multiline_display()); - res.push(format!( - "Partition spec = {{ {} }}", - self.partition_spec.multiline_display().join(", ") - )); - res - } -} - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct TabularWriteCsv { pub schema: SchemaRef, diff --git a/src/daft-plan/src/physical_ops/explode.rs b/src/daft-plan/src/physical_ops/explode.rs index b0e6308ddc..7bb29785d7 100644 --- a/src/daft-plan/src/physical_ops/explode.rs +++ b/src/daft-plan/src/physical_ops/explode.rs @@ -82,18 +82,22 @@ mod tests { use daft_core::{datatypes::Field, DataType}; use daft_dsl::{col, Expr}; - use crate::{planner::plan, test::dummy_scan_node, PartitionSchemeConfig, PartitionSpec}; + use crate::{ + planner::plan, + test::{dummy_scan_node, dummy_scan_operator}, + PartitionSchemeConfig, PartitionSpec, + }; /// do not destroy the partition spec. #[test] fn test_partition_spec_preserving() -> DaftResult<()> { let cfg = DaftExecutionConfig::default().into(); - let logical_plan = dummy_scan_node(vec![ + let logical_plan = dummy_scan_node(dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::List(Box::new(DataType::Int64))), Field::new("c", DataType::Int64), - ]) + ])) .repartition( Some(3), vec![Expr::Column("a".into())], @@ -123,11 +127,11 @@ mod tests { fn test_partition_spec_destroying() -> DaftResult<()> { let cfg = DaftExecutionConfig::default().into(); - let logical_plan = dummy_scan_node(vec![ + let logical_plan = dummy_scan_node(dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::List(Box::new(DataType::Int64))), Field::new("c", DataType::Int64), - ]) + ])) .repartition( Some(3), vec![Expr::Column("a".into()), Expr::Column("b".into())], diff --git a/src/daft-plan/src/physical_ops/json.rs b/src/daft-plan/src/physical_ops/json.rs index 6d0f51229d..1aea9c4a41 100644 --- a/src/daft-plan/src/physical_ops/json.rs +++ b/src/daft-plan/src/physical_ops/json.rs @@ -1,53 +1,8 @@ -use std::sync::Arc; - use daft_core::schema::SchemaRef; -use daft_scan::Pushdowns; -use crate::{ - physical_plan::PhysicalPlanRef, sink_info::OutputFileInfo, source_info::LegacyExternalInfo, - PartitionSpec, -}; +use crate::{physical_plan::PhysicalPlanRef, sink_info::OutputFileInfo}; use serde::{Deserialize, Serialize}; -#[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] -pub struct TabularScanJson { - pub projection_schema: SchemaRef, - pub external_info: LegacyExternalInfo, - pub partition_spec: Arc, - pub pushdowns: Pushdowns, -} - -impl TabularScanJson { - pub(crate) fn new( - projection_schema: SchemaRef, - external_info: LegacyExternalInfo, - partition_spec: Arc, - pushdowns: Pushdowns, - ) -> Self { - Self { - projection_schema, - external_info, - partition_spec, - pushdowns, - } - } - - pub fn multiline_display(&self) -> Vec { - let mut res = vec![]; - res.push("TabularScanJson:".to_string()); - res.push(format!( - "Projection schema = {}", - self.projection_schema.short_string() - )); - res.extend(self.external_info.multiline_display()); - res.push(format!( - "Partition spec = {{ {} }}", - self.partition_spec.multiline_display().join(", ") - )); - res - } -} - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct TabularWriteJson { pub schema: SchemaRef, diff --git a/src/daft-plan/src/physical_ops/mod.rs b/src/daft-plan/src/physical_ops/mod.rs index 849a9470cb..346462f796 100644 --- a/src/daft-plan/src/physical_ops/mod.rs +++ b/src/daft-plan/src/physical_ops/mod.rs @@ -27,7 +27,7 @@ pub use agg::Aggregate; pub use broadcast_join::BroadcastJoin; pub use coalesce::Coalesce; pub use concat::Concat; -pub use csv::{TabularScanCsv, TabularWriteCsv}; +pub use csv::TabularWriteCsv; pub use empty_scan::EmptyScan; pub use explode::Explode; pub use fanout::{FanoutByHash, FanoutByRange, FanoutRandom}; @@ -36,10 +36,10 @@ pub use flatten::Flatten; pub use hash_join::HashJoin; #[cfg(feature = "python")] pub use in_memory::InMemoryScan; -pub use json::{TabularScanJson, TabularWriteJson}; +pub use json::TabularWriteJson; pub use limit::Limit; pub use monotonically_increasing_id::MonotonicallyIncreasingId; -pub use parquet::{TabularScanParquet, TabularWriteParquet}; +pub use parquet::TabularWriteParquet; pub use project::Project; pub use reduce::ReduceMerge; pub use sample::Sample; diff --git a/src/daft-plan/src/physical_ops/parquet.rs b/src/daft-plan/src/physical_ops/parquet.rs index a93cac5fc2..371babc4ae 100644 --- a/src/daft-plan/src/physical_ops/parquet.rs +++ b/src/daft-plan/src/physical_ops/parquet.rs @@ -1,53 +1,8 @@ -use std::sync::Arc; - use daft_core::schema::SchemaRef; -use daft_scan::Pushdowns; -use crate::{ - physical_plan::PhysicalPlanRef, sink_info::OutputFileInfo, - source_info::LegacyExternalInfo as ExternalSourceInfo, PartitionSpec, -}; +use crate::{physical_plan::PhysicalPlanRef, sink_info::OutputFileInfo}; use serde::{Deserialize, Serialize}; -#[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] -pub struct TabularScanParquet { - pub projection_schema: SchemaRef, - pub external_info: ExternalSourceInfo, - pub partition_spec: Arc, - pub pushdowns: Pushdowns, -} - -impl TabularScanParquet { - pub(crate) fn new( - projection_schema: SchemaRef, - external_info: ExternalSourceInfo, - partition_spec: Arc, - pushdowns: Pushdowns, - ) -> Self { - Self { - projection_schema, - external_info, - partition_spec, - pushdowns, - } - } - - pub fn multiline_display(&self) -> Vec { - let mut res = vec![]; - res.push("TabularScanParquet:".to_string()); - res.push(format!( - "Projection schema = {}", - self.projection_schema.short_string() - )); - res.extend(self.external_info.multiline_display()); - res.push(format!( - "Partition spec = {{ {} }}", - self.partition_spec.multiline_display().join(", ") - )); - res - } -} - #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct TabularWriteParquet { pub schema: SchemaRef, diff --git a/src/daft-plan/src/physical_ops/project.rs b/src/daft-plan/src/physical_ops/project.rs index 5fba44d476..5a86035fe2 100644 --- a/src/daft-plan/src/physical_ops/project.rs +++ b/src/daft-plan/src/physical_ops/project.rs @@ -235,7 +235,10 @@ mod tests { use rstest::rstest; use crate::{ - partitioning::PartitionSchemeConfig, planner::plan, test::dummy_scan_node, PartitionSpec, + partitioning::PartitionSchemeConfig, + planner::plan, + test::{dummy_scan_node, dummy_scan_operator}, + PartitionSpec, }; /// Test that projections preserving column inputs, even through aliasing, @@ -248,11 +251,11 @@ mod tests { col("b"), col("a").alias("aa"), ]; - let logical_plan = dummy_scan_node(vec![ + let logical_plan = dummy_scan_node(dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Int64), Field::new("c", DataType::Int64), - ]) + ])) .repartition( Some(3), vec![Expr::Column("a".into()), Expr::Column("b".into())], @@ -292,11 +295,11 @@ mod tests { use crate::partitioning::PartitionSchemeConfig; let cfg = DaftExecutionConfig::default().into(); - let logical_plan = dummy_scan_node(vec![ + let logical_plan = dummy_scan_node(dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Int64), Field::new("c", DataType::Int64), - ]) + ])) .repartition( Some(3), vec![Expr::Column("a".into()), Expr::Column("b".into())], @@ -324,11 +327,11 @@ mod tests { let cfg = DaftExecutionConfig::default().into(); let expressions = vec![col("a").alias("y"), col("a"), col("a").alias("z"), col("b")]; - let logical_plan = dummy_scan_node(vec![ + let logical_plan = dummy_scan_node(dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Int64), Field::new("c", DataType::Int64), - ]) + ])) .repartition( Some(3), vec![Expr::Column("a".into()), Expr::Column("b".into())], diff --git a/src/daft-plan/src/physical_plan.rs b/src/daft-plan/src/physical_plan.rs index e306d1e593..02d311171c 100644 --- a/src/daft-plan/src/physical_plan.rs +++ b/src/daft-plan/src/physical_plan.rs @@ -1,21 +1,12 @@ #[cfg(feature = "python")] use { - crate::{ - partitioning::PyPartitionSpec, - sink_info::OutputFileInfo, - source_info::{FileInfos, InMemoryInfo, LegacyExternalInfo}, - }, + crate::{partitioning::PyPartitionSpec, sink_info::OutputFileInfo, source_info::InMemoryInfo}, common_io_config::IOConfig, daft_core::python::schema::PySchema, daft_core::schema::SchemaRef, daft_dsl::python::PyExpr, daft_dsl::Expr, - daft_scan::{ - file_format::{FileFormat, FileFormatConfig, PyFileFormatConfig}, - python::pylib::PyScanTask, - storage_config::{PyStorageConfig, StorageConfig}, - Pushdowns, - }, + daft_scan::{file_format::FileFormat, python::pylib::PyScanTask}, pyo3::{ pyclass, pymethods, types::PyBytes, PyObject, PyRef, PyRefMut, PyResult, PyTypeInfo, Python, ToPyObject, @@ -41,9 +32,6 @@ pub(crate) type PhysicalPlanRef = Arc; pub enum PhysicalPlan { #[cfg(feature = "python")] InMemoryScan(InMemoryScan), - TabularScanParquet(TabularScanParquet), - TabularScanCsv(TabularScanCsv), - TabularScanJson(TabularScanJson), TabularScan(TabularScan), EmptyScan(EmptyScan), Project(Project), @@ -78,11 +66,6 @@ impl PhysicalPlan { Self::InMemoryScan(InMemoryScan { partition_spec, .. }) => partition_spec.clone(), Self::TabularScan(TabularScan { partition_spec, .. }) => partition_spec.clone(), Self::EmptyScan(EmptyScan { partition_spec, .. }) => partition_spec.clone(), - Self::TabularScanParquet(TabularScanParquet { partition_spec, .. }) => { - partition_spec.clone() - } - Self::TabularScanCsv(TabularScanCsv { partition_spec, .. }) => partition_spec.clone(), - Self::TabularScanJson(TabularScanJson { partition_spec, .. }) => partition_spec.clone(), Self::Project(Project { partition_spec, .. }) => partition_spec.clone(), Self::Filter(Filter { input, .. }) => input.partition_spec(), Self::Limit(Limit { input, .. }) => input.partition_spec(), @@ -279,10 +262,6 @@ impl PhysicalPlan { } // TODO(Clark): Approximate post-aggregation sizes via grouping estimates + aggregation type. Self::Aggregate(_) => None, - // No size approximation support for legacy I/O. - Self::TabularScanParquet(_) | Self::TabularScanCsv(_) | Self::TabularScanJson(_) => { - None - } // Post-write DataFrame will contain paths to files that were written. // TODO(Clark): Estimate output size via root directory and estimates for # of partitions given partitioning column. Self::TabularWriteParquet(_) | Self::TabularWriteCsv(_) | Self::TabularWriteJson(_) => { @@ -295,11 +274,7 @@ impl PhysicalPlan { match self { #[cfg(feature = "python")] Self::InMemoryScan(..) => vec![], - Self::TabularScan(..) - | Self::EmptyScan(..) - | Self::TabularScanParquet(..) - | Self::TabularScanCsv(..) - | Self::TabularScanJson(..) => vec![], + Self::TabularScan(..) | Self::EmptyScan(..) => vec![], Self::Project(Project { input, .. }) => vec![input], Self::Filter(Filter { input, .. }) => vec![input], Self::Limit(Limit { input, .. }) => vec![input], @@ -335,10 +310,7 @@ impl PhysicalPlan { #[cfg(feature = "python")] Self::InMemoryScan(..) => panic!("Source nodes don't have children, with_new_children() should never be called for source ops"), Self::TabularScan(..) - | Self::EmptyScan(..) - | Self::TabularScanParquet(..) - | Self::TabularScanCsv(..) - | Self::TabularScanJson(..) => panic!("Source nodes don't have children, with_new_children() should never be called for source ops"), + | Self::EmptyScan(..) => panic!("Source nodes don't have children, with_new_children() should never be called for source ops"), Self::Project(Project { projection, resource_request, partition_spec, .. }) => Self::Project(Project::try_new( input.clone(), projection.clone(), resource_request.clone(), partition_spec.clone(), ).unwrap()), @@ -364,10 +336,7 @@ impl PhysicalPlan { #[cfg(feature = "python")] Self::InMemoryScan(..) => panic!("Source nodes don't have children, with_new_children() should never be called for source ops"), Self::TabularScan(..) - | Self::EmptyScan(..) - | Self::TabularScanParquet(..) - | Self::TabularScanCsv(..) - | Self::TabularScanJson(..) => panic!("Source nodes don't have children, with_new_children() should never be called for source ops"), + | Self::EmptyScan(..) => panic!("Source nodes don't have children, with_new_children() should never be called for source ops"), Self::HashJoin(HashJoin { left_on, right_on, join_type, .. }) => Self::HashJoin(HashJoin::new(input1.clone(), input2.clone(), left_on.clone(), right_on.clone(), *join_type)), Self::BroadcastJoin(BroadcastJoin { left_on, @@ -390,9 +359,6 @@ impl PhysicalPlan { Self::InMemoryScan(..) => "InMemoryScan", Self::TabularScan(..) => "TabularScan", Self::EmptyScan(..) => "EmptyScan", - Self::TabularScanParquet(..) => "TabularScanParquet", - Self::TabularScanCsv(..) => "TabularScanCsv", - Self::TabularScanJson(..) => "TabularScanJson", Self::Project(..) => "Project", Self::Filter(..) => "Filter", Self::Limit(..) => "Limit", @@ -425,11 +391,6 @@ impl PhysicalPlan { Self::InMemoryScan(in_memory_scan) => in_memory_scan.multiline_display(), Self::TabularScan(tabular_scan) => tabular_scan.multiline_display(), Self::EmptyScan(empty_scan) => empty_scan.multiline_display(), - Self::TabularScanParquet(tabular_scan_parquet) => { - tabular_scan_parquet.multiline_display() - } - Self::TabularScanCsv(tabular_scan_csv) => tabular_scan_csv.multiline_display(), - Self::TabularScanJson(tabular_scan_json) => tabular_scan_json.multiline_display(), Self::Project(project) => project.multiline_display(), Self::Filter(filter) => filter.multiline_display(), Self::Limit(limit) => limit.multiline_display(), @@ -493,12 +454,8 @@ impl PhysicalPlanScheduler { Ok(self.plan.repr_ascii(simple)) } /// Converts the contained physical plan into an iterator of executable partition tasks. - pub fn to_partition_tasks( - &self, - psets: HashMap>, - is_ray_runner: bool, - ) -> PyResult { - Python::with_gil(|py| self.plan.to_partition_tasks(py, &psets, is_ray_runner)) + pub fn to_partition_tasks(&self, psets: HashMap>) -> PyResult { + Python::with_gil(|py| self.plan.to_partition_tasks(py, &psets)) } } @@ -530,46 +487,6 @@ impl PartitionIterator { } } -#[cfg(feature = "python")] -#[allow(clippy::too_many_arguments)] -fn tabular_scan( - py: Python<'_>, - source_schema: &SchemaRef, - projection_schema: &SchemaRef, - file_infos: &Arc, - file_format_config: &Arc, - storage_config: &Arc, - pushdowns: &Pushdowns, - is_ray_runner: bool, -) -> PyResult { - let columns_to_read = if projection_schema.names() != source_schema.names() { - Some( - projection_schema - .fields - .iter() - .map(|(name, _)| name) - .cloned() - .collect::>(), - ) - } else { - None - }; - let py_iter = py - .import(pyo3::intern!(py, "daft.execution.rust_physical_plan_shim"))? - .getattr(pyo3::intern!(py, "tabular_scan"))? - .call1(( - PySchema::from(source_schema.clone()), - columns_to_read, - file_infos.to_table()?, - PyFileFormatConfig::from(file_format_config.clone()), - PyStorageConfig::from(storage_config.clone()), - pushdowns.limit, - is_ray_runner, - ))?; - - Ok(py_iter.into()) -} - #[allow(clippy::too_many_arguments)] #[cfg(feature = "python")] fn tabular_write( @@ -612,7 +529,6 @@ impl PhysicalPlan { &self, py: Python<'_>, psets: &HashMap>, - is_ray_runner: bool, ) -> PyResult { match self { PhysicalPlan::InMemoryScan(InMemoryScan { @@ -655,79 +571,13 @@ impl PhysicalPlan { Ok(py_iter.into()) } - PhysicalPlan::TabularScanParquet(TabularScanParquet { - projection_schema, - external_info: - LegacyExternalInfo { - source_schema, - file_infos, - file_format_config, - storage_config, - pushdowns, - .. - }, - .. - }) => tabular_scan( - py, - source_schema, - projection_schema, - file_infos, - file_format_config, - storage_config, - pushdowns, - is_ray_runner, - ), - PhysicalPlan::TabularScanCsv(TabularScanCsv { - projection_schema, - external_info: - LegacyExternalInfo { - source_schema, - file_infos, - file_format_config, - storage_config, - pushdowns, - .. - }, - .. - }) => tabular_scan( - py, - source_schema, - projection_schema, - file_infos, - file_format_config, - storage_config, - pushdowns, - is_ray_runner, - ), - PhysicalPlan::TabularScanJson(TabularScanJson { - projection_schema, - external_info: - LegacyExternalInfo { - source_schema, - file_infos, - file_format_config, - storage_config, - pushdowns, - .. - }, - .. - }) => tabular_scan( - py, - source_schema, - projection_schema, - file_infos, - file_format_config, - storage_config, - pushdowns, - is_ray_runner, - ), PhysicalPlan::Project(Project { input, projection, resource_request, .. }) => { - let upstream_iter = input.to_partition_tasks(py, psets, is_ray_runner)?; + let upstream_iter = input.to_partition_tasks(py, psets)?; let projection_pyexprs: Vec = projection .iter() .map(|expr| PyExpr::from(expr.clone())) @@ -739,7 +589,7 @@ impl PhysicalPlan { Ok(py_iter.into()) } PhysicalPlan::Filter(Filter { input, predicate }) => { - let upstream_iter = input.to_partition_tasks(py, psets, is_ray_runner)?; + let upstream_iter = input.to_partition_tasks(py, psets)?; let expressions_mod = py.import(pyo3::intern!(py, "daft.expressions.expressions"))?; let py_predicate = expressions_mod @@ -769,7 +619,7 @@ impl PhysicalPlan { eager, num_partitions, }) => { - let upstream_iter = input.to_partition_tasks(py, psets, is_ray_runner)?; + let upstream_iter = input.to_partition_tasks(py, psets)?; let py_physical_plan = py.import(pyo3::intern!(py, "daft.execution.physical_plan"))?; let global_limit_iter = py_physical_plan @@ -780,7 +630,7 @@ impl PhysicalPlan { PhysicalPlan::Explode(Explode { input, to_explode, .. }) => { - let upstream_iter = input.to_partition_tasks(py, psets, is_ray_runner)?; + let upstream_iter = input.to_partition_tasks(py, psets)?; let explode_pyexprs: Vec = to_explode .iter() .map(|expr| PyExpr::from(expr.clone())) @@ -797,7 +647,7 @@ impl PhysicalPlan { with_replacement, seed, }) => { - let upstream_iter = input.to_partition_tasks(py, psets, is_ray_runner)?; + let upstream_iter = input.to_partition_tasks(py, psets)?; let py_iter = py .import(pyo3::intern!(py, "daft.execution.rust_physical_plan_shim"))? .getattr(pyo3::intern!(py, "sample"))? @@ -808,7 +658,7 @@ impl PhysicalPlan { input, column_name, }) => { - let upstream_iter = input.to_partition_tasks(py, psets, is_ray_runner)?; + let upstream_iter = input.to_partition_tasks(py, psets)?; let py_iter = py .import(pyo3::intern!(py, "daft.execution.physical_plan"))? .getattr(pyo3::intern!(py, "monotonically_increasing_id"))? @@ -821,7 +671,7 @@ impl PhysicalPlan { descending, num_partitions, }) => { - let upstream_iter = input.to_partition_tasks(py, psets, is_ray_runner)?; + let upstream_iter = input.to_partition_tasks(py, psets)?; let sort_by_pyexprs: Vec = sort_by .iter() .map(|expr| PyExpr::from(expr.clone())) @@ -842,7 +692,7 @@ impl PhysicalPlan { input_num_partitions, output_num_partitions, }) => { - let upstream_iter = input.to_partition_tasks(py, psets, is_ray_runner)?; + let upstream_iter = input.to_partition_tasks(py, psets)?; let py_iter = py .import(pyo3::intern!(py, "daft.execution.physical_plan"))? .getattr(pyo3::intern!(py, "split"))? @@ -850,7 +700,7 @@ impl PhysicalPlan { Ok(py_iter.into()) } PhysicalPlan::Flatten(Flatten { input }) => { - let upstream_iter = input.to_partition_tasks(py, psets, is_ray_runner)?; + let upstream_iter = input.to_partition_tasks(py, psets)?; let py_iter = py .import(pyo3::intern!(py, "daft.execution.physical_plan"))? .getattr(pyo3::intern!(py, "flatten_plan"))? @@ -861,7 +711,7 @@ impl PhysicalPlan { input, num_partitions, }) => { - let upstream_iter = input.to_partition_tasks(py, psets, is_ray_runner)?; + let upstream_iter = input.to_partition_tasks(py, psets)?; let py_iter = py .import(pyo3::intern!(py, "daft.execution.physical_plan"))? .getattr(pyo3::intern!(py, "fanout_random"))? @@ -873,7 +723,7 @@ impl PhysicalPlan { num_partitions, partition_by, }) => { - let upstream_iter = input.to_partition_tasks(py, psets, is_ray_runner)?; + let upstream_iter = input.to_partition_tasks(py, psets)?; let partition_by_pyexprs: Vec = partition_by .iter() .map(|expr| PyExpr::from(expr.clone())) @@ -888,7 +738,7 @@ impl PhysicalPlan { "FanoutByRange not implemented, since only use case (sorting) doesn't need it yet." ), PhysicalPlan::ReduceMerge(ReduceMerge { input }) => { - let upstream_iter = input.to_partition_tasks(py, psets, is_ray_runner)?; + let upstream_iter = input.to_partition_tasks(py, psets)?; let py_iter = py .import(pyo3::intern!(py, "daft.execution.rust_physical_plan_shim"))? .getattr(pyo3::intern!(py, "reduce_merge"))? @@ -901,7 +751,7 @@ impl PhysicalPlan { input, .. }) => { - let upstream_iter = input.to_partition_tasks(py, psets, is_ray_runner)?; + let upstream_iter = input.to_partition_tasks(py, psets)?; let aggs_as_pyexprs: Vec = aggregations .iter() .map(|agg_expr| PyExpr::from(Expr::Agg(agg_expr.clone()))) @@ -921,7 +771,7 @@ impl PhysicalPlan { num_from, num_to, }) => { - let upstream_iter = input.to_partition_tasks(py, psets, is_ray_runner)?; + let upstream_iter = input.to_partition_tasks(py, psets)?; let py_iter = py .import(pyo3::intern!(py, "daft.execution.physical_plan"))? .getattr(pyo3::intern!(py, "coalesce"))? @@ -929,8 +779,8 @@ impl PhysicalPlan { Ok(py_iter.into()) } PhysicalPlan::Concat(Concat { other, input }) => { - let upstream_input_iter = input.to_partition_tasks(py, psets, is_ray_runner)?; - let upstream_other_iter = other.to_partition_tasks(py, psets, is_ray_runner)?; + let upstream_input_iter = input.to_partition_tasks(py, psets)?; + let upstream_other_iter = other.to_partition_tasks(py, psets)?; let py_iter = py .import(pyo3::intern!(py, "daft.execution.physical_plan"))? .getattr(pyo3::intern!(py, "concat"))? @@ -945,8 +795,8 @@ impl PhysicalPlan { join_type, .. }) => { - let upstream_left_iter = left.to_partition_tasks(py, psets, is_ray_runner)?; - let upstream_right_iter = right.to_partition_tasks(py, psets, is_ray_runner)?; + let upstream_left_iter = left.to_partition_tasks(py, psets)?; + let upstream_right_iter = right.to_partition_tasks(py, psets)?; let left_on_pyexprs: Vec = left_on .iter() .map(|expr| PyExpr::from(expr.clone())) @@ -977,8 +827,8 @@ impl PhysicalPlan { left_is_larger, needs_presort, }) => { - let left_iter = left.to_partition_tasks(py, psets, is_ray_runner)?; - let right_iter = right.to_partition_tasks(py, psets, is_ray_runner)?; + let left_iter = left.to_partition_tasks(py, psets)?; + let right_iter = right.to_partition_tasks(py, psets)?; let left_on_pyexprs: Vec = left_on .iter() .map(|expr| PyExpr::from(expr.clone())) @@ -1022,8 +872,8 @@ impl PhysicalPlan { join_type, is_swapped, }) => { - let upstream_left_iter = left.to_partition_tasks(py, psets, is_ray_runner)?; - let upstream_right_iter = right.to_partition_tasks(py, psets, is_ray_runner)?; + let upstream_left_iter = left.to_partition_tasks(py, psets)?; + let upstream_right_iter = right.to_partition_tasks(py, psets)?; let left_on_pyexprs: Vec = left_on .iter() .map(|expr| PyExpr::from(expr.clone())) @@ -1058,7 +908,7 @@ impl PhysicalPlan { input, }) => tabular_write( py, - input.to_partition_tasks(py, psets, is_ray_runner)?, + input.to_partition_tasks(py, psets)?, file_format, schema, root_dir, @@ -1079,7 +929,7 @@ impl PhysicalPlan { input, }) => tabular_write( py, - input.to_partition_tasks(py, psets, is_ray_runner)?, + input.to_partition_tasks(py, psets)?, file_format, schema, root_dir, @@ -1100,7 +950,7 @@ impl PhysicalPlan { input, }) => tabular_write( py, - input.to_partition_tasks(py, psets, is_ray_runner)?, + input.to_partition_tasks(py, psets)?, file_format, schema, root_dir, diff --git a/src/daft-plan/src/planner.rs b/src/daft-plan/src/planner.rs index e5d999b2a3..6a507a72b7 100644 --- a/src/daft-plan/src/planner.rs +++ b/src/daft-plan/src/planner.rs @@ -10,7 +10,6 @@ use common_error::DaftResult; use daft_core::count_mode::CountMode; use daft_core::DataType; use daft_dsl::Expr; -use daft_scan::file_format::FileFormatConfig; use daft_scan::ScanExternalInfo; use crate::logical_ops::{ @@ -24,7 +23,7 @@ use crate::logical_plan::LogicalPlan; use crate::partitioning::PartitionSchemeConfig; use crate::physical_plan::PhysicalPlan; use crate::sink_info::{OutputFileInfo, SinkInfo}; -use crate::source_info::{ExternalInfo as ExternalSourceInfo, LegacyExternalInfo, SourceInfo}; +use crate::source_info::SourceInfo; use crate::FileFormat; use crate::{physical_ops::*, JoinStrategy, PartitionSpec}; @@ -34,56 +33,13 @@ use crate::physical_ops::InMemoryScan; /// Translate a logical plan to a physical plan. pub fn plan(logical_plan: &LogicalPlan, cfg: Arc) -> DaftResult { match logical_plan { - LogicalPlan::Source(Source { - output_schema, - source_info, - }) => match source_info.as_ref() { - SourceInfo::ExternalInfo(ExternalSourceInfo::Legacy( - ext_info @ LegacyExternalInfo { - file_format_config, - file_infos, - pushdowns, - .. - }, - )) => { - let partition_spec = Arc::new(PartitionSpec::new( - PartitionSchemeConfig::Unknown(Default::default()), - file_infos.len(), - None, - )); - match file_format_config.as_ref() { - FileFormatConfig::Parquet(_) => { - Ok(PhysicalPlan::TabularScanParquet(TabularScanParquet::new( - output_schema.clone(), - ext_info.clone(), - partition_spec, - pushdowns.clone(), - ))) - } - FileFormatConfig::Csv(_) => { - Ok(PhysicalPlan::TabularScanCsv(TabularScanCsv::new( - output_schema.clone(), - ext_info.clone(), - partition_spec, - pushdowns.clone(), - ))) - } - FileFormatConfig::Json(_) => { - Ok(PhysicalPlan::TabularScanJson(TabularScanJson::new( - output_schema.clone(), - ext_info.clone(), - partition_spec, - pushdowns.clone(), - ))) - } - } - } - SourceInfo::ExternalInfo(ExternalSourceInfo::Scan(ScanExternalInfo { + LogicalPlan::Source(Source { source_info, .. }) => match source_info.as_ref() { + SourceInfo::ExternalInfo(ScanExternalInfo { pushdowns, scan_op, source_schema, .. - })) => { + }) => { let scan_tasks = scan_op.0.to_scan_tasks(pushdowns.clone())?; let scan_tasks = daft_scan::scan_task_iters::split_by_row_groups( @@ -796,7 +752,7 @@ mod tests { use crate::physical_plan::PhysicalPlan; use crate::planner::plan; - use crate::test::dummy_scan_node; + use crate::test::{dummy_scan_node, dummy_scan_operator}; /// Tests that planner drops a simple Repartition (e.g. df.into_partitions()) the child already has the desired number of partitions. /// @@ -805,10 +761,10 @@ mod tests { fn repartition_dropped_redundant_into_partitions() -> DaftResult<()> { let cfg: Arc = DaftExecutionConfig::default().into(); // dummy_scan_node() will create the default PartitionSpec, which only has a single partition. - let builder = dummy_scan_node(vec![ + let builder = dummy_scan_node(dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Utf8), - ]) + ])) .repartition(Some(10), vec![], Default::default())? .filter(col("a").lt(&lit(2)))?; assert_eq!( @@ -833,10 +789,10 @@ mod tests { fn repartition_dropped_single_partition() -> DaftResult<()> { let cfg: Arc = DaftExecutionConfig::default().into(); // dummy_scan_node() will create the default PartitionSpec, which only has a single partition. - let builder = dummy_scan_node(vec![ + let builder = dummy_scan_node(dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Utf8), - ]); + ])); assert_eq!( plan(builder.build().as_ref(), cfg.clone())? .partition_spec() @@ -847,7 +803,7 @@ mod tests { .repartition(Some(1), vec![col("a")], Default::default())? .build(); let physical_plan = plan(logical_plan.as_ref(), cfg.clone())?; - assert_matches!(physical_plan, PhysicalPlan::TabularScanJson(_)); + assert_matches!(physical_plan, PhysicalPlan::TabularScan(_)); Ok(()) } @@ -857,10 +813,10 @@ mod tests { #[test] fn repartition_dropped_same_partition_spec() -> DaftResult<()> { let cfg = DaftExecutionConfig::default().into(); - let logical_plan = dummy_scan_node(vec![ + let logical_plan = dummy_scan_node(dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Utf8), - ]) + ])) .repartition(Some(10), vec![col("a")], Default::default())? .filter(col("a").lt(&lit(2)))? .repartition(Some(10), vec![col("a")], Default::default())? @@ -877,10 +833,10 @@ mod tests { #[test] fn repartition_dropped_same_partition_spec_agg() -> DaftResult<()> { let cfg = DaftExecutionConfig::default().into(); - let logical_plan = dummy_scan_node(vec![ + let logical_plan = dummy_scan_node(dummy_scan_operator(vec![ Field::new("a", DataType::Int64), Field::new("b", DataType::Int64), - ]) + ])) .repartition(Some(10), vec![col("a")], Default::default())? .aggregate( vec![Expr::Agg(AggExpr::Sum(col("a").into()))], diff --git a/src/daft-plan/src/source_info/mod.rs b/src/daft-plan/src/source_info/mod.rs index c6053fb8b6..3d2f7acbed 100644 --- a/src/daft-plan/src/source_info/mod.rs +++ b/src/daft-plan/src/source_info/mod.rs @@ -1,11 +1,9 @@ pub mod file_info; use daft_core::schema::SchemaRef; -use daft_scan::storage_config::StorageConfig; use daft_scan::ScanExternalInfo; -use daft_scan::{file_format::FileFormatConfig, Pushdowns}; pub use file_info::{FileInfo, FileInfos}; use serde::{Deserialize, Serialize}; -use std::{hash::Hash, sync::Arc}; +use std::hash::Hash; #[cfg(feature = "python")] use { daft_scan::py_object_serde::{deserialize_py_object, serialize_py_object}, @@ -17,7 +15,7 @@ use { pub enum SourceInfo { #[cfg(feature = "python")] InMemoryInfo(InMemoryInfo), - ExternalInfo(ExternalInfo), + ExternalInfo(ScanExternalInfo), } #[cfg(feature = "python")] @@ -82,88 +80,3 @@ impl Hash for InMemoryInfo { } } } - -#[derive(Debug, Clone, PartialEq, Eq, Hash)] -pub enum ExternalInfo { - Scan(ScanExternalInfo), - Legacy(LegacyExternalInfo), -} - -impl ExternalInfo { - pub fn pushdowns(&self) -> &Pushdowns { - match self { - Self::Scan(ScanExternalInfo { pushdowns, .. }) - | Self::Legacy(LegacyExternalInfo { pushdowns, .. }) => pushdowns, - } - } - - pub fn with_pushdowns(&self, pushdowns: Pushdowns) -> Self { - match self { - Self::Scan(external_info) => Self::Scan(ScanExternalInfo { - pushdowns, - ..external_info.clone() - }), - Self::Legacy(external_info) => Self::Legacy(LegacyExternalInfo { - pushdowns, - ..external_info.clone() - }), - } - } -} - -#[derive(Debug, Clone, PartialEq, Eq, Hash, Serialize, Deserialize)] -pub struct LegacyExternalInfo { - pub source_schema: SchemaRef, - pub file_infos: Arc, - pub file_format_config: Arc, - pub storage_config: Arc, - pub pushdowns: Pushdowns, -} - -impl LegacyExternalInfo { - pub fn new( - source_schema: SchemaRef, - file_infos: Arc, - file_format_config: Arc, - storage_config: Arc, - pushdowns: Pushdowns, - ) -> Self { - Self { - source_schema, - file_infos, - file_format_config, - storage_config, - pushdowns, - } - } - - pub fn multiline_display(&self) -> Vec { - let mut res = vec![]; - res.push(format!( - "File paths = [{}]", - self.file_infos.file_paths.join(", ") - )); - res.push(format!( - "File schema = {}", - self.source_schema.short_string() - )); - let file_format = self.file_format_config.multiline_display(); - if !file_format.is_empty() { - res.push(format!( - "{} config= {}", - self.file_format_config.var_name(), - file_format.join(", ") - )); - } - let storage_config = self.storage_config.multiline_display(); - if !storage_config.is_empty() { - res.push(format!( - "{} storage config = {{ {} }}", - self.storage_config.var_name(), - storage_config.join(", ") - )); - } - res.extend(self.pushdowns.multiline_display()); - res - } -} diff --git a/src/daft-plan/src/test/mod.rs b/src/daft-plan/src/test/mod.rs index 37dbff01ee..b3a852a0a9 100644 --- a/src/daft-plan/src/test/mod.rs +++ b/src/daft-plan/src/test/mod.rs @@ -1,51 +1,33 @@ use std::sync::Arc; use daft_core::{datatypes::Field, schema::Schema}; -use daft_scan::{file_format::FileFormatConfig, storage_config::StorageConfig, Pushdowns}; +use daft_scan::{ + file_format::FileFormatConfig, storage_config::StorageConfig, AnonymousScanOperator, Pushdowns, + ScanOperator, +}; -use crate::{builder::LogicalPlanBuilder, source_info::FileInfos, NativeStorageConfig}; - -/// Create a dummy scan node containing the provided fields in its schema. -pub fn dummy_scan_node(fields: Vec) -> LogicalPlanBuilder { - dummy_scan_node_with_pushdowns(fields, Default::default()) -} +use crate::{builder::LogicalPlanBuilder, NativeStorageConfig}; /// Create a dummy scan node containing the provided fields in its schema and the provided limit. -pub fn dummy_scan_node_with_pushdowns( - fields: Vec, - pushdowns: Pushdowns, -) -> LogicalPlanBuilder { +pub fn dummy_scan_operator(fields: Vec) -> Arc { let schema = Arc::new(Schema::new(fields).unwrap()); - - LogicalPlanBuilder::table_scan_with_pushdowns( - FileInfos::new_internal(vec!["/foo".to_string()], vec![None], vec![None]), + Arc::new(AnonymousScanOperator::new( + vec!["/foo".to_string()], schema, FileFormatConfig::Json(Default::default()).into(), StorageConfig::Native(NativeStorageConfig::new_internal(true, None).into()).into(), - pushdowns, - ) - .unwrap() + )) } -pub fn dummy_scan_operator_node(fields: Vec) -> LogicalPlanBuilder { - dummy_scan_operator_node_with_pushdowns(fields, Default::default()) +/// Create a dummy scan node containing the provided fields in its schema. +pub fn dummy_scan_node(scan_op: Arc) -> LogicalPlanBuilder { + dummy_scan_node_with_pushdowns(scan_op, Default::default()) } /// Create a dummy scan node containing the provided fields in its schema and the provided limit. -pub fn dummy_scan_operator_node_with_pushdowns( - fields: Vec, +pub fn dummy_scan_node_with_pushdowns( + scan_op: Arc, pushdowns: Pushdowns, ) -> LogicalPlanBuilder { - let schema = Arc::new(Schema::new(fields).unwrap()); - let anon = daft_scan::AnonymousScanOperator::new( - vec!["/foo".to_string()], - schema, - FileFormatConfig::Json(Default::default()).into(), - StorageConfig::Native(NativeStorageConfig::new_internal(true, None).into()).into(), - ); - LogicalPlanBuilder::table_scan_with_scan_operator( - daft_scan::ScanOperatorRef(Arc::new(anon)), - Some(pushdowns), - ) - .unwrap() + LogicalPlanBuilder::table_scan(daft_scan::ScanOperatorRef(scan_op), Some(pushdowns)).unwrap() } diff --git a/src/daft-scan/src/anonymous.rs b/src/daft-scan/src/anonymous.rs index a3db188b22..c19f6ceb25 100644 --- a/src/daft-scan/src/anonymous.rs +++ b/src/daft-scan/src/anonymous.rs @@ -50,6 +50,19 @@ impl ScanOperator for AnonymousScanOperator { false } + // fn is_eq(&self, other: &dyn ScanOperator) -> Option { + // Some( + // (other as &dyn Any) + // .downcast_ref::() + // .map(|o| self == o) + // .unwrap_or(false), + // ) + // } + + // fn hash_op(&self, state: &mut dyn Hasher) -> Option<()> { + // Some(self.hash(state)) + // } + fn multiline_display(&self) -> Vec { let mut lines = vec![ "AnonymousScanOperator".to_string(), diff --git a/src/daft-scan/src/glob.rs b/src/daft-scan/src/glob.rs index 2ce839eb33..1c6ac32702 100644 --- a/src/daft-scan/src/glob.rs +++ b/src/daft-scan/src/glob.rs @@ -13,7 +13,7 @@ use crate::{ storage_config::StorageConfig, DataFileSource, PartitionField, Pushdowns, ScanOperator, ScanTask, ScanTaskRef, }; -#[derive(Debug, PartialEq, Hash)] +#[derive(Debug)] pub struct GlobScanOperator { glob_paths: Vec, file_format_config: Arc, @@ -236,6 +236,15 @@ impl ScanOperator for GlobScanOperator { false } + // fn is_eq(&self, other: &dyn ScanOperator) -> Option { + // Some( + // (other as &dyn Any) + // .downcast_ref::() + // .map(|o| self == o) + // .unwrap_or(false), + // ) + // } + fn multiline_display(&self) -> Vec { let mut lines = vec![ "GlobScanOperator".to_string(), diff --git a/src/daft-scan/src/lib.rs b/src/daft-scan/src/lib.rs index fab6b271f7..4ca49b965e 100644 --- a/src/daft-scan/src/lib.rs +++ b/src/daft-scan/src/lib.rs @@ -1,6 +1,8 @@ #![feature(if_let_guard)] #![feature(let_chains)] +#![feature(trait_upcasting)] use std::{ + any::Any, fmt::{Debug, Display}, hash::{Hash, Hasher}, sync::Arc, @@ -536,7 +538,7 @@ impl Display for PartitionTransform { } } -pub trait ScanOperator: Send + Sync + Debug { +pub trait ScanOperator: Send + Sync + Debug + Any { fn schema(&self) -> SchemaRef; fn partitioning_keys(&self) -> &[PartitionField]; @@ -544,6 +546,8 @@ pub trait ScanOperator: Send + Sync + Debug { fn can_absorb_select(&self) -> bool; fn can_absorb_limit(&self) -> bool; fn multiline_display(&self) -> Vec; + // fn is_eq(&self, _: &dyn ScanOperator) -> Option; + // fn hash_op(&self, state: &mut dyn Hasher) -> Option<()>; fn to_scan_tasks( &self, pushdowns: Pushdowns, @@ -577,6 +581,9 @@ impl Hash for ScanOperatorRef { impl PartialEq for ScanOperatorRef { fn eq(&self, other: &ScanOperatorRef) -> bool { Arc::ptr_eq(&self.0, &other.0) + // self.0 + // .is_eq(other.0.as_ref()) + // .unwrap_or_else(|| Arc::ptr_eq(&self.0, &other.0)) } } @@ -610,6 +617,15 @@ impl ScanExternalInfo { pushdowns, } } + + pub fn with_pushdowns(&self, pushdowns: Pushdowns) -> Self { + Self { + scan_op: self.scan_op.clone(), + source_schema: self.source_schema.clone(), + partitioning_keys: self.partitioning_keys.clone(), + pushdowns, + } + } } #[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq, Hash)] diff --git a/src/daft-scan/src/python.rs b/src/daft-scan/src/python.rs index 93cdcbc374..827521590f 100644 --- a/src/daft-scan/src/python.rs +++ b/src/daft-scan/src/python.rs @@ -190,6 +190,10 @@ pub mod pylib { self.can_absorb_select } + // fn is_eq(&self, _: &dyn ScanOperator) -> Option { + // None + // } + fn multiline_display(&self) -> Vec { let lines = vec![format!("PythonScanOperator: {}", self.display_name)]; lines diff --git a/tests/dataframe/test_creation.py b/tests/dataframe/test_creation.py index 7d92478ecd..52ab451824 100644 --- a/tests/dataframe/test_creation.py +++ b/tests/dataframe/test_creation.py @@ -424,11 +424,7 @@ def test_create_dataframe_csv_generate_headers(valid_data: list[dict[str, float] writer.writerows([[item[col] for col in header] for item in valid_data]) f.flush() - cnames = ( - [f"column_{i}" for i in range(1, 6)] - if use_native_downloader or os.environ.get("DAFT_MICROPARTITIONS", "1") == "1" - else [f"f{i}" for i in range(5)] - ) + cnames = [f"column_{i}" for i in range(1, 6)] df = daft.read_csv(fname, has_headers=False, use_native_downloader=use_native_downloader) assert df.column_names == cnames @@ -524,15 +520,6 @@ def test_create_dataframe_csv_specify_schema_no_headers( "column_5": DataType.string(), } - if use_native_downloader == False and os.environ.get("DAFT_MICROPARTITIONS") == "0": - schema_hints_for_csv_without_headers = { - "f0": DataType.float64(), - "f1": DataType.float64(), - "f2": DataType.float64(), - "f3": DataType.float64(), - "f4": DataType.string(), - } - df = daft.read_csv( fname, delimiter="\t", diff --git a/tests/integration/io/parquet/test_reads_s3_minio.py b/tests/integration/io/parquet/test_reads_s3_minio.py index 44ae5e3633..f640513f65 100644 --- a/tests/integration/io/parquet/test_reads_s3_minio.py +++ b/tests/integration/io/parquet/test_reads_s3_minio.py @@ -1,7 +1,5 @@ from __future__ import annotations -import os - import pyarrow as pa import pytest from pyarrow import parquet as pq @@ -36,10 +34,5 @@ def test_minio_parquet_read_no_files(minio_io_config): with minio_create_bucket(minio_io_config, bucket_name=bucket_name) as fs: fs.touch("s3://data-engineering-prod/foo/file.txt") - msg = ( - "Glob path had no matches:" - if os.getenv("DAFT_MICROPARTITIONS", "1") == "1" - else "No files found at s3://data-engineering-prod/foo/\\*\\*.parquet" - ) - with pytest.raises(FileNotFoundError, match=msg): + with pytest.raises(FileNotFoundError, match="Glob path had no matches:"): daft.read_parquet("s3://data-engineering-prod/foo/**.parquet", io_config=minio_io_config) diff --git a/tests/io/test_merge_scan_tasks.py b/tests/io/test_merge_scan_tasks.py index 75c426874b..e69d1a1105 100644 --- a/tests/io/test_merge_scan_tasks.py +++ b/tests/io/test_merge_scan_tasks.py @@ -1,7 +1,6 @@ from __future__ import annotations import contextlib -import os import pytest @@ -33,7 +32,6 @@ def csv_files(tmpdir): return tmpdir -@pytest.mark.skipif(os.getenv("DAFT_MICROPARTITIONS", "1") == "0", reason="Test can only run on micropartitions") def test_merge_scan_task_exceed_max(csv_files): with override_merge_scan_tasks_configs(0, 0): df = daft.read_csv(str(csv_files)) @@ -42,7 +40,6 @@ def test_merge_scan_task_exceed_max(csv_files): ), "Should have 3 partitions since all merges are more than the maximum (>0 bytes)" -@pytest.mark.skipif(os.getenv("DAFT_MICROPARTITIONS", "1") == "0", reason="Test can only run on micropartitions") def test_merge_scan_task_below_max(csv_files): with override_merge_scan_tasks_configs(21, 22): df = daft.read_csv(str(csv_files)) @@ -51,7 +48,6 @@ def test_merge_scan_task_below_max(csv_files): ), "Should have 2 partitions [(CSV1, CSV2), (CSV3)] since the second merge is too large (>22 bytes)" -@pytest.mark.skipif(os.getenv("DAFT_MICROPARTITIONS", "1") == "0", reason="Test can only run on micropartitions") def test_merge_scan_task_above_min(csv_files): with override_merge_scan_tasks_configs(19, 40): df = daft.read_csv(str(csv_files)) @@ -60,7 +56,6 @@ def test_merge_scan_task_above_min(csv_files): ), "Should have 2 partitions [(CSV1, CSV2), (CSV3)] since the first merge is above the minimum (>19 bytes)" -@pytest.mark.skipif(os.getenv("DAFT_MICROPARTITIONS", "1") == "0", reason="Test can only run on micropartitions") def test_merge_scan_task_below_min(csv_files): with override_merge_scan_tasks_configs(35, 40): df = daft.read_csv(str(csv_files))