diff --git a/.github/working-dir/shuffle_testing.py b/.github/working-dir/shuffle_testing.py index fdf45cbba4..1610b819c9 100644 --- a/.github/working-dir/shuffle_testing.py +++ b/.github/working-dir/shuffle_testing.py @@ -14,7 +14,7 @@ import daft import daft.context from daft.io._generator import read_generator -from daft.table.table import Table +from daft.recordbatch.recordbatch import RecordBatch # Constants GB = 1 << 30 @@ -110,7 +110,7 @@ def generate( delay = random.uniform(0, timing_variation) time.sleep(delay) - yield Table.from_pydict(data) + yield RecordBatch.from_pydict(data) def generator( diff --git a/Cargo.lock b/Cargo.lock index fd5cba331a..042e32b2a2 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1945,11 +1945,11 @@ dependencies = [ "daft-parquet", "daft-physical-plan", "daft-py-runners", + "daft-recordbatch", "daft-scan", "daft-scheduler", "daft-sql", "daft-stats", - "daft-table", "daft-writers", "lazy_static", "libc", @@ -2020,10 +2020,10 @@ dependencies = [ "daft-dsl", "daft-logical-plan", "daft-micropartition", + "daft-recordbatch", "daft-scan", "daft-schema", "daft-sql", - "daft-table", "dashmap", "futures", "itertools 0.11.0", @@ -2107,7 +2107,7 @@ dependencies = [ "daft-decoding", "daft-dsl", "daft-io", - "daft-table", + "daft-recordbatch", "futures", "memchr", "parking_lot 0.12.3", @@ -2288,7 +2288,7 @@ dependencies = [ "daft-decoding", "daft-dsl", "daft-io", - "daft-table", + "daft-recordbatch", "futures", "indexmap 2.7.0", "memchr", @@ -2330,8 +2330,8 @@ dependencies = [ "daft-micropartition", "daft-parquet", "daft-physical-plan", + "daft-recordbatch", "daft-scan", - "daft-table", "daft-writers", "futures", "indexmap 2.7.0", @@ -2416,9 +2416,9 @@ dependencies = [ "daft-io", "daft-json", "daft-parquet", + "daft-recordbatch", "daft-scan", "daft-stats", - "daft-table", "dashmap", "futures", "parquet2", @@ -2457,8 +2457,8 @@ dependencies = [ "daft-core", "daft-dsl", "daft-io", + "daft-recordbatch", "daft-stats", - "daft-table", "futures", "indexmap 2.7.0", "itertools 0.11.0", @@ -2506,6 +2506,27 @@ dependencies = [ "pyo3", ] +[[package]] +name = "daft-recordbatch" +version = "0.3.0-dev0" +dependencies = [ + "arrow2", + "comfy-table 7.1.3", + "common-arrow-ffi", + "common-display", + "common-error", + "daft-core", + "daft-dsl", + "daft-image", + "daft-logical-plan", + "html-escape", + "indexmap 2.7.0", + "num-traits", + "pyo3", + "rand 0.8.5", + "serde", +] + [[package]] name = "daft-scan" version = "0.3.0-dev0" @@ -2527,9 +2548,9 @@ dependencies = [ "daft-json", "daft-logical-plan", "daft-parquet", + "daft-recordbatch", "daft-schema", "daft-stats", - "daft-table", "futures", "indexmap 2.7.0", "itertools 0.11.0", @@ -2627,33 +2648,12 @@ dependencies = [ "common-error", "daft-core", "daft-dsl", - "daft-table", + "daft-recordbatch", "indexmap 2.7.0", "serde", "snafu", ] -[[package]] -name = "daft-table" -version = "0.3.0-dev0" -dependencies = [ - "arrow2", - "comfy-table 7.1.3", - "common-arrow-ffi", - "common-display", - "common-error", - "daft-core", - "daft-dsl", - "daft-image", - "daft-logical-plan", - "html-escape", - "indexmap 2.7.0", - "num-traits", - "pyo3", - "rand 0.8.5", - "serde", -] - [[package]] name = "daft-writers" version = "0.3.0-dev0" @@ -2666,7 +2666,7 @@ dependencies = [ "daft-io", "daft-logical-plan", "daft-micropartition", - "daft-table", + "daft-recordbatch", "pyo3", ] diff --git a/Cargo.toml b/Cargo.toml index 228ac4f1b3..ac487a27fb 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -33,11 +33,11 @@ daft-minhash = {path = "src/daft-minhash", default-features = false} daft-parquet = {path = "src/daft-parquet", default-features = false} daft-physical-plan = {path = "src/daft-physical-plan", default-features = false} daft-py-runners = {path = "src/daft-py-runners", default-features = false} +daft-recordbatch = {path = "src/daft-recordbatch", default-features = false} daft-scan = {path = "src/daft-scan", default-features = false} daft-scheduler = {path = "src/daft-scheduler", default-features = false} daft-sql = {path = "src/daft-sql", default-features = false} daft-stats = {path = "src/daft-stats", default-features = false} -daft-table = {path = "src/daft-table", default-features = false} daft-writers = {path = "src/daft-writers", default-features = false} lazy_static = {workspace = true} log = {workspace = true} @@ -76,7 +76,7 @@ python = [ "daft-scheduler/python", "daft-sql/python", "daft-stats/python", - "daft-table/python", + "daft-recordbatch/python", "daft-writers/python", "daft-context/python", "dep:daft-catalog-python-catalog", @@ -168,7 +168,7 @@ members = [ "src/daft-scheduler", "src/daft-sketch", "src/daft-sql", - "src/daft-table", + "src/daft-recordbatch", "src/daft-writers", "src/hyperloglog", "src/daft-connect", @@ -207,10 +207,10 @@ daft-local-execution = {path = "src/daft-local-execution"} daft-logical-plan = {path = "src/daft-logical-plan"} daft-micropartition = {path = "src/daft-micropartition"} daft-py-runners = {path = "src/daft-py-runners"} +daft-recordbatch = {path = "src/daft-recordbatch"} daft-scan = {path = "src/daft-scan"} daft-schema = {path = "src/daft-schema"} daft-sql = {path = "src/daft-sql"} -daft-table = {path = "src/daft-table"} derivative = "2.2.0" derive_builder = "0.20.2" futures = "0.3.30" diff --git a/benchmarking/parquet/conftest.py b/benchmarking/parquet/conftest.py index 19774f359b..506750e6eb 100644 --- a/benchmarking/parquet/conftest.py +++ b/benchmarking/parquet/conftest.py @@ -41,12 +41,12 @@ def boto3_get_object_read(path: str, columns: list[str] | None = None) -> pa.Tab def daft_native_read(path: str, columns: list[str] | None = None) -> pa.Table: - tbl = daft.table.MicroPartition.read_parquet(path, columns=columns) + tbl = daft.recordbatch.MicroPartition.read_parquet(path, columns=columns) return tbl.to_arrow() def daft_native_read_to_arrow(path: str, columns: list[str] | None = None) -> pa.Table: - return daft.table.read_parquet_into_pyarrow(path, columns=columns) + return daft.recordbatch.read_parquet_into_pyarrow(path, columns=columns) def daft_dataframe_read(path: str, columns: list[str] | None = None) -> pa.Table: @@ -85,12 +85,12 @@ def fn(files: list[str]) -> list[pa.Table]: def daft_bulk_read(paths: list[str], columns: list[str] | None = None) -> list[pa.Table]: - tables = daft.table.Table.read_parquet_bulk(paths, columns=columns) + tables = daft.recordbatch.read_parquet_bulk(paths, columns=columns) return [t.to_arrow() for t in tables] def daft_into_pyarrow_bulk_read(paths: list[str], columns: list[str] | None = None) -> list[pa.Table]: - return daft.table.read_parquet_into_pyarrow_bulk(paths, columns=columns) + return daft.recordbatch.read_parquet_into_pyarrow_bulk(paths, columns=columns) def pyarrow_bulk_read(paths: list[str], columns: list[str] | None = None) -> list[pa.Table]: diff --git a/daft/daft/__init__.pyi b/daft/daft/__init__.pyi index d59f6f0fff..b6a12c614f 100644 --- a/daft/daft/__init__.pyi +++ b/daft/daft/__init__.pyi @@ -622,8 +622,8 @@ class ScanTask: size_bytes: int | None, iceberg_delete_files: list[str] | None, pushdowns: Pushdowns | None, - partition_values: PyTable | None, - stats: PyTable | None, + partition_values: PyRecordBatch | None, + stats: PyRecordBatch | None, ) -> ScanTask | None: """Create a Catalog Scan Task.""" ... @@ -637,7 +637,7 @@ class ScanTask: num_rows: int | None, size_bytes: int | None, pushdowns: Pushdowns | None, - stats: PyTable | None, + stats: PyRecordBatch | None, ) -> ScanTask: """Create a SQL Scan Task.""" ... @@ -651,7 +651,7 @@ class ScanTask: num_rows: int | None, size_bytes: int | None, pushdowns: Pushdowns | None, - stats: PyTable | None, + stats: PyRecordBatch | None, ) -> ScanTask: """Create a Python factory function Scan Task.""" ... @@ -1369,48 +1369,48 @@ class PySeries: @staticmethod def _debug_bincode_deserialize(b: bytes) -> PySeries: ... -class PyTable: +class PyRecordBatch: def schema(self) -> PySchema: ... - def cast_to_schema(self, schema: PySchema) -> PyTable: ... - def eval_expression_list(self, exprs: list[PyExpr]) -> PyTable: ... - def take(self, idx: PySeries) -> PyTable: ... - def filter(self, exprs: list[PyExpr]) -> PyTable: ... - def sort(self, sort_keys: list[PyExpr], descending: list[bool], nulls_first: list[bool]) -> PyTable: ... + def cast_to_schema(self, schema: PySchema) -> PyRecordBatch: ... + def eval_expression_list(self, exprs: list[PyExpr]) -> PyRecordBatch: ... + def take(self, idx: PySeries) -> PyRecordBatch: ... + def filter(self, exprs: list[PyExpr]) -> PyRecordBatch: ... + def sort(self, sort_keys: list[PyExpr], descending: list[bool], nulls_first: list[bool]) -> PyRecordBatch: ... def argsort(self, sort_keys: list[PyExpr], descending: list[bool], nulls_first: list[bool]) -> PySeries: ... - def agg(self, to_agg: list[PyExpr], group_by: list[PyExpr]) -> PyTable: ... + def agg(self, to_agg: list[PyExpr], group_by: list[PyExpr]) -> PyRecordBatch: ... def pivot( self, group_by: list[PyExpr], pivot_column: PyExpr, values_column: PyExpr, names: list[str], - ) -> PyTable: ... + ) -> PyRecordBatch: ... def hash_join( self, - right: PyTable, + right: PyRecordBatch, left_on: list[PyExpr], right_on: list[PyExpr], how: JoinType, - ) -> PyTable: ... + ) -> PyRecordBatch: ... def sort_merge_join( self, - right: PyTable, + right: PyRecordBatch, left_on: list[PyExpr], right_on: list[PyExpr], is_sorted: bool, - ) -> PyTable: ... - def explode(self, to_explode: list[PyExpr]) -> PyTable: ... - def head(self, num: int) -> PyTable: ... - def sample_by_fraction(self, fraction: float, with_replacement: bool, seed: int | None) -> PyTable: ... - def sample_by_size(self, size: int, with_replacement: bool, seed: int | None) -> PyTable: ... - def quantiles(self, num: int) -> PyTable: ... - def partition_by_hash(self, exprs: list[PyExpr], num_partitions: int) -> list[PyTable]: ... - def partition_by_random(self, num_partitions: int, seed: int) -> list[PyTable]: ... + ) -> PyRecordBatch: ... + def explode(self, to_explode: list[PyExpr]) -> PyRecordBatch: ... + def head(self, num: int) -> PyRecordBatch: ... + def sample_by_fraction(self, fraction: float, with_replacement: bool, seed: int | None) -> PyRecordBatch: ... + def sample_by_size(self, size: int, with_replacement: bool, seed: int | None) -> PyRecordBatch: ... + def quantiles(self, num: int) -> PyRecordBatch: ... + def partition_by_hash(self, exprs: list[PyExpr], num_partitions: int) -> list[PyRecordBatch]: ... + def partition_by_random(self, num_partitions: int, seed: int) -> list[PyRecordBatch]: ... def partition_by_range( - self, partition_keys: list[PyExpr], boundaries: PyTable, descending: list[bool] - ) -> list[PyTable]: ... - def partition_by_value(self, partition_keys: list[PyExpr]) -> tuple[list[PyTable], PyTable]: ... - def add_monotonically_increasing_id(self, partition_num: int, column_name: str) -> PyTable: ... + self, partition_keys: list[PyExpr], boundaries: PyRecordBatch, descending: list[bool] + ) -> list[PyRecordBatch]: ... + def partition_by_value(self, partition_keys: list[PyExpr]) -> tuple[list[PyRecordBatch], PyRecordBatch]: ... + def add_monotonically_increasing_id(self, partition_num: int, column_name: str) -> PyRecordBatch: ... def __repr__(self) -> str: ... def _repr_html_(self) -> str: ... def __len__(self) -> int: ... @@ -1419,17 +1419,17 @@ class PyTable: def get_column(self, name: str) -> PySeries: ... def get_column_by_index(self, idx: int) -> PySeries: ... @staticmethod - def concat(tables: list[PyTable]) -> PyTable: ... - def slice(self, start: int, end: int) -> PyTable: ... + def concat(tables: list[PyRecordBatch]) -> PyRecordBatch: ... + def slice(self, start: int, end: int) -> PyRecordBatch: ... @staticmethod - def from_arrow_record_batches(record_batches: list[pa.RecordBatch], schema: PySchema) -> PyTable: ... + def from_arrow_record_batches(record_batches: list[pa.RecordBatch], schema: PySchema) -> PyRecordBatch: ... @staticmethod - def from_pylist_series(dict: dict[str, PySeries]) -> PyTable: ... + def from_pylist_series(dict: dict[str, PySeries]) -> PyRecordBatch: ... def to_arrow_record_batch(self) -> pa.RecordBatch: ... @staticmethod - def empty(schema: PySchema | None = None) -> PyTable: ... + def empty(schema: PySchema | None = None) -> PyRecordBatch: ... @staticmethod - def from_file_infos(file_infos: FileInfos) -> PyTable: ... + def from_file_infos(file_infos: FileInfos) -> PyRecordBatch: ... def to_file_infos(self) -> FileInfos: ... class PyMicroPartition: @@ -1443,13 +1443,13 @@ class PyMicroPartition: @staticmethod def from_scan_task(scan_task: ScanTask) -> PyMicroPartition: ... @staticmethod - def from_tables(tables: list[PyTable]) -> PyMicroPartition: ... + def from_tables(tables: list[PyRecordBatch]) -> PyMicroPartition: ... @staticmethod def from_arrow_record_batches(record_batches: list[pa.RecordBatch], schema: PySchema) -> PyMicroPartition: ... @staticmethod def concat(tables: list[PyMicroPartition]) -> PyMicroPartition: ... def slice(self, start: int, end: int) -> PyMicroPartition: ... - def to_table(self) -> PyTable: ... + def to_table(self) -> PyRecordBatch: ... def cast_to_schema(self, schema: PySchema) -> PyMicroPartition: ... def eval_expression_list(self, exprs: list[PyExpr]) -> PyMicroPartition: ... def take(self, idx: PySeries) -> PyMicroPartition: ... @@ -1499,7 +1499,7 @@ class PyMicroPartition: def partition_by_hash(self, exprs: list[PyExpr], num_partitions: int) -> list[PyMicroPartition]: ... def partition_by_random(self, num_partitions: int, seed: int) -> list[PyMicroPartition]: ... def partition_by_range( - self, partition_keys: list[PyExpr], boundaries: PyTable, descending: list[bool] + self, partition_keys: list[PyExpr], boundaries: PyRecordBatch, descending: list[bool] ) -> list[PyMicroPartition]: ... def partition_by_value(self, exprs: list[PyExpr]) -> tuple[list[PyMicroPartition], PyMicroPartition]: ... def add_monotonically_increasing_id(self, partition_num: int, column_name: str) -> PyMicroPartition: ... diff --git a/daft/dataframe/dataframe.py b/daft/dataframe/dataframe.py index 686a7fe971..5c12629b6b 100644 --- a/daft/dataframe/dataframe.py +++ b/daft/dataframe/dataframe.py @@ -40,8 +40,8 @@ from daft.expressions import Expression, ExpressionsProjection, col, lit from daft.filesystem import overwrite_files from daft.logical.builder import LogicalPlanBuilder +from daft.recordbatch import MicroPartition from daft.runners.partitioning import LocalPartitionSet, PartitionCacheEntry, PartitionSet -from daft.table import MicroPartition from daft.viz import DataFrameDisplay if TYPE_CHECKING: @@ -66,7 +66,7 @@ def to_logical_plan_builder(*parts: MicroPartition) -> LogicalPlanBuilder: - """Creates a Daft DataFrame from a single Table. + """Creates a Daft DataFrame from a single RecordBatch. Args: parts: The Tables that we wish to convert into a Daft DataFrame. @@ -388,7 +388,7 @@ def iter_partitions( ) -> Iterator[Union[MicroPartition, "ray.ObjectRef[MicroPartition]"]]: """Begin executing this dataframe and return an iterator over the partitions. - Each partition will be returned as a daft.Table object (if using Python runner backend) + Each partition will be returned as a daft.recordbatch object (if using Python runner backend) or a ray ObjectRef (if using Ray runner backend). .. NOTE:: @@ -539,7 +539,7 @@ def _from_pandas(cls, data: Union["pandas.DataFrame", List["pandas.DataFrame"]]) @classmethod def _from_tables(cls, *parts: MicroPartition) -> "DataFrame": - """Creates a Daft DataFrame from a single Table. + """Creates a Daft DataFrame from a single RecordBatch. Args: parts: The Tables that we wish to convert into a Daft DataFrame. @@ -659,7 +659,7 @@ def write_parquet( return result_df else: from daft import from_pydict - from daft.table.table_io import write_empty_tabular + from daft.recordbatch.recordbatch_io import write_empty_tabular file_path = write_empty_tabular( root_dir, FileFormat.Parquet, self.schema(), compression=compression, io_config=io_config @@ -732,7 +732,7 @@ def write_csv( return result_df else: from daft import from_pydict - from daft.table.table_io import write_empty_tabular + from daft.recordbatch.recordbatch_io import write_empty_tabular file_path = write_empty_tabular(root_dir, FileFormat.Csv, self.schema(), io_config=io_config) diff --git a/daft/dataframe/preview.py b/daft/dataframe/preview.py index c6e04e0045..9b38ce090e 100644 --- a/daft/dataframe/preview.py +++ b/daft/dataframe/preview.py @@ -4,7 +4,7 @@ from typing import TYPE_CHECKING if TYPE_CHECKING: - from daft.table import MicroPartition + from daft.recordbatch import MicroPartition @dataclass(frozen=True) diff --git a/daft/delta_lake/delta_lake_scan.py b/daft/delta_lake/delta_lake_scan.py index b81b0128e9..791b3e8f98 100644 --- a/daft/delta_lake/delta_lake_scan.py +++ b/daft/delta_lake/delta_lake_scan.py @@ -191,7 +191,7 @@ def to_scan_tasks(self, pushdowns: Pushdowns) -> Iterator[ScanTask]: # pyarrow < 13.0.0 doesn't accept pyarrow scalars in the array constructor. arrow_arr = pa.array([part_values[field_name].as_py()], type=dtype.field(field_idx).type) arrays[field_name] = daft.Series.from_arrow(arrow_arr, field_name) - partition_values = daft.table.Table.from_pydict(arrays)._table + partition_values = daft.recordbatch.RecordBatch.from_pydict(arrays)._table else: partition_values = None @@ -217,7 +217,7 @@ def to_scan_tasks(self, pushdowns: Pushdowns) -> Iterator[ScanTask]: type=dtype.field(field_idx).type, ) arrays[field_name] = daft.Series.from_arrow(arrow_arr, field_name) - stats = daft.table.Table.from_pydict(arrays) + stats = daft.recordbatch.RecordBatch.from_pydict(arrays) else: stats = None st = ScanTask.catalog_scan_task( diff --git a/daft/delta_lake/delta_lake_write.py b/daft/delta_lake/delta_lake_write.py index e762c059cc..26cbf98a14 100644 --- a/daft/delta_lake/delta_lake_write.py +++ b/daft/delta_lake/delta_lake_write.py @@ -5,8 +5,8 @@ from daft.datatype import DataType from daft.dependencies import pa from daft.io.common import _get_schema_from_dict -from daft.table.micropartition import MicroPartition -from daft.table.partitioning import PartitionedTable, partition_strings_to_path +from daft.recordbatch.micropartition import MicroPartition +from daft.recordbatch.partitioning import PartitionedTable, partition_strings_to_path if TYPE_CHECKING: from deltalake.writer import AddAction diff --git a/daft/execution/actor_pool_udf.py b/daft/execution/actor_pool_udf.py index 9ecadc3f91..110a3543d8 100644 --- a/daft/execution/actor_pool_udf.py +++ b/daft/execution/actor_pool_udf.py @@ -5,7 +5,7 @@ from typing import TYPE_CHECKING from daft.expressions import Expression, ExpressionsProjection -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition if TYPE_CHECKING: from multiprocessing.connection import Connection diff --git a/daft/execution/execution_step.py b/daft/execution/execution_step.py index 9670ff42b7..4a60489957 100644 --- a/daft/execution/execution_step.py +++ b/daft/execution/execution_step.py @@ -7,6 +7,7 @@ from daft.context import get_context from daft.daft import JoinSide, ResourceRequest from daft.expressions import Expression, ExpressionsProjection, col +from daft.recordbatch import MicroPartition, recordbatch_io from daft.runners.partitioning import ( Boundaries, MaterializedResult, @@ -14,7 +15,6 @@ PartitionMetadata, PartitionT, ) -from daft.table import MicroPartition, table_io if TYPE_CHECKING: import pathlib @@ -423,7 +423,7 @@ def run_partial_metadata(self, input_metadatas: list[PartialPartitionMetadata]) ] def _handle_file_write(self, input: MicroPartition) -> MicroPartition: - return table_io.write_tabular( + return recordbatch_io.write_tabular( input, path=self.root_dir, schema=self.schema, @@ -463,7 +463,7 @@ def run_partial_metadata(self, input_metadatas: list[PartialPartitionMetadata]) ] def _handle_file_write(self, input: MicroPartition) -> MicroPartition: - return table_io.write_iceberg( + return recordbatch_io.write_iceberg( input, base_path=self.base_path, schema=self.iceberg_schema, @@ -502,7 +502,7 @@ def run_partial_metadata(self, input_metadatas: list[PartialPartitionMetadata]) ] def _handle_file_write(self, input: MicroPartition) -> MicroPartition: - return table_io.write_deltalake( + return recordbatch_io.write_deltalake( input, large_dtypes=self.large_dtypes, base_path=self.base_path, @@ -539,7 +539,7 @@ def run_partial_metadata(self, input_metadatas: list[PartialPartitionMetadata]) ] def _handle_file_write(self, input: MicroPartition) -> MicroPartition: - return table_io.write_lance( + return recordbatch_io.write_lance( input, base_path=self.base_path, mode=self.mode, diff --git a/daft/execution/native_executor.py b/daft/execution/native_executor.py index 6ff88626fe..cb83df9b9e 100644 --- a/daft/execution/native_executor.py +++ b/daft/execution/native_executor.py @@ -6,7 +6,7 @@ NativeExecutor as _NativeExecutor, ) from daft.dataframe.display import MermaidOptions -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition if TYPE_CHECKING: from daft.daft import PyDaftExecutionConfig diff --git a/daft/execution/physical_plan.py b/daft/execution/physical_plan.py index 7d1ae3cbe2..ecac8e1e2a 100644 --- a/daft/execution/physical_plan.py +++ b/daft/execution/physical_plan.py @@ -43,11 +43,11 @@ calculate_cross_join_stats, ) from daft.expressions import ExpressionsProjection +from daft.recordbatch.micropartition import MicroPartition from daft.runners.partitioning import ( MaterializedResult, PartitionT, ) -from daft.table.micropartition import MicroPartition logger = logging.getLogger(__name__) diff --git a/daft/execution/rust_physical_plan_shim.py b/daft/execution/rust_physical_plan_shim.py index cd316ae431..7784137bf4 100644 --- a/daft/execution/rust_physical_plan_shim.py +++ b/daft/execution/rust_physical_plan_shim.py @@ -22,7 +22,7 @@ from pyiceberg.schema import Schema as IcebergSchema from pyiceberg.table import TableProperties as IcebergTableProperties - from daft.table import MicroPartition + from daft.recordbatch import MicroPartition def scan_with_tasks( diff --git a/daft/filesystem.py b/daft/filesystem.py index 309ffb09ac..2c9c5d6731 100644 --- a/daft/filesystem.py +++ b/daft/filesystem.py @@ -13,7 +13,7 @@ from daft.daft import FileFormat, FileInfos, IOConfig, io_glob from daft.dependencies import fsspec, pafs from daft.expressions.expressions import col, lit -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition if TYPE_CHECKING: from daft import DataFrame diff --git a/daft/hudi/hudi_scan.py b/daft/hudi/hudi_scan.py index 2f342e4a6a..b9411dce39 100644 --- a/daft/hudi/hudi_scan.py +++ b/daft/hudi/hudi_scan.py @@ -91,7 +91,7 @@ def to_scan_tasks(self, pushdowns: Pushdowns) -> Iterator[ScanTask]: partition_paths = { HUDI_METAFIELD_PARTITION_PATH: daft.Series.from_arrow(arrow_arr, HUDI_METAFIELD_PARTITION_PATH), } - partition_values = daft.table.Table.from_pydict(partition_paths)._table + partition_values = daft.recordbatch.RecordBatch.from_pydict(partition_paths)._table else: partition_values = None @@ -115,7 +115,7 @@ def to_scan_tasks(self, pushdowns: Pushdowns) -> Iterator[ScanTask]: type=field_type, ) arrays[field_name] = daft.Series.from_arrow(arrow_arr, field_name) - stats = daft.table.Table.from_pydict(arrays)._table + stats = daft.recordbatch.RecordBatch.from_pydict(arrays)._table st = ScanTask.catalog_scan_task( file=path, diff --git a/daft/iceberg/iceberg_scan.py b/daft/iceberg/iceberg_scan.py index 261e162b80..217d1554c9 100644 --- a/daft/iceberg/iceberg_scan.py +++ b/daft/iceberg/iceberg_scan.py @@ -124,7 +124,9 @@ def display_name(self) -> str: def partitioning_keys(self) -> list[PartitionField]: return self._partition_keys - def _iceberg_record_to_partition_spec(self, spec: IcebergPartitionSpec, record: Record) -> daft.table.Table | None: + def _iceberg_record_to_partition_spec( + self, spec: IcebergPartitionSpec, record: Record + ) -> daft.recordbatch.RecordBatch | None: partition_fields = iceberg_partition_spec_to_fields(self._table.schema(), spec) arrays = dict() assert len(record._position_to_field_name) == len(partition_fields) @@ -137,7 +139,7 @@ def _iceberg_record_to_partition_spec(self, spec: IcebergPartitionSpec, record: field_dtype ) if len(arrays) > 0: - return daft.table.Table.from_pydict(arrays) + return daft.recordbatch.RecordBatch.from_pydict(arrays) else: return None diff --git a/daft/iceberg/iceberg_write.py b/daft/iceberg/iceberg_write.py index 4888b83351..2427bf4b53 100644 --- a/daft/iceberg/iceberg_write.py +++ b/daft/iceberg/iceberg_write.py @@ -7,8 +7,8 @@ from daft.datatype import DataType from daft.expressions.expressions import ExpressionsProjection from daft.io.common import _get_schema_from_dict -from daft.table import MicroPartition -from daft.table.partitioning import PartitionedTable, partition_strings_to_path +from daft.recordbatch import MicroPartition +from daft.recordbatch.partitioning import PartitionedTable, partition_strings_to_path if TYPE_CHECKING: import pyarrow as pa diff --git a/daft/io/_deltalake.py b/daft/io/_deltalake.py index 8cbf6a6cf5..b233fac570 100644 --- a/daft/io/_deltalake.py +++ b/daft/io/_deltalake.py @@ -70,9 +70,9 @@ def read_deltalake( table_uri = table.table_uri # Override the storage_config with the one provided by Unity catalog - table_io_config = table.io_config - if table_io_config is not None: - storage_config = StorageConfig(multithreaded_io, table_io_config) + recordbatch_io_config = table.io_config + if recordbatch_io_config is not None: + storage_config = StorageConfig(multithreaded_io, recordbatch_io_config) else: raise ValueError( f"table argument must be a table URI string, DataCatalogTable or UnityCatalogTable instance, but got: {type(table)}, {table}" diff --git a/daft/io/_generator.py b/daft/io/_generator.py index e5c185e464..f27e931487 100644 --- a/daft/io/_generator.py +++ b/daft/io/_generator.py @@ -2,23 +2,23 @@ from typing import TYPE_CHECKING, Callable, Iterator, List -from daft.daft import Pushdowns, PyTable, ScanOperatorHandle, ScanTask +from daft.daft import Pushdowns, PyRecordBatch, ScanOperatorHandle, ScanTask from daft.dataframe import DataFrame from daft.io.scan import PartitionField, ScanOperator from daft.logical.builder import LogicalPlanBuilder from daft.logical.schema import Schema if TYPE_CHECKING: - from daft.table.table import Table + from daft.recordbatch.recordbatch import RecordBatch -def _generator_factory_function(func: Callable[[], Iterator["Table"]]) -> Iterator["PyTable"]: +def _generator_factory_function(func: Callable[[], Iterator["RecordBatch"]]) -> Iterator["PyRecordBatch"]: for table in func(): yield table._table def read_generator( - generators: Iterator[Callable[[], Iterator["Table"]]], + generators: Iterator[Callable[[], Iterator["RecordBatch"]]], schema: Schema, ) -> DataFrame: """Create a DataFrame from a generator function. @@ -26,7 +26,7 @@ def read_generator( Example: >>> import daft >>> from daft.io._generator import read_generator - >>> from daft.table.table import Table + >>> from daft.recordbatch.recordbatch import RecordBatch >>> from functools import partial >>> >>> # Set runner to Ray for distributed processing @@ -35,7 +35,7 @@ def read_generator( >>> # Helper function to generate data for each partition >>> def generate(num_rows: int): ... data = {"ints": [i for i in range(num_rows)]} - ... yield Table.from_pydict(data) + ... yield RecordBatch.from_pydict(data) >>> >>> # Generator function that yields partial functions for each partition >>> def generator(num_partitions: int): @@ -53,7 +53,7 @@ def read_generator( ... ) Args: - generator (Callable[[int, Any], Iterator[Table]]): a generator function that generates data + generator (Callable[[int, Any], Iterator[RecordBatch]]): a generator function that generates data num_partitions (int): the number of partitions to generate schema (Schema): the schema of the generated data generator_args (Any): additional arguments to pass to the generator @@ -73,7 +73,7 @@ def read_generator( class GeneratorScanOperator(ScanOperator): def __init__( self, - generators: Iterator[Callable[[], Iterator["Table"]]], + generators: Iterator[Callable[[], Iterator["RecordBatch"]]], schema: Schema, ): self._generators = generators diff --git a/daft/io/_lance.py b/daft/io/_lance.py index bce5bb527e..f60baa06ce 100644 --- a/daft/io/_lance.py +++ b/daft/io/_lance.py @@ -4,13 +4,13 @@ from daft import context from daft.api_annotations import PublicAPI -from daft.daft import IOConfig, Pushdowns, PyTable, ScanOperatorHandle, ScanTask +from daft.daft import IOConfig, Pushdowns, PyRecordBatch, ScanOperatorHandle, ScanTask from daft.dataframe import DataFrame from daft.io.object_store_options import io_config_to_storage_options from daft.io.scan import PartitionField, ScanOperator from daft.logical.builder import LogicalPlanBuilder from daft.logical.schema import Schema -from daft.table import Table +from daft.recordbatch import RecordBatch if TYPE_CHECKING: import lance @@ -18,9 +18,10 @@ def _lancedb_table_factory_function( fragment: "lance.LanceFragment", required_columns: Optional[List[str]] -) -> Iterator["PyTable"]: +) -> Iterator["PyRecordBatch"]: return ( - Table.from_arrow_record_batches([rb], rb.schema)._table for rb in fragment.to_batches(columns=required_columns) + RecordBatch.from_arrow_record_batches([rb], rb.schema)._table + for rb in fragment.to_batches(columns=required_columns) ) diff --git a/daft/io/_range.py b/daft/io/_range.py index 8a28bed8ab..5ac09b5d28 100644 --- a/daft/io/_range.py +++ b/daft/io/_range.py @@ -8,10 +8,12 @@ from daft import DataType from daft.io._generator import GeneratorScanOperator from daft.logical.schema import Schema -from daft.table.table import Table +from daft.recordbatch.recordbatch import RecordBatch -def _range_generators(start: int, end: int, step: int, partitions: int) -> Iterator[Callable[[], Iterator[Table]]]: +def _range_generators( + start: int, end: int, step: int, partitions: int +) -> Iterator[Callable[[], Iterator[RecordBatch]]]: # TODO: Partitioning with range scan is currently untested and unused. # There may be issues with balanced partitions and step size. @@ -22,10 +24,10 @@ def _range_generators(start: int, end: int, step: int, partitions: int) -> Itera for i in range(partitions) ] - def generator(partition_idx: int) -> Iterator[Table]: + def generator(partition_idx: int) -> Iterator[RecordBatch]: partition_start, partition_end = partition_bounds[partition_idx] values = list(range(partition_start, partition_end, step)) - yield Table.from_pydict({"id": values}) + yield RecordBatch.from_pydict({"id": values}) from functools import partial diff --git a/daft/io/file_path.py b/daft/io/file_path.py index 70cd1d08bf..b07bd94f48 100644 --- a/daft/io/file_path.py +++ b/daft/io/file_path.py @@ -6,11 +6,11 @@ from daft.api_annotations import PublicAPI from daft.context import get_context from daft.daft import IOConfig -from daft.daft import PyTable as _PyTable +from daft.daft import PyRecordBatch as _PyRecordBatch from daft.dataframe import DataFrame from daft.logical.builder import LogicalPlanBuilder +from daft.recordbatch import MicroPartition from daft.runners.partitioning import LocalPartitionSet -from daft.table import MicroPartition @PublicAPI @@ -47,7 +47,7 @@ def from_glob_path(path: str, io_config: Optional[IOConfig] = None) -> DataFrame io_config = context.daft_planning_config.default_io_config if io_config is None else io_config runner_io = context.get_or_create_runner().runner_io() file_infos = runner_io.glob_paths_details([path], io_config=io_config) - file_infos_table = MicroPartition._from_pytable(_PyTable.from_file_infos(file_infos)) + file_infos_table = MicroPartition._from_pytable(_PyRecordBatch.from_file_infos(file_infos)) partition = LocalPartitionSet() partition.set_partition_from_table(0, file_infos_table) cache_entry = context.get_or_create_runner().put_partition_set_into_cache(partition) diff --git a/daft/io/writer.py b/daft/io/writer.py index 59258b21b8..311e2d58b6 100644 --- a/daft/io/writer.py +++ b/daft/io/writer.py @@ -15,13 +15,13 @@ make_iceberg_data_file, make_iceberg_record, ) -from daft.series import Series -from daft.table.micropartition import MicroPartition -from daft.table.partitioning import ( +from daft.recordbatch.micropartition import MicroPartition +from daft.recordbatch.partitioning import ( partition_strings_to_path, partition_values_to_str_mapping, ) -from daft.table.table import Table +from daft.recordbatch.recordbatch import RecordBatch +from daft.series import Series if TYPE_CHECKING: from pyiceberg.schema import Schema as IcebergSchema @@ -34,7 +34,7 @@ def __init__( root_dir: str, file_idx: int, file_format: str, - partition_values: Optional[Table] = None, + partition_values: Optional[RecordBatch] = None, compression: Optional[str] = None, io_config: Optional[IOConfig] = None, version: Optional[int] = None, @@ -93,11 +93,11 @@ def write(self, table: MicroPartition) -> int: pass @abstractmethod - def close(self) -> Table: + def close(self) -> RecordBatch: """Close the writer and return metadata about the written file. Write should not be called after close. Returns: - Table containing metadata about the written file, including path and partition values. + RecordBatch containing metadata about the written file, including path and partition values. """ pass @@ -107,7 +107,7 @@ def __init__( self, root_dir: str, file_idx: int, - partition_values: Optional[Table] = None, + partition_values: Optional[RecordBatch] = None, compression: Optional[str] = None, io_config: Optional[IOConfig] = None, version: Optional[int] = None, @@ -152,7 +152,7 @@ def write(self, table: MicroPartition) -> int: self.position = current_position return bytes_written - def close(self) -> Table: + def close(self) -> RecordBatch: if self.current_writer is not None: self.current_writer.close() @@ -161,7 +161,7 @@ def close(self) -> Table: if self.partition_values is not None: for col_name in self.partition_values.column_names(): metadata[col_name] = self.partition_values.get_column(col_name) - return Table.from_pydict(metadata) + return RecordBatch.from_pydict(metadata) class CSVFileWriter(FileWriterBase): @@ -169,7 +169,7 @@ def __init__( self, root_dir: str, file_idx: int, - partition_values: Optional[Table] = None, + partition_values: Optional[RecordBatch] = None, io_config: Optional[IOConfig] = None, ): super().__init__( @@ -202,7 +202,7 @@ def write(self, table: MicroPartition) -> int: self.position = current_position return bytes_written - def close(self) -> Table: + def close(self) -> RecordBatch: if self.current_writer is not None: self.current_writer.close() @@ -211,7 +211,7 @@ def close(self) -> Table: if self.partition_values is not None: for col_name in self.partition_values.column_names(): metadata[col_name] = self.partition_values.get_column(col_name) - return Table.from_pydict(metadata) + return RecordBatch.from_pydict(metadata) class IcebergWriter(ParquetFileWriter): @@ -222,7 +222,7 @@ def __init__( schema: "IcebergSchema", properties: "IcebergTableProperties", partition_spec_id: int, - partition_values: Optional[Table] = None, + partition_values: Optional[RecordBatch] = None, io_config: Optional[IOConfig] = None, ): from pyiceberg.io.pyarrow import schema_to_pyarrow @@ -258,7 +258,7 @@ def write(self, table: MicroPartition) -> int: self.position = current_position return bytes_written - def close(self) -> Table: + def close(self) -> RecordBatch: if self.current_writer is not None: self.current_writer.close() self.is_closed = True @@ -275,7 +275,7 @@ def close(self) -> Table: self.iceberg_schema, self.properties, ) - return Table.from_pydict({"data_file": [data_file]}) + return RecordBatch.from_pydict({"data_file": [data_file]}) class DeltalakeWriter(ParquetFileWriter): @@ -285,7 +285,7 @@ def __init__( file_idx: int, version: int, large_dtypes: bool, - partition_values: Optional[Table] = None, + partition_values: Optional[RecordBatch] = None, io_config: Optional[IOConfig] = None, ): super().__init__( @@ -321,7 +321,7 @@ def write(self, table: MicroPartition) -> int: self.position = current_position return bytes_written - def close(self) -> Table: + def close(self) -> RecordBatch: if self.current_writer is not None: self.current_writer.close() self.is_closed = True @@ -336,4 +336,4 @@ def close(self) -> Table: partition_values=self.partition_strings, ) - return Table.from_pydict({"add_action": [add_action]}) + return RecordBatch.from_pydict({"add_action": [add_action]}) diff --git a/daft/logical/map_partition_ops.py b/daft/logical/map_partition_ops.py index 935894c861..927cb1d184 100644 --- a/daft/logical/map_partition_ops.py +++ b/daft/logical/map_partition_ops.py @@ -7,7 +7,7 @@ from daft.logical.schema import Schema if TYPE_CHECKING: - from daft.table import MicroPartition + from daft.recordbatch import MicroPartition class MapPartitionOp: diff --git a/daft/recordbatch/__init__.py b/daft/recordbatch/__init__.py new file mode 100644 index 0000000000..fcb7c24f60 --- /dev/null +++ b/daft/recordbatch/__init__.py @@ -0,0 +1,7 @@ +from __future__ import annotations + +# Need to import after `.table` due to circular dep issue otherwise +from .micropartition import MicroPartition +from .recordbatch import RecordBatch, read_parquet_into_pyarrow, read_parquet_into_pyarrow_bulk + +__all__ = ["MicroPartition", "RecordBatch", "read_parquet_into_pyarrow", "read_parquet_into_pyarrow_bulk"] diff --git a/daft/table/micropartition.py b/daft/recordbatch/micropartition.py similarity index 94% rename from daft/table/micropartition.py rename to daft/recordbatch/micropartition.py index b1c60a9c3f..7263c5c7ae 100644 --- a/daft/table/micropartition.py +++ b/daft/recordbatch/micropartition.py @@ -15,13 +15,13 @@ JsonReadOptions, ) from daft.daft import PyMicroPartition as _PyMicroPartition -from daft.daft import PyTable as _PyTable +from daft.daft import PyRecordBatch as _PyRecordBatch from daft.daft import ScanTask as _ScanTask from daft.datatype import DataType, TimeUnit from daft.expressions import Expression, ExpressionsProjection from daft.logical.schema import Schema +from daft.recordbatch.recordbatch import RecordBatch from daft.series import Series -from daft.table.table import Table if TYPE_CHECKING: import pandas as pd @@ -72,8 +72,8 @@ def _from_scan_task(scan_task: _ScanTask) -> MicroPartition: return MicroPartition._from_pymicropartition(_PyMicroPartition.from_scan_task(scan_task)) @staticmethod - def _from_pytable(pyt: _PyTable) -> MicroPartition: - assert isinstance(pyt, _PyTable) + def _from_pytable(pyt: _PyRecordBatch) -> MicroPartition: + assert isinstance(pyt, _PyRecordBatch) return MicroPartition._from_pymicropartition(_PyMicroPartition.from_tables([pyt])) @staticmethod @@ -84,12 +84,12 @@ def _from_pymicropartition(pym: _PyMicroPartition) -> MicroPartition: return tab @staticmethod - def _from_tables(tables: list[Table]) -> MicroPartition: + def _from_tables(tables: list[RecordBatch]) -> MicroPartition: return MicroPartition._from_pymicropartition(_PyMicroPartition.from_tables([t._table for t in tables])) @staticmethod def from_arrow(arrow_table: pa.Table) -> MicroPartition: - table = Table.from_arrow(arrow_table) + table = RecordBatch.from_arrow(arrow_table) return MicroPartition._from_tables([table]) @staticmethod @@ -100,12 +100,12 @@ def from_arrow_record_batches(rbs: list[pa.RecordBatch], arrow_schema: pa.Schema @staticmethod def from_pandas(pd_df: pd.DataFrame) -> MicroPartition: - table = Table.from_pandas(pd_df) + table = RecordBatch.from_pandas(pd_df) return MicroPartition._from_tables([table]) @staticmethod def from_pydict(data: dict) -> MicroPartition: - table = Table.from_pydict(data) + table = RecordBatch.from_pydict(data) return MicroPartition._from_tables([table]) @classmethod @@ -128,8 +128,8 @@ def slice(self, start: int, end: int) -> MicroPartition: # Exporting methods ### - def to_table(self) -> Table: - return Table._from_pytable(self._micropartition.to_table()) + def to_table(self) -> RecordBatch: + return RecordBatch._from_pytable(self._micropartition.to_table()) def to_arrow(self) -> pa.Table: return self.to_table().to_arrow() @@ -339,10 +339,10 @@ def partition_by_hash(self, exprs: ExpressionsProjection, num_partitions: int) - ] def partition_by_range( - self, partition_keys: ExpressionsProjection, boundaries: Table, descending: list[bool] + self, partition_keys: ExpressionsProjection, boundaries: RecordBatch, descending: list[bool] ) -> list[MicroPartition]: - if not isinstance(boundaries, Table): - raise TypeError(f"Expected a Table for `boundaries` in partition_by_range but got {type(boundaries)}") + if not isinstance(boundaries, RecordBatch): + raise TypeError(f"Expected a RecordBatch for `boundaries` in partition_by_range but got {type(boundaries)}") exprs = [e._expr for e in partition_keys] return [ @@ -364,10 +364,10 @@ def partition_by_random(self, num_partitions: int, seed: int) -> list[MicroParti def partition_by_value(self, partition_keys: ExpressionsProjection) -> tuple[list[MicroPartition], MicroPartition]: exprs = [e._expr for e in partition_keys] - pytables, values = self._micropartition.partition_by_value(exprs) - return [MicroPartition._from_pymicropartition(t) for t in pytables], MicroPartition._from_pymicropartition( - values - ) + PyRecordBatchs, values = self._micropartition.partition_by_value(exprs) + return [ + MicroPartition._from_pymicropartition(t) for t in PyRecordBatchs + ], MicroPartition._from_pymicropartition(values) def add_monotonically_increasing_id(self, partition_num: int, column_name: str) -> MicroPartition: return MicroPartition._from_pymicropartition( @@ -424,8 +424,8 @@ def read_parquet_statistics( paths: Series | list[str], io_config: IOConfig | None = None, multithreaded_io: bool | None = None, - ) -> Table: - return Table.read_parquet_statistics(paths=paths, io_config=io_config, multithreaded_io=multithreaded_io) + ) -> RecordBatch: + return RecordBatch.read_parquet_statistics(paths=paths, io_config=io_config, multithreaded_io=multithreaded_io) @classmethod def read_parquet( diff --git a/daft/table/partitioning.py b/daft/recordbatch/partitioning.py similarity index 96% rename from daft/table/partitioning.py rename to daft/recordbatch/partitioning.py index f7b097f066..0b9d3c9968 100644 --- a/daft/table/partitioning.py +++ b/daft/recordbatch/partitioning.py @@ -2,7 +2,7 @@ from daft import Series from daft.expressions import ExpressionsProjection -from daft.table.table import Table +from daft.recordbatch.recordbatch import RecordBatch from .micropartition import MicroPartition @@ -19,7 +19,7 @@ def partition_strings_to_path( def partition_values_to_str_mapping( - partition_values: Union[MicroPartition, Table], + partition_values: Union[MicroPartition, RecordBatch], ) -> Dict[str, Series]: null_part = Series.from_pylist( [None] diff --git a/daft/table/table.py b/daft/recordbatch/recordbatch.py similarity index 80% rename from daft/table/table.py rename to daft/recordbatch/recordbatch.py index d77fc2a3f8..854802de2b 100644 --- a/daft/table/table.py +++ b/daft/recordbatch/recordbatch.py @@ -13,7 +13,7 @@ JsonParseOptions, JsonReadOptions, ) -from daft.daft import PyTable as _PyTable +from daft.daft import PyRecordBatch as _PyRecordBatch from daft.daft import ScanTask as _ScanTask from daft.daft import read_csv as _read_csv from daft.daft import read_json as _read_json @@ -34,11 +34,11 @@ logger = logging.getLogger(__name__) -class Table: - _table: _PyTable +class RecordBatch: + _table: _PyRecordBatch def __init__(self) -> None: - raise NotImplementedError("We do not support creating a Table via __init__ ") + raise NotImplementedError("We do not support creating a RecordBatch via __init__ ") def schema(self) -> Schema: return Schema._from_pyschema(self._table.schema()) @@ -66,23 +66,23 @@ def _repr_html_(self) -> str: ### @staticmethod - def empty(schema: Schema | None = None) -> Table: - pyt = _PyTable.empty(None) if schema is None else _PyTable.empty(schema._schema) - return Table._from_pytable(pyt) + def empty(schema: Schema | None = None) -> RecordBatch: + pyt = _PyRecordBatch.empty(None) if schema is None else _PyRecordBatch.empty(schema._schema) + return RecordBatch._from_pytable(pyt) @staticmethod - def _from_scan_task(_: _ScanTask) -> Table: - raise NotImplementedError("_from_scan_task is not implemented for legacy Python Table.") + def _from_scan_task(_: _ScanTask) -> RecordBatch: + raise NotImplementedError("_from_scan_task is not implemented for legacy Python RecordBatch.") @staticmethod - def _from_pytable(pyt: _PyTable) -> Table: - assert isinstance(pyt, _PyTable) - tab = Table.__new__(Table) + def _from_pytable(pyt: _PyRecordBatch) -> RecordBatch: + assert isinstance(pyt, _PyRecordBatch) + tab = RecordBatch.__new__(RecordBatch) tab._table = pyt return tab @staticmethod - def from_arrow(arrow_table: pa.Table) -> Table: + def from_arrow(arrow_table: pa.Table) -> RecordBatch: assert isinstance(arrow_table, pa.Table) schema = Schema._from_field_name_and_types( [(f.name, DataType.from_arrow_type(f.type)) for f in arrow_table.schema] @@ -100,21 +100,21 @@ def from_arrow(arrow_table: pa.Table) -> Table: # If there are any contained Arrow types that are not natively supported, go through Table.from_pydict() # path. logger.debug("Unsupported Arrow types detected for columns: %s", non_native_fields) - return Table.from_pydict(dict(zip(arrow_table.column_names, arrow_table.columns))) + return RecordBatch.from_pydict(dict(zip(arrow_table.column_names, arrow_table.columns))) else: # Otherwise, go through record batch happy path. arrow_table = ensure_table(arrow_table) - pyt = _PyTable.from_arrow_record_batches(arrow_table.to_batches(), schema._schema) - return Table._from_pytable(pyt) + pyt = _PyRecordBatch.from_arrow_record_batches(arrow_table.to_batches(), schema._schema) + return RecordBatch._from_pytable(pyt) @staticmethod - def from_arrow_record_batches(rbs: list[pa.RecordBatch], arrow_schema: pa.Schema) -> Table: + def from_arrow_record_batches(rbs: list[pa.RecordBatch], arrow_schema: pa.Schema) -> RecordBatch: schema = Schema._from_field_name_and_types([(f.name, DataType.from_arrow_type(f.type)) for f in arrow_schema]) - pyt = _PyTable.from_arrow_record_batches(rbs, schema._schema) - return Table._from_pytable(pyt) + pyt = _PyRecordBatch.from_arrow_record_batches(rbs, schema._schema) + return RecordBatch._from_pytable(pyt) @staticmethod - def from_pandas(pd_df: pd.DataFrame) -> Table: + def from_pandas(pd_df: pd.DataFrame) -> RecordBatch: if not pd.module_available(): raise ImportError("Unable to import Pandas - please ensure that it is installed.") assert isinstance(pd_df, pd.DataFrame) @@ -123,40 +123,40 @@ def from_pandas(pd_df: pd.DataFrame) -> Table: except pa.ArrowInvalid: pass else: - return Table.from_arrow(arrow_table) + return RecordBatch.from_arrow(arrow_table) # Fall back to pydict path. df_as_dict = pd_df.to_dict(orient="series") - return Table.from_pydict(df_as_dict) + return RecordBatch.from_pydict(df_as_dict) @staticmethod - def from_pydict(data: dict) -> Table: + def from_pydict(data: dict) -> RecordBatch: series_dict = dict() for k, v in data.items(): series = item_to_series(k, v) series_dict[k] = series._series - return Table._from_pytable(_PyTable.from_pylist_series(series_dict)) + return RecordBatch._from_pytable(_PyRecordBatch.from_pylist_series(series_dict)) @classmethod - def concat(cls, to_merge: list[Table]) -> Table: + def concat(cls, to_merge: list[RecordBatch]) -> RecordBatch: tables = [] for t in to_merge: - if not isinstance(t, Table): + if not isinstance(t, RecordBatch): raise TypeError(f"Expected a Table for concat, got {type(t)}") tables.append(t._table) - return Table._from_pytable(_PyTable.concat(tables)) + return RecordBatch._from_pytable(_PyRecordBatch.concat(tables)) - def slice(self, start: int, end: int) -> Table: + def slice(self, start: int, end: int) -> RecordBatch: if not isinstance(start, int): raise TypeError(f"expected int for start but got {type(start)}") if not isinstance(end, int): raise TypeError(f"expected int for end but got {type(end)}") - return Table._from_pytable(self._table.slice(start, end)) + return RecordBatch._from_pytable(self._table.slice(start, end)) ### # Exporting methods ### - def to_table(self) -> Table: + def to_table(self) -> RecordBatch: """For compatibility with MicroPartition.""" return self @@ -220,33 +220,33 @@ def to_pandas( # Compute methods (Table -> Table) ### - def cast_to_schema(self, schema: Schema) -> Table: - """Casts a Table into the provided schema.""" - return Table._from_pytable(self._table.cast_to_schema(schema._schema)) + def cast_to_schema(self, schema: Schema) -> RecordBatch: + """Casts a RecordBatch into the provided schema.""" + return RecordBatch._from_pytable(self._table.cast_to_schema(schema._schema)) - def eval_expression_list(self, exprs: ExpressionsProjection) -> Table: + def eval_expression_list(self, exprs: ExpressionsProjection) -> RecordBatch: assert all(isinstance(e, Expression) for e in exprs) pyexprs = [e._expr for e in exprs] - return Table._from_pytable(self._table.eval_expression_list(pyexprs)) + return RecordBatch._from_pytable(self._table.eval_expression_list(pyexprs)) - def head(self, num: int) -> Table: - return Table._from_pytable(self._table.head(num)) + def head(self, num: int) -> RecordBatch: + return RecordBatch._from_pytable(self._table.head(num)) - def take(self, indices: Series) -> Table: + def take(self, indices: Series) -> RecordBatch: assert isinstance(indices, Series) - return Table._from_pytable(self._table.take(indices._series)) + return RecordBatch._from_pytable(self._table.take(indices._series)) - def filter(self, exprs: ExpressionsProjection) -> Table: + def filter(self, exprs: ExpressionsProjection) -> RecordBatch: assert all(isinstance(e, Expression) for e in exprs) pyexprs = [e._expr for e in exprs] - return Table._from_pytable(self._table.filter(pyexprs)) + return RecordBatch._from_pytable(self._table.filter(pyexprs)) def sort( self, sort_keys: ExpressionsProjection, descending: bool | list[bool] | None = None, nulls_first: bool | list[bool] | None = None, - ) -> Table: + ) -> RecordBatch: assert all(isinstance(e, Expression) for e in sort_keys) pyexprs = [e._expr for e in sort_keys] if descending is None: @@ -273,7 +273,7 @@ def sort( ) else: nulls_first = [bool(x) for x in nulls_first] - return Table._from_pytable(self._table.sort(pyexprs, descending, nulls_first)) + return RecordBatch._from_pytable(self._table.sort(pyexprs, descending, nulls_first)) def sample( self, @@ -281,65 +281,67 @@ def sample( size: int | None = None, with_replacement: bool = False, seed: int | None = None, - ) -> Table: + ) -> RecordBatch: if fraction is not None and size is not None: raise ValueError("Must specify either `fraction` or `size`, but not both") elif fraction is not None: - return Table._from_pytable(self._table.sample_by_fraction(fraction, with_replacement, seed)) + return RecordBatch._from_pytable(self._table.sample_by_fraction(fraction, with_replacement, seed)) elif size is not None: - return Table._from_pytable(self._table.sample_by_size(size, with_replacement, seed)) + return RecordBatch._from_pytable(self._table.sample_by_size(size, with_replacement, seed)) else: raise ValueError("Must specify either `fraction` or `size`") - def agg(self, to_agg: list[Expression], group_by: ExpressionsProjection | None = None) -> Table: + def agg(self, to_agg: list[Expression], group_by: ExpressionsProjection | None = None) -> RecordBatch: to_agg_pyexprs = [e._expr for e in to_agg] group_by_pyexprs = [e._expr for e in group_by] if group_by is not None else [] - return Table._from_pytable(self._table.agg(to_agg_pyexprs, group_by_pyexprs)) + return RecordBatch._from_pytable(self._table.agg(to_agg_pyexprs, group_by_pyexprs)) def pivot( self, group_by: ExpressionsProjection, pivot_column: Expression, values_column: Expression, names: list[str] - ) -> Table: + ) -> RecordBatch: group_by_pyexprs = [e._expr for e in group_by] - return Table._from_pytable(self._table.pivot(group_by_pyexprs, pivot_column._expr, values_column._expr, names)) + return RecordBatch._from_pytable( + self._table.pivot(group_by_pyexprs, pivot_column._expr, values_column._expr, names) + ) - def quantiles(self, num: int) -> Table: - return Table._from_pytable(self._table.quantiles(num)) + def quantiles(self, num: int) -> RecordBatch: + return RecordBatch._from_pytable(self._table.quantiles(num)) - def explode(self, columns: ExpressionsProjection) -> Table: + def explode(self, columns: ExpressionsProjection) -> RecordBatch: """NOTE: Expressions here must be Explode expressions.""" to_explode_pyexprs = [e._expr for e in columns] - return Table._from_pytable(self._table.explode(to_explode_pyexprs)) + return RecordBatch._from_pytable(self._table.explode(to_explode_pyexprs)) def hash_join( self, - right: Table, + right: RecordBatch, left_on: ExpressionsProjection, right_on: ExpressionsProjection, how: JoinType = JoinType.Inner, - ) -> Table: + ) -> RecordBatch: if len(left_on) != len(right_on): raise ValueError( f"Mismatch of number of join keys, left_on: {len(left_on)}, right_on: {len(right_on)}\nleft_on {left_on}\nright_on {right_on}" ) - if not isinstance(right, Table): + if not isinstance(right, RecordBatch): raise TypeError(f"Expected a Table for `right` in join but got {type(right)}") left_exprs = [e._expr for e in left_on] right_exprs = [e._expr for e in right_on] - return Table._from_pytable( + return RecordBatch._from_pytable( self._table.hash_join(right._table, left_on=left_exprs, right_on=right_exprs, how=how) ) def sort_merge_join( self, - right: Table, + right: RecordBatch, left_on: ExpressionsProjection, right_on: ExpressionsProjection, how: JoinType = JoinType.Inner, is_sorted: bool = False, - ) -> Table: + ) -> RecordBatch: if how != JoinType.Inner: raise NotImplementedError("TODO: [RUST] Implement Other Join types") if len(left_on) != len(right_on): @@ -347,49 +349,51 @@ def sort_merge_join( f"Mismatch of number of join keys, left_on: {len(left_on)}, right_on: {len(right_on)}\nleft_on {left_on}\nright_on {right_on}" ) - if not isinstance(right, Table): + if not isinstance(right, RecordBatch): raise TypeError(f"Expected a Table for `right` in join but got {type(right)}") left_exprs = [e._expr for e in left_on] right_exprs = [e._expr for e in right_on] - return Table._from_pytable( + return RecordBatch._from_pytable( self._table.sort_merge_join(right._table, left_on=left_exprs, right_on=right_exprs, is_sorted=is_sorted) ) - def partition_by_hash(self, exprs: ExpressionsProjection, num_partitions: int) -> list[Table]: + def partition_by_hash(self, exprs: ExpressionsProjection, num_partitions: int) -> list[RecordBatch]: if not isinstance(num_partitions, int): raise TypeError(f"Expected a num_partitions to be int, got {type(num_partitions)}") pyexprs = [e._expr for e in exprs] - return [Table._from_pytable(t) for t in self._table.partition_by_hash(pyexprs, num_partitions)] + return [RecordBatch._from_pytable(t) for t in self._table.partition_by_hash(pyexprs, num_partitions)] def partition_by_range( - self, partition_keys: ExpressionsProjection, boundaries: Table, descending: list[bool] - ) -> list[Table]: - if not isinstance(boundaries, Table): - raise TypeError(f"Expected a Table for `boundaries` in partition_by_range but got {type(boundaries)}") + self, partition_keys: ExpressionsProjection, boundaries: RecordBatch, descending: list[bool] + ) -> list[RecordBatch]: + if not isinstance(boundaries, RecordBatch): + raise TypeError(f"Expected a RecordBatch for `boundaries` in partition_by_range but got {type(boundaries)}") exprs = [e._expr for e in partition_keys] - return [Table._from_pytable(t) for t in self._table.partition_by_range(exprs, boundaries._table, descending)] + return [ + RecordBatch._from_pytable(t) for t in self._table.partition_by_range(exprs, boundaries._table, descending) + ] - def partition_by_random(self, num_partitions: int, seed: int) -> list[Table]: + def partition_by_random(self, num_partitions: int, seed: int) -> list[RecordBatch]: if not isinstance(num_partitions, int): raise TypeError(f"Expected a num_partitions to be int, got {type(num_partitions)}") if not isinstance(seed, int): raise TypeError(f"Expected a seed to be int, got {type(seed)}") - return [Table._from_pytable(t) for t in self._table.partition_by_random(num_partitions, seed)] + return [RecordBatch._from_pytable(t) for t in self._table.partition_by_random(num_partitions, seed)] - def partition_by_value(self, partition_keys: ExpressionsProjection) -> tuple[list[Table], Table]: + def partition_by_value(self, partition_keys: ExpressionsProjection) -> tuple[list[RecordBatch], RecordBatch]: exprs = [e._expr for e in partition_keys] - pytables, values = self._table.partition_by_value(exprs) + PyRecordBatchs, values = self._table.partition_by_value(exprs) - return [Table._from_pytable(t) for t in pytables], Table._from_pytable(values) + return [RecordBatch._from_pytable(t) for t in PyRecordBatchs], RecordBatch._from_pytable(values) - def add_monotonically_increasing_id(self, partition_num: int, column_name: str) -> Table: - return Table._from_pytable(self._table.add_monotonically_increasing_id(partition_num, column_name)) + def add_monotonically_increasing_id(self, partition_num: int, column_name: str) -> RecordBatch: + return RecordBatch._from_pytable(self._table.add_monotonically_increasing_id(partition_num, column_name)) ### # Compute methods (Table -> Series) @@ -433,7 +437,7 @@ def argsort( def __reduce__(self) -> tuple: names = self.column_names() - return Table.from_pydict, ({name: self.get_column(name) for name in names},) + return RecordBatch.from_pydict, ({name: self.get_column(name) for name in names},) @classmethod def read_parquet( @@ -447,8 +451,8 @@ def read_parquet( io_config: IOConfig | None = None, multithreaded_io: bool | None = None, coerce_int96_timestamp_unit: TimeUnit = TimeUnit.ns(), - ) -> Table: - return Table._from_pytable( + ) -> RecordBatch: + return RecordBatch._from_pytable( _read_parquet( uri=path, columns=columns, @@ -475,8 +479,8 @@ def read_parquet_bulk( num_parallel_tasks: int | None = 128, multithreaded_io: bool | None = None, coerce_int96_timestamp_unit: TimeUnit = TimeUnit.ns(), - ) -> list[Table]: - pytables = _read_parquet_bulk( + ) -> list[RecordBatch]: + PyRecordBatchs = _read_parquet_bulk( uris=paths, columns=columns, start_offset=start_offset, @@ -488,7 +492,7 @@ def read_parquet_bulk( multithreaded_io=multithreaded_io, coerce_int96_timestamp_unit=coerce_int96_timestamp_unit._timeunit, ) - return [Table._from_pytable(t) for t in pytables] + return [RecordBatch._from_pytable(t) for t in PyRecordBatchs] @classmethod def read_parquet_statistics( @@ -496,11 +500,11 @@ def read_parquet_statistics( paths: Series | list[str], io_config: IOConfig | None = None, multithreaded_io: bool | None = None, - ) -> Table: + ) -> RecordBatch: if not isinstance(paths, Series): paths = Series.from_pylist(paths, name="uris").cast(DataType.string()) assert paths.name() == "uris", f"Expected input series to have name 'uris', but found: {paths.name()}" - return Table._from_pytable( + return RecordBatch._from_pytable( _read_parquet_statistics( uris=paths._series, io_config=io_config, @@ -517,8 +521,8 @@ def read_csv( read_options: CsvReadOptions | None = None, io_config: IOConfig | None = None, multithreaded_io: bool | None = None, - ) -> Table: - return Table._from_pytable( + ) -> RecordBatch: + return RecordBatch._from_pytable( _read_csv( uri=path, convert_options=convert_options, @@ -539,8 +543,8 @@ def read_json( io_config: IOConfig | None = None, multithreaded_io: bool | None = None, max_chunks_in_flight: int | None = None, - ) -> Table: - return Table._from_pytable( + ) -> RecordBatch: + return RecordBatch._from_pytable( _read_json( uri=path, convert_options=convert_options, diff --git a/daft/table/table_io.py b/daft/recordbatch/recordbatch_io.py similarity index 100% rename from daft/table/table_io.py rename to daft/recordbatch/recordbatch_io.py diff --git a/daft/runners/native_runner.py b/daft/runners/native_runner.py index a03e14c93a..fa294b1040 100644 --- a/daft/runners/native_runner.py +++ b/daft/runners/native_runner.py @@ -7,6 +7,7 @@ from daft.daft import FileFormatConfig, FileInfos, IOConfig from daft.execution.native_executor import NativeExecutor from daft.filesystem import glob_path_with_stats +from daft.recordbatch import MicroPartition from daft.runners import runner_io from daft.runners.partitioning import ( LocalMaterializedResult, @@ -15,7 +16,6 @@ PartitionSetCache, ) from daft.runners.runner import LOCAL_PARTITION_SET_CACHE, Runner -from daft.table import MicroPartition if TYPE_CHECKING: from daft.logical.builder import LogicalPlanBuilder diff --git a/daft/runners/partitioning.py b/daft/runners/partitioning.py index 502dabcff8..cc3c81745d 100644 --- a/daft/runners/partitioning.py +++ b/daft/runners/partitioning.py @@ -8,7 +8,7 @@ from uuid import uuid4 from daft.datatype import TimeUnit -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition if TYPE_CHECKING: import pandas as pd diff --git a/daft/runners/pyrunner.py b/daft/runners/pyrunner.py index 48be64921b..01deda7f63 100644 --- a/daft/runners/pyrunner.py +++ b/daft/runners/pyrunner.py @@ -17,6 +17,7 @@ from daft.expressions import ExpressionsProjection from daft.filesystem import glob_path_with_stats from daft.internal.gpu import cuda_visible_devices +from daft.recordbatch import MicroPartition from daft.runners import runner_io from daft.runners.partitioning import ( LocalMaterializedResult, @@ -30,7 +31,6 @@ from daft.runners.profiler import profiler from daft.runners.progress_bar import ProgressBar from daft.runners.runner import LOCAL_PARTITION_SET_CACHE, Runner -from daft.table import MicroPartition if TYPE_CHECKING: from daft.execution import physical_plan diff --git a/daft/runners/ray_runner.py b/daft/runners/ray_runner.py index dba2b92a55..fc801af734 100644 --- a/daft/runners/ray_runner.py +++ b/daft/runners/ray_runner.py @@ -18,12 +18,12 @@ from daft.arrow_utils import ensure_array from daft.context import execution_config_ctx, get_context -from daft.daft import PyTable as _PyTable +from daft.daft import PyRecordBatch as _PyRecordBatch from daft.dependencies import np +from daft.recordbatch import RecordBatch from daft.runners import ray_tracing from daft.runners.progress_bar import ProgressBar from daft.series import Series, item_to_series -from daft.table import Table logger = logging.getLogger(__name__) @@ -56,6 +56,7 @@ from daft.execution.physical_plan import ActorPoolManager from daft.expressions import ExpressionsProjection from daft.filesystem import glob_path_with_stats +from daft.recordbatch import MicroPartition from daft.runners import runner_io from daft.runners.partitioning import ( LocalPartitionSet, @@ -68,7 +69,6 @@ ) from daft.runners.profiler import profiler from daft.runners.runner import Runner -from daft.table import MicroPartition if TYPE_CHECKING: import dask @@ -216,7 +216,7 @@ def _micropartition_from_arrow_with_ray_data_extensions(arrow_table: pa.Table) - else item_to_series(name, column) ) series_dict[name] = series._series - return MicroPartition._from_tables([Table._from_pytable(_PyTable.from_pylist_series(series_dict))]) + return MicroPartition._from_tables([RecordBatch._from_pytable(_PyRecordBatch.from_pylist_series(series_dict))]) return MicroPartition.from_arrow(arrow_table) diff --git a/daft/runners/runner.py b/daft/runners/runner.py index 34a81fda6a..b6734d3087 100644 --- a/daft/runners/runner.py +++ b/daft/runners/runner.py @@ -13,8 +13,8 @@ if TYPE_CHECKING: from daft.logical.builder import LogicalPlanBuilder + from daft.recordbatch import MicroPartition from daft.runners.runner_io import RunnerIO - from daft.table import MicroPartition LOCAL_PARTITION_SET_CACHE = PartitionSetCache() diff --git a/daft/sql/sql_scan.py b/daft/sql/sql_scan.py index 7c090ef90a..8a3de1f51f 100644 --- a/daft/sql/sql_scan.py +++ b/daft/sql/sql_scan.py @@ -11,14 +11,14 @@ DatabaseSourceConfig, FileFormatConfig, Pushdowns, - PyTable, + PyRecordBatch, ScanTask, StorageConfig, ) from daft.expressions.expressions import lit from daft.io.common import _get_schema_from_dict from daft.io.scan import PartitionField, ScanOperator -from daft.table import Table +from daft.recordbatch import RecordBatch if TYPE_CHECKING: from collections.abc import Iterator @@ -111,7 +111,7 @@ def to_scan_tasks(self, pushdowns: Pushdowns) -> Iterator[ScanTask]: right_clause = ( f"{self._partition_col} {'<' if i < num_scan_tasks - 1 else '<='} {partition_bounds_sql[i + 1]}" ) - stats = Table.from_pydict({self._partition_col: [partition_bounds[i], partition_bounds[i + 1]]}) + stats = RecordBatch.from_pydict({self._partition_col: [partition_bounds[i], partition_bounds[i + 1]]}) scan_task = self._construct_scan_task( pushdowns, num_rows=None, @@ -210,7 +210,7 @@ def _get_partition_bounds(self, num_scan_tasks: int) -> list[Any]: if pa_table.num_columns != num_scan_tasks + 1: raise RuntimeError(f"Expected {num_scan_tasks + 1} percentiles, but got {pa_table.num_columns}.") - pydict = Table.from_arrow(pa_table).to_pydict() + pydict = RecordBatch.from_arrow(pa_table).to_pydict() assert pydict.keys() == {f"bound_{i}" for i in range(num_scan_tasks + 1)} return [pydict[f"bound_{i}"][0] for i in range(num_scan_tasks + 1)] @@ -232,7 +232,7 @@ def _get_partition_bounds(self, num_scan_tasks: int) -> list[Any]: if pa_table.num_columns != 2: raise RuntimeError(f"Failed to get partition bounds: expected 2 columns, but got {pa_table.num_columns}.") - pydict = Table.from_arrow(pa_table).to_pydict() + pydict = RecordBatch.from_arrow(pa_table).to_pydict() assert pydict.keys() == {"min", "max"} min_val = pydict["min"][0] max_val = pydict["max"][0] @@ -248,7 +248,7 @@ def _construct_scan_task( num_rows: int | None = None, size_bytes: int | None = None, partition_bounds: tuple[str, str] | None = None, - stats: PyTable | None = None, + stats: PyRecordBatch | None = None, ) -> ScanTask: predicate_sql = pushdowns.filters.to_sql() if pushdowns.filters is not None else None apply_pushdowns_to_sql = not self._disable_pushdowns_to_sql and ( diff --git a/daft/table/__init__.py b/daft/table/__init__.py deleted file mode 100644 index 5ecab85779..0000000000 --- a/daft/table/__init__.py +++ /dev/null @@ -1,7 +0,0 @@ -from __future__ import annotations - -# Need to import after `.table` due to circular dep issue otherwise -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/docs/mkdocs/integrations/delta_lake.md b/docs/mkdocs/integrations/delta_lake.md index 2e282a314a..4cf08162a2 100644 --- a/docs/mkdocs/integrations/delta_lake.md +++ b/docs/mkdocs/integrations/delta_lake.md @@ -81,7 +81,7 @@ You can use [`df.write_deltalake`](../{{ api_path }}/dataframe_methods/daft.Data === "🐍 Python" ```python - df.write_deltalake("tmp/daft-table", mode="overwrite") + df.write_deltalake("tmp/daft-recordbatch", mode="overwrite") ``` Daft supports multiple write modes. See the API docs for [`daft.DataFrame.write_deltalake`](../{{ api_path }}/dataframe_methods/daft.DataFrame.write_deltalake.html) for more details. diff --git a/src/daft-connect/Cargo.toml b/src/daft-connect/Cargo.toml index 46daa95f18..76c9300ab9 100644 --- a/src/daft-connect/Cargo.toml +++ b/src/daft-connect/Cargo.toml @@ -14,10 +14,10 @@ daft-logical-plan = {workspace = true, optional = true, features = ["python"]} daft-micropartition = {workspace = true, optional = true, features = [ "python" ]} +daft-recordbatch = {workspace = true, optional = true, features = ["python"]} daft-scan = {workspace = true, optional = true, features = ["python"]} daft-schema = {workspace = true, optional = true, features = ["python"]} daft-sql = {workspace = true, optional = true, features = ["python"]} -daft-table = {workspace = true, optional = true, features = ["python"]} dashmap = "6.1.0" futures = "0.3.31" itertools = {workspace = true} @@ -43,7 +43,7 @@ python = [ "dep:daft-scan", "dep:daft-schema", "dep:daft-sql", - "dep:daft-table", + "dep:daft-recordbatch", "dep:daft-context", "dep:daft-catalog" ] diff --git a/src/daft-connect/src/execute.rs b/src/daft-connect/src/execute.rs index bef6da8503..0f1a1904b4 100644 --- a/src/daft-connect/src/execute.rs +++ b/src/daft-connect/src/execute.rs @@ -6,7 +6,7 @@ use daft_context::get_context; use daft_dsl::LiteralValue; use daft_logical_plan::LogicalPlanBuilder; use daft_micropartition::MicroPartition; -use daft_table::Table; +use daft_recordbatch::RecordBatch; use futures::{ stream::{self, BoxStream}, StreamExt, TryStreamExt, @@ -357,14 +357,14 @@ impl Session { .ok_or_else(|| ConnectError::internal("no results"))?; let tbls = single_batch.get_tables()?; - let tbl = Table::concat(&tbls)?; + let tbl = RecordBatch::concat(&tbls)?; let output = tbl.to_comfy_table(None).to_string(); let s = LiteralValue::Utf8(output) .into_single_value_series()? .rename("show_string"); - let tbl = Table::from_nonempty_columns(vec![s])?; + let tbl = RecordBatch::from_nonempty_columns(vec![s])?; response_builder.arrow_batch_response(&tbl) } } diff --git a/src/daft-connect/src/response_builder.rs b/src/daft-connect/src/response_builder.rs index d0a800242b..2c713857ed 100644 --- a/src/daft-connect/src/response_builder.rs +++ b/src/daft-connect/src/response_builder.rs @@ -1,5 +1,5 @@ use arrow2::io::ipc::write::StreamWriter; -use daft_table::Table; +use daft_recordbatch::RecordBatch; use spark_connect::{ analyze_plan_response, execute_plan_response::{ArrowBatch, ResponseType, ResultComplete}, @@ -65,7 +65,7 @@ impl ResponseBuilder { } /// Send an arrow batch response to the client - pub fn arrow_batch_response(&self, table: &Table) -> ConnectResult { + pub fn arrow_batch_response(&self, table: &RecordBatch) -> ConnectResult { let mut data = Vec::new(); let mut writer = StreamWriter::new( diff --git a/src/daft-connect/src/spark_analyzer.rs b/src/daft-connect/src/spark_analyzer.rs index d1b13fb9a4..c67925d28a 100644 --- a/src/daft-connect/src/spark_analyzer.rs +++ b/src/daft-connect/src/spark_analyzer.rs @@ -10,10 +10,10 @@ use daft_core::series::Series; use daft_dsl::col; use daft_logical_plan::{LogicalPlanBuilder, PyLogicalPlanBuilder}; use daft_micropartition::{self, python::PyMicroPartition, MicroPartition}; +use daft_recordbatch::RecordBatch; use daft_scan::builder::{CsvScanBuilder, ParquetScanBuilder}; use daft_schema::schema::{Schema, SchemaRef}; use daft_sql::SQLPlanner; -use daft_table::Table; use datatype::to_daft_datatype; pub use datatype::to_spark_datatype; use itertools::zip_eq; @@ -56,7 +56,7 @@ impl SparkAnalyzer<'_> { /// Creates a logical source (scan) operator from a vec of tables. /// - /// Consider moving into LogicalBuilder, but this would re-introduce the daft-table dependency. + /// Consider moving into LogicalBuilder, but this would re-introduce the daft-recordbatch dependency. /// /// TODOs /// * https://github.com/Eventual-Inc/Daft/pull/3250 @@ -66,18 +66,17 @@ impl SparkAnalyzer<'_> { &self, _plan_id: usize, schema: Arc, - tables: Vec, + tables: Vec, ) -> ConnectResult { let mp = MicroPartition::new_loaded(schema, Arc::new(tables), None); Python::with_gil(|py| { // Convert MicroPartition to a logical plan using Python interop. let py_micropartition = py - .import(intern!(py, "daft.table"))? + .import(intern!(py, "daft.recordbatch"))? .getattr(intern!(py, "MicroPartition"))? .getattr(intern!(py, "_from_pymicropartition"))? .call1((PyMicroPartition::from(mp),))?; - // ERROR: 2: AttributeError: 'daft.daft.PySchema' object has no attribute '_schema' let py_plan_builder = py .import(intern!(py, "daft.dataframe.dataframe"))? .getattr(intern!(py, "to_logical_plan_builder"))? @@ -465,7 +464,7 @@ impl SparkAnalyzer<'_> { }) .collect::>>()?; - let batch = Table::from_nonempty_columns(columns)?; + let batch = RecordBatch::from_nonempty_columns(columns)?; Ok(batch) }).collect::>>()?; diff --git a/src/daft-csv/Cargo.toml b/src/daft-csv/Cargo.toml index 4829b8a80b..c6645a35c1 100644 --- a/src/daft-csv/Cargo.toml +++ b/src/daft-csv/Cargo.toml @@ -11,7 +11,7 @@ daft-core = {path = "../daft-core", default-features = false} daft-decoding = {path = "../daft-decoding"} daft-dsl = {path = "../daft-dsl", default-features = false} daft-io = {path = "../daft-io", default-features = false} -daft-table = {path = "../daft-table", default-features = false} +daft-recordbatch = {path = "../daft-recordbatch", default-features = false} futures = {workspace = true} memchr = "2.7.2" parking_lot = "0.12.3" @@ -27,7 +27,7 @@ tokio-util = {workspace = true} rstest = {workspace = true} [features] -python = ["dep:pyo3", "common-error/python", "common-py-serde/python", "daft-core/python", "daft-io/python", "daft-table/python", "daft-dsl/python"] +python = ["dep:pyo3", "common-error/python", "common-py-serde/python", "daft-core/python", "daft-io/python", "daft-recordbatch/python", "daft-dsl/python"] [lints] workspace = true diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs index 8dd3f4f927..758302f480 100644 --- a/src/daft-csv/src/local.rs +++ b/src/daft-csv/src/local.rs @@ -16,7 +16,7 @@ use daft_core::{ use daft_decoding::deserialize::deserialize_column; use daft_dsl::{optimization::get_required_columns, Expr}; use daft_io::{IOClient, IOStatsRef}; -use daft_table::Table; +use daft_recordbatch::RecordBatch; use futures::{Stream, StreamExt, TryStreamExt}; use rayon::{ iter::IndexedParallelIterator, @@ -149,7 +149,7 @@ pub async fn read_csv_local( io_client: Arc, io_stats: Option, max_chunks_in_flight: Option, -) -> DaftResult
{ +) -> DaftResult { let stream = stream_csv_local( uri, convert_options.clone(), @@ -176,7 +176,7 @@ pub async fn read_csv_local( io_stats, ) .await?; - return Table::empty(Some(Arc::new(Schema::try_from(&schema)?))); + return RecordBatch::empty(Some(Arc::new(Schema::try_from(&schema)?))); } let concated_table = tables_concat(collected_tables)?; @@ -200,7 +200,7 @@ pub async fn stream_csv_local( io_client: Arc, io_stats: Option, max_chunks_in_flight: Option, -) -> DaftResult> + Send> { +) -> DaftResult> + Send> { let uri = uri.trim_start_matches("file://"); let file = std::fs::File::open(uri)?; @@ -544,7 +544,7 @@ fn stream_csv_as_tables( predicate: Option>, limit: Option, n_threads: usize, -) -> DaftResult> + Send> { +) -> DaftResult> + Send> { // Create a slab iterator over the file. let slabpool = FileSlabPool::new(); let slab_iterator = SlabIterator::new(file, slabpool); @@ -606,7 +606,7 @@ fn stream_csv_as_tables( .context(super::OneShotRecvSnafu {})? }); let flattened = stream - .map(|result: DaftResult>| { + .map(|result: DaftResult>| { let tables = result?; DaftResult::Ok(futures::stream::iter(tables.into_iter().map(Ok))) }) @@ -831,7 +831,7 @@ fn collect_tables( include_columns: Option>, predicate: Option>, limit: Option, -) -> DaftResult> +) -> DaftResult> where R: std::io::Read, { @@ -871,7 +871,7 @@ fn parse_csv_chunk( include_columns: Option>, predicate: Option>, limit: Option, -) -> DaftResult> +) -> DaftResult> where R: std::io::Read, { @@ -898,7 +898,7 @@ where }) .collect::>>()?; let num_rows = chunk.first().map(|s| s.len()).unwrap_or(0); - let table = Table::new_unchecked(read_schema.clone(), chunk, num_rows); + let table = RecordBatch::new_unchecked(read_schema.clone(), chunk, num_rows); let table = if let Some(predicate) = &predicate { let filtered = table.filter(&[predicate.clone()])?; if let Some(include_columns) = &include_columns { diff --git a/src/daft-csv/src/python.rs b/src/daft-csv/src/python.rs index 875a58e3a0..a2869b09d7 100644 --- a/src/daft-csv/src/python.rs +++ b/src/daft-csv/src/python.rs @@ -3,7 +3,7 @@ pub mod pylib { use daft_core::python::PySchema; use daft_io::{get_io_client, python::IOConfig, IOStatsContext}; - use daft_table::python::PyTable; + use daft_recordbatch::python::PyRecordBatch; use pyo3::{pyfunction, PyResult, Python}; use crate::{CsvConvertOptions, CsvParseOptions, CsvReadOptions}; @@ -24,7 +24,7 @@ pub mod pylib { read_options: Option, io_config: Option, multithreaded_io: Option, - ) -> PyResult { + ) -> PyResult { py.allow_threads(|| { let io_stats = IOStatsContext::new(format!("read_csv: for uri {uri}")); diff --git a/src/daft-csv/src/read.rs b/src/daft-csv/src/read.rs index 11b1cfff85..7937d37d63 100644 --- a/src/daft-csv/src/read.rs +++ b/src/daft-csv/src/read.rs @@ -17,7 +17,7 @@ use daft_core::{prelude::*, utils::arrow::cast_array_for_daft_if_needed}; use daft_decoding::deserialize::deserialize_column; use daft_dsl::optimization::get_required_columns; use daft_io::{parse_url, GetResult, IOClient, IOStatsRef, SourceType}; -use daft_table::Table; +use daft_recordbatch::RecordBatch; use futures::{stream::BoxStream, Stream, StreamExt, TryStreamExt}; use rayon::{ iter::{IndexedParallelIterator, IntoParallelIterator}, @@ -48,7 +48,7 @@ impl ByteRecordChunkStream for S where use crate::local::{read_csv_local, stream_csv_local}; type TableChunkResult = - super::Result>, super::JoinSnafu, super::Error>>; + super::Result>, super::JoinSnafu, super::Error>>; trait TableStream: Stream {} impl TableStream for S where S: Stream {} @@ -62,7 +62,7 @@ pub fn read_csv( io_stats: Option, multithreaded_io: bool, max_chunks_in_flight: Option, -) -> DaftResult
{ +) -> DaftResult { let runtime_handle = get_io_runtime(multithreaded_io); runtime_handle.block_on_current_thread(async { read_csv_single_into_table( @@ -89,7 +89,7 @@ pub fn read_csv_bulk( multithreaded_io: bool, max_chunks_in_flight: Option, num_parallel_tasks: usize, -) -> DaftResult> { +) -> DaftResult> { let runtime_handle = get_io_runtime(multithreaded_io); let tables = runtime_handle.block_on_current_thread(async move { // Launch a read task per URI, throttling the number of concurrent file reads to num_parallel tasks. @@ -154,7 +154,7 @@ pub async fn stream_csv( io_client: Arc, io_stats: Option, max_chunks_in_flight: Option, -) -> DaftResult>> { +) -> DaftResult>> { let uri = uri.as_str(); let (source_type, _) = parse_url(uri)?; let is_compressed = CompressionCodec::from_uri(uri).is_some(); @@ -185,7 +185,7 @@ pub async fn stream_csv( } } -pub fn tables_concat(mut tables: Vec
) -> DaftResult
{ +pub fn tables_concat(mut tables: Vec) -> DaftResult { if tables.is_empty() { return Err(DaftError::ValueError( "Need at least 1 Table to perform concat".to_string(), @@ -216,10 +216,10 @@ pub fn tables_concat(mut tables: Vec
) -> DaftResult
{ Series::concat(series_to_cat.as_slice()) }) .collect::>>()?; - Table::new_with_size( + RecordBatch::new_with_size( first_table.schema.clone(), new_series, - tables.iter().map(daft_table::Table::len).sum(), + tables.iter().map(daft_recordbatch::RecordBatch::len).sum(), ) } @@ -232,7 +232,7 @@ async fn read_csv_single_into_table( io_client: Arc, io_stats: Option, max_chunks_in_flight: Option, -) -> DaftResult
{ +) -> DaftResult { let (source_type, _) = parse_url(uri)?; let is_compressed = CompressionCodec::from_uri(uri).is_some(); if matches!(source_type, SourceType::File) && !is_compressed { @@ -350,7 +350,7 @@ async fn read_csv_single_into_table( .collect::>>()?; // Handle empty table case. if collected_tables.is_empty() { - return Table::empty(Some(schema)); + return RecordBatch::empty(Some(schema)); } // // TODO(Clark): Don't concatenate all chunks from a file into a single table, since MicroPartition is natively chunked. @@ -373,7 +373,7 @@ pub async fn stream_csv_single( io_client: Arc, io_stats: Option, max_chunks_in_flight: Option, -) -> DaftResult> + Send> { +) -> DaftResult> + Send> { let predicate = convert_options .as_ref() .and_then(|opts| opts.predicate.clone()); @@ -667,7 +667,7 @@ fn parse_into_column_array_chunk_stream( }) .collect::>>()?; let num_rows = chunk.first().map_or(0, daft_core::series::Series::len); - Ok(Table::new_unchecked(read_schema, chunk, num_rows)) + Ok(RecordBatch::new_unchecked(read_schema, chunk, num_rows)) })(); let _ = send.send(result); }); @@ -713,7 +713,7 @@ mod tests { utils::arrow::{cast_array_for_daft_if_needed, cast_array_from_daft_if_needed}, }; use daft_io::{IOClient, IOConfig}; - use daft_table::Table; + use daft_recordbatch::RecordBatch; use rstest::rstest; use super::read_csv; @@ -722,7 +722,7 @@ mod tests { #[allow(clippy::too_many_arguments)] fn check_equal_local_arrow2( path: &str, - out: &Table, + out: &RecordBatch, has_header: bool, delimiter: Option, double_quote: bool, diff --git a/src/daft-json/Cargo.toml b/src/daft-json/Cargo.toml index 9baec41d2d..e3cee2fa7c 100644 --- a/src/daft-json/Cargo.toml +++ b/src/daft-json/Cargo.toml @@ -9,7 +9,7 @@ daft-core = {path = "../daft-core", default-features = false} daft-decoding = {path = "../daft-decoding"} daft-dsl = {path = "../daft-dsl", default-features = false} daft-io = {path = "../daft-io", default-features = false} -daft-table = {path = "../daft-table", default-features = false} +daft-recordbatch = {path = "../daft-recordbatch", default-features = false} futures = {workspace = true} indexmap = {workspace = true} memchr = "2.7.2" @@ -35,7 +35,7 @@ python = [ "common-py-serde/python", "daft-core/python", "daft-io/python", - "daft-table/python", + "daft-recordbatch/python", "daft-dsl/python" ] diff --git a/src/daft-json/src/local.rs b/src/daft-json/src/local.rs index d5c9828921..6be0f25c5c 100644 --- a/src/daft-json/src/local.rs +++ b/src/daft-json/src/local.rs @@ -3,7 +3,7 @@ use std::{borrow::Cow, collections::HashSet, num::NonZeroUsize, sync::Arc}; use common_error::DaftResult; use daft_core::{prelude::*, utils::arrow::cast_array_for_daft_if_needed}; use daft_dsl::Expr; -use daft_table::Table; +use daft_recordbatch::RecordBatch; use indexmap::IndexMap; use num_traits::Pow; use rayon::{prelude::*, ThreadPoolBuilder}; @@ -28,7 +28,7 @@ pub fn read_json_local( parse_options: Option, read_options: Option, max_chunks_in_flight: Option, -) -> DaftResult
{ +) -> DaftResult { let uri = uri.trim_start_matches("file://"); let file = std::fs::File::open(uri)?; // SAFETY: mmapping is inherently unsafe. @@ -111,7 +111,7 @@ impl<'a> JsonReader<'a> { }) } - pub fn finish(&self) -> DaftResult
{ + pub fn finish(&self) -> DaftResult { let mut bytes = self.bytes; let mut n_threads = self.n_threads; let mut total_rows = 128; @@ -148,7 +148,7 @@ impl<'a> JsonReader<'a> { let chunk = &bytes[start..stop]; self.parse_json_chunk(chunk, chunk_size) }) - .collect::>>() + .collect::>>() })?; let tbl = tables_concat(tbls)?; @@ -162,7 +162,7 @@ impl<'a> JsonReader<'a> { Ok(tbl) } - fn parse_json_chunk(&self, bytes: &[u8], chunk_size: usize) -> DaftResult
{ + fn parse_json_chunk(&self, bytes: &[u8], chunk_size: usize) -> DaftResult { let mut scratch = vec![]; let scratch = &mut scratch; @@ -229,7 +229,7 @@ impl<'a> JsonReader<'a> { }) .collect::>>()?; - let tbl = Table::new_unchecked(self.schema.clone(), columns, num_rows); + let tbl = RecordBatch::new_unchecked(self.schema.clone(), columns, num_rows); if let Some(pred) = &self.predicate { tbl.filter(&[pred.clone()]) diff --git a/src/daft-json/src/python.rs b/src/daft-json/src/python.rs index 01deebca3e..f40480c969 100644 --- a/src/daft-json/src/python.rs +++ b/src/daft-json/src/python.rs @@ -3,7 +3,7 @@ pub mod pylib { use daft_core::python::PySchema; use daft_io::{get_io_client, python::IOConfig, IOStatsContext}; - use daft_table::python::PyTable; + use daft_recordbatch::python::PyRecordBatch; use pyo3::{pyfunction, PyResult, Python}; use crate::{JsonConvertOptions, JsonParseOptions, JsonReadOptions}; @@ -27,7 +27,7 @@ pub mod pylib { io_config: Option, multithreaded_io: Option, max_chunks_in_flight: Option, - ) -> PyResult { + ) -> PyResult { py.allow_threads(|| { let io_stats = IOStatsContext::new(format!("read_json: for uri {uri}")); diff --git a/src/daft-json/src/read.rs b/src/daft-json/src/read.rs index 9503c77152..c7e4abe2bd 100644 --- a/src/daft-json/src/read.rs +++ b/src/daft-json/src/read.rs @@ -6,7 +6,7 @@ use daft_compression::CompressionCodec; use daft_core::{prelude::*, utils::arrow::cast_array_for_daft_if_needed}; use daft_dsl::optimization::get_required_columns; use daft_io::{parse_url, GetResult, IOClient, IOStatsRef, SourceType}; -use daft_table::Table; +use daft_recordbatch::RecordBatch; use futures::{stream::BoxStream, Stream, StreamExt, TryStreamExt}; use rayon::prelude::{IntoParallelIterator, ParallelIterator}; use snafu::{ @@ -26,7 +26,7 @@ use crate::{ }; type TableChunkResult = - super::Result>, super::JoinSnafu, super::Error>>; + super::Result>, super::JoinSnafu, super::Error>>; type LineChunkResult = super::Result>; @@ -46,7 +46,7 @@ pub fn read_json( io_stats: Option, multithreaded_io: bool, max_chunks_in_flight: Option, -) -> DaftResult
{ +) -> DaftResult { let runtime_handle = get_io_runtime(multithreaded_io); runtime_handle.block_on_current_thread(async { read_json_single_into_table( @@ -73,7 +73,7 @@ pub fn read_json_bulk( multithreaded_io: bool, max_chunks_in_flight: Option, num_parallel_tasks: usize, -) -> DaftResult> { +) -> DaftResult> { let runtime_handle = get_io_runtime(multithreaded_io); let tables = runtime_handle.block_on_current_thread(async move { // Launch a read task per URI, throttling the number of concurrent file reads to num_parallel tasks. @@ -131,7 +131,7 @@ pub fn read_json_bulk( // Parallel version of table concat // get rid of this once Table APIs are parallel -pub(crate) fn tables_concat(mut tables: Vec
) -> DaftResult
{ +pub(crate) fn tables_concat(mut tables: Vec) -> DaftResult { if tables.is_empty() { return Err(DaftError::ValueError( "Need at least 1 Table to perform concat".to_string(), @@ -162,10 +162,10 @@ pub(crate) fn tables_concat(mut tables: Vec
) -> DaftResult
{ Series::concat(series_to_cat.as_slice()) }) .collect::>>()?; - Table::new_with_size( + RecordBatch::new_with_size( first_table.schema.clone(), new_series, - tables.iter().map(daft_table::Table::len).sum(), + tables.iter().map(daft_recordbatch::RecordBatch::len).sum(), ) } @@ -177,7 +177,7 @@ async fn read_json_single_into_table( io_client: Arc, io_stats: Option, max_chunks_in_flight: Option, -) -> DaftResult
{ +) -> DaftResult { let (source_type, fixed_uri) = parse_url(uri)?; let is_compressed = CompressionCodec::from_uri(uri).is_some(); if matches!(source_type, SourceType::File) && !is_compressed { @@ -273,7 +273,7 @@ async fn read_json_single_into_table( // Handle empty table case. if collected_tables.is_empty() { let daft_schema = Arc::new(Schema::try_from(&schema)?); - return Table::empty(Some(daft_schema)); + return RecordBatch::empty(Some(daft_schema)); } // // TODO(Clark): Don't concatenate all chunks from a file into a single table, since MicroPartition is natively chunked. let concated_table = tables_concat(collected_tables)?; @@ -295,7 +295,7 @@ pub async fn stream_json( io_client: Arc, io_stats: Option, max_chunks_in_flight: Option, -) -> DaftResult>> { +) -> DaftResult>> { let predicate = convert_options .as_ref() .and_then(|opts| opts.predicate.clone()); @@ -545,7 +545,7 @@ fn parse_into_column_array_chunk_stream( ) }) .collect::>>()?; - Ok(Table::new_unchecked( + Ok(RecordBatch::new_unchecked( daft_schema.clone(), all_series, num_rows, @@ -569,7 +569,7 @@ mod tests { utils::arrow::{cast_array_for_daft_if_needed, cast_array_from_daft_if_needed}, }; use daft_io::{IOClient, IOConfig}; - use daft_table::Table; + use daft_recordbatch::RecordBatch; use indexmap::IndexMap; use rstest::rstest; @@ -582,7 +582,7 @@ mod tests { fn check_equal_local_arrow2( path: &str, - out: &Table, + out: &RecordBatch, limit: Option, projection: Option>, ) { diff --git a/src/daft-local-execution/Cargo.toml b/src/daft-local-execution/Cargo.toml index da11ef0039..e235b2d667 100644 --- a/src/daft-local-execution/Cargo.toml +++ b/src/daft-local-execution/Cargo.toml @@ -19,8 +19,8 @@ daft-logical-plan = {path = "../daft-logical-plan", default-features = false} daft-micropartition = {path = "../daft-micropartition", default-features = false} daft-parquet = {path = "../daft-parquet", default-features = false} daft-physical-plan = {path = "../daft-physical-plan", default-features = false} +daft-recordbatch = {path = "../daft-recordbatch", default-features = false} daft-scan = {path = "../daft-scan", default-features = false} -daft-table = {path = "../daft-table", default-features = false} daft-writers = {path = "../daft-writers", default-features = false} futures = {workspace = true} indexmap = {workspace = true} diff --git a/src/daft-local-execution/src/intermediate_ops/cross_join.rs b/src/daft-local-execution/src/intermediate_ops/cross_join.rs index c160cb7bd6..ff64a26faf 100644 --- a/src/daft-local-execution/src/intermediate_ops/cross_join.rs +++ b/src/daft-local-execution/src/intermediate_ops/cross_join.rs @@ -3,7 +3,7 @@ use std::sync::Arc; use common_error::DaftResult; use daft_core::{join::JoinSide, prelude::SchemaRef}; use daft_micropartition::MicroPartition; -use daft_table::Table; +use daft_recordbatch::RecordBatch; use tracing::{instrument, Span}; use super::intermediate_op::{ @@ -13,13 +13,13 @@ use super::intermediate_op::{ use crate::{state_bridge::BroadcastStateBridgeRef, ExecutionTaskSpawner}; struct CrossJoinState { - bridge: BroadcastStateBridgeRef>, + bridge: BroadcastStateBridgeRef>, stream_idx: usize, collect_idx: usize, } impl CrossJoinState { - fn new(bridge: BroadcastStateBridgeRef>) -> Self { + fn new(bridge: BroadcastStateBridgeRef>) -> Self { Self { bridge, stream_idx: 0, @@ -37,14 +37,14 @@ impl IntermediateOpState for CrossJoinState { pub struct CrossJoinOperator { output_schema: SchemaRef, stream_side: JoinSide, - state_bridge: BroadcastStateBridgeRef>, + state_bridge: BroadcastStateBridgeRef>, } impl CrossJoinOperator { pub(crate) fn new( output_schema: SchemaRef, stream_side: JoinSide, - state_bridge: BroadcastStateBridgeRef>, + state_bridge: BroadcastStateBridgeRef>, ) -> Self { Self { output_schema, diff --git a/src/daft-local-execution/src/intermediate_ops/inner_hash_join_probe.rs b/src/daft-local-execution/src/intermediate_ops/inner_hash_join_probe.rs index af8f387a4c..46bf1a7209 100644 --- a/src/daft-local-execution/src/intermediate_ops/inner_hash_join_probe.rs +++ b/src/daft-local-execution/src/intermediate_ops/inner_hash_join_probe.rs @@ -4,7 +4,7 @@ use common_error::DaftResult; use daft_core::prelude::SchemaRef; use daft_dsl::ExprRef; use daft_micropartition::MicroPartition; -use daft_table::{GrowableTable, ProbeState}; +use daft_recordbatch::{GrowableRecordBatch, ProbeState}; use indexmap::IndexSet; use itertools::Itertools; use tracing::{info_span, instrument, Span}; @@ -105,7 +105,7 @@ impl InnerHashJoinProbeOperator { let _growables = info_span!("InnerHashJoinOperator::build_growables").entered(); - let mut build_side_growable = GrowableTable::new( + let mut build_side_growable = GrowableRecordBatch::new( &tables.iter().collect::>(), false, Self::DEFAULT_GROWABLE_SIZE, @@ -113,7 +113,7 @@ impl InnerHashJoinProbeOperator { let input_tables = input.get_tables()?; - let mut probe_side_growable = GrowableTable::new( + let mut probe_side_growable = GrowableRecordBatch::new( &input_tables.iter().collect::>(), false, Self::DEFAULT_GROWABLE_SIZE, diff --git a/src/daft-local-execution/src/sinks/anti_semi_hash_join_probe.rs b/src/daft-local-execution/src/sinks/anti_semi_hash_join_probe.rs index b1c6ba49c0..88fc17c5a7 100644 --- a/src/daft-local-execution/src/sinks/anti_semi_hash_join_probe.rs +++ b/src/daft-local-execution/src/sinks/anti_semi_hash_join_probe.rs @@ -5,7 +5,7 @@ use daft_core::{prelude::SchemaRef, series::IntoSeries}; use daft_dsl::ExprRef; use daft_logical_plan::JoinType; use daft_micropartition::MicroPartition; -use daft_table::{GrowableTable, ProbeState, Probeable, Table}; +use daft_recordbatch::{GrowableRecordBatch, ProbeState, Probeable, RecordBatch}; use futures::{stream, StreamExt}; use itertools::Itertools; use tracing::{info_span, instrument, Span}; @@ -99,7 +99,7 @@ impl AntiSemiProbeSink { let input_tables = input.get_tables()?; - let mut probe_side_growable = GrowableTable::new( + let mut probe_side_growable = GrowableRecordBatch::new( &input_tables.iter().collect::>(), false, Self::DEFAULT_GROWABLE_SIZE, @@ -214,7 +214,7 @@ impl AntiSemiProbeSink { .map(|(bitmap, table)| table.mask_filter(&bitmap.into_series())) .collect::>>()?; - let build_side_table = Table::concat(&leftovers)?; + let build_side_table = RecordBatch::concat(&leftovers)?; Ok(Some(Arc::new(MicroPartition::new_loaded( build_side_table.schema.clone(), Arc::new(vec![build_side_table]), diff --git a/src/daft-local-execution/src/sinks/cross_join_collect.rs b/src/daft-local-execution/src/sinks/cross_join_collect.rs index c4d615806f..a7139cb7f9 100644 --- a/src/daft-local-execution/src/sinks/cross_join_collect.rs +++ b/src/daft-local-execution/src/sinks/cross_join_collect.rs @@ -2,7 +2,7 @@ use std::sync::Arc; use common_error::DaftResult; use daft_micropartition::MicroPartition; -use daft_table::Table; +use daft_recordbatch::RecordBatch; use tracing::{info_span, instrument}; use super::blocking_sink::{ @@ -11,7 +11,7 @@ use super::blocking_sink::{ }; use crate::{state_bridge::BroadcastStateBridgeRef, ExecutionTaskSpawner}; -struct CrossJoinCollectState(Option>); +struct CrossJoinCollectState(Option>); impl BlockingSinkState for CrossJoinCollectState { fn as_any_mut(&mut self) -> &mut dyn std::any::Any { @@ -20,11 +20,11 @@ impl BlockingSinkState for CrossJoinCollectState { } pub struct CrossJoinCollectSink { - state_bridge: BroadcastStateBridgeRef>, + state_bridge: BroadcastStateBridgeRef>, } impl CrossJoinCollectSink { - pub(crate) fn new(state_bridge: BroadcastStateBridgeRef>) -> Self { + pub(crate) fn new(state_bridge: BroadcastStateBridgeRef>) -> Self { Self { state_bridge } } } diff --git a/src/daft-local-execution/src/sinks/hash_join_build.rs b/src/daft-local-execution/src/sinks/hash_join_build.rs index 20d4a22182..44e988d942 100644 --- a/src/daft-local-execution/src/sinks/hash_join_build.rs +++ b/src/daft-local-execution/src/sinks/hash_join_build.rs @@ -4,7 +4,7 @@ use common_error::DaftResult; use daft_core::prelude::SchemaRef; use daft_dsl::ExprRef; use daft_micropartition::MicroPartition; -use daft_table::{make_probeable_builder, ProbeState, ProbeableBuilder, Table}; +use daft_recordbatch::{make_probeable_builder, ProbeState, ProbeableBuilder, RecordBatch}; use itertools::Itertools; use tracing::{info_span, instrument}; @@ -18,7 +18,7 @@ enum ProbeTableState { Building { probe_table_builder: Option>, projection: Vec, - tables: Vec
, + tables: Vec, }, Done, } @@ -51,7 +51,7 @@ impl ProbeTableState { let probe_table_builder = probe_table_builder.as_mut().unwrap(); let input_tables = input.get_tables()?; if input_tables.is_empty() { - tables.push(Table::empty(Some(input.schema()))?); + tables.push(RecordBatch::empty(Some(input.schema()))?); return Ok(()); } for table in input_tables.iter() { diff --git a/src/daft-local-execution/src/sinks/outer_hash_join_probe.rs b/src/daft-local-execution/src/sinks/outer_hash_join_probe.rs index 847a8c9e6a..0fe088e56c 100644 --- a/src/daft-local-execution/src/sinks/outer_hash_join_probe.rs +++ b/src/daft-local-execution/src/sinks/outer_hash_join_probe.rs @@ -10,7 +10,7 @@ use daft_core::{ use daft_dsl::ExprRef; use daft_logical_plan::JoinType; use daft_micropartition::MicroPartition; -use daft_table::{GrowableTable, ProbeState, Table}; +use daft_recordbatch::{GrowableRecordBatch, ProbeState, RecordBatch}; use futures::{stream, StreamExt}; use indexmap::IndexSet; use itertools::Itertools; @@ -31,7 +31,7 @@ pub(crate) struct IndexBitmapBuilder { } impl IndexBitmapBuilder { - pub fn new(tables: &[Table]) -> Self { + pub fn new(tables: &[RecordBatch]) -> Self { Self { mutable_bitmaps: tables .iter() @@ -222,7 +222,7 @@ impl OuterHashJoinProbeSink { let tables = probe_state.get_tables(); let _growables = info_span!("OuterHashJoinProbeSink::build_growables").entered(); - let mut build_side_growable = GrowableTable::new( + let mut build_side_growable = GrowableRecordBatch::new( &tables.iter().collect::>(), false, tables.iter().map(|t| t.len()).sum(), @@ -230,7 +230,7 @@ impl OuterHashJoinProbeSink { let input_tables = input.get_tables()?; let mut probe_side_growable = - GrowableTable::new(&input_tables.iter().collect::>(), false, input.len())?; + GrowableRecordBatch::new(&input_tables.iter().collect::>(), false, input.len())?; drop(_growables); { @@ -289,7 +289,7 @@ impl OuterHashJoinProbeSink { let tables = probe_state.get_tables(); let _growables = info_span!("OuterHashJoinProbeSink::build_growables").entered(); - let mut build_side_growable = GrowableTable::new( + let mut build_side_growable = GrowableRecordBatch::new( &tables.iter().collect::>(), true, tables.iter().map(|t| t.len()).sum(), @@ -297,7 +297,7 @@ impl OuterHashJoinProbeSink { let input_tables = input.get_tables()?; let mut probe_side_growable = - GrowableTable::new(&input_tables.iter().collect::>(), false, input.len())?; + GrowableRecordBatch::new(&input_tables.iter().collect::>(), false, input.len())?; drop(_growables); { @@ -362,7 +362,7 @@ impl OuterHashJoinProbeSink { let _growables = info_span!("OuterHashJoinProbeSink::build_growables").entered(); // Need to set use_validity to true here because we add nulls to the build side - let mut build_side_growable = GrowableTable::new( + let mut build_side_growable = GrowableRecordBatch::new( &tables.iter().collect::>(), true, tables.iter().map(|t| t.len()).sum(), @@ -370,7 +370,7 @@ impl OuterHashJoinProbeSink { let input_tables = input.get_tables()?; let mut probe_side_growable = - GrowableTable::new(&input_tables.iter().collect::>(), false, input.len())?; + GrowableRecordBatch::new(&input_tables.iter().collect::>(), false, input.len())?; drop(_growables); { @@ -420,7 +420,7 @@ impl OuterHashJoinProbeSink { async fn merge_bitmaps_and_construct_null_table( mut states: Vec>, - ) -> DaftResult
{ + ) -> DaftResult { let mut states_iter = states.iter_mut(); let first_state = states_iter .next() @@ -470,7 +470,7 @@ impl OuterHashJoinProbeSink { .map(|(bitmap, table)| table.mask_filter(&bitmap.into_series())) .collect::>>()?; - Table::concat(&leftovers) + RecordBatch::concat(&leftovers) } async fn finalize_outer( @@ -492,7 +492,7 @@ impl OuterHashJoinProbeSink { .values() .map(|field| Series::full_null(&field.name, &field.dtype, left.len())) .collect::>(); - Table::new_unchecked(right_non_join_schema.clone(), columns, left.len()) + RecordBatch::new_unchecked(right_non_join_schema.clone(), columns, left.len()) }; // If we built the probe table on the right, flip the order of union. let (left, right) = if build_on_left { @@ -523,7 +523,7 @@ impl OuterHashJoinProbeSink { .values() .map(|field| Series::full_null(&field.name, &field.dtype, left.len())) .collect::>(); - Table::new_unchecked(right_non_join_schema.clone(), columns, left.len()) + RecordBatch::new_unchecked(right_non_join_schema.clone(), columns, left.len()) }; let final_table = join_table.union(&left)?.union(&right)?; Ok(Some(Arc::new(MicroPartition::new_loaded( @@ -547,7 +547,7 @@ impl OuterHashJoinProbeSink { .values() .map(|field| Series::full_null(&field.name, &field.dtype, build_side_table.len())) .collect::>(); - Table::new_unchecked( + RecordBatch::new_unchecked( left_non_join_schema.clone(), columns, build_side_table.len(), diff --git a/src/daft-local-execution/src/sinks/write.rs b/src/daft-local-execution/src/sinks/write.rs index 3ae84b0e87..d67ebc213c 100644 --- a/src/daft-local-execution/src/sinks/write.rs +++ b/src/daft-local-execution/src/sinks/write.rs @@ -4,7 +4,7 @@ use common_error::DaftResult; use daft_core::prelude::SchemaRef; use daft_dsl::ExprRef; use daft_micropartition::MicroPartition; -use daft_table::Table; +use daft_recordbatch::RecordBatch; use daft_writers::{FileWriter, WriterFactory}; use tracing::{instrument, Span}; @@ -31,12 +31,12 @@ pub enum WriteFormat { } struct WriteState { - writer: Box, Result = Vec
>>, + writer: Box, Result = Vec>>, } impl WriteState { pub fn new( - writer: Box, Result = Vec
>>, + writer: Box, Result = Vec>>, ) -> Self { Self { writer } } @@ -50,7 +50,7 @@ impl BlockingSinkState for WriteState { pub(crate) struct WriteSink { write_format: WriteFormat, - writer_factory: Arc, Result = Vec
>>, + writer_factory: Arc, Result = Vec>>, partition_by: Option>, file_schema: SchemaRef, } @@ -58,7 +58,9 @@ pub(crate) struct WriteSink { impl WriteSink { pub(crate) fn new( write_format: WriteFormat, - writer_factory: Arc, Result = Vec
>>, + writer_factory: Arc< + dyn WriterFactory, Result = Vec>, + >, partition_by: Option>, file_schema: SchemaRef, ) -> Self { diff --git a/src/daft-micropartition/Cargo.toml b/src/daft-micropartition/Cargo.toml index b9bbd35ca7..636abaa742 100644 --- a/src/daft-micropartition/Cargo.toml +++ b/src/daft-micropartition/Cargo.toml @@ -12,9 +12,9 @@ daft-dsl = {path = "../daft-dsl", default-features = false} daft-io = {path = "../daft-io", default-features = false} daft-json = {path = "../daft-json", default-features = false} daft-parquet = {path = "../daft-parquet", default-features = false} +daft-recordbatch = {path = "../daft-recordbatch", default-features = false} daft-scan = {path = "../daft-scan", default-features = false} daft-stats = {path = "../daft-stats", default-features = false} -daft-table = {path = "../daft-table", default-features = false} dashmap = "6.1.0" futures = {workspace = true} parquet2 = {workspace = true} @@ -32,7 +32,7 @@ python = [ "common-scan-info/python", "daft-core/python", "daft-dsl/python", - "daft-table/python", + "daft-recordbatch/python", "daft-io/python", "daft-parquet/python", "daft-scan/python", diff --git a/src/daft-micropartition/src/micropartition.rs b/src/daft-micropartition/src/micropartition.rs index 0edc5ea144..743713f4b0 100644 --- a/src/daft-micropartition/src/micropartition.rs +++ b/src/daft-micropartition/src/micropartition.rs @@ -21,9 +21,9 @@ use daft_json::{JsonConvertOptions, JsonParseOptions, JsonReadOptions}; use daft_parquet::read::{ read_parquet_bulk, read_parquet_metadata_bulk, ParquetSchemaInferenceOptions, }; +use daft_recordbatch::RecordBatch; use daft_scan::{storage_config::StorageConfig, ChunkSpec, DataSource, ScanTask}; use daft_stats::{PartitionSpec, TableMetadata, TableStatistics}; -use daft_table::Table; use futures::{Future, Stream}; use parquet2::metadata::FileMetaData; use snafu::ResultExt; @@ -33,7 +33,7 @@ use crate::{DaftCSVSnafu, DaftCoreComputeSnafu}; #[derive(Debug)] pub enum TableState { Unloaded(Arc), - Loaded(Arc>), + Loaded(Arc>), } impl Display for TableState { @@ -87,7 +87,7 @@ pub struct MicroPartition { pub(crate) statistics: Option, } -/// Helper to run all the IO and compute required to materialize a [`ScanTask`] into a `Vec
` +/// Helper to run all the IO and compute required to materialize a [`ScanTask`] into a `Vec` /// /// All [`Table`] objects returned will have the same [`Schema`] as [`ScanTask::materialized_schema`]. /// @@ -98,7 +98,7 @@ pub struct MicroPartition { fn materialize_scan_task( scan_task: Arc, io_stats: Option, -) -> crate::Result<(Vec
, SchemaRef)> { +) -> crate::Result<(Vec, SchemaRef)> { let pushdown_columns = scan_task.pushdowns.columns.as_ref().map(|v| { v.iter() .map(std::string::String::as_str) @@ -352,11 +352,11 @@ impl MicroPartition { #[must_use] pub fn new_loaded( schema: SchemaRef, - tables: Arc>, + record_batches: Arc>, statistics: Option, ) -> Self { // Check and validate invariants with asserts - for table in tables.iter() { + for table in record_batches.iter() { assert!( table.schema == schema, "Loaded MicroPartition's tables' schema must match its own schema exactly" @@ -368,11 +368,14 @@ impl MicroPartition { .cast_to_schema(schema.clone()) .expect("Statistics cannot be casted to schema") }); - let tables_len_sum = tables.iter().map(daft_table::Table::len).sum(); + let tables_len_sum = record_batches + .iter() + .map(daft_recordbatch::RecordBatch::len) + .sum(); Self { schema, - state: Mutex::new(TableState::Loaded(tables)), + state: Mutex::new(TableState::Loaded(record_batches)), metadata: TableMetadata { length: tables_len_sum, }, @@ -508,7 +511,7 @@ impl MicroPartition { let size_bytes = if let TableState::Loaded(tables) = &*guard { let total_size: usize = tables .iter() - .map(daft_table::Table::size_bytes) + .map(daft_recordbatch::RecordBatch::size_bytes) .collect::>>()? .iter() .sum(); @@ -532,7 +535,10 @@ impl MicroPartition { /// /// "Reading if necessary" means I/O operations only occur for unloaded data, /// optimizing performance by avoiding redundant reads. - pub(crate) fn tables_or_read(&self, io_stats: IOStatsRef) -> crate::Result>> { + pub(crate) fn tables_or_read( + &self, + io_stats: IOStatsRef, + ) -> crate::Result>> { let mut guard = self.state.lock().unwrap(); match &*guard { TableState::Unloaded(scan_task) => { @@ -548,12 +554,12 @@ impl MicroPartition { } } - pub fn get_tables(&self) -> crate::Result>> { + pub fn get_tables(&self) -> crate::Result>> { let tables = self.tables_or_read(IOStatsContext::new("get tables"))?; Ok(tables) } - pub fn concat_or_get(&self, io_stats: IOStatsRef) -> crate::Result>> { + pub fn concat_or_get(&self, io_stats: IOStatsRef) -> crate::Result>> { let tables = self.tables_or_read(io_stats)?; if tables.len() <= 1 { return Ok(tables); @@ -562,7 +568,7 @@ impl MicroPartition { let mut guard = self.state.lock().unwrap(); if tables.len() > 1 { - let new_table = Table::concat(tables.iter().collect::>().as_slice()) + let new_table = RecordBatch::concat(tables.iter().collect::>().as_slice()) .context(DaftCoreComputeSnafu)?; *guard = TableState::Loaded(Arc::new(vec![new_table])); }; @@ -1193,11 +1199,11 @@ impl Display for MicroPartition { struct MicroPartitionStreamAdapter { state: TableState, current: usize, - pending_task: Option>>>, + pending_task: Option>>>, } impl Stream for MicroPartitionStreamAdapter { - type Item = DaftResult
; + type Item = DaftResult; fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let this = self.get_mut(); @@ -1246,7 +1252,7 @@ impl Stream for MicroPartitionStreamAdapter { } } impl MicroPartition { - pub fn into_stream(self: Arc) -> DaftResult>> { + pub fn into_stream(self: Arc) -> DaftResult>> { let state = match &*self.state.lock().unwrap() { TableState::Unloaded(scan_task) => TableState::Unloaded(scan_task.clone()), TableState::Loaded(tables) => TableState::Loaded(tables.clone()), @@ -1270,7 +1276,7 @@ mod tests { prelude::Schema, series::IntoSeries, }; - use daft_table::Table; + use daft_recordbatch::RecordBatch; use futures::StreamExt; use crate::MicroPartition; @@ -1281,8 +1287,8 @@ mod tests { let columns2 = vec![Int32Array::from_values("a", vec![2].into_iter()).into_series()]; let schema = Schema::new(vec![Field::new("a", DataType::Int32)])?; - let table1 = Table::from_nonempty_columns(columns)?; - let table2 = Table::from_nonempty_columns(columns2)?; + let table1 = RecordBatch::from_nonempty_columns(columns)?; + let table2 = RecordBatch::from_nonempty_columns(columns2)?; let mp = MicroPartition::new_loaded( Arc::new(schema), diff --git a/src/daft-micropartition/src/ops/agg.rs b/src/daft-micropartition/src/ops/agg.rs index 18f265c3eb..39f1f7f660 100644 --- a/src/daft-micropartition/src/ops/agg.rs +++ b/src/daft-micropartition/src/ops/agg.rs @@ -1,7 +1,7 @@ use common_error::DaftResult; use daft_dsl::ExprRef; use daft_io::IOStatsContext; -use daft_table::Table; +use daft_recordbatch::RecordBatch; use crate::micropartition::MicroPartition; @@ -13,7 +13,7 @@ impl MicroPartition { match tables.as_slice() { [] => { - let empty_table = Table::empty(Some(self.schema.clone()))?; + let empty_table = RecordBatch::empty(Some(self.schema.clone()))?; let agged = empty_table.agg(to_agg, group_by)?; Ok(Self::new_loaded( agged.schema.clone(), diff --git a/src/daft-micropartition/src/ops/concat.rs b/src/daft-micropartition/src/ops/concat.rs index 2ca6160178..1235226fb1 100644 --- a/src/daft-micropartition/src/ops/concat.rs +++ b/src/daft-micropartition/src/ops/concat.rs @@ -53,7 +53,10 @@ impl MicroPartition { all_stats = Some(curr_stats.union(stats)?); } } - let new_len = all_tables.iter().map(daft_table::Table::len).sum(); + let new_len = all_tables + .iter() + .map(daft_recordbatch::RecordBatch::len) + .sum(); Ok(Self { schema: first_schema.clone(), diff --git a/src/daft-micropartition/src/ops/join.rs b/src/daft-micropartition/src/ops/join.rs index 7438706a42..d51a239a17 100644 --- a/src/daft-micropartition/src/ops/join.rs +++ b/src/daft-micropartition/src/ops/join.rs @@ -7,8 +7,8 @@ use daft_core::{ }; use daft_dsl::{join::infer_join_schema, ExprRef}; use daft_io::IOStatsContext; +use daft_recordbatch::RecordBatch; use daft_stats::TruthValue; -use daft_table::Table; use crate::micropartition::MicroPartition; @@ -23,7 +23,13 @@ impl MicroPartition { table_join: F, ) -> DaftResult where - F: FnOnce(&Table, &Table, &[ExprRef], &[ExprRef], JoinType) -> DaftResult
, + F: FnOnce( + &RecordBatch, + &RecordBatch, + &[ExprRef], + &[ExprRef], + JoinType, + ) -> DaftResult, { let join_schema = infer_join_schema(&self.schema, &right.schema, how)?; match (how, self.len(), right.len()) { @@ -91,8 +97,8 @@ impl MicroPartition { let io_stats = IOStatsContext::new("MicroPartition::hash_join"); let null_equals_nulls = null_equals_nulls.unwrap_or_else(|| vec![false; left_on.len()]); let table_join = - |lt: &Table, rt: &Table, lo: &[ExprRef], ro: &[ExprRef], _how: JoinType| { - Table::hash_join(lt, rt, lo, ro, null_equals_nulls.as_slice(), _how) + |lt: &RecordBatch, rt: &RecordBatch, lo: &[ExprRef], ro: &[ExprRef], _how: JoinType| { + RecordBatch::hash_join(lt, rt, lo, ro, null_equals_nulls.as_slice(), _how) }; self.join(right, io_stats, left_on, right_on, how, table_join) @@ -107,8 +113,8 @@ impl MicroPartition { ) -> DaftResult { let io_stats = IOStatsContext::new("MicroPartition::sort_merge_join"); let table_join = - |lt: &Table, rt: &Table, lo: &[ExprRef], ro: &[ExprRef], _how: JoinType| { - Table::sort_merge_join(lt, rt, lo, ro, is_sorted) + |lt: &RecordBatch, rt: &RecordBatch, lo: &[ExprRef], ro: &[ExprRef], _how: JoinType| { + RecordBatch::sort_merge_join(lt, rt, lo, ro, is_sorted) }; self.join( @@ -124,9 +130,10 @@ impl MicroPartition { pub fn cross_join(&self, right: &Self, outer_loop_side: JoinSide) -> DaftResult { let io_stats = IOStatsContext::new("MicroPartition::cross_join"); - let table_join = |lt: &Table, rt: &Table, _: &[ExprRef], _: &[ExprRef], _: JoinType| { - Table::cross_join(lt, rt, outer_loop_side) - }; + let table_join = + |lt: &RecordBatch, rt: &RecordBatch, _: &[ExprRef], _: &[ExprRef], _: JoinType| { + RecordBatch::cross_join(lt, rt, outer_loop_side) + }; self.join(right, io_stats, &[], &[], JoinType::Inner, table_join) } diff --git a/src/daft-micropartition/src/ops/partition.rs b/src/daft-micropartition/src/ops/partition.rs index 14f20bd11d..f0fb643c97 100644 --- a/src/daft-micropartition/src/ops/partition.rs +++ b/src/daft-micropartition/src/ops/partition.rs @@ -3,7 +3,7 @@ use std::sync::Arc; use common_error::DaftResult; use daft_dsl::ExprRef; use daft_io::IOStatsContext; -use daft_table::Table; +use daft_recordbatch::RecordBatch; use crate::micropartition::MicroPartition; @@ -27,7 +27,7 @@ fn transpose2(v: Vec>) -> Vec> { } impl MicroPartition { - fn vec_part_tables_to_mps(&self, part_tables: Vec>) -> DaftResult> { + fn vec_part_tables_to_mps(&self, part_tables: Vec>) -> DaftResult> { let part_tables = transpose2(part_tables); Ok(part_tables .into_iter() @@ -83,7 +83,7 @@ impl MicroPartition { pub fn partition_by_range( &self, partition_keys: &[ExprRef], - boundaries: &Table, + boundaries: &RecordBatch, descending: &[bool], ) -> DaftResult> { let io_stats = IOStatsContext::new("MicroPartition::partition_by_range"); diff --git a/src/daft-micropartition/src/ops/pivot.rs b/src/daft-micropartition/src/ops/pivot.rs index 15ff085382..1e97a1a4ab 100644 --- a/src/daft-micropartition/src/ops/pivot.rs +++ b/src/daft-micropartition/src/ops/pivot.rs @@ -1,7 +1,7 @@ use common_error::{DaftError, DaftResult}; use daft_dsl::ExprRef; use daft_io::IOStatsContext; -use daft_table::Table; +use daft_recordbatch::RecordBatch; use crate::micropartition::MicroPartition; @@ -19,7 +19,7 @@ impl MicroPartition { match tables.as_slice() { [] => { - let empty_table = Table::empty(Some(self.schema.clone()))?; + let empty_table = RecordBatch::empty(Some(self.schema.clone()))?; let pivoted = empty_table.pivot(group_by, pivot_col, values_col, names)?; Ok(Self::empty(Some(pivoted.schema))) } diff --git a/src/daft-micropartition/src/ops/sort.rs b/src/daft-micropartition/src/ops/sort.rs index f2be0bb38e..f33af3b8a0 100644 --- a/src/daft-micropartition/src/ops/sort.rs +++ b/src/daft-micropartition/src/ops/sort.rs @@ -4,7 +4,7 @@ use common_error::DaftResult; use daft_core::series::Series; use daft_dsl::ExprRef; use daft_io::IOStatsContext; -use daft_table::Table; +use daft_recordbatch::RecordBatch; use crate::micropartition::MicroPartition; @@ -43,7 +43,7 @@ impl MicroPartition { let tables = self.concat_or_get(io_stats)?; match tables.as_slice() { [] => { - let empty_table = Table::empty(Some(self.schema.clone()))?; + let empty_table = RecordBatch::empty(Some(self.schema.clone()))?; empty_table.argsort(sort_keys, descending, nulls_first) } [single] => single.argsort(sort_keys, descending, nulls_first), diff --git a/src/daft-micropartition/src/ops/take.rs b/src/daft-micropartition/src/ops/take.rs index 4476ac5f82..de204e7945 100644 --- a/src/daft-micropartition/src/ops/take.rs +++ b/src/daft-micropartition/src/ops/take.rs @@ -3,7 +3,7 @@ use std::sync::Arc; use common_error::DaftResult; use daft_core::series::Series; use daft_io::IOStatsContext; -use daft_table::Table; +use daft_recordbatch::RecordBatch; use crate::micropartition::MicroPartition; @@ -19,7 +19,7 @@ impl MicroPartition { match tables.as_slice() { // Fallback onto `[empty_table]` behavior [] => { - let empty_table = Table::empty(Some(self.schema.clone()))?; + let empty_table = RecordBatch::empty(Some(self.schema.clone()))?; let taken = empty_table.take(idx)?; Ok(Self::new_loaded( self.schema.clone(), diff --git a/src/daft-micropartition/src/partitioning.rs b/src/daft-micropartition/src/partitioning.rs index a2d8d19c00..5b7982b09a 100644 --- a/src/daft-micropartition/src/partitioning.rs +++ b/src/daft-micropartition/src/partitioning.rs @@ -5,7 +5,7 @@ use std::{ use common_error::{DaftError, DaftResult}; pub use common_partitioning::*; -use daft_table::Table; +use daft_recordbatch::RecordBatch; use dashmap::DashMap; use futures::stream::BoxStream; @@ -48,7 +48,7 @@ impl MicroPartitionSet { Self::default() } - pub fn from_tables(id: PartitionId, tables: Vec
) -> DaftResult { + pub fn from_tables(id: PartitionId, tables: Vec) -> DaftResult { if tables.is_empty() { return Ok(Self::empty()); } diff --git a/src/daft-micropartition/src/python.rs b/src/daft-micropartition/src/python.rs index 53a302dc3e..d17abdb16c 100644 --- a/src/daft-micropartition/src/python.rs +++ b/src/daft-micropartition/src/python.rs @@ -12,11 +12,11 @@ use daft_dsl::python::PyExpr; use daft_io::{python::IOConfig, IOStatsContext}; use daft_json::{JsonConvertOptions, JsonParseOptions, JsonReadOptions}; use daft_parquet::read::ParquetSchemaInferenceOptions; +use daft_recordbatch::{python::PyRecordBatch, RecordBatch}; use daft_scan::{ python::pylib::PyScanTask, storage_config::StorageConfig, DataSource, ScanTask, ScanTaskRef, }; use daft_stats::{TableMetadata, TableStatistics}; -use daft_table::{python::PyTable, Table}; use pyo3::{exceptions::PyValueError, prelude::*, types::PyBytes, PyTypeInfo}; use snafu::ResultExt; @@ -89,7 +89,7 @@ impl PyMicroPartition { } #[staticmethod] - pub fn from_tables(tables: Vec) -> PyResult { + pub fn from_tables(tables: Vec) -> PyResult { match &tables[..] { [] => Ok(MicroPartition::empty(None).into()), [first, ..] => { @@ -124,21 +124,23 @@ impl PyMicroPartition { // TODO: Cleanup and refactor code for sharing with Table let tables = record_batches .into_iter() - .map(|rb| daft_table::ffi::record_batches_to_table(py, &[rb], schema.schema.clone())) + .map(|rb| { + daft_recordbatch::ffi::record_batches_to_table(py, &[rb], schema.schema.clone()) + }) .collect::>>()?; Ok(MicroPartition::new_loaded(schema.schema.clone(), Arc::new(tables), None).into()) } // Export Methods - pub fn to_table(&self, py: Python) -> PyResult { + pub fn to_table(&self, py: Python) -> PyResult { let concatted = py.allow_threads(|| { let io_stats = IOStatsContext::new("PyMicroPartition::to_table"); self.inner.concat_or_get(io_stats) })?; match &concatted.as_ref()[..] { - [] => PyTable::empty(Some(self.schema()?)), - [table] => Ok(PyTable { + [] => PyRecordBatch::empty(Some(self.schema()?)), + [table] => Ok(PyRecordBatch { table: table.clone(), }), [..] => unreachable!("concat_or_get should return one or none"), @@ -489,7 +491,7 @@ impl PyMicroPartition { &self, py: Python, partition_keys: Vec, - boundaries: &PyTable, + boundaries: &PyRecordBatch, descending: Vec, ) -> PyResult> { let exprs: Vec = partition_keys @@ -796,7 +798,7 @@ impl PyMicroPartition { .into_iter() .map(|p| { Ok(p.getattr(py, pyo3::intern!(py, "_table"))? - .extract::(py)? + .extract::(py)? .table) }) .collect::>>()?; @@ -820,11 +822,11 @@ impl PyMicroPartition { let guard = self.inner.state.lock().unwrap(); if let TableState::Loaded(tables) = &*guard { let _from_pytable = py - .import(pyo3::intern!(py, "daft.table"))? - .getattr(pyo3::intern!(py, "Table"))? + .import(pyo3::intern!(py, "daft.recordbatch"))? + .getattr(pyo3::intern!(py, "RecordBatch"))? .getattr(pyo3::intern!(py, "_from_pytable"))?; - let pytables = tables.iter().map(|t| PyTable { table: t.clone() }); + let pytables = tables.iter().map(|t| PyRecordBatch { table: t.clone() }); let pyobjs = pytables .map(|pt| _from_pytable.call1((pt,))) .collect::>>()?; @@ -864,7 +866,7 @@ pub fn read_json_into_py_table( storage_config: StorageConfig, include_columns: Option>, num_rows: Option, -) -> PyResult { +) -> PyResult { let read_options = py .import(pyo3::intern!(py, "daft.runners.partitioning"))? .getattr(pyo3::intern!(py, "TableReadOptions"))? @@ -874,7 +876,7 @@ pub fn read_json_into_py_table( .getattr(pyo3::intern!(py, "Schema"))? .getattr(pyo3::intern!(py, "_from_pyschema"))? .call1((schema,))?; - py.import(pyo3::intern!(py, "daft.table.table_io"))? + py.import(pyo3::intern!(py, "daft.recordbatch.recordbatch_io"))? .getattr(pyo3::intern!(py, "read_json"))? .call1((uri, py_schema, storage_config, read_options))? .getattr(pyo3::intern!(py, "to_table"))? @@ -894,7 +896,7 @@ pub fn read_csv_into_py_table( storage_config: StorageConfig, include_columns: Option>, num_rows: Option, -) -> PyResult { +) -> PyResult { let py_schema = py .import(pyo3::intern!(py, "daft.logical.schema"))? .getattr(pyo3::intern!(py, "Schema"))? @@ -909,7 +911,7 @@ pub fn read_csv_into_py_table( .import(pyo3::intern!(py, "daft.runners.partitioning"))? .getattr(pyo3::intern!(py, "TableParseCSVOptions"))? .call1((delimiter, header_idx, double_quote))?; - py.import(pyo3::intern!(py, "daft.table.table_io"))? + py.import(pyo3::intern!(py, "daft.recordbatch.recordbatch_io"))? .getattr(pyo3::intern!(py, "read_csv"))? .call1((uri, py_schema, storage_config, parse_options, read_options))? .getattr(pyo3::intern!(py, "to_table"))? @@ -926,7 +928,7 @@ pub fn read_parquet_into_py_table( storage_config: StorageConfig, include_columns: Option>, num_rows: Option, -) -> PyResult { +) -> PyResult { let py_schema = py .import(pyo3::intern!(py, "daft.logical.schema"))? .getattr(pyo3::intern!(py, "Schema"))? @@ -945,7 +947,7 @@ pub fn read_parquet_into_py_table( .import(pyo3::intern!(py, "daft.runners.partitioning"))? .getattr(pyo3::intern!(py, "TableParseParquetOptions"))? .call1((py_coerce_int96_timestamp_unit,))?; - py.import(pyo3::intern!(py, "daft.table.table_io"))? + py.import(pyo3::intern!(py, "daft.recordbatch.recordbatch_io"))? .getattr(pyo3::intern!(py, "read_parquet"))? .call1((uri, py_schema, storage_config, read_options, parse_options))? .getattr(pyo3::intern!(py, "to_table"))? @@ -962,7 +964,7 @@ pub fn read_sql_into_py_table( schema: PySchema, include_columns: Option>, num_rows: Option, -) -> PyResult { +) -> PyResult { let py_schema = py .import(pyo3::intern!(py, "daft.logical.schema"))? .getattr(pyo3::intern!(py, "Schema"))? @@ -981,7 +983,7 @@ pub fn read_sql_into_py_table( .import(pyo3::intern!(py, "daft.runners.partitioning"))? .getattr(pyo3::intern!(py, "TableReadOptions"))? .call1((num_rows, include_columns))?; - py.import(pyo3::intern!(py, "daft.table.table_io"))? + py.import(pyo3::intern!(py, "daft.recordbatch.recordbatch_io"))? .getattr(pyo3::intern!(py, "read_sql"))? .call1((sql, conn, py_schema, read_options, py_predicate))? .getattr(pyo3::intern!(py, "to_table"))? @@ -992,7 +994,7 @@ pub fn read_sql_into_py_table( pub fn read_pyfunc_into_table_iter( scan_task: &ScanTaskRef, -) -> crate::Result>> { +) -> crate::Result>> { let table_iterators = scan_task.sources.iter().map(|source| { // Call Python function to create an Iterator (Grabs the GIL and then releases it) match source { @@ -1030,8 +1032,8 @@ pub fn read_pyfunc_into_table_iter( .map(|result| { result .map(|tbl| { - tbl.extract::() - .expect("Must be a PyTable") + tbl.extract::() + .expect("Must be a PyRecordBatch") .table }) .with_context(|_| PyIOSnafu) @@ -1050,7 +1052,7 @@ pub fn read_pyfunc_into_table_iter( Err(e) => Some(Err(e)), Ok(table) => { // Apply filters - let post_pushdown_table = || -> crate::Result
{ + let post_pushdown_table = || -> crate::Result { let table = if let Some(filters) = scan_task_filters.as_ref() { table .filter(&[filters.clone()]) diff --git a/src/daft-parquet/Cargo.toml b/src/daft-parquet/Cargo.toml index 317cf48b29..2737838639 100644 --- a/src/daft-parquet/Cargo.toml +++ b/src/daft-parquet/Cargo.toml @@ -9,8 +9,8 @@ common-runtime = {path = "../common/runtime", default-features = false} daft-core = {path = "../daft-core", default-features = false} daft-dsl = {path = "../daft-dsl", default-features = false} daft-io = {path = "../daft-io", default-features = false} +daft-recordbatch = {path = "../daft-recordbatch", default-features = false} daft-stats = {path = "../daft-stats", default-features = false} -daft-table = {path = "../daft-table", default-features = false} futures = {workspace = true} indexmap = {workspace = true} itertools = {workspace = true} @@ -29,7 +29,7 @@ bincode = {workspace = true} path_macro = {workspace = true} [features] -python = ["dep:pyo3", "common-error/python", "daft-core/python", "daft-io/python", "daft-table/python", "daft-stats/python", "daft-dsl/python", "common-arrow-ffi/python"] +python = ["dep:pyo3", "common-error/python", "daft-core/python", "daft-io/python", "daft-recordbatch/python", "daft-stats/python", "daft-dsl/python", "common-arrow-ffi/python"] [lints] workspace = true diff --git a/src/daft-parquet/src/file.rs b/src/daft-parquet/src/file.rs index 024a7463fd..88b03faa52 100644 --- a/src/daft-parquet/src/file.rs +++ b/src/daft-parquet/src/file.rs @@ -10,8 +10,8 @@ use common_runtime::get_compute_runtime; use daft_core::{prelude::*, utils::arrow::cast_array_for_daft_if_needed}; use daft_dsl::ExprRef; use daft_io::{IOClient, IOStatsRef}; +use daft_recordbatch::RecordBatch; use daft_stats::TruthValue; -use daft_table::Table; use futures::{future::try_join_all, stream::BoxStream, StreamExt}; use parquet2::{ page::{CompressedPage, Page}, @@ -405,7 +405,7 @@ impl ParquetFileReader { original_columns: Option>, original_num_rows: Option, delete_rows: Option>, - ) -> DaftResult>> { + ) -> DaftResult>> { let daft_schema = Arc::new(daft_core::prelude::Schema::try_from( self.arrow_schema.as_ref(), )?); @@ -556,7 +556,7 @@ impl ParquetFileReader { pub async fn read_from_ranges_into_table( self, ranges: Arc, - ) -> DaftResult
{ + ) -> DaftResult { let metadata = self.metadata; let all_handles = self .arrow_schema @@ -731,7 +731,7 @@ impl ParquetFileReader { .into_iter() .collect::>>()?; - Table::new_with_size( + RecordBatch::new_with_size( Schema::new(all_series.iter().map(|s| s.field().clone()).collect())?, all_series, self.row_ranges.as_ref().iter().map(|rr| rr.num_rows).sum(), diff --git a/src/daft-parquet/src/python.rs b/src/daft-parquet/src/python.rs index 88763798fc..c5250054c7 100644 --- a/src/daft-parquet/src/python.rs +++ b/src/daft-parquet/src/python.rs @@ -7,7 +7,7 @@ pub mod pylib { use daft_core::python::{PySchema, PySeries, PyTimeUnit}; use daft_dsl::python::PyExpr; use daft_io::{get_io_client, python::IOConfig, IOStatsContext}; - use daft_table::python::PyTable; + use daft_recordbatch::python::PyRecordBatch; use pyo3::{pyfunction, types::PyModule, Bound, PyResult, Python}; use crate::read::{ @@ -36,7 +36,7 @@ pub mod pylib { io_config: Option, multithreaded_io: Option, coerce_int96_timestamp_unit: Option, - ) -> PyResult { + ) -> PyResult { py.allow_threads(|| { let io_stats = IOStatsContext::new(format!("read_parquet: for uri {uri}")); @@ -175,7 +175,7 @@ pub mod pylib { num_parallel_tasks: Option, multithreaded_io: Option, coerce_int96_timestamp_unit: Option, - ) -> PyResult> { + ) -> PyResult> { py.allow_threads(|| { let io_stats = IOStatsContext::new("read_parquet_bulk"); @@ -313,7 +313,7 @@ pub mod pylib { uris: PySeries, io_config: Option, multithreaded_io: Option, - ) -> PyResult { + ) -> PyResult { py.allow_threads(|| { let io_stats = IOStatsContext::new("read_parquet_statistics"); diff --git a/src/daft-parquet/src/read.rs b/src/daft-parquet/src/read.rs index 10a018c113..220bb5bde3 100644 --- a/src/daft-parquet/src/read.rs +++ b/src/daft-parquet/src/read.rs @@ -17,7 +17,7 @@ use daft_core::prelude::*; use daft_core::python::PyTimeUnit; use daft_dsl::{optimization::get_required_columns, ExprRef}; use daft_io::{parse_url, IOClient, IOStatsRef, SourceType}; -use daft_table::Table; +use daft_recordbatch::RecordBatch; use futures::{ future::{join_all, try_join_all}, stream::BoxStream, @@ -158,7 +158,7 @@ async fn read_parquet_single( metadata: Option>, delete_rows: Option>, chunk_size: Option, -) -> DaftResult
{ +) -> DaftResult { let field_id_mapping_provided = field_id_mapping.is_some(); let mut columns_to_read = columns.clone(); let columns_to_return = columns; @@ -376,7 +376,7 @@ async fn stream_parquet_single( delete_rows: Option>, maintain_order: bool, chunk_size: Option, -) -> DaftResult> + Send> { +) -> DaftResult> + Send> { let field_id_mapping_provided = field_id_mapping.is_some(); let columns_to_return = columns.map(|s| s.iter().map(|s| (*s).to_string()).collect_vec()); let num_rows_to_return = num_rows; @@ -680,7 +680,7 @@ pub fn read_parquet( multithreaded_io: bool, schema_infer_options: ParquetSchemaInferenceOptions, metadata: Option>, -) -> DaftResult
{ +) -> DaftResult { let runtime_handle = get_io_runtime(multithreaded_io); runtime_handle.block_on_current_thread(async { @@ -766,7 +766,7 @@ pub fn read_parquet_bulk>( metadata: Option>>, delete_map: Option>>, chunk_size: Option, -) -> DaftResult> { +) -> DaftResult> { let runtime_handle = get_io_runtime(multithreaded_io); let columns = columns.map(|s| s.iter().map(|v| v.as_ref().to_string()).collect::>()); @@ -815,7 +815,7 @@ pub async fn read_parquet_bulk_async( metadata: Option>>, delete_map: Option>>, chunk_size: Option, -) -> DaftResult>> { +) -> DaftResult>> { let task_stream = futures::stream::iter(uris.into_iter().enumerate().map(|(i, uri)| { let owned_columns = columns.clone(); let owned_row_group = row_groups.as_ref().and_then(|rgs| rgs[i].clone()); @@ -879,7 +879,7 @@ pub async fn stream_parquet( maintain_order: bool, delete_rows: Option>, chunk_size: Option, -) -> DaftResult>> { +) -> DaftResult>> { let stream = stream_parquet_single( uri.to_string(), columns, @@ -1021,7 +1021,7 @@ pub fn read_parquet_statistics( io_client: Arc, io_stats: Option, field_id_mapping: Option>>, -) -> DaftResult
{ +) -> DaftResult { let runtime_handle = get_io_runtime(true); if uris.data_type() != &DataType::Utf8 { @@ -1093,7 +1093,7 @@ pub fn read_parquet_statistics( )), )); - Table::from_nonempty_columns(vec![ + RecordBatch::from_nonempty_columns(vec![ uris.clone(), row_count_series.into_series(), row_group_series.into_series(), diff --git a/src/daft-parquet/src/stream_reader.rs b/src/daft-parquet/src/stream_reader.rs index 8ece492908..4ccf94f75b 100644 --- a/src/daft-parquet/src/stream_reader.rs +++ b/src/daft-parquet/src/stream_reader.rs @@ -12,7 +12,7 @@ use common_runtime::{get_compute_runtime, RuntimeTask}; use daft_core::{prelude::*, utils::arrow::cast_array_for_daft_if_needed}; use daft_dsl::ExprRef; use daft_io::IOStatsRef; -use daft_table::Table; +use daft_recordbatch::RecordBatch; use futures::{stream::BoxStream, StreamExt}; use itertools::Itertools; use rayon::{ @@ -65,7 +65,7 @@ fn arrow_chunk_to_table( predicate: Option, original_columns: Option<&[String]>, original_num_rows: Option, -) -> DaftResult
{ +) -> DaftResult { let all_series = arrow_chunk .into_iter() .zip(schema_ref.fields.iter()) @@ -96,7 +96,7 @@ fn arrow_chunk_to_table( .into()); } - let mut table = Table::new_with_size( + let mut table = RecordBatch::new_with_size( Schema::new(all_series.iter().map(|s| s.field().clone()).collect())?, all_series, len, @@ -148,7 +148,7 @@ pub fn spawn_column_iters_to_table_task( original_columns: Option>, original_num_rows: Option, delete_rows: Option>, - output_sender: tokio::sync::mpsc::Sender>, + output_sender: tokio::sync::mpsc::Sender>, permit: tokio::sync::OwnedSemaphorePermit, channel_size: usize, ) -> RuntimeTask> { @@ -177,7 +177,7 @@ pub fn spawn_column_iters_to_table_task( compute_runtime.spawn(async move { if deserializer_handles.is_empty() { - let empty = Table::new_with_size(schema_ref.clone(), vec![], rg_range.num_rows); + let empty = RecordBatch::new_with_size(schema_ref.clone(), vec![], rg_range.num_rows); let _ = output_sender.send(empty).await; return Ok(()); } @@ -534,7 +534,7 @@ pub async fn local_parquet_read_async( schema_infer_options: ParquetSchemaInferenceOptions, metadata: Option>, chunk_size: Option, -) -> DaftResult<(Arc, Table)> { +) -> DaftResult<(Arc, RecordBatch)> { let (send, recv) = tokio::sync::oneshot::channel(); let uri = uri.to_string(); rayon::spawn(move || { @@ -572,7 +572,7 @@ pub async fn local_parquet_read_async( .collect::, _>>()?; Ok(( metadata, - Table::new_with_size( + RecordBatch::new_with_size( Schema::new(converted_arrays.iter().map(|s| s.field().clone()).collect())?, converted_arrays, num_rows_read, @@ -602,7 +602,7 @@ pub async fn local_parquet_stream( chunk_size: Option, ) -> DaftResult<( Arc, - BoxStream<'static, DaftResult
>, + BoxStream<'static, DaftResult>, )> { let chunk_size = chunk_size.unwrap_or(PARQUET_MORSEL_SIZE); let (metadata, schema_ref, row_ranges, column_iters) = local_parquet_read_into_column_iters( diff --git a/src/daft-table/Cargo.toml b/src/daft-recordbatch/Cargo.toml similarity index 97% rename from src/daft-table/Cargo.toml rename to src/daft-recordbatch/Cargo.toml index 2a68b4124a..c6dc83e282 100644 --- a/src/daft-table/Cargo.toml +++ b/src/daft-recordbatch/Cargo.toml @@ -23,5 +23,5 @@ workspace = true [package] edition = {workspace = true} -name = "daft-table" +name = "daft-recordbatch" version = {workspace = true} diff --git a/src/daft-table/src/ffi.rs b/src/daft-recordbatch/src/ffi.rs similarity index 86% rename from src/daft-table/src/ffi.rs rename to src/daft-recordbatch/src/ffi.rs index 81d495728c..681f50d0ab 100644 --- a/src/daft-table/src/ffi.rs +++ b/src/daft-recordbatch/src/ffi.rs @@ -6,15 +6,15 @@ use daft_core::{ }; use pyo3::{exceptions::PyValueError, prelude::*, types::PyList}; -use crate::Table; +use crate::RecordBatch; pub fn record_batches_to_table( py: Python, batches: &[Bound], schema: SchemaRef, -) -> PyResult
{ +) -> PyResult { if batches.is_empty() { - return Ok(Table::empty(Some(schema))?); + return Ok(RecordBatch::empty(Some(schema))?); } let names = schema.names(); @@ -37,7 +37,7 @@ pub fn record_batches_to_table( } // Now do the heavy lifting (casting and concats) without the GIL. py.allow_threads(|| { - let mut tables: Vec
= Vec::with_capacity(num_batches); + let mut tables: Vec = Vec::with_capacity(num_batches); for (cols, num_rows) in extracted_arrow_arrays { let columns = cols .into_iter() @@ -47,15 +47,19 @@ pub fn record_batches_to_table( Series::try_from((names.get(i).unwrap().as_str(), cast_array)) }) .collect::>>()?; - tables.push(Table::new_with_size(schema.clone(), columns, num_rows)?); + tables.push(RecordBatch::new_with_size( + schema.clone(), + columns, + num_rows, + )?); } - Ok(Table::concat(tables.as_slice())?) + Ok(RecordBatch::concat(tables.as_slice())?) }) } pub fn table_to_record_batch( py: Python, - table: &Table, + table: &RecordBatch, pyarrow: Bound, ) -> PyResult { let mut arrays = Vec::with_capacity(table.num_columns()); diff --git a/src/daft-table/src/growable/mod.rs b/src/daft-recordbatch/src/growable/mod.rs similarity index 85% rename from src/daft-table/src/growable/mod.rs rename to src/daft-recordbatch/src/growable/mod.rs index 6404a78c04..32c40bdaf0 100644 --- a/src/daft-table/src/growable/mod.rs +++ b/src/daft-recordbatch/src/growable/mod.rs @@ -4,14 +4,18 @@ use daft_core::{ series::Series, }; -use crate::Table; +use crate::RecordBatch; -pub struct GrowableTable<'a> { +pub struct GrowableRecordBatch<'a> { growables: Vec>, } -impl<'a> GrowableTable<'a> { - pub fn new(tables: &[&'a Table], use_validity: bool, capacity: usize) -> DaftResult { +impl<'a> GrowableRecordBatch<'a> { + pub fn new( + tables: &[&'a RecordBatch], + use_validity: bool, + capacity: usize, + ) -> DaftResult { let num_tables = tables.len(); if tables.is_empty() { return Err(DaftError::ValueError( @@ -65,16 +69,16 @@ impl<'a> GrowableTable<'a> { } /// Builds an array from the [`Growable`] - pub fn build(&mut self) -> DaftResult
{ + pub fn build(&mut self) -> DaftResult { if self.growables.is_empty() { - Table::empty(None) + RecordBatch::empty(None) } else { let columns = self .growables .iter_mut() .map(|g| g.build()) .collect::>>()?; - Table::from_nonempty_columns(columns) + RecordBatch::from_nonempty_columns(columns) } } } diff --git a/src/daft-table/src/lib.rs b/src/daft-recordbatch/src/lib.rs similarity index 98% rename from src/daft-table/src/lib.rs rename to src/daft-recordbatch/src/lib.rs index d9f4ea29d7..9a2a6882b5 100644 --- a/src/daft-table/src/lib.rs +++ b/src/daft-recordbatch/src/lib.rs @@ -32,7 +32,7 @@ mod ops; mod probeable; mod repr_html; -pub use growable::GrowableTable; +pub use growable::GrowableRecordBatch; pub use probeable::{make_probeable_builder, ProbeState, Probeable, ProbeableBuilder}; #[cfg(feature = "python")] @@ -43,13 +43,13 @@ use rand::seq::index::sample; use repr_html::html_value; #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] -pub struct Table { +pub struct RecordBatch { pub schema: SchemaRef, columns: Arc>, num_rows: usize, } -impl Hash for Table { +impl Hash for RecordBatch { fn hash(&self, state: &mut H) { self.schema.hash(state); for col in &*self.columns { @@ -73,7 +73,7 @@ fn _validate_schema(schema: &Schema, columns: &[Series]) -> DaftResult<()> { Ok(()) } -impl Table { +impl RecordBatch { /// Create a new [`Table`] and handle broadcasting of any unit-length columns /// /// Note that this function is slow. You might instead be looking for [`Table::new_with_size`] which does not perform broadcasting @@ -850,10 +850,10 @@ impl Table { ) } } -impl TryFrom
for FileInfos { +impl TryFrom for FileInfos { type Error = DaftError; - fn try_from(table: Table) -> DaftResult { + fn try_from(table: RecordBatch) -> DaftResult { let file_paths = table .get_column("path")? .utf8()? @@ -888,7 +888,7 @@ impl TryFrom
for FileInfos { } } -impl TryFrom<&FileInfos> for Table { +impl TryFrom<&FileInfos> for RecordBatch { type Error = DaftError; fn try_from(file_info: &FileInfos) -> DaftResult { @@ -911,7 +911,7 @@ impl TryFrom<&FileInfos> for Table { } } -impl PartialEq for Table { +impl PartialEq for RecordBatch { fn eq(&self, other: &Self) -> bool { if self.len() != other.len() { return false; @@ -928,7 +928,7 @@ impl PartialEq for Table { } } -impl Display for Table { +impl Display for RecordBatch { // `f` is a buffer, and this method must write the formatted string into it fn fmt(&self, f: &mut Formatter) -> Result { let table = self.to_comfy_table(Some(32)); @@ -936,13 +936,13 @@ impl Display for Table { } } -impl AsRef for Table { +impl AsRef for RecordBatch { fn as_ref(&self) -> &Self { self } } -impl<'a> IntoIterator for &'a Table { +impl<'a> IntoIterator for &'a RecordBatch { type Item = &'a Series; type IntoIter = slice::Iter<'a, Series>; fn into_iter(self) -> Self::IntoIter { @@ -956,7 +956,7 @@ mod test { use daft_core::prelude::*; use daft_dsl::col; - use crate::Table; + use crate::RecordBatch; #[test] fn add_int_and_float_expression() -> DaftResult<()> { @@ -966,7 +966,7 @@ mod test { a.field().clone().rename("a"), b.field().clone().rename("b"), ])?; - let table = Table::from_nonempty_columns(vec![a, b])?; + let table = RecordBatch::from_nonempty_columns(vec![a, b])?; let e1 = col("a").add(col("b")); let result = table.eval_expression(&e1)?; assert_eq!(*result.data_type(), DataType::Float64); diff --git a/src/daft-table/src/ops/agg.rs b/src/daft-recordbatch/src/ops/agg.rs similarity index 99% rename from src/daft-table/src/ops/agg.rs rename to src/daft-recordbatch/src/ops/agg.rs index df0150257b..89e99d3b6e 100644 --- a/src/daft-table/src/ops/agg.rs +++ b/src/daft-recordbatch/src/ops/agg.rs @@ -2,9 +2,9 @@ use common_error::{DaftError, DaftResult}; use daft_core::{array::ops::IntoGroups, prelude::*}; use daft_dsl::{functions::FunctionExpr, AggExpr, Expr, ExprRef}; -use crate::Table; +use crate::RecordBatch; -impl Table { +impl RecordBatch { pub fn agg(&self, to_agg: &[ExprRef], group_by: &[ExprRef]) -> DaftResult { // Dispatch depending on whether we're doing groupby or just a global agg. match group_by.len() { diff --git a/src/daft-table/src/ops/explode.rs b/src/daft-recordbatch/src/ops/explode.rs similarity index 98% rename from src/daft-table/src/ops/explode.rs rename to src/daft-recordbatch/src/ops/explode.rs index cee00485ac..ef3bd39d0b 100644 --- a/src/daft-table/src/ops/explode.rs +++ b/src/daft-recordbatch/src/ops/explode.rs @@ -9,7 +9,7 @@ use daft_core::{ }; use daft_dsl::Expr; -use crate::Table; +use crate::RecordBatch; fn lengths_to_indices(lengths: &UInt64Array, capacity: usize) -> DaftResult { let mut indices = Vec::with_capacity(capacity); @@ -20,7 +20,7 @@ fn lengths_to_indices(lengths: &UInt64Array, capacity: usize) -> DaftResult>(&self, exprs: &[E]) -> DaftResult { if exprs.is_empty() { return Err(DaftError::ValueError(format!( diff --git a/src/daft-table/src/ops/groups.rs b/src/daft-recordbatch/src/ops/groups.rs similarity index 98% rename from src/daft-table/src/ops/groups.rs rename to src/daft-recordbatch/src/ops/groups.rs index 7239c5a35c..a51513cb73 100644 --- a/src/daft-table/src/ops/groups.rs +++ b/src/daft-recordbatch/src/ops/groups.rs @@ -8,9 +8,9 @@ use daft_core::{ series::Series, }; -use crate::Table; +use crate::RecordBatch; -impl Table { +impl RecordBatch { fn hash_grouper(&self) -> DaftResult { // Group equal rows together. // @@ -111,7 +111,7 @@ impl Table { } } -impl IntoGroups for Table { +impl IntoGroups for RecordBatch { fn make_groups(&self) -> DaftResult { self.as_physical()?.hash_grouper() } diff --git a/src/daft-table/src/ops/hash.rs b/src/daft-recordbatch/src/ops/hash.rs similarity index 98% rename from src/daft-table/src/ops/hash.rs rename to src/daft-recordbatch/src/ops/hash.rs index 5421988518..118b500101 100644 --- a/src/daft-table/src/ops/hash.rs +++ b/src/daft-recordbatch/src/ops/hash.rs @@ -7,9 +7,9 @@ use daft_core::{ utils::identity_hash_set::{IdentityBuildHasher, IndexHash}, }; -use crate::Table; +use crate::RecordBatch; -impl Table { +impl RecordBatch { pub fn hash_rows(&self) -> DaftResult { if self.num_columns() == 0 { return Err(DaftError::ValueError( diff --git a/src/daft-table/src/ops/joins/hash_join.rs b/src/daft-recordbatch/src/ops/joins/hash_join.rs similarity index 95% rename from src/daft-table/src/ops/joins/hash_join.rs rename to src/daft-recordbatch/src/ops/joins/hash_join.rs index dccbe574a7..6b7929afa5 100644 --- a/src/daft-table/src/ops/joins/hash_join.rs +++ b/src/daft-recordbatch/src/ops/joins/hash_join.rs @@ -12,14 +12,14 @@ use daft_dsl::{ }; use super::{add_non_join_key_columns, match_types_for_tables}; -use crate::Table; +use crate::RecordBatch; pub(super) fn hash_inner_join( - left: &Table, - right: &Table, + left: &RecordBatch, + right: &RecordBatch, left_on: &[ExprRef], right_on: &[ExprRef], null_equals_nulls: &[bool], -) -> DaftResult
{ +) -> DaftResult { let join_schema = infer_join_schema(&left.schema, &right.schema, JoinType::Inner)?; let lkeys = left.eval_expression_list(left_on)?; let rkeys = right.eval_expression_list(right_on)?; @@ -95,17 +95,17 @@ pub(super) fn hash_inner_join( let num_rows = lidx.len(); join_series = add_non_join_key_columns(left, right, lidx, ridx, join_series)?; - Table::new_with_size(join_schema, join_series, num_rows) + RecordBatch::new_with_size(join_schema, join_series, num_rows) } pub(super) fn hash_left_right_join( - left: &Table, - right: &Table, + left: &RecordBatch, + right: &RecordBatch, left_on: &[ExprRef], right_on: &[ExprRef], null_equals_nulls: &[bool], left_side: bool, -) -> DaftResult
{ +) -> DaftResult { let join_schema = infer_join_schema(&left.schema, &right.schema, JoinType::Right)?; let lkeys = left.eval_expression_list(left_on)?; let rkeys = right.eval_expression_list(right_on)?; @@ -205,17 +205,17 @@ pub(super) fn hash_left_right_join( let num_rows = lidx.len(); join_series = add_non_join_key_columns(left, right, lidx, ridx, join_series)?; - Table::new_with_size(join_schema, join_series, num_rows) + RecordBatch::new_with_size(join_schema, join_series, num_rows) } pub(super) fn hash_semi_anti_join( - left: &Table, - right: &Table, + left: &RecordBatch, + right: &RecordBatch, left_on: &[ExprRef], right_on: &[ExprRef], null_equals_nulls: &[bool], is_anti: bool, -) -> DaftResult
{ +) -> DaftResult { let lkeys = left.eval_expression_list(left_on)?; let rkeys = right.eval_expression_list(right_on)?; @@ -270,12 +270,12 @@ pub(super) fn hash_semi_anti_join( } pub(super) fn hash_outer_join( - left: &Table, - right: &Table, + left: &RecordBatch, + right: &RecordBatch, left_on: &[ExprRef], right_on: &[ExprRef], null_equals_nulls: &[bool], -) -> DaftResult
{ +) -> DaftResult { let join_schema = infer_join_schema(&left.schema, &right.schema, JoinType::Outer)?; let lkeys = left.eval_expression_list(left_on)?; let rkeys = right.eval_expression_list(right_on)?; @@ -409,5 +409,5 @@ pub(super) fn hash_outer_join( let num_rows = lidx.len(); join_series = add_non_join_key_columns(left, right, lidx, ridx, join_series)?; - Table::new_with_size(join_schema, join_series, num_rows) + RecordBatch::new_with_size(join_schema, join_series, num_rows) } diff --git a/src/daft-table/src/ops/joins/merge_join.rs b/src/daft-recordbatch/src/ops/joins/merge_join.rs similarity index 98% rename from src/daft-table/src/ops/joins/merge_join.rs rename to src/daft-recordbatch/src/ops/joins/merge_join.rs index 4b5a861811..1218e4358c 100644 --- a/src/daft-table/src/ops/joins/merge_join.rs +++ b/src/daft-recordbatch/src/ops/joins/merge_join.rs @@ -8,7 +8,7 @@ use daft_core::{ series::{IntoSeries, Series}, }; -use crate::Table; +use crate::RecordBatch; /// A state machine for the below merge-join algorithm. /// @@ -41,7 +41,7 @@ enum MergeJoinState { StagedRightEqualRun(usize), } -pub fn merge_inner_join(left: &Table, right: &Table) -> DaftResult<(Series, Series)> { +pub fn merge_inner_join(left: &RecordBatch, right: &RecordBatch) -> DaftResult<(Series, Series)> { if left.num_columns() != right.num_columns() { return Err(DaftError::ValueError(format!( "Mismatch of join on clauses: left: {:?} vs right: {:?}", diff --git a/src/daft-table/src/ops/joins/mod.rs b/src/daft-recordbatch/src/ops/joins/mod.rs similarity index 92% rename from src/daft-table/src/ops/joins/mod.rs rename to src/daft-recordbatch/src/ops/joins/mod.rs index 8ffc86b847..237f744b63 100644 --- a/src/daft-table/src/ops/joins/mod.rs +++ b/src/daft-recordbatch/src/ops/joins/mod.rs @@ -11,11 +11,14 @@ use daft_dsl::{ use hash_join::hash_semi_anti_join; use self::hash_join::{hash_inner_join, hash_left_right_join, hash_outer_join}; -use crate::Table; +use crate::RecordBatch; mod hash_join; mod merge_join; -fn match_types_for_tables(left: &Table, right: &Table) -> DaftResult<(Table, Table)> { +fn match_types_for_tables( + left: &RecordBatch, + right: &RecordBatch, +) -> DaftResult<(RecordBatch, RecordBatch)> { let mut lseries = vec![]; let mut rseries = vec![]; @@ -33,14 +36,14 @@ fn match_types_for_tables(left: &Table, right: &Table) -> DaftResult<(Table, Tab } } Ok(( - Table::from_nonempty_columns(lseries)?, - Table::from_nonempty_columns(rseries)?, + RecordBatch::from_nonempty_columns(lseries)?, + RecordBatch::from_nonempty_columns(rseries)?, )) } fn add_non_join_key_columns( - left: &Table, - right: &Table, + left: &RecordBatch, + right: &RecordBatch, lidx: Series, ridx: Series, mut join_series: Vec, @@ -71,7 +74,7 @@ fn add_non_join_key_columns( Ok(join_series) } -impl Table { +impl RecordBatch { pub fn hash_join( &self, right: &Self, @@ -195,7 +198,10 @@ impl Table { pub fn cross_join(&self, right: &Self, outer_loop_side: JoinSide) -> DaftResult { /// Create a new table by repeating each column of the input table `inner_len` times in a row, thus preserving sort order. - fn create_outer_loop_table(input: &Table, inner_len: usize) -> DaftResult
{ + fn create_outer_loop_table( + input: &RecordBatch, + inner_len: usize, + ) -> DaftResult { let idx = (0..input.len() as u64) .flat_map(|i| std::iter::repeat(i).take(inner_len)) .collect::>(); @@ -206,8 +212,11 @@ impl Table { } /// Create a enw table by repeating the entire table `outer_len` number of times - fn create_inner_loop_table(input: &Table, outer_len: usize) -> DaftResult
{ - Table::concat(&vec![input; outer_len]) + fn create_inner_loop_table( + input: &RecordBatch, + outer_len: usize, + ) -> DaftResult { + RecordBatch::concat(&vec![input; outer_len]) } let (left_table, right_table) = match outer_loop_side { diff --git a/src/daft-table/src/ops/mod.rs b/src/daft-recordbatch/src/ops/mod.rs similarity index 100% rename from src/daft-table/src/ops/mod.rs rename to src/daft-recordbatch/src/ops/mod.rs diff --git a/src/daft-table/src/ops/partition.rs b/src/daft-recordbatch/src/ops/partition.rs similarity index 98% rename from src/daft-table/src/ops/partition.rs rename to src/daft-recordbatch/src/ops/partition.rs index 93d61f1547..cc9b2539f9 100644 --- a/src/daft-table/src/ops/partition.rs +++ b/src/daft-recordbatch/src/ops/partition.rs @@ -10,9 +10,9 @@ use daft_core::{ use daft_dsl::ExprRef; use rand::SeedableRng; -use crate::Table; +use crate::RecordBatch; -impl Table { +impl RecordBatch { fn partition_by_index( &self, targets: &UInt64Array, diff --git a/src/daft-table/src/ops/pivot.rs b/src/daft-recordbatch/src/ops/pivot.rs similarity index 99% rename from src/daft-table/src/ops/pivot.rs rename to src/daft-recordbatch/src/ops/pivot.rs index 2eaf7274e6..59944d516c 100644 --- a/src/daft-table/src/ops/pivot.rs +++ b/src/daft-recordbatch/src/ops/pivot.rs @@ -2,7 +2,7 @@ use common_error::{DaftError, DaftResult}; use daft_core::{array::ops::IntoGroups, prelude::*}; use daft_dsl::ExprRef; -use crate::Table; +use crate::RecordBatch; fn map_name_to_pivot_key_idx<'a>( pivot_series: &'a Series, @@ -66,7 +66,7 @@ fn map_pivot_key_idx_to_values_indices( Ok(pivot_key_idx_to_values_indices) } -impl Table { +impl RecordBatch { pub fn pivot( &self, group_by: &[ExprRef], diff --git a/src/daft-table/src/ops/search_sorted.rs b/src/daft-recordbatch/src/ops/search_sorted.rs similarity index 97% rename from src/daft-table/src/ops/search_sorted.rs rename to src/daft-recordbatch/src/ops/search_sorted.rs index 95b3738493..7a89276d20 100644 --- a/src/daft-table/src/ops/search_sorted.rs +++ b/src/daft-recordbatch/src/ops/search_sorted.rs @@ -4,9 +4,9 @@ use daft_core::{ series::Series, }; -use crate::Table; +use crate::RecordBatch; -impl Table { +impl RecordBatch { pub fn search_sorted(&self, keys: &Self, descending: &[bool]) -> DaftResult { if self.schema != keys.schema { return Err(DaftError::SchemaMismatch(format!( diff --git a/src/daft-table/src/ops/sort.rs b/src/daft-recordbatch/src/ops/sort.rs similarity index 96% rename from src/daft-table/src/ops/sort.rs rename to src/daft-recordbatch/src/ops/sort.rs index 540013a02e..26500a8afc 100644 --- a/src/daft-table/src/ops/sort.rs +++ b/src/daft-recordbatch/src/ops/sort.rs @@ -2,9 +2,9 @@ use common_error::{DaftError, DaftResult}; use daft_core::series::Series; use daft_dsl::ExprRef; -use crate::Table; +use crate::RecordBatch; -impl Table { +impl RecordBatch { pub fn sort( &self, sort_keys: &[ExprRef], diff --git a/src/daft-table/src/ops/unpivot.rs b/src/daft-recordbatch/src/ops/unpivot.rs similarity index 97% rename from src/daft-table/src/ops/unpivot.rs rename to src/daft-recordbatch/src/ops/unpivot.rs index e916e53e5a..621d112570 100644 --- a/src/daft-table/src/ops/unpivot.rs +++ b/src/daft-recordbatch/src/ops/unpivot.rs @@ -4,9 +4,9 @@ use common_error::{DaftError, DaftResult}; use daft_core::{prelude::*, series::cast_series_to_supertype}; use daft_dsl::ExprRef; -use crate::Table; +use crate::RecordBatch; -impl Table { +impl RecordBatch { pub fn unpivot( &self, ids: &[ExprRef], diff --git a/src/daft-table/src/probeable/mod.rs b/src/daft-recordbatch/src/probeable/mod.rs similarity index 87% rename from src/daft-table/src/probeable/mod.rs rename to src/daft-recordbatch/src/probeable/mod.rs index c68cb60884..e46c58cc45 100644 --- a/src/daft-table/src/probeable/mod.rs +++ b/src/daft-recordbatch/src/probeable/mod.rs @@ -8,7 +8,7 @@ use daft_core::prelude::SchemaRef; use probe_set::{ProbeSet, ProbeSetBuilder}; use probe_table::{ProbeTable, ProbeTableBuilder}; -use crate::Table; +use crate::RecordBatch; struct ArrowTableEntry(Vec>); @@ -31,7 +31,7 @@ pub fn make_probeable_builder( } pub trait ProbeableBuilder: Send + Sync { - fn add_table(&mut self, table: &Table) -> DaftResult<()>; + fn add_table(&mut self, table: &RecordBatch) -> DaftResult<()>; fn build(self: Box) -> Arc; } @@ -76,23 +76,23 @@ pub trait Probeable: Send + Sync { /// The inner iterator, if present, iterates over the rows of the left table that match the right row. /// Otherwise, if the inner iterator is None, indicates that the right row has no matches. /// NOTE: This function only works if track_indices is true. - fn probe_indices<'a>(&'a self, table: &'a Table) -> DaftResult>; + fn probe_indices<'a>(&'a self, table: &'a RecordBatch) -> DaftResult>; /// Probe_exists returns an iterator of booleans. The iterator iterates over the rows of the right table. fn probe_exists<'a>( &'a self, - table: &'a Table, + table: &'a RecordBatch, ) -> DaftResult + 'a>>; } #[derive(Clone)] pub struct ProbeState { probeable: Arc, - tables: Arc>, + tables: Arc>, } impl ProbeState { - pub fn new(probeable: Arc, tables: Arc>) -> Self { + pub fn new(probeable: Arc, tables: Arc>) -> Self { Self { probeable, tables } } @@ -100,7 +100,7 @@ impl ProbeState { &self.probeable } - pub fn get_tables(&self) -> &Arc> { + pub fn get_tables(&self) -> &Arc> { &self.tables } } diff --git a/src/daft-table/src/probeable/probe_set.rs b/src/daft-recordbatch/src/probeable/probe_set.rs similarity index 93% rename from src/daft-table/src/probeable/probe_set.rs rename to src/daft-recordbatch/src/probeable/probe_set.rs index 40ab4ab86e..baa9d09c3a 100644 --- a/src/daft-table/src/probeable/probe_set.rs +++ b/src/daft-recordbatch/src/probeable/probe_set.rs @@ -14,7 +14,7 @@ use daft_core::{ }; use super::{ArrowTableEntry, IndicesMapper, Probeable, ProbeableBuilder}; -use crate::Table; +use crate::RecordBatch; pub struct ProbeSet { schema: SchemaRef, hash_table: HashMap, @@ -61,7 +61,7 @@ impl ProbeSet { }) } - fn probe<'a>(&'a self, input: &'a Table) -> DaftResult + 'a> { + fn probe<'a>(&'a self, input: &'a RecordBatch) -> DaftResult + 'a> { assert_eq!(self.schema.len(), input.schema.len()); assert!(self .schema @@ -101,7 +101,7 @@ impl ProbeSet { })) } - fn add_table(&mut self, table: &Table) -> DaftResult<()> { + fn add_table(&mut self, table: &RecordBatch) -> DaftResult<()> { // we have to cast to the join key schema assert_eq!(table.schema, self.schema); let hashes = table.hash_rows()?; @@ -160,12 +160,12 @@ impl ProbeSet { impl Probeable for ProbeSet { fn probe_exists<'a>( &'a self, - table: &'a Table, + table: &'a RecordBatch, ) -> DaftResult + 'a>> { Ok(Box::new(self.probe(table)?)) } - fn probe_indices<'a>(&'a self, _table: &'a Table) -> DaftResult> { + fn probe_indices<'a>(&'a self, _table: &'a RecordBatch) -> DaftResult> { panic!("Probe indices is not supported for ProbeSet") } } @@ -173,7 +173,7 @@ impl Probeable for ProbeSet { pub struct ProbeSetBuilder(pub ProbeSet); impl ProbeableBuilder for ProbeSetBuilder { - fn add_table(&mut self, table: &Table) -> DaftResult<()> { + fn add_table(&mut self, table: &RecordBatch) -> DaftResult<()> { self.0.add_table(table) } diff --git a/src/daft-table/src/probeable/probe_table.rs b/src/daft-recordbatch/src/probeable/probe_table.rs similarity index 95% rename from src/daft-table/src/probeable/probe_table.rs rename to src/daft-recordbatch/src/probeable/probe_table.rs index ba4c11dc41..bc72689906 100644 --- a/src/daft-table/src/probeable/probe_table.rs +++ b/src/daft-recordbatch/src/probeable/probe_table.rs @@ -14,7 +14,7 @@ use daft_core::{ }; use super::{ArrowTableEntry, IndicesMapper, Probeable, ProbeableBuilder}; -use crate::Table; +use crate::RecordBatch; pub struct ProbeTable { schema: SchemaRef, @@ -62,7 +62,7 @@ impl ProbeTable { fn probe<'a>( &'a self, - input: &'a Table, + input: &'a RecordBatch, ) -> DaftResult> + 'a> { assert_eq!(self.schema.len(), input.schema.len()); assert!(self @@ -107,7 +107,7 @@ impl ProbeTable { }))) } - fn add_table(&mut self, table: &Table) -> DaftResult<()> { + fn add_table(&mut self, table: &RecordBatch) -> DaftResult<()> { // we have to cast to the join key schema assert_eq!(table.schema, self.schema); let hashes = table.hash_rows()?; @@ -166,7 +166,7 @@ impl ProbeTable { } impl Probeable for ProbeTable { - fn probe_indices<'a>(&'a self, table: &'a Table) -> DaftResult> { + fn probe_indices<'a>(&'a self, table: &'a RecordBatch) -> DaftResult> { let iter = self.probe(table)?; Ok(IndicesMapper::new( Box::new(iter), @@ -177,7 +177,7 @@ impl Probeable for ProbeTable { fn probe_exists<'a>( &'a self, - table: &'a Table, + table: &'a RecordBatch, ) -> DaftResult + 'a>> { let iter = self.probe(table)?; Ok(Box::new(iter.map(|indices| indices.is_some()))) @@ -187,7 +187,7 @@ impl Probeable for ProbeTable { pub struct ProbeTableBuilder(pub ProbeTable); impl ProbeableBuilder for ProbeTableBuilder { - fn add_table(&mut self, table: &Table) -> DaftResult<()> { + fn add_table(&mut self, table: &RecordBatch) -> DaftResult<()> { self.0.add_table(table) } diff --git a/src/daft-table/src/python.rs b/src/daft-recordbatch/src/python.rs similarity index 95% rename from src/daft-table/src/python.rs rename to src/daft-recordbatch/src/python.rs index ad53cb03a6..db9983e0da 100644 --- a/src/daft-table/src/python.rs +++ b/src/daft-recordbatch/src/python.rs @@ -9,16 +9,16 @@ use daft_logical_plan::FileInfos; use indexmap::IndexMap; use pyo3::{exceptions::PyValueError, prelude::*}; -use crate::{ffi, Table}; +use crate::{ffi, RecordBatch}; #[pyclass] #[derive(Clone)] -pub struct PyTable { - pub table: Table, +pub struct PyRecordBatch { + pub table: RecordBatch, } #[pymethods] -impl PyTable { +impl PyRecordBatch { pub fn schema(&self) -> PyResult { Ok(PySchema { schema: self.table.schema.clone(), @@ -349,12 +349,12 @@ impl PyTable { .collect(); py.allow_threads(|| { let (tables, values) = self.table.partition_by_value(exprs.as_slice())?; - let pytables = tables + let pyrecordbatches = tables .into_iter() .map(std::convert::Into::into) .collect::>(); let values = values.into(); - Ok((pytables, values)) + Ok((pyrecordbatches, values)) }) } @@ -409,7 +409,7 @@ impl PyTable { #[staticmethod] pub fn concat(py: Python, tables: Vec) -> PyResult { let tables: Vec<_> = tables.iter().map(|t| &t.table).collect(); - py.allow_threads(|| Ok(Table::concat(tables.as_slice())?.into())) + py.allow_threads(|| Ok(RecordBatch::concat(tables.as_slice())?.into())) } pub fn slice(&self, start: i64, end: i64) -> PyResult { @@ -471,7 +471,7 @@ impl PyTable { } Ok(Self { - table: Table::new_with_broadcast(Schema::new(fields)?, columns, num_rows)?, + table: RecordBatch::new_with_broadcast(Schema::new(fields)?, columns, num_rows)?, }) } @@ -485,7 +485,7 @@ impl PyTable { #[staticmethod] #[pyo3(signature = (schema=None))] pub fn empty(schema: Option) -> PyResult { - Ok(Table::empty(match schema { + Ok(RecordBatch::empty(match schema { Some(s) => Some(s.schema), None => None, })? @@ -499,30 +499,30 @@ impl PyTable { #[staticmethod] pub fn from_file_infos(file_infos: &FileInfos) -> PyResult { - let table: Table = file_infos.try_into()?; + let table: RecordBatch = file_infos.try_into()?; Ok(table.into()) } } -impl From
for PyTable { - fn from(value: Table) -> Self { +impl From for PyRecordBatch { + fn from(value: RecordBatch) -> Self { Self { table: value } } } -impl From for Table { - fn from(item: PyTable) -> Self { +impl From for RecordBatch { + fn from(item: PyRecordBatch) -> Self { item.table } } -impl AsRef
for PyTable { - fn as_ref(&self) -> &Table { +impl AsRef for PyRecordBatch { + fn as_ref(&self) -> &RecordBatch { &self.table } } pub fn register_modules(parent: &Bound) -> PyResult<()> { - parent.add_class::()?; + parent.add_class::()?; Ok(()) } diff --git a/src/daft-table/src/repr_html.rs b/src/daft-recordbatch/src/repr_html.rs similarity index 100% rename from src/daft-table/src/repr_html.rs rename to src/daft-recordbatch/src/repr_html.rs diff --git a/src/daft-scan/Cargo.toml b/src/daft-scan/Cargo.toml index 49344c53ab..7448f24f4f 100644 --- a/src/daft-scan/Cargo.toml +++ b/src/daft-scan/Cargo.toml @@ -16,9 +16,9 @@ daft-io = {path = "../daft-io", default-features = false} daft-json = {path = "../daft-json", default-features = false} daft-logical-plan = {path = "../daft-logical-plan", default-features = false} daft-parquet = {path = "../daft-parquet", default-features = false} +daft-recordbatch = {path = "../daft-recordbatch", default-features = false} daft-schema = {path = "../daft-schema", default-features = false} daft-stats = {path = "../daft-stats", default-features = false} -daft-table = {path = "../daft-table", default-features = false} futures = {workspace = true} indexmap = {workspace = true} itertools = {workspace = true} @@ -33,7 +33,7 @@ urlencoding = "2.1.3" tokio = {workspace = true, features = ["full"]} [features] -python = ["dep:pyo3", "common-error/python", "daft-core/python", "daft-logical-plan/python", "daft-table/python", "daft-stats/python", "common-file-formats/python", "common-io-config/python", "common-daft-config/python", "common-scan-info/python", "daft-schema/python"] +python = ["dep:pyo3", "common-error/python", "daft-core/python", "daft-logical-plan/python", "daft-recordbatch/python", "daft-stats/python", "common-file-formats/python", "common-io-config/python", "common-daft-config/python", "common-scan-info/python", "daft-schema/python"] [lints] workspace = true diff --git a/src/daft-scan/src/glob.rs b/src/daft-scan/src/glob.rs index bf91d4205c..233643c286 100644 --- a/src/daft-scan/src/glob.rs +++ b/src/daft-scan/src/glob.rs @@ -8,13 +8,13 @@ use daft_core::{prelude::Utf8Array, series::IntoSeries}; use daft_csv::CsvParseOptions; use daft_io::{parse_url, FileMetadata, IOClient, IOStatsContext, IOStatsRef}; use daft_parquet::read::ParquetSchemaInferenceOptions; +use daft_recordbatch::RecordBatch; use daft_schema::{ dtype::DataType, field::Field, schema::{Schema, SchemaRef}, }; use daft_stats::PartitionSpec; -use daft_table::Table; use futures::{stream::BoxStream, Stream, StreamExt, TryStreamExt}; use snafu::Snafu; @@ -421,7 +421,7 @@ impl ScanOperator for GlobScanOperator { } let (partition_spec, generated_fields) = if !partition_values.is_empty() { let partition_values_table = - Table::from_nonempty_columns(partition_values)?; + RecordBatch::from_nonempty_columns(partition_values)?; // If there are partition values, evaluate them against partition filters, if any. if let Some(partition_filters) = &pushdowns.partition_filters { let filter_result = diff --git a/src/daft-scan/src/python.rs b/src/daft-scan/src/python.rs index a13d16cfdc..48e5e4c1c8 100644 --- a/src/daft-scan/src/python.rs +++ b/src/daft-scan/src/python.rs @@ -77,9 +77,9 @@ pub mod pylib { PartitionField, Pushdowns, ScanOperator, ScanOperatorRef, ScanTaskLike, ScanTaskLikeRef, }; use daft_logical_plan::{LogicalPlanBuilder, PyLogicalPlanBuilder}; + use daft_recordbatch::{python::PyRecordBatch, RecordBatch}; use daft_schema::{python::schema::PySchema, schema::SchemaRef}; use daft_stats::{PartitionSpec, TableMetadata, TableStatistics}; - use daft_table::{python::PyTable, Table}; use pyo3::{prelude::*, pyclass, types::PyIterator}; use serde::{Deserialize, Serialize}; @@ -368,8 +368,8 @@ pub mod pylib { size_bytes: Option, iceberg_delete_files: Option>, pushdowns: Option, - partition_values: Option, - stats: Option, + partition_values: Option, + stats: Option, ) -> PyResult> { if let Some(ref pvalues) = partition_values && let Some(Some(ref partition_filters)) = @@ -391,7 +391,8 @@ pub mod pylib { // TODO(Clark): Filter out scan tasks with pushed down filters + table stats? let pspec = PartitionSpec { - keys: partition_values.map_or_else(|| Table::empty(None).unwrap(), |p| p.table), + keys: partition_values + .map_or_else(|| RecordBatch::empty(None).unwrap(), |p| p.table), }; let statistics = stats .map(|s| TableStatistics::from_stats_table(&s.table)) @@ -441,7 +442,7 @@ pub mod pylib { num_rows: Option, size_bytes: Option, pushdowns: Option, - stats: Option, + stats: Option, ) -> PyResult { let statistics = stats .map(|s| TableStatistics::from_stats_table(&s.table)) @@ -484,7 +485,7 @@ pub mod pylib { num_rows: Option, size_bytes: Option, pushdowns: Option, - stats: Option, + stats: Option, ) -> PyResult { let statistics = stats .map(|s| TableStatistics::from_stats_table(&s.table)) diff --git a/src/daft-stats/Cargo.toml b/src/daft-stats/Cargo.toml index 7cea2fe37d..0012898cae 100644 --- a/src/daft-stats/Cargo.toml +++ b/src/daft-stats/Cargo.toml @@ -2,13 +2,13 @@ common-error = {path = "../common/error", default-features = false} daft-core = {path = "../daft-core", default-features = false} daft-dsl = {path = "../daft-dsl", default-features = false} -daft-table = {path = "../daft-table", default-features = false} +daft-recordbatch = {path = "../daft-recordbatch", default-features = false} indexmap = {workspace = true, features = ["serde"]} serde = {workspace = true} snafu = {workspace = true} [features] -python = ["common-error/python", "daft-core/python", "daft-dsl/python", "daft-table/python"] +python = ["common-error/python", "daft-core/python", "daft-dsl/python", "daft-recordbatch/python"] [lints] workspace = true diff --git a/src/daft-stats/src/partition_spec.rs b/src/daft-stats/src/partition_spec.rs index 79fe4c21b4..125688190d 100644 --- a/src/daft-stats/src/partition_spec.rs +++ b/src/daft-stats/src/partition_spec.rs @@ -5,11 +5,11 @@ use std::{ use daft_core::array::ops::{DaftCompare, DaftLogical}; use daft_dsl::{ExprRef, Literal}; -use daft_table::Table; +use daft_recordbatch::RecordBatch; #[derive(Debug, Clone, serde::Serialize, serde::Deserialize)] pub struct PartitionSpec { - pub keys: Table, + pub keys: RecordBatch, } impl PartitionSpec { diff --git a/src/daft-stats/src/table_stats.rs b/src/daft-stats/src/table_stats.rs index 5f6f32a5a8..2238f4c62a 100644 --- a/src/daft-stats/src/table_stats.rs +++ b/src/daft-stats/src/table_stats.rs @@ -8,7 +8,7 @@ use std::{ use common_error::{DaftError, DaftResult}; use daft_core::prelude::*; use daft_dsl::{Expr, ExprRef}; -use daft_table::Table; +use daft_recordbatch::RecordBatch; use indexmap::{IndexMap, IndexSet}; use crate::column_stats::ColumnRangeStatistics; @@ -28,7 +28,7 @@ impl Hash for TableStatistics { } impl TableStatistics { - pub fn from_stats_table(table: &Table) -> DaftResult { + pub fn from_stats_table(table: &RecordBatch) -> DaftResult { // Assumed format is each column having 2 rows: // - row 0: Minimum value for the column. // - row 1: Maximum value for the column. @@ -45,7 +45,7 @@ impl TableStatistics { } #[must_use] - pub fn from_table(table: &Table) -> Self { + pub fn from_table(table: &RecordBatch) -> Self { let mut columns = IndexMap::with_capacity(table.num_columns()); for name in table.column_names() { let col = table.get_column(&name).unwrap(); @@ -202,7 +202,7 @@ impl Display for TableStatistics { .map(|(s, c)| c.combined_series().unwrap().rename(s)) .collect::>(); let tbl_schema = Schema::new(columns.iter().map(|s| s.field().clone()).collect()).unwrap(); - let tab = Table::new_with_size(tbl_schema, columns, 2).unwrap(); + let tab = RecordBatch::new_with_size(tbl_schema, columns, 2).unwrap(); write!(f, "{tab}") } } @@ -211,17 +211,18 @@ impl Display for TableStatistics { mod test { use daft_core::prelude::*; use daft_dsl::{col, lit}; - use daft_table::Table; + use daft_recordbatch::RecordBatch; use super::TableStatistics; use crate::column_stats::TruthValue; #[test] fn test_equal() -> crate::Result<()> { - let table = Table::from_nonempty_columns(vec![ - Int64Array::from(("a", vec![1, 2, 3, 4])).into_series() - ]) - .unwrap(); + let table = + RecordBatch::from_nonempty_columns(vec![ + Int64Array::from(("a", vec![1, 2, 3, 4])).into_series() + ]) + .unwrap(); let table_stats = TableStatistics::from_table(&table); // False case @@ -236,9 +237,9 @@ mod test { // True case let table = - Table::from_nonempty_columns( - vec![Int64Array::from(("a", vec![0, 0, 0])).into_series()], - ) + RecordBatch::from_nonempty_columns(vec![ + Int64Array::from(("a", vec![0, 0, 0])).into_series() + ]) .unwrap(); let table_stats = TableStatistics::from_table(&table); diff --git a/src/daft-writers/Cargo.toml b/src/daft-writers/Cargo.toml index 8d3cf5ecfa..2134c5ac36 100644 --- a/src/daft-writers/Cargo.toml +++ b/src/daft-writers/Cargo.toml @@ -7,7 +7,7 @@ daft-dsl = {path = "../daft-dsl", default-features = false} daft-io = {path = "../daft-io", default-features = false} daft-logical-plan = {path = "../daft-logical-plan", default-features = false} daft-micropartition = {path = "../daft-micropartition", default-features = false} -daft-table = {path = "../daft-table", default-features = false} +daft-recordbatch = {path = "../daft-recordbatch", default-features = false} pyo3 = {workspace = true, optional = true} [features] diff --git a/src/daft-writers/src/batch.rs b/src/daft-writers/src/batch.rs index a0333cb1f3..2f1a9640f3 100644 --- a/src/daft-writers/src/batch.rs +++ b/src/daft-writers/src/batch.rs @@ -6,7 +6,7 @@ use std::{ use common_error::DaftResult; use daft_micropartition::MicroPartition; -use daft_table::Table; +use daft_recordbatch::RecordBatch; use crate::{FileWriter, TargetInMemorySizeBytesCalculator, WriterFactory}; @@ -15,7 +15,7 @@ use crate::{FileWriter, TargetInMemorySizeBytesCalculator, WriterFactory}; // Having a size range instead of exact size allows for more flexibility in the size of the produced Micropartitions, // and reducing the amount of '.slice' operations. struct SizeBasedBuffer { - buffer: VecDeque<(Table, usize)>, + buffer: VecDeque<(RecordBatch, usize)>, size_bytes: usize, } @@ -121,7 +121,7 @@ impl SizeBasedBuffer { // a row group at a time. pub struct TargetBatchWriter { size_calculator: Arc, - writer: Box, Result = Option
>>, + writer: Box, Result = Option>>, buffer: SizeBasedBuffer, is_closed: bool, } @@ -133,7 +133,7 @@ impl TargetBatchWriter { pub fn new( size_calculator: Arc, - writer: Box, Result = Option
>>, + writer: Box, Result = Option>>, ) -> Self { Self { size_calculator, @@ -157,7 +157,7 @@ impl TargetBatchWriter { impl FileWriter for TargetBatchWriter { type Input = Arc; - type Result = Option
; + type Result = Option; fn write(&mut self, input: Arc) -> DaftResult { assert!( @@ -200,13 +200,16 @@ impl FileWriter for TargetBatchWriter { } pub struct TargetBatchWriterFactory { - writer_factory: Arc, Result = Option
>>, + writer_factory: + Arc, Result = Option>>, size_calculator: Arc, } impl TargetBatchWriterFactory { pub fn new( - writer_factory: Arc, Result = Option
>>, + writer_factory: Arc< + dyn WriterFactory, Result = Option>, + >, size_calculator: Arc, ) -> Self { Self { @@ -218,12 +221,12 @@ impl TargetBatchWriterFactory { impl WriterFactory for TargetBatchWriterFactory { type Input = Arc; - type Result = Option
; + type Result = Option; fn create_writer( &self, file_idx: usize, - partition_values: Option<&Table>, + partition_values: Option<&RecordBatch>, ) -> DaftResult>> { let writer = self .writer_factory diff --git a/src/daft-writers/src/catalog.rs b/src/daft-writers/src/catalog.rs index 1520e2ad8f..8109533578 100644 --- a/src/daft-writers/src/catalog.rs +++ b/src/daft-writers/src/catalog.rs @@ -3,7 +3,7 @@ use std::sync::Arc; use common_error::{DaftError, DaftResult}; use daft_logical_plan::{CatalogType, DeltaLakeCatalogInfo, IcebergCatalogInfo}; use daft_micropartition::MicroPartition; -use daft_table::Table; +use daft_recordbatch::RecordBatch; use crate::{pyarrow::PyArrowWriter, FileWriter, WriterFactory}; @@ -24,12 +24,12 @@ impl CatalogWriterFactory { impl WriterFactory for CatalogWriterFactory { type Input = Arc; - type Result = Option
; + type Result = Option; fn create_writer( &self, file_idx: usize, - partition_values: Option<&Table>, + partition_values: Option<&RecordBatch>, ) -> DaftResult>> { match self.native { true => unimplemented!(), @@ -44,9 +44,9 @@ impl WriterFactory for CatalogWriterFactory { pub fn create_pyarrow_catalog_writer( file_idx: usize, - partition_values: Option<&Table>, + partition_values: Option<&RecordBatch>, catalog_info: &CatalogType, -) -> DaftResult, Result = Option
>>> { +) -> DaftResult, Result = Option>>> { match catalog_info { CatalogType::DeltaLake(DeltaLakeCatalogInfo { path, diff --git a/src/daft-writers/src/file.rs b/src/daft-writers/src/file.rs index b46c8f2fc9..e4344bde7a 100644 --- a/src/daft-writers/src/file.rs +++ b/src/daft-writers/src/file.rs @@ -2,7 +2,7 @@ use std::{cmp::max, sync::Arc}; use common_error::DaftResult; use daft_micropartition::MicroPartition; -use daft_table::Table; +use daft_recordbatch::RecordBatch; use crate::{FileWriter, TargetInMemorySizeBytesCalculator, WriterFactory}; @@ -12,21 +12,24 @@ struct TargetFileSizeWriter { current_in_memory_size_estimate: usize, current_in_memory_bytes_written: usize, total_physical_bytes_written: usize, - current_writer: Box, Result = Option
>>, - writer_factory: Arc, Result = Option
>>, + current_writer: Box, Result = Option>>, + writer_factory: + Arc, Result = Option>>, size_calculator: Arc, - results: Vec
, - partition_values: Option
, + results: Vec, + partition_values: Option, is_closed: bool, } impl TargetFileSizeWriter { fn new( - writer_factory: Arc, Result = Option
>>, - partition_values: Option
, + writer_factory: Arc< + dyn WriterFactory, Result = Option>, + >, + partition_values: Option, size_calculator: Arc, ) -> DaftResult { - let writer: Box, Result = Option
>> = + let writer: Box, Result = Option>> = writer_factory.create_writer(0, partition_values.as_ref())?; let estimate = size_calculator.calculate_target_in_memory_size_bytes(); Ok(Self { @@ -87,7 +90,7 @@ impl TargetFileSizeWriter { impl FileWriter for TargetFileSizeWriter { type Input = Arc; - type Result = Vec
; + type Result = Vec; fn write(&mut self, input: Arc) -> DaftResult { assert!( @@ -152,13 +155,16 @@ impl FileWriter for TargetFileSizeWriter { } pub(crate) struct TargetFileSizeWriterFactory { - writer_factory: Arc, Result = Option
>>, + writer_factory: + Arc, Result = Option>>, size_calculator: Arc, } impl TargetFileSizeWriterFactory { pub(crate) fn new( - writer_factory: Arc, Result = Option
>>, + writer_factory: Arc< + dyn WriterFactory, Result = Option>, + >, size_calculator: Arc, ) -> Self { Self { @@ -170,12 +176,12 @@ impl TargetFileSizeWriterFactory { impl WriterFactory for TargetFileSizeWriterFactory { type Input = Arc; - type Result = Vec
; + type Result = Vec; fn create_writer( &self, _file_idx: usize, - partition_values: Option<&Table>, + partition_values: Option<&RecordBatch>, ) -> DaftResult>> { Ok(Box::new(TargetFileSizeWriter::new( self.writer_factory.clone(), diff --git a/src/daft-writers/src/lance.rs b/src/daft-writers/src/lance.rs index c06023f294..7f6c1206a3 100644 --- a/src/daft-writers/src/lance.rs +++ b/src/daft-writers/src/lance.rs @@ -3,7 +3,7 @@ use std::sync::Arc; use common_error::DaftResult; use daft_logical_plan::LanceCatalogInfo; use daft_micropartition::{python::PyMicroPartition, MicroPartition}; -use daft_table::{python::PyTable, Table}; +use daft_recordbatch::{python::PyRecordBatch, RecordBatch}; use pyo3::{types::PyAnyMethods, Python}; use crate::{FileWriter, WriterFactory}; @@ -11,7 +11,7 @@ use crate::{FileWriter, WriterFactory}; pub struct LanceWriter { is_closed: bool, lance_info: LanceCatalogInfo, - results: Vec
, + results: Vec, bytes_written: usize, } @@ -28,7 +28,7 @@ impl LanceWriter { impl FileWriter for LanceWriter { type Input = Arc; - type Result = Vec
; + type Result = Vec; fn write(&mut self, data: Self::Input) -> DaftResult { assert!(!self.is_closed, "Cannot write to a closed LanceWriter"); @@ -37,12 +37,12 @@ impl FileWriter for LanceWriter { .expect("MicroPartition should have size_bytes for LanceWriter"); Python::with_gil(|py| { let py_micropartition = py - .import(pyo3::intern!(py, "daft.table"))? + .import(pyo3::intern!(py, "daft.recordbatch"))? .getattr(pyo3::intern!(py, "MicroPartition"))? .getattr(pyo3::intern!(py, "_from_pymicropartition"))? .call1((PyMicroPartition::from(data),))?; - let written_fragments: PyTable = py - .import(pyo3::intern!(py, "daft.table.table_io"))? + let written_fragments: PyRecordBatch = py + .import(pyo3::intern!(py, "daft.recordbatch.recordbatch_io"))? .getattr(pyo3::intern!(py, "write_lance"))? .call1(( py_micropartition, @@ -77,7 +77,7 @@ impl FileWriter for LanceWriter { pub fn make_lance_writer_factory( lance_info: LanceCatalogInfo, -) -> Arc, Result = Vec
>> { +) -> Arc, Result = Vec>> { Arc::new(LanceWriterFactory { lance_info }) } @@ -88,12 +88,12 @@ pub struct LanceWriterFactory { impl WriterFactory for LanceWriterFactory { type Input = Arc; - type Result = Vec
; + type Result = Vec; fn create_writer( &self, _file_idx: usize, - _partition_values: Option<&Table>, + _partition_values: Option<&RecordBatch>, ) -> DaftResult>> { let writer = LanceWriter::new(self.lance_info.clone()); Ok(Box::new(writer)) diff --git a/src/daft-writers/src/lib.rs b/src/daft-writers/src/lib.rs index e0f98a3b6d..794401658a 100644 --- a/src/daft-writers/src/lib.rs +++ b/src/daft-writers/src/lib.rs @@ -27,7 +27,7 @@ use common_file_formats::FileFormat; use daft_dsl::ExprRef; use daft_logical_plan::OutputFileInfo; use daft_micropartition::MicroPartition; -use daft_table::Table; +use daft_recordbatch::RecordBatch; use file::TargetFileSizeWriterFactory; #[cfg(feature = "python")] pub use lance::make_lance_writer_factory; @@ -60,14 +60,14 @@ pub trait WriterFactory: Send + Sync { fn create_writer( &self, file_idx: usize, - partition_values: Option<&Table>, + partition_values: Option<&RecordBatch>, ) -> DaftResult>>; } pub fn make_physical_writer_factory( file_info: &OutputFileInfo, cfg: &DaftExecutionConfig, -) -> Arc, Result = Vec
>> { +) -> Arc, Result = Vec>> { let base_writer_factory = PhysicalWriterFactory::new(file_info.clone()); match file_info.file_format { FileFormat::Parquet => { @@ -131,7 +131,7 @@ pub fn make_catalog_writer_factory( catalog_info: &daft_logical_plan::CatalogType, partition_cols: &Option>, cfg: &DaftExecutionConfig, -) -> Arc, Result = Vec
>> { +) -> Arc, Result = Vec>> { use catalog::CatalogWriterFactory; let base_writer_factory = CatalogWriterFactory::new(catalog_info.clone()); diff --git a/src/daft-writers/src/partition.rs b/src/daft-writers/src/partition.rs index 77871cbd01..21dcf2de64 100644 --- a/src/daft-writers/src/partition.rs +++ b/src/daft-writers/src/partition.rs @@ -8,7 +8,7 @@ use daft_core::{array::ops::as_arrow::AsArrow, utils::identity_hash_set::IndexHa use daft_dsl::ExprRef; use daft_io::IOStatsContext; use daft_micropartition::MicroPartition; -use daft_table::Table; +use daft_recordbatch::RecordBatch; use crate::{FileWriter, WriterFactory}; @@ -16,17 +16,21 @@ use crate::{FileWriter, WriterFactory}; /// to a separate file. It uses a map to keep track of the writers for each partition. struct PartitionedWriter { // TODO: Figure out a way to NOT use the IndexHash + RawEntryMut pattern here. Ideally we want to store ScalarValues, aka. single Rows of the partition values as keys for the hashmap. - per_partition_writers: - HashMap, Result = Vec
>>>, - saved_partition_values: Vec
, - writer_factory: Arc, Result = Vec
>>, + per_partition_writers: HashMap< + IndexHash, + Box, Result = Vec>>, + >, + saved_partition_values: Vec, + writer_factory: Arc, Result = Vec>>, partition_by: Vec, is_closed: bool, } impl PartitionedWriter { pub fn new( - writer_factory: Arc, Result = Vec
>>, + writer_factory: Arc< + dyn WriterFactory, Result = Vec>, + >, partition_by: Vec, ) -> Self { Self { @@ -41,7 +45,7 @@ impl PartitionedWriter { fn partition( partition_cols: &[ExprRef], data: Arc, - ) -> DaftResult<(Vec
, Table)> { + ) -> DaftResult<(Vec, RecordBatch)> { let data = data.concat_or_get(IOStatsContext::new("MicroPartition::partition_by_value"))?; let table = data.first().unwrap(); let (split_tables, partition_values) = table.partition_by_value(partition_cols)?; @@ -51,7 +55,7 @@ impl PartitionedWriter { impl FileWriter for PartitionedWriter { type Input = Arc; - type Result = Vec
; + type Result = Vec; fn write(&mut self, input: Arc) -> DaftResult { assert!( @@ -130,13 +134,15 @@ impl FileWriter for PartitionedWriter { } pub(crate) struct PartitionedWriterFactory { - writer_factory: Arc, Result = Vec
>>, + writer_factory: Arc, Result = Vec>>, partition_cols: Vec, } impl PartitionedWriterFactory { pub(crate) fn new( - writer_factory: Arc, Result = Vec
>>, + writer_factory: Arc< + dyn WriterFactory, Result = Vec>, + >, partition_cols: Vec, ) -> Self { Self { @@ -147,12 +153,12 @@ impl PartitionedWriterFactory { } impl WriterFactory for PartitionedWriterFactory { type Input = Arc; - type Result = Vec
; + type Result = Vec; fn create_writer( &self, _file_idx: usize, - _partition_values: Option<&Table>, + _partition_values: Option<&RecordBatch>, ) -> DaftResult>> { Ok(Box::new(PartitionedWriter::new( self.writer_factory.clone(), diff --git a/src/daft-writers/src/physical.rs b/src/daft-writers/src/physical.rs index fb5e3ca5d6..e1c3a090c9 100644 --- a/src/daft-writers/src/physical.rs +++ b/src/daft-writers/src/physical.rs @@ -4,7 +4,7 @@ use common_error::{DaftError, DaftResult}; use common_file_formats::FileFormat; use daft_logical_plan::OutputFileInfo; use daft_micropartition::MicroPartition; -use daft_table::Table; +use daft_recordbatch::RecordBatch; use crate::{FileWriter, WriterFactory}; @@ -25,12 +25,12 @@ impl PhysicalWriterFactory { impl WriterFactory for PhysicalWriterFactory { type Input = Arc; - type Result = Option
; + type Result = Option; fn create_writer( &self, file_idx: usize, - partition_values: Option<&Table>, + partition_values: Option<&RecordBatch>, ) -> DaftResult>> { match self.native { true => unimplemented!(), @@ -55,8 +55,8 @@ pub fn create_pyarrow_file_writer( compression: &Option, io_config: &Option, format: FileFormat, - partition: Option<&Table>, -) -> DaftResult, Result = Option
>>> { + partition: Option<&RecordBatch>, +) -> DaftResult, Result = Option>>> { match format { #[cfg(feature = "python")] FileFormat::Parquet => Ok(Box::new(crate::pyarrow::PyArrowWriter::new_parquet_writer( diff --git a/src/daft-writers/src/pyarrow.rs b/src/daft-writers/src/pyarrow.rs index d969998a50..3148f546f2 100644 --- a/src/daft-writers/src/pyarrow.rs +++ b/src/daft-writers/src/pyarrow.rs @@ -2,7 +2,7 @@ use std::sync::Arc; use common_error::DaftResult; use daft_micropartition::{python::PyMicroPartition, MicroPartition}; -use daft_table::{python::PyTable, Table}; +use daft_recordbatch::{python::PyRecordBatch, RecordBatch}; use pyo3::{types::PyAnyMethods, PyObject, Python}; use crate::FileWriter; @@ -19,18 +19,18 @@ impl PyArrowWriter { file_idx: usize, compression: &Option, io_config: &Option, - partition_values: Option<&Table>, + partition_values: Option<&RecordBatch>, ) -> DaftResult { Python::with_gil(|py| { let file_writer_module = py.import(pyo3::intern!(py, "daft.io.writer"))?; let file_writer_class = file_writer_module.getattr("ParquetFileWriter")?; let _from_pytable = py - .import(pyo3::intern!(py, "daft.table"))? - .getattr(pyo3::intern!(py, "Table"))? + .import(pyo3::intern!(py, "daft.recordbatch"))? + .getattr(pyo3::intern!(py, "RecordBatch"))? .getattr(pyo3::intern!(py, "_from_pytable"))?; let partition_values = match partition_values { Some(pv) => { - let py_table = _from_pytable.call1((PyTable::from(pv.clone()),))?; + let py_table = _from_pytable.call1((PyRecordBatch::from(pv.clone()),))?; Some(py_table) } None => None, @@ -57,18 +57,18 @@ impl PyArrowWriter { root_dir: &str, file_idx: usize, io_config: &Option, - partition_values: Option<&Table>, + partition_values: Option<&RecordBatch>, ) -> DaftResult { Python::with_gil(|py| { let file_writer_module = py.import(pyo3::intern!(py, "daft.io.writer"))?; let file_writer_class = file_writer_module.getattr("CSVFileWriter")?; let _from_pytable = py - .import(pyo3::intern!(py, "daft.table"))? - .getattr(pyo3::intern!(py, "Table"))? + .import(pyo3::intern!(py, "daft.recordbatch"))? + .getattr(pyo3::intern!(py, "RecordBatch"))? .getattr(pyo3::intern!(py, "_from_pytable"))?; let partition_values = match partition_values { Some(pv) => { - let py_table = _from_pytable.call1((PyTable::from(pv.clone()),))?; + let py_table = _from_pytable.call1((PyRecordBatch::from(pv.clone()),))?; Some(py_table) } None => None, @@ -95,19 +95,19 @@ impl PyArrowWriter { schema: &pyo3::Py, properties: &pyo3::Py, partition_spec_id: i64, - partition_values: Option<&Table>, + partition_values: Option<&RecordBatch>, io_config: &Option, ) -> DaftResult { Python::with_gil(|py| { let file_writer_module = py.import(pyo3::intern!(py, "daft.io.writer"))?; let file_writer_class = file_writer_module.getattr("IcebergWriter")?; let _from_pytable = py - .import(pyo3::intern!(py, "daft.table"))? - .getattr(pyo3::intern!(py, "Table"))? + .import(pyo3::intern!(py, "daft.recordbatch"))? + .getattr(pyo3::intern!(py, "RecordBatch"))? .getattr(pyo3::intern!(py, "_from_pytable"))?; let partition_values = match partition_values { Some(pv) => { - let py_table = _from_pytable.call1((PyTable::from(pv.clone()),))?; + let py_table = _from_pytable.call1((PyRecordBatch::from(pv.clone()),))?; Some(py_table) } None => None, @@ -136,19 +136,19 @@ impl PyArrowWriter { file_idx: usize, version: i32, large_dtypes: bool, - partition_values: Option<&Table>, + partition_values: Option<&RecordBatch>, io_config: &Option, ) -> DaftResult { Python::with_gil(|py| { let file_writer_module = py.import(pyo3::intern!(py, "daft.io.writer"))?; let file_writer_class = file_writer_module.getattr("DeltalakeWriter")?; let _from_pytable = py - .import(pyo3::intern!(py, "daft.table"))? - .getattr(pyo3::intern!(py, "Table"))? + .import(pyo3::intern!(py, "daft.recordbatch"))? + .getattr(pyo3::intern!(py, "RecordBatch"))? .getattr(pyo3::intern!(py, "_from_pytable"))?; let partition_values = match partition_values { Some(pv) => { - let py_table = _from_pytable.call1((PyTable::from(pv.clone()),))?; + let py_table = _from_pytable.call1((PyRecordBatch::from(pv.clone()),))?; Some(py_table) } None => None, @@ -174,13 +174,13 @@ impl PyArrowWriter { impl FileWriter for PyArrowWriter { type Input = Arc; - type Result = Option
; + type Result = Option; fn write(&mut self, data: Self::Input) -> DaftResult { assert!(!self.is_closed, "Cannot write to a closed PyArrowWriter"); let bytes_written = Python::with_gil(|py| { let py_micropartition = py - .import(pyo3::intern!(py, "daft.table"))? + .import(pyo3::intern!(py, "daft.recordbatch"))? .getattr(pyo3::intern!(py, "MicroPartition"))? .getattr(pyo3::intern!(py, "_from_pymicropartition"))? .call1((PyMicroPartition::from(data),))?; @@ -203,7 +203,7 @@ impl FileWriter for PyArrowWriter { .py_writer .call_method0(py, pyo3::intern!(py, "close"))? .getattr(py, pyo3::intern!(py, "_table"))?; - Ok(Some(result.extract::(py)?.into())) + Ok(Some(result.extract::(py)?.into())) }) } } diff --git a/src/daft-writers/src/test.rs b/src/daft-writers/src/test.rs index e982cdad0a..f0bedb3c50 100644 --- a/src/daft-writers/src/test.rs +++ b/src/daft-writers/src/test.rs @@ -6,7 +6,7 @@ use daft_core::{ series::IntoSeries, }; use daft_micropartition::MicroPartition; -use daft_table::Table; +use daft_recordbatch::RecordBatch; use crate::{FileWriter, WriterFactory}; @@ -14,12 +14,12 @@ pub(crate) struct DummyWriterFactory; impl WriterFactory for DummyWriterFactory { type Input = Arc; - type Result = Option
; + type Result = Option; fn create_writer( &self, file_idx: usize, - partition_values: Option<&Table>, + partition_values: Option<&RecordBatch>, ) -> DaftResult>> { Ok(Box::new(DummyWriter { file_idx: file_idx.to_string(), @@ -35,14 +35,14 @@ impl WriterFactory for DummyWriterFactory { pub(crate) struct DummyWriter { file_idx: String, - partition_values: Option
, + partition_values: Option, write_count: usize, byte_count: usize, } impl FileWriter for DummyWriter { type Input = Arc; - type Result = Option
; + type Result = Option; fn write(&mut self, input: Self::Input) -> DaftResult { self.write_count += 1; @@ -61,7 +61,7 @@ impl FileWriter for DummyWriter { let write_count_series = UInt64Array::from_values("write_count", std::iter::once(self.write_count as u64)) .into_series(); - let path_table = Table::new_unchecked( + let path_table = RecordBatch::new_unchecked( Schema::new(vec![ path_series.field().clone(), write_count_series.field().clone(), @@ -83,7 +83,7 @@ pub(crate) fn make_dummy_mp(size_bytes: usize) -> Arc { let series = UInt8Array::from_values("ints", std::iter::repeat(42).take(size_bytes)).into_series(); let schema = Arc::new(Schema::new(vec![series.field().clone()]).unwrap()); - let table = Table::new_unchecked(schema.clone(), vec![series.into()], size_bytes); + let table = RecordBatch::new_unchecked(schema.clone(), vec![series.into()], size_bytes); Arc::new(MicroPartition::new_loaded( schema.into(), vec![table].into(), diff --git a/src/lib.rs b/src/lib.rs index 7a39ddf361..483dfe58fe 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -107,7 +107,7 @@ pub mod pylib { daft_core::python::register_modules(m)?; daft_local_execution::register_modules(m)?; daft_dsl::register_modules(m)?; - daft_table::register_modules(m)?; + daft_recordbatch::register_modules(m)?; daft_io::register_modules(m)?; daft_parquet::register_modules(m)?; daft_csv::register_modules(m)?; diff --git a/tests/actor_pool/test_pyactor_pool.py b/tests/actor_pool/test_pyactor_pool.py index f97b3bf8a8..3ec5a0d32b 100644 --- a/tests/actor_pool/test_pyactor_pool.py +++ b/tests/actor_pool/test_pyactor_pool.py @@ -8,9 +8,9 @@ from daft.context import get_context from daft.execution.execution_step import ActorPoolProject from daft.expressions import ExpressionsProjection +from daft.recordbatch import MicroPartition from daft.runners.partitioning import PartialPartitionMetadata from daft.runners.pyrunner import AcquiredResources, PyActorPool, PyRunner -from daft.table import MicroPartition from tests.conftest import get_tests_daft_runner_name diff --git a/tests/actor_pool/test_ray_actor_pool.py b/tests/actor_pool/test_ray_actor_pool.py index 086dcda2a9..94c91f5eda 100644 --- a/tests/actor_pool/test_ray_actor_pool.py +++ b/tests/actor_pool/test_ray_actor_pool.py @@ -4,9 +4,9 @@ from daft import DataType, ResourceRequest from daft.daft import PyDaftExecutionConfig from daft.expressions import ExpressionsProjection +from daft.recordbatch import MicroPartition from daft.runners.partitioning import PartialPartitionMetadata from daft.runners.ray_runner import RayRoundRobinActorPool -from daft.table import MicroPartition @daft.udf(return_dtype=DataType.int64()) diff --git a/tests/conftest.py b/tests/conftest.py index ad5d761f28..a958430928 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -9,7 +9,7 @@ import daft import daft.context -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition # import all conftest from tests.integration.io.conftest import * @@ -118,11 +118,11 @@ def _make_df( import pyarrow.parquet as papq name = str(uuid.uuid4()) - daft_table = MicroPartition.from_arrow(pa_table) + daft_recordbatch = MicroPartition.from_arrow(pa_table) partitioned_tables = ( - daft_table.partition_by_random(repartition, 0) + daft_recordbatch.partition_by_random(repartition, 0) if len(repartition_columns) == 0 - else daft_table.partition_by_hash([daft.col(c) for c in repartition_columns], repartition) + else daft_recordbatch.partition_by_hash([daft.col(c) for c in repartition_columns], repartition) ) for i, tbl in enumerate(partitioned_tables): tmp_file = tmp_path / (name + f"-{i}") diff --git a/tests/dataframe/test_describe.py b/tests/dataframe/test_describe.py index 7970b2a338..ddc11a0f39 100644 --- a/tests/dataframe/test_describe.py +++ b/tests/dataframe/test_describe.py @@ -1,5 +1,5 @@ import daft -import daft.table +import daft.recordbatch from daft import DataFrame, Schema from daft import DataType as dt from daft.logical.schema import Field @@ -36,7 +36,7 @@ def test_describe(): Field.create("c_sparse_tensor", dt.sparse_tensor(dt.float32())), ] # create an empty table with known schema - mp = daft.table.MicroPartition.empty(Schema._from_fields(fields)) + mp = daft.recordbatch.MicroPartition.empty(Schema._from_fields(fields)) df = DataFrame._from_tables(mp) # describe.. df = df.describe().collect() diff --git a/tests/dataframe/test_monotonically_increasing_id.py b/tests/dataframe/test_monotonically_increasing_id.py index 2d52ec4869..c513370e46 100644 --- a/tests/dataframe/test_monotonically_increasing_id.py +++ b/tests/dataframe/test_monotonically_increasing_id.py @@ -4,7 +4,7 @@ from daft.datatype import DataType from daft.io._generator import read_generator -from daft.table.table import Table +from daft.recordbatch.recordbatch import RecordBatch from tests.conftest import get_tests_daft_runner_name @@ -57,7 +57,7 @@ def test_monotonically_increasing_id_multiple_partitions_with_into_partition(mak def test_monotonically_increasing_id_from_generator() -> None: ITEMS = list(range(10)) - table = Table.from_pydict({"a": ITEMS}) + table = RecordBatch.from_pydict({"a": ITEMS}) num_tables = 3 num_generators = 3 diff --git a/tests/dataframe/test_shuffles.py b/tests/dataframe/test_shuffles.py index 4b7cae6cb9..e52e56c862 100644 --- a/tests/dataframe/test_shuffles.py +++ b/tests/dataframe/test_shuffles.py @@ -8,7 +8,7 @@ import daft from daft.io._generator import read_generator -from daft.table.table import Table +from daft.recordbatch.recordbatch import RecordBatch from tests.conftest import get_tests_daft_runner_name @@ -17,7 +17,7 @@ def generate(num_rows: int, bytes_per_row: int): "ints": np.random.randint(0, num_rows, num_rows, dtype=np.uint64), "bytes": [os.urandom(bytes_per_row) for _ in range(num_rows)], } - yield Table.from_pydict(data) + yield RecordBatch.from_pydict(data) def generator( diff --git a/tests/expressions/test_expressions.py b/tests/expressions/test_expressions.py index 723747f9be..73fc3b40c8 100644 --- a/tests/expressions/test_expressions.py +++ b/tests/expressions/test_expressions.py @@ -10,8 +10,8 @@ from daft.datatype import DataType, TimeUnit from daft.expressions import col, lit from daft.expressions.testing import expr_structurally_equal +from daft.recordbatch import MicroPartition from daft.series import Series -from daft.table import MicroPartition @pytest.mark.parametrize( diff --git a/tests/expressions/test_expressions_projection.py b/tests/expressions/test_expressions_projection.py index f4f5a89444..970580afc4 100644 --- a/tests/expressions/test_expressions_projection.py +++ b/tests/expressions/test_expressions_projection.py @@ -4,7 +4,7 @@ from daft.expressions import Expression, ExpressionsProjection, col from daft.expressions.testing import expr_structurally_equal -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition def test_expressions_projection_error_dup_name(): diff --git a/tests/expressions/test_null_safe_equals.py b/tests/expressions/test_null_safe_equals.py index 3f01f748d4..50dc1375a4 100644 --- a/tests/expressions/test_null_safe_equals.py +++ b/tests/expressions/test_null_safe_equals.py @@ -4,7 +4,7 @@ import pytest from daft.expressions import col, lit -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition @pytest.mark.parametrize( diff --git a/tests/expressions/test_udf.py b/tests/expressions/test_udf.py index ff43df639e..60514e5c92 100644 --- a/tests/expressions/test_udf.py +++ b/tests/expressions/test_udf.py @@ -9,8 +9,8 @@ from daft.datatype import DataType from daft.expressions import Expression from daft.expressions.testing import expr_structurally_equal +from daft.recordbatch import MicroPartition from daft.series import Series -from daft.table import MicroPartition from daft.udf import udf diff --git a/tests/expressions/typing/conftest.py b/tests/expressions/typing/conftest.py index ee5222b8a0..417b9e833b 100644 --- a/tests/expressions/typing/conftest.py +++ b/tests/expressions/typing/conftest.py @@ -10,8 +10,8 @@ from daft.datatype import DataType from daft.expressions import Expression, ExpressionsProjection +from daft.recordbatch import MicroPartition from daft.series import Series -from daft.table import MicroPartition ALL_DTYPES = [ (DataType.int8(), pa.array([1, 2, None], type=pa.int8())), diff --git a/tests/integration/io/csv/test_read_pushdowns.py b/tests/integration/io/csv/test_read_pushdowns.py index b65369c731..8fc4ae256d 100644 --- a/tests/integration/io/csv/test_read_pushdowns.py +++ b/tests/integration/io/csv/test_read_pushdowns.py @@ -6,7 +6,7 @@ import daft from daft.daft import CsvConvertOptions -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition PRED_PUSHDOWN_FILES = [ "s3://daft-public-data/test_fixtures/csv-dev/sampled-tpch.csv", diff --git a/tests/integration/io/jsonl/test_read_pushdown.py b/tests/integration/io/jsonl/test_read_pushdown.py index 811d0934bd..b4d50be374 100644 --- a/tests/integration/io/jsonl/test_read_pushdown.py +++ b/tests/integration/io/jsonl/test_read_pushdown.py @@ -6,7 +6,7 @@ import daft from daft.daft import JsonConvertOptions -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition PRED_PUSHDOWN_FILES = [ "s3://daft-public-data/test_fixtures/json-dev/sampled-tpch.jsonl", diff --git a/tests/integration/io/parquet/test_read_pushdowns.py b/tests/integration/io/parquet/test_read_pushdowns.py index 2e001501eb..bdcdafe9c4 100644 --- a/tests/integration/io/parquet/test_read_pushdowns.py +++ b/tests/integration/io/parquet/test_read_pushdowns.py @@ -5,7 +5,7 @@ import pytest import daft -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition PRED_PUSHDOWN_FILES = [ "s3://daft-public-data/test_fixtures/parquet-dev/sampled-tpch-with-stats.parquet", diff --git a/tests/integration/io/parquet/test_reads_local_fixtures.py b/tests/integration/io/parquet/test_reads_local_fixtures.py index bb36125c20..a00dd82ef4 100644 --- a/tests/integration/io/parquet/test_reads_local_fixtures.py +++ b/tests/integration/io/parquet/test_reads_local_fixtures.py @@ -4,7 +4,7 @@ import pytest -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition BUCKETS = ["head-retries-parquet-bucket", "get-retries-parquet-bucket"] diff --git a/tests/integration/io/parquet/test_reads_public_data.py b/tests/integration/io/parquet/test_reads_public_data.py index dff82b5fb0..ecf6410666 100644 --- a/tests/integration/io/parquet/test_reads_public_data.py +++ b/tests/integration/io/parquet/test_reads_public_data.py @@ -7,10 +7,10 @@ from pyarrow import parquet as pq import daft -import daft.table +import daft.recordbatch from daft.exceptions import ConnectTimeoutError, ReadTimeoutError from daft.filesystem import get_filesystem, get_protocol_from_path -from daft.table import MicroPartition, Table +from daft.recordbatch import MicroPartition, RecordBatch def get_filesystem_from_path(path: str, **kwargs) -> fsspec.AbstractFileSystem: @@ -259,7 +259,7 @@ def test_parquet_read_table(parquet_file, public_storage_io_config, multithreade ) def test_parquet_read_table_into_pyarrow(parquet_file, public_storage_io_config, multithreaded_io): _, url = parquet_file - daft_native_read = daft.table.read_parquet_into_pyarrow( + daft_native_read = daft.recordbatch.read_parquet_into_pyarrow( url, io_config=public_storage_io_config, multithreaded_io=multithreaded_io ) pa_read = read_parquet_with_pyarrow(url) @@ -281,7 +281,7 @@ def test_parquet_read_table_bulk(parquet_file, public_storage_io_config, multith pa_read = MicroPartition.from_arrow(read_parquet_with_pyarrow(url)) # Legacy Table returns a list[Table] - if MicroPartition == Table: + if MicroPartition == RecordBatch: for daft_native_read in daft_native_reads: assert daft_native_read.schema() == pa_read.schema() pd.testing.assert_frame_equal(daft_native_read.to_pandas(), pa_read.to_pandas()) @@ -300,7 +300,7 @@ def test_parquet_read_table_bulk(parquet_file, public_storage_io_config, multith ) def test_parquet_into_pyarrow_bulk(parquet_file, public_storage_io_config, multithreaded_io): _, url = parquet_file - daft_native_reads = daft.table.read_parquet_into_pyarrow_bulk( + daft_native_reads = daft.recordbatch.read_parquet_into_pyarrow_bulk( [url] * 2, io_config=public_storage_io_config, multithreaded_io=multithreaded_io ) pa_read = read_parquet_with_pyarrow(url) @@ -365,7 +365,7 @@ def test_row_groups_selection_bulk(public_storage_io_config, multithreaded_io): url = ["s3://daft-public-data/test_fixtures/parquet-dev/mvp.parquet"] * 11 row_groups = [list(range(10))] + [[i] for i in range(10)] - if MicroPartition == Table: + if MicroPartition == RecordBatch: first, *rest = MicroPartition.read_parquet_bulk( url, io_config=public_storage_io_config, multithreaded_io=multithreaded_io, row_groups_per_path=row_groups ) @@ -392,7 +392,7 @@ def test_row_groups_selection_bulk(public_storage_io_config, multithreaded_io): def test_row_groups_selection_into_pyarrow_bulk(public_storage_io_config, multithreaded_io): url = ["s3://daft-public-data/test_fixtures/parquet-dev/mvp.parquet"] * 11 row_groups = [list(range(10))] + [[i] for i in range(10)] - first, *rest = daft.table.read_parquet_into_pyarrow_bulk( + first, *rest = daft.recordbatch.read_parquet_into_pyarrow_bulk( url, io_config=public_storage_io_config, multithreaded_io=multithreaded_io, row_groups_per_path=row_groups ) assert len(first) == 100 @@ -458,7 +458,7 @@ def test_read_file_level_timeout(): ) with pytest.raises((ReadTimeoutError), match=f"Parquet reader timed out while trying to read: {url}"): - daft.table.read_parquet_into_pyarrow(url, io_config=read_timeout_config, file_timeout_ms=2) + daft.recordbatch.read_parquet_into_pyarrow(url, io_config=read_timeout_config, file_timeout_ms=2) @pytest.mark.integration() diff --git a/tests/integration/io/parquet/test_reads_s3_minio.py b/tests/integration/io/parquet/test_reads_s3_minio.py index cd5d9ef988..2d3b918cf3 100644 --- a/tests/integration/io/parquet/test_reads_s3_minio.py +++ b/tests/integration/io/parquet/test_reads_s3_minio.py @@ -22,7 +22,7 @@ def test_minio_parquet_bulk_readback(minio_io_config): for path in target_paths: pq.write_table(pa_table, path, filesystem=fs) - readback = daft.table.read_parquet_into_pyarrow_bulk(target_paths, io_config=minio_io_config) + readback = daft.recordbatch.read_parquet_into_pyarrow_bulk(target_paths, io_config=minio_io_config) assert len(readback) == len(target_paths) for tab in readback: assert tab.to_pydict() == data diff --git a/tests/io/test_hive_style_partitions.py b/tests/io/test_hive_style_partitions.py index e6e67bb040..8941f90764 100644 --- a/tests/io/test_hive_style_partitions.py +++ b/tests/io/test_hive_style_partitions.py @@ -43,12 +43,12 @@ def unify_timestamp(table): ) -def assert_tables_equal(daft_table, pa_table): +def assert_tables_equal(daft_recordbatch, pa_table): sorted_pa_table = pa_table.sort_by([("id", "ascending")]).select(SCHEMA.names) sorted_pa_table = unify_timestamp(sorted_pa_table) - sorted_daft_table = daft_table.sort_by([("id", "ascending")]).select(SCHEMA.names) - sorted_daft_table = unify_timestamp(sorted_daft_table) - assert sorted_pa_table == sorted_daft_table + sorted_daft_recordbatch = daft_recordbatch.sort_by([("id", "ascending")]).select(SCHEMA.names) + sorted_daft_recordbatch = unify_timestamp(sorted_daft_recordbatch) + assert sorted_pa_table == sorted_daft_recordbatch @pytest.mark.parametrize( diff --git a/tests/io/test_parquet.py b/tests/io/test_parquet.py index 2557809012..8c6231d206 100644 --- a/tests/io/test_parquet.py +++ b/tests/io/test_parquet.py @@ -15,7 +15,7 @@ from daft.datatype import DataType, TimeUnit from daft.expressions import col from daft.logical.schema import Schema -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition from ..integration.io.conftest import minio_create_bucket diff --git a/tests/microbenchmarks/test_filter.py b/tests/microbenchmarks/test_filter.py index 6415331ccf..5e7fe34ab4 100644 --- a/tests/microbenchmarks/test_filter.py +++ b/tests/microbenchmarks/test_filter.py @@ -4,7 +4,7 @@ import daft from daft import DataFrame -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition NUM_ROWS = 1_000_000 diff --git a/tests/microbenchmarks/test_if_else.py b/tests/microbenchmarks/test_if_else.py index f1e2c79151..26e5203401 100644 --- a/tests/microbenchmarks/test_if_else.py +++ b/tests/microbenchmarks/test_if_else.py @@ -6,7 +6,7 @@ import daft from daft import DataFrame -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition NUM_ROWS = 1_000_000 diff --git a/tests/microbenchmarks/test_take.py b/tests/microbenchmarks/test_take.py index bb7b5f46dc..bb1fb9a29a 100644 --- a/tests/microbenchmarks/test_take.py +++ b/tests/microbenchmarks/test_take.py @@ -4,7 +4,7 @@ import daft from daft import DataFrame, Series -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition NUM_ROWS = 10_000_000 diff --git a/tests/table/__init__.py b/tests/recordbatch/__init__.py similarity index 100% rename from tests/table/__init__.py rename to tests/recordbatch/__init__.py diff --git a/tests/table/binary/test_concat.py b/tests/recordbatch/binary/test_concat.py similarity index 99% rename from tests/table/binary/test_concat.py rename to tests/recordbatch/binary/test_concat.py index cc603d5d39..46835db7cf 100644 --- a/tests/table/binary/test_concat.py +++ b/tests/recordbatch/binary/test_concat.py @@ -3,7 +3,7 @@ import pytest from daft.expressions import col, lit -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition @pytest.mark.parametrize( diff --git a/tests/table/binary/test_fixed_size_binary_concat.py b/tests/recordbatch/binary/test_fixed_size_binary_concat.py similarity index 99% rename from tests/table/binary/test_fixed_size_binary_concat.py rename to tests/recordbatch/binary/test_fixed_size_binary_concat.py index 9d87a5de4e..b0c58cb948 100644 --- a/tests/table/binary/test_fixed_size_binary_concat.py +++ b/tests/recordbatch/binary/test_fixed_size_binary_concat.py @@ -5,7 +5,7 @@ from daft import DataType from daft.expressions import col, lit -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition @pytest.mark.parametrize( diff --git a/tests/table/binary/test_fixed_size_binary_length.py b/tests/recordbatch/binary/test_fixed_size_binary_length.py similarity index 98% rename from tests/table/binary/test_fixed_size_binary_length.py rename to tests/recordbatch/binary/test_fixed_size_binary_length.py index 3ea737c7fc..90a76c1f76 100644 --- a/tests/table/binary/test_fixed_size_binary_length.py +++ b/tests/recordbatch/binary/test_fixed_size_binary_length.py @@ -5,7 +5,7 @@ from daft import DataType from daft.expressions import col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition @pytest.mark.parametrize( diff --git a/tests/table/binary/test_fixed_size_binary_slice.py b/tests/recordbatch/binary/test_fixed_size_binary_slice.py similarity index 99% rename from tests/table/binary/test_fixed_size_binary_slice.py rename to tests/recordbatch/binary/test_fixed_size_binary_slice.py index 81ce4e122e..19310a5204 100644 --- a/tests/table/binary/test_fixed_size_binary_slice.py +++ b/tests/recordbatch/binary/test_fixed_size_binary_slice.py @@ -5,7 +5,7 @@ from daft import DataType from daft.expressions import col, lit -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition @pytest.mark.parametrize( diff --git a/tests/table/binary/test_length.py b/tests/recordbatch/binary/test_length.py similarity index 98% rename from tests/table/binary/test_length.py rename to tests/recordbatch/binary/test_length.py index 6c497dc015..ec5838ddc3 100644 --- a/tests/table/binary/test_length.py +++ b/tests/recordbatch/binary/test_length.py @@ -3,7 +3,7 @@ import pytest from daft.expressions import col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition def test_binary_length() -> None: diff --git a/tests/table/binary/test_slice.py b/tests/recordbatch/binary/test_slice.py similarity index 99% rename from tests/table/binary/test_slice.py rename to tests/recordbatch/binary/test_slice.py index 7bf63d2ca8..ab9a11d92b 100644 --- a/tests/table/binary/test_slice.py +++ b/tests/recordbatch/binary/test_slice.py @@ -4,7 +4,7 @@ from daft import DataType from daft.expressions import col, lit -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition def test_binary_slice() -> None: diff --git a/tests/table/image/__init__.py b/tests/recordbatch/image/__init__.py similarity index 100% rename from tests/table/image/__init__.py rename to tests/recordbatch/image/__init__.py diff --git a/tests/table/image/conftest.py b/tests/recordbatch/image/conftest.py similarity index 100% rename from tests/table/image/conftest.py rename to tests/recordbatch/image/conftest.py diff --git a/tests/table/image/test_crop.py b/tests/recordbatch/image/test_crop.py similarity index 98% rename from tests/table/image/test_crop.py rename to tests/recordbatch/image/test_crop.py index 93d4fb6f8e..59da1f28eb 100644 --- a/tests/table/image/test_crop.py +++ b/tests/recordbatch/image/test_crop.py @@ -5,7 +5,7 @@ import pytest import daft -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition MODES = ["L", "LA", "RGB", "RGBA"] MODE_TO_NP_DTYPE = { diff --git a/tests/table/image/test_decode.py b/tests/recordbatch/image/test_decode.py similarity index 100% rename from tests/table/image/test_decode.py rename to tests/recordbatch/image/test_decode.py diff --git a/tests/table/image/test_to_mode.py b/tests/recordbatch/image/test_to_mode.py similarity index 96% rename from tests/table/image/test_to_mode.py rename to tests/recordbatch/image/test_to_mode.py index f5e4d70674..7abb4f380b 100644 --- a/tests/table/image/test_to_mode.py +++ b/tests/recordbatch/image/test_to_mode.py @@ -4,7 +4,7 @@ import daft from daft import DataType -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition MODE_TO_NUM_CHANNELS = { "L": 1, diff --git a/tests/table/json/__init__.py b/tests/recordbatch/json/__init__.py similarity index 100% rename from tests/table/json/__init__.py rename to tests/recordbatch/json/__init__.py diff --git a/tests/table/json/test_json_query.py b/tests/recordbatch/json/test_json_query.py similarity index 98% rename from tests/table/json/test_json_query.py rename to tests/recordbatch/json/test_json_query.py index d3e429fa47..7bd1b5cf7a 100644 --- a/tests/table/json/test_json_query.py +++ b/tests/recordbatch/json/test_json_query.py @@ -3,7 +3,7 @@ import pytest from daft.expressions import col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition # Chose a non-exhaustive sample of the filters provided at https://github.com/01mf02/jaq/tree/main diff --git a/tests/table/list/__init__.py b/tests/recordbatch/list/__init__.py similarity index 100% rename from tests/table/list/__init__.py rename to tests/recordbatch/list/__init__.py diff --git a/tests/table/list/test_list_bool_aggs.py b/tests/recordbatch/list/test_list_bool_aggs.py similarity index 96% rename from tests/table/list/test_list_bool_aggs.py rename to tests/recordbatch/list/test_list_bool_aggs.py index 077a01f214..530d183d99 100644 --- a/tests/table/list/test_list_bool_aggs.py +++ b/tests/recordbatch/list/test_list_bool_aggs.py @@ -4,7 +4,7 @@ from daft.datatype import DataType from daft.expressions import col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition # Test data includes various combinations of boolean values and nulls table = MicroPartition.from_pydict( diff --git a/tests/table/list/test_list_chunk.py b/tests/recordbatch/list/test_list_chunk.py similarity index 99% rename from tests/table/list/test_list_chunk.py rename to tests/recordbatch/list/test_list_chunk.py index 59b5a2ee3b..16e0da52d6 100644 --- a/tests/table/list/test_list_chunk.py +++ b/tests/recordbatch/list/test_list_chunk.py @@ -3,7 +3,7 @@ from daft.datatype import DataType from daft.expressions import col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition def test_list_chunk_empty_series(): diff --git a/tests/table/list/test_list_count_length.py b/tests/recordbatch/list/test_list_count_length.py similarity index 98% rename from tests/table/list/test_list_count_length.py rename to tests/recordbatch/list/test_list_count_length.py index 7e48f3088d..009c4eacb6 100644 --- a/tests/table/list/test_list_count_length.py +++ b/tests/recordbatch/list/test_list_count_length.py @@ -5,7 +5,7 @@ from daft.daft import CountMode from daft.datatype import DataType from daft.expressions import col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition @pytest.fixture diff --git a/tests/table/list/test_list_get.py b/tests/recordbatch/list/test_list_get.py similarity index 98% rename from tests/table/list/test_list_get.py rename to tests/recordbatch/list/test_list_get.py index a46fcbd926..fce07c1193 100644 --- a/tests/table/list/test_list_get.py +++ b/tests/recordbatch/list/test_list_get.py @@ -4,7 +4,7 @@ from daft.datatype import DataType from daft.expressions import col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition def test_list_get(): diff --git a/tests/table/list/test_list_join.py b/tests/recordbatch/list/test_list_join.py similarity index 96% rename from tests/table/list/test_list_join.py rename to tests/recordbatch/list/test_list_join.py index 8cc489ec1a..3cd466205d 100644 --- a/tests/table/list/test_list_join.py +++ b/tests/recordbatch/list/test_list_join.py @@ -3,7 +3,7 @@ import pytest from daft.expressions import col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition def test_list_join(): diff --git a/tests/table/list/test_list_numeric_aggs.py b/tests/recordbatch/list/test_list_numeric_aggs.py similarity index 96% rename from tests/table/list/test_list_numeric_aggs.py rename to tests/recordbatch/list/test_list_numeric_aggs.py index c77d2fe3dc..d228ebaf7c 100644 --- a/tests/table/list/test_list_numeric_aggs.py +++ b/tests/recordbatch/list/test_list_numeric_aggs.py @@ -4,7 +4,7 @@ from daft.datatype import DataType from daft.expressions import col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition table = MicroPartition.from_pydict({"a": [[1, 2], [3, 4], [5, None], [None, None], None]}) fixed_dtype = DataType.fixed_size_list(DataType.int64(), 2) diff --git a/tests/table/list/test_list_slice.py b/tests/recordbatch/list/test_list_slice.py similarity index 99% rename from tests/table/list/test_list_slice.py rename to tests/recordbatch/list/test_list_slice.py index 9ae0db4335..6eaa9b0d6e 100644 --- a/tests/table/list/test_list_slice.py +++ b/tests/recordbatch/list/test_list_slice.py @@ -3,7 +3,7 @@ from daft.datatype import DataType from daft.expressions import col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition def test_list_slice_empty_series(): diff --git a/tests/table/list/test_list_sort.py b/tests/recordbatch/list/test_list_sort.py similarity index 97% rename from tests/table/list/test_list_sort.py rename to tests/recordbatch/list/test_list_sort.py index 9f03ffb8c8..b9892b6dc3 100644 --- a/tests/table/list/test_list_sort.py +++ b/tests/recordbatch/list/test_list_sort.py @@ -2,7 +2,7 @@ from daft.datatype import DataType from daft.expressions import col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition def test_list_sort(): diff --git a/tests/table/map/__init__.py b/tests/recordbatch/map/__init__.py similarity index 100% rename from tests/table/map/__init__.py rename to tests/recordbatch/map/__init__.py diff --git a/tests/table/map/test_map_get.py b/tests/recordbatch/map/test_map_get.py similarity index 97% rename from tests/table/map/test_map_get.py rename to tests/recordbatch/map/test_map_get.py index 9c7548d1e3..1992915ca2 100644 --- a/tests/table/map/test_map_get.py +++ b/tests/recordbatch/map/test_map_get.py @@ -6,7 +6,7 @@ import pytest from daft.expressions import col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition def test_map_get(): diff --git a/tests/table/numeric/test_numeric.py b/tests/recordbatch/numeric/test_numeric.py similarity index 97% rename from tests/table/numeric/test_numeric.py rename to tests/recordbatch/numeric/test_numeric.py index 3e15685055..3758792d7f 100644 --- a/tests/table/numeric/test_numeric.py +++ b/tests/recordbatch/numeric/test_numeric.py @@ -9,8 +9,8 @@ import pytest from daft import col, lit -from daft.table import MicroPartition -from tests.table import daft_numeric_types +from daft.recordbatch import MicroPartition +from tests.recordbatch import daft_numeric_types OPS = [ ops.add, @@ -46,15 +46,15 @@ def test_table_numeric_expressions(data_dtype, op) -> None: a, b = [5, 6, 7, 8], [1, 2, 3, 4] pa_table = pa.Table.from_pydict({"a": a, "b": b}) - daft_table = MicroPartition.from_arrow(pa_table) - daft_table = daft_table.eval_expression_list( + daft_recordbatch = MicroPartition.from_arrow(pa_table) + daft_recordbatch = daft_recordbatch.eval_expression_list( [op(col("a").cast(data_dtype), col("b").cast(data_dtype)).alias("result")] ) - assert len(daft_table) == 4 - assert daft_table.column_names() == ["result"] + assert len(daft_recordbatch) == 4 + assert daft_recordbatch.column_names() == ["result"] pyresult = [op(left, right) for left, right in zip(a, b)] - assert daft_table.get_column("result").to_pylist() == pyresult + assert daft_recordbatch.get_column("result").to_pylist() == pyresult @pytest.mark.parametrize("data_dtype, op", itertools.product(daft_numeric_types, OPS)) @@ -62,17 +62,17 @@ def test_table_numeric_expressions_with_nulls(data_dtype, op) -> None: a, b = [5, 6, None, 8, None], [1, 2, 3, None, None] pa_table = pa.Table.from_pydict({"a": a, "b": b}) - daft_table = MicroPartition.from_arrow(pa_table) - daft_table = daft_table.eval_expression_list( + daft_recordbatch = MicroPartition.from_arrow(pa_table) + daft_recordbatch = daft_recordbatch.eval_expression_list( [op(col("a").cast(data_dtype), col("b").cast(data_dtype)).alias("result")] ) - assert len(daft_table) == 5 - assert daft_table.column_names() == ["result"] + assert len(daft_recordbatch) == 5 + assert daft_recordbatch.column_names() == ["result"] pyresult = [op(left, right) for left, right in zip(a[:2], b[:2])] - assert daft_table.get_column("result").to_pylist()[:2] == pyresult + assert daft_recordbatch.get_column("result").to_pylist()[:2] == pyresult - assert daft_table.get_column("result").to_pylist()[2:] == [None, None, None] + assert daft_recordbatch.get_column("result").to_pylist()[2:] == [None, None, None] def test_table_numeric_abs() -> None: diff --git a/tests/table/struct/__init__.py b/tests/recordbatch/recordbatch_io/__init__.py similarity index 100% rename from tests/table/struct/__init__.py rename to tests/recordbatch/recordbatch_io/__init__.py diff --git a/tests/table/table_io/test_csv.py b/tests/recordbatch/recordbatch_io/test_csv.py similarity index 94% rename from tests/table/table_io/test_csv.py rename to tests/recordbatch/recordbatch_io/test_csv.py index ea0118db3b..1338b1f089 100644 --- a/tests/table/table_io/test_csv.py +++ b/tests/recordbatch/recordbatch_io/test_csv.py @@ -12,8 +12,8 @@ from daft.daft import CsvParseOptions from daft.datatype import DataType from daft.logical.schema import Schema +from daft.recordbatch import MicroPartition, recordbatch_io from daft.runners.partitioning import TableParseCSVOptions, TableReadOptions -from daft.table import MicroPartition, table_io from daft.utils import get_arrow_version @@ -29,8 +29,8 @@ def test_read_input(tmpdir): schema = Schema._from_field_name_and_types([("foo", DataType.int64())]) # Test pathlib, str and IO - assert table_io.read_csv(tmpdir / "file.csv", schema=schema).to_pydict() == data - assert table_io.read_csv(str(tmpdir / "file.csv"), schema=schema).to_pydict() == data + assert recordbatch_io.read_csv(tmpdir / "file.csv", schema=schema).to_pydict() == data + assert recordbatch_io.read_csv(str(tmpdir / "file.csv"), schema=schema).to_pydict() == data @contextlib.contextmanager @@ -123,7 +123,7 @@ def test_csv_read_data(data, expected_data_series): "data": expected_data_series, } ) - table = table_io.read_csv(f, schema) + table = recordbatch_io.read_csv(f, schema) assert table.to_arrow() == expected.to_arrow(), f"Expected:\n{expected}\n\nReceived:\n{table}" @@ -143,7 +143,7 @@ def test_csv_read_data_csv_limit_rows(): "data": [1, 2], } ) - table = table_io.read_csv( + table = recordbatch_io.read_csv( f, schema, read_options=TableReadOptions(num_rows=2), @@ -166,7 +166,7 @@ def test_csv_read_data_csv_select_columns(): "data": [1, 2, None], } ) - table = table_io.read_csv( + table = recordbatch_io.read_csv( f, schema, read_options=TableReadOptions(column_names=["data"]), @@ -191,7 +191,7 @@ def test_csv_read_data_csv_custom_delimiter(): "data": [1, 2, None], } ) - table = table_io.read_csv( + table = recordbatch_io.read_csv( f, schema, csv_options=TableParseCSVOptions(delimiter="|"), @@ -215,7 +215,7 @@ def test_csv_read_data_csv_no_header(): "data": [1, 2, None], } ) - table = table_io.read_csv( + table = recordbatch_io.read_csv( f, schema, csv_options=TableParseCSVOptions(header_index=None), @@ -240,7 +240,7 @@ def test_csv_read_data_csv_custom_quote(): "data": ["aa", "aa", "aa"], } ) - table = table_io.read_csv( + table = recordbatch_io.read_csv( f, schema, csv_options=TableParseCSVOptions(quote="'"), @@ -269,7 +269,7 @@ def test_csv_read_data_custom_escape(): "data": ['a"a"a', "aa", "aa"], } ) - table = table_io.read_csv( + table = recordbatch_io.read_csv( f, schema, csv_options=TableParseCSVOptions(escape_char="\\", double_quote=False), @@ -298,7 +298,7 @@ def test_csv_read_data_custom_comment(): # Skipping test for arrow < 7.0.0 as comments are not supported in pyarrow arrow_version = get_arrow_version() if arrow_version >= (7, 0, 0): - table = table_io.read_csv( + table = recordbatch_io.read_csv( file, schema, csv_options=TableParseCSVOptions(comment="#"), @@ -326,7 +326,7 @@ def test_csv_read_data_variable_missing_columns(): "data": [None, 2], } ) - table = table_io.read_csv( + table = recordbatch_io.read_csv( f, schema, csv_options=TableParseCSVOptions(allow_variable_columns=True), @@ -354,7 +354,7 @@ def test_csv_read_data_variable_extra_columns(): "data": [1, 2], } ) - table = table_io.read_csv( + table = recordbatch_io.read_csv( f, schema, csv_options=TableParseCSVOptions(allow_variable_columns=True), @@ -382,7 +382,7 @@ def test_csv_read_data_variable_columns_with_non_matching_types(): "data": [None, 2], } ) - table = table_io.read_csv( + table = recordbatch_io.read_csv( f, schema, csv_options=TableParseCSVOptions(allow_variable_columns=True), diff --git a/tests/table/table_io/test_json.py b/tests/recordbatch/recordbatch_io/test_json.py similarity index 89% rename from tests/table/table_io/test_json.py rename to tests/recordbatch/recordbatch_io/test_json.py index 08798c6a9b..fbbfb4e3ff 100644 --- a/tests/table/table_io/test_json.py +++ b/tests/recordbatch/recordbatch_io/test_json.py @@ -12,8 +12,8 @@ import daft from daft.datatype import DataType from daft.logical.schema import Schema +from daft.recordbatch import MicroPartition, recordbatch_io from daft.runners.partitioning import TableReadOptions -from daft.table import MicroPartition, table_io def test_read_input(tmpdir): @@ -27,8 +27,8 @@ def test_read_input(tmpdir): schema = Schema._from_field_name_and_types([("foo", DataType.int64())]) # Test pathlib, str and IO - assert table_io.read_json(tmpdir / "file.json", schema=schema).to_pydict() == data - assert table_io.read_json(str(tmpdir / "file.json"), schema=schema).to_pydict() == data + assert recordbatch_io.read_json(tmpdir / "file.json", schema=schema).to_pydict() == data + assert recordbatch_io.read_json(str(tmpdir / "file.json"), schema=schema).to_pydict() == data @contextlib.contextmanager @@ -99,7 +99,7 @@ def test_json_read_data(data, expected_data_series): "data": expected_data_series, } ) - table = table_io.read_json(f, schema) + table = recordbatch_io.read_json(f, schema) assert table.to_arrow() == expected.to_arrow(), f"Expected:\n{expected}\n\nReceived:\n{table}" @@ -117,7 +117,7 @@ def test_json_read_data_limit_rows(): "data": [1, 2], } ) - table = table_io.read_json(f, schema, read_options=TableReadOptions(num_rows=2)) + table = recordbatch_io.read_json(f, schema, read_options=TableReadOptions(num_rows=2)) assert table.to_arrow() == expected.to_arrow(), f"Expected:\n{expected}\n\nReceived:\n{table}" @@ -134,5 +134,5 @@ def test_json_read_data_select_columns(): "data": [1, 2, None], } ) - table = table_io.read_json(f, schema, read_options=TableReadOptions(column_names=["data"])) + table = recordbatch_io.read_json(f, schema, read_options=TableReadOptions(column_names=["data"])) assert table.to_arrow() == expected.to_arrow(), f"Expected:\n{expected}\n\nReceived:\n{table}" diff --git a/tests/table/table_io/test_parquet.py b/tests/recordbatch/recordbatch_io/test_parquet.py similarity index 95% rename from tests/table/table_io/test_parquet.py rename to tests/recordbatch/recordbatch_io/test_parquet.py index 7dadf8deb0..1e36e216ad 100644 --- a/tests/table/table_io/test_parquet.py +++ b/tests/recordbatch/recordbatch_io/test_parquet.py @@ -15,13 +15,13 @@ from daft.datatype import DataType, TimeUnit from daft.exceptions import DaftCoreException from daft.logical.schema import Schema -from daft.runners.partitioning import TableParseParquetOptions, TableReadOptions -from daft.table import ( +from daft.recordbatch import ( MicroPartition, read_parquet_into_pyarrow, read_parquet_into_pyarrow_bulk, - table_io, + recordbatch_io, ) +from daft.runners.partitioning import TableParseParquetOptions, TableReadOptions PYARROW_GE_11_0_0 = tuple(int(s) for s in pa.__version__.split(".") if s.isnumeric()) >= (11, 0, 0) PYARROW_GE_13_0_0 = tuple(int(s) for s in pa.__version__.split(".") if s.isnumeric()) >= (13, 0, 0) @@ -36,8 +36,8 @@ def test_read_input(tmpdir): schema = Schema._from_field_name_and_types([("foo", DataType.int64())]) # Test pathlib, str and IO - assert table_io.read_parquet(tmpdir / "file.parquet", schema=schema).to_arrow() == data - assert table_io.read_parquet(str(tmpdir / "file.parquet"), schema=schema).to_arrow() == data + assert recordbatch_io.read_parquet(tmpdir / "file.parquet", schema=schema).to_arrow() == data + assert recordbatch_io.read_parquet(str(tmpdir / "file.parquet"), schema=schema).to_arrow() == data @contextlib.contextmanager @@ -81,7 +81,7 @@ def test_parquet_read_empty(): with _parquet_write_helper(pa.Table.from_pydict({"foo": pa.array([], type=pa.int64())})) as f: schema = Schema._from_field_name_and_types([("foo", DataType.int64())]) expected = MicroPartition.from_pydict({"foo": pa.array([], type=pa.int64())}) - table = table_io.read_parquet(f, schema) + table = recordbatch_io.read_parquet(f, schema) assert table.to_arrow() == expected.to_arrow(), f"Expected:\n{expected}\n\nReceived:\n{table}" @@ -120,7 +120,7 @@ def test_parquet_read_data(data, expected_data_series): "data": expected_data_series, } ) - table = table_io.read_parquet(f, schema) + table = recordbatch_io.read_parquet(f, schema) assert table.to_arrow() == expected.to_arrow(), f"Expected:\n{expected}\n\nReceived:\n{table}" @@ -142,7 +142,7 @@ def test_parquet_read_data_limit_rows(row_group_size): "data": [1, 2], } ) - table = table_io.read_parquet(f, schema, read_options=TableReadOptions(num_rows=2)) + table = recordbatch_io.read_parquet(f, schema, read_options=TableReadOptions(num_rows=2)) assert table.to_arrow() == expected.to_arrow(), f"Expected:\n{expected}\n\nReceived:\n{table}" @@ -168,7 +168,7 @@ def test_parquet_read_data_select_columns(): "data": [1, 2, None], } ) - table = table_io.read_parquet(f, schema, read_options=TableReadOptions(column_names=["data"])) + table = recordbatch_io.read_parquet(f, schema, read_options=TableReadOptions(column_names=["data"])) assert table.to_arrow() == expected.to_arrow(), f"Expected:\n{expected}\n\nReceived:\n{table}" @@ -205,7 +205,7 @@ def test_parquet_read_int96_timestamps(use_deprecated_int96_timestamps): ) as f: schema = Schema._from_field_name_and_types(schema) expected = MicroPartition.from_pydict(data) - table = table_io.read_parquet( + table = recordbatch_io.read_parquet( f, schema, read_options=TableReadOptions(column_names=schema.column_names()), @@ -240,7 +240,7 @@ def test_parquet_read_int96_timestamps_overflow(coerce_to): ) as f: schema = Schema._from_field_name_and_types(schema) expected = MicroPartition.from_pydict(data) - table = table_io.read_parquet( + table = recordbatch_io.read_parquet( f, schema, read_options=TableReadOptions(column_names=schema.column_names()), diff --git a/tests/table/table_io/test_read_time_cast.py b/tests/recordbatch/recordbatch_io/test_read_time_cast.py similarity index 91% rename from tests/table/table_io/test_read_time_cast.py rename to tests/recordbatch/recordbatch_io/test_read_time_cast.py index 5d59806a42..2dfc3ca8a3 100644 --- a/tests/table/table_io/test_read_time_cast.py +++ b/tests/recordbatch/recordbatch_io/test_read_time_cast.py @@ -6,8 +6,8 @@ import daft from daft import DataType from daft.logical.schema import Schema -from daft.table import MicroPartition, table_io -from tests.table.table_io.test_parquet import _parquet_write_helper +from daft.recordbatch import MicroPartition, recordbatch_io +from tests.recordbatch.recordbatch_io.test_parquet import _parquet_write_helper @pytest.mark.parametrize( @@ -44,6 +44,6 @@ ) def test_parquet_cast_at_read_time(data, schema, expected): with _parquet_write_helper(data) as f: - table = table_io.read_parquet(f, schema) + table = recordbatch_io.read_parquet(f, schema) assert table.schema() == schema assert table.to_arrow() == expected.to_arrow() diff --git a/tests/table/table_io/__init__.py b/tests/recordbatch/struct/__init__.py similarity index 100% rename from tests/table/table_io/__init__.py rename to tests/recordbatch/struct/__init__.py diff --git a/tests/table/struct/test_struct_get.py b/tests/recordbatch/struct/test_struct_get.py similarity index 97% rename from tests/table/struct/test_struct_get.py rename to tests/recordbatch/struct/test_struct_get.py index 142272f6ca..a43b0270a7 100644 --- a/tests/table/struct/test_struct_get.py +++ b/tests/recordbatch/struct/test_struct_get.py @@ -5,7 +5,7 @@ import pytest from daft.expressions import col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition def test_struct_get(): diff --git a/tests/table/struct/test_to_struct.py b/tests/recordbatch/struct/test_to_struct.py similarity index 97% rename from tests/table/struct/test_to_struct.py rename to tests/recordbatch/struct/test_to_struct.py index 4543fc2762..4fb7d058f2 100644 --- a/tests/table/struct/test_to_struct.py +++ b/tests/recordbatch/struct/test_to_struct.py @@ -2,7 +2,7 @@ import daft from daft.expressions import col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition def test_to_struct(): diff --git a/tests/table/temporal/__init__.py b/tests/recordbatch/temporal/__init__.py similarity index 100% rename from tests/table/temporal/__init__.py rename to tests/recordbatch/temporal/__init__.py diff --git a/tests/table/temporal/test_temporal.py b/tests/recordbatch/temporal/test_temporal.py similarity index 98% rename from tests/table/temporal/test_temporal.py rename to tests/recordbatch/temporal/test_temporal.py index 80590de0eb..ad19067523 100644 --- a/tests/table/temporal/test_temporal.py +++ b/tests/recordbatch/temporal/test_temporal.py @@ -3,7 +3,7 @@ import pytest from daft import col -from daft.table.micropartition import MicroPartition +from daft.recordbatch.micropartition import MicroPartition DATETIMES_WITHOUT_NULL = [ datetime.datetime(2021, 1, 1, 23, 59, 58), diff --git a/tests/table/test_between.py b/tests/recordbatch/test_between.py similarity index 88% rename from tests/table/test_between.py rename to tests/recordbatch/test_between.py index 989a29b895..b1011b88c4 100644 --- a/tests/table/test_between.py +++ b/tests/recordbatch/test_between.py @@ -4,7 +4,7 @@ import pytest from daft import col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition @pytest.mark.parametrize( @@ -52,9 +52,9 @@ ], ) def test_table_expr_between_scalars(value, lower, upper, expected) -> None: - daft_table = MicroPartition.from_pydict({"value": value}) - daft_table = daft_table.eval_expression_list([col("value").between(lower, upper)]) - pydict = daft_table.to_pydict() + daft_recordbatch = MicroPartition.from_pydict({"value": value}) + daft_recordbatch = daft_recordbatch.eval_expression_list([col("value").between(lower, upper)]) + pydict = daft_recordbatch.to_pydict() assert pydict["value"] == expected @@ -130,9 +130,9 @@ def test_table_expr_between_scalars(value, lower, upper, expected) -> None: ) def test_between_columns(value, lower, upper, expected) -> None: table = {"value": value, "lower": lower, "upper": upper} - daft_table = MicroPartition.from_pydict(table) - daft_table = daft_table.eval_expression_list([col("value").between(col("lower"), col("upper"))]) - pydict = daft_table.to_pydict() + daft_recordbatch = MicroPartition.from_pydict(table) + daft_recordbatch = daft_recordbatch.eval_expression_list([col("value").between(col("lower"), col("upper"))]) + pydict = daft_recordbatch.to_pydict() assert pydict["value"] == expected @@ -144,15 +144,15 @@ def test_between_columns(value, lower, upper, expected) -> None: ], ) def test_between_between_different_types(value, lower, upper) -> None: - daft_table = MicroPartition.from_pydict({"a": value}) + daft_recordbatch = MicroPartition.from_pydict({"a": value}) with pytest.raises(ValueError): - daft_table = daft_table.eval_expression_list([col("a").between(lower, upper)]) + daft_recordbatch = daft_recordbatch.eval_expression_list([col("a").between(lower, upper)]) def test_between_bad_input() -> None: - daft_table = MicroPartition.from_pydict({"a": [1, 2, 3]}) + daft_recordbatch = MicroPartition.from_pydict({"a": [1, 2, 3]}) with pytest.raises(TypeError): - daft_table = daft_table.eval_expression_list([col("a").between([1, 2, 3], 1)]) + daft_recordbatch = daft_recordbatch.eval_expression_list([col("a").between([1, 2, 3], 1)]) @pytest.mark.parametrize( @@ -184,7 +184,7 @@ def test_between_bad_input() -> None: ) def test_table_expr_between_col_and_scalar(value, lower, upper, expected) -> None: table = {"value": value, "lower": lower} - daft_table = MicroPartition.from_pydict(table) - daft_table = daft_table.eval_expression_list([col("value").between(col("lower"), upper)]) - pydict = daft_table.to_pydict() + daft_recordbatch = MicroPartition.from_pydict(table) + daft_recordbatch = daft_recordbatch.eval_expression_list([col("value").between(col("lower"), upper)]) + pydict = daft_recordbatch.to_pydict() assert pydict["value"] == expected diff --git a/tests/table/test_bitwise.py b/tests/recordbatch/test_bitwise.py similarity index 98% rename from tests/table/test_bitwise.py rename to tests/recordbatch/test_bitwise.py index b3733ace3d..43d226e638 100644 --- a/tests/table/test_bitwise.py +++ b/tests/recordbatch/test_bitwise.py @@ -5,7 +5,7 @@ import pytest from daft.expressions import col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition BITWISE_OPERATORS = [and_, or_, xor] diff --git a/tests/table/test_blackbox_kernels.py b/tests/recordbatch/test_blackbox_kernels.py similarity index 96% rename from tests/table/test_blackbox_kernels.py rename to tests/recordbatch/test_blackbox_kernels.py index 2e40e897f2..8ceac1e262 100644 --- a/tests/table/test_blackbox_kernels.py +++ b/tests/recordbatch/test_blackbox_kernels.py @@ -1,7 +1,7 @@ from __future__ import annotations from daft.expressions import col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition def test_pyobjects_blackbox_kernels() -> None: diff --git a/tests/table/test_broadcasts.py b/tests/recordbatch/test_broadcasts.py similarity index 94% rename from tests/table/test_broadcasts.py rename to tests/recordbatch/test_broadcasts.py index 127a8d6a43..366e6446ee 100644 --- a/tests/table/test_broadcasts.py +++ b/tests/recordbatch/test_broadcasts.py @@ -4,7 +4,7 @@ import daft from daft.expressions import col, lit -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition @pytest.mark.parametrize("data", [1, "a", True, b"Y", 0.5, None, [1, 2, 3], object()]) diff --git a/tests/table/test_concat.py b/tests/recordbatch/test_concat.py similarity index 98% rename from tests/table/test_concat.py rename to tests/recordbatch/test_concat.py index 4b6f3399a4..5b482215c4 100644 --- a/tests/table/test_concat.py +++ b/tests/recordbatch/test_concat.py @@ -2,7 +2,7 @@ import pytest -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition def test_table_concat() -> None: diff --git a/tests/table/test_eval.py b/tests/recordbatch/test_eval.py similarity index 66% rename from tests/table/test_eval.py rename to tests/recordbatch/test_eval.py index 6758b2f842..09d915a920 100644 --- a/tests/table/test_eval.py +++ b/tests/recordbatch/test_eval.py @@ -4,19 +4,19 @@ import pytest from daft import DataType, col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition def test_table_eval_expressions() -> None: pa_table = pa.Table.from_pydict({"a": [1, 2, 3, 4], "b": [5, 6, 7, 8]}) - daft_table = MicroPartition.from_arrow(pa_table) - assert len(daft_table) == 4 - assert daft_table.column_names() == ["a", "b"] + daft_recordbatch = MicroPartition.from_arrow(pa_table) + assert len(daft_recordbatch) == 4 + assert daft_recordbatch.column_names() == ["a", "b"] exprs = [col("a") + col("b"), col("b") * 2] - new_table = daft_table.eval_expression_list(exprs) - assert len(daft_table) == 4 - assert daft_table.column_names() == ["a", "b"] + new_table = daft_recordbatch.eval_expression_list(exprs) + assert len(daft_recordbatch) == 4 + assert daft_recordbatch.column_names() == ["a", "b"] result = new_table.to_pydict() assert result["a"] == [6, 8, 10, 12] assert result["b"] == [10, 12, 14, 16] @@ -24,14 +24,14 @@ def test_table_eval_expressions() -> None: def test_table_eval_expressions_conflict() -> None: pa_table = pa.Table.from_pydict({"a": [1, 2, 3, 4], "b": [5, 6, 7, 8]}) - daft_table = MicroPartition.from_arrow(pa_table) - assert len(daft_table) == 4 - assert daft_table.column_names() == ["a", "b"] + daft_recordbatch = MicroPartition.from_arrow(pa_table) + assert len(daft_recordbatch) == 4 + assert daft_recordbatch.column_names() == ["a", "b"] exprs = [col("a") + col("b"), col("a") * 2] with pytest.raises(ValueError, match="Duplicate name"): - daft_table.eval_expression_list(exprs) + daft_recordbatch.eval_expression_list(exprs) @pytest.mark.parametrize( @@ -49,18 +49,18 @@ def test_table_eval_expressions_conflict() -> None: ) def test_table_expr_not(input, expr, expected) -> None: """Test logical not expression.""" - daft_table = MicroPartition.from_pydict({"input": input}) - daft_table = daft_table.eval_expression_list([expr]) - pydict = daft_table.to_pydict() + daft_recordbatch = MicroPartition.from_pydict({"input": input}) + daft_recordbatch = daft_recordbatch.eval_expression_list([expr]) + pydict = daft_recordbatch.to_pydict() assert pydict["input"] == expected def test_table_expr_not_wrong() -> None: - daft_table = MicroPartition.from_pydict({"input": [None, 0, 1]}) + daft_recordbatch = MicroPartition.from_pydict({"input": [None, 0, 1]}) with pytest.raises(ValueError): - daft_table = daft_table.eval_expression_list([~col("input")]) + daft_recordbatch = daft_recordbatch.eval_expression_list([~col("input")]) @pytest.mark.parametrize( @@ -76,9 +76,9 @@ def test_table_expr_not_wrong() -> None: ) def test_table_expr_is_null(input, expected) -> None: """Test is_null expression.""" - daft_table = MicroPartition.from_pydict({"input": input}) - daft_table = daft_table.eval_expression_list([col("input").is_null()]) - pydict = daft_table.to_pydict() + daft_recordbatch = MicroPartition.from_pydict({"input": input}) + daft_recordbatch = daft_recordbatch.eval_expression_list([col("input").is_null()]) + pydict = daft_recordbatch.to_pydict() assert pydict["input"] == expected @@ -96,8 +96,8 @@ def test_table_expr_is_null(input, expected) -> None: ) def test_table_expr_not_null(input, expected) -> None: """Test not_null expression.""" - daft_table = MicroPartition.from_pydict({"input": input}) - daft_table = daft_table.eval_expression_list([col("input").not_null()]) - pydict = daft_table.to_pydict() + daft_recordbatch = MicroPartition.from_pydict({"input": input}) + daft_recordbatch = daft_recordbatch.eval_expression_list([col("input").not_null()]) + pydict = daft_recordbatch.to_pydict() assert pydict["input"] == expected diff --git a/tests/table/test_explodes.py b/tests/recordbatch/test_explodes.py similarity index 98% rename from tests/table/test_explodes.py rename to tests/recordbatch/test_explodes.py index 9565c6cc1f..8004487a65 100644 --- a/tests/table/test_explodes.py +++ b/tests/recordbatch/test_explodes.py @@ -4,8 +4,8 @@ import pytest from daft.expressions import col +from daft.recordbatch import MicroPartition from daft.series import Series -from daft.table import MicroPartition TEST_DATA = [ Series.from_arrow(pa.array([[1, 2], [3, 4], None, []], type=pa.list_(pa.int64()))), diff --git a/tests/table/test_fill.py b/tests/recordbatch/test_fill.py similarity index 77% rename from tests/table/test_fill.py rename to tests/recordbatch/test_fill.py index fc873fb193..1111acab9c 100644 --- a/tests/table/test_fill.py +++ b/tests/recordbatch/test_fill.py @@ -6,7 +6,7 @@ from daft.datatype import DataType from daft.expressions.expressions import col -from daft.table.micropartition import MicroPartition +from daft.recordbatch.micropartition import MicroPartition @pytest.mark.parametrize( @@ -39,9 +39,9 @@ ], ) def test_table_expr_fill_null(input, fill_value, expected) -> None: - daft_table = MicroPartition.from_pydict({"input": input}) - daft_table = daft_table.eval_expression_list([col("input").fill_null(fill_value)]) - pydict = daft_table.to_pydict() + daft_recordbatch = MicroPartition.from_pydict({"input": input}) + daft_recordbatch = daft_recordbatch.eval_expression_list([col("input").fill_null(fill_value)]) + pydict = daft_recordbatch.to_pydict() assert pydict["input"] == expected @@ -55,8 +55,10 @@ def test_table_expr_fill_nan(float_dtype) -> None: fill_value = 2.0 expected = [1.0, None, 3.0, 2.0] - daft_table = MicroPartition.from_pydict({"input": input}) - daft_table = daft_table.eval_expression_list([col("input").cast(float_dtype).float.fill_nan(fill_value)]) - pydict = daft_table.to_pydict() + daft_recordbatch = MicroPartition.from_pydict({"input": input}) + daft_recordbatch = daft_recordbatch.eval_expression_list( + [col("input").cast(float_dtype).float.fill_nan(fill_value)] + ) + pydict = daft_recordbatch.to_pydict() assert pydict["input"] == expected diff --git a/tests/table/test_filter.py b/tests/recordbatch/test_filter.py similarity index 75% rename from tests/table/test_filter.py rename to tests/recordbatch/test_filter.py index 68fddbf842..cf941d6abc 100644 --- a/tests/table/test_filter.py +++ b/tests/recordbatch/test_filter.py @@ -5,17 +5,17 @@ import pytest from daft import DataType, col, lit -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition def test_table_filter_all_pass() -> None: pa_table = pa.Table.from_pydict({"a": [1, 2, 3, 4], "b": [5, 6, 7, 8]}) - daft_table = MicroPartition.from_arrow(pa_table) - assert len(daft_table) == 4 - assert daft_table.column_names() == ["a", "b"] + daft_recordbatch = MicroPartition.from_arrow(pa_table) + assert len(daft_recordbatch) == 4 + assert daft_recordbatch.column_names() == ["a", "b"] exprs = [col("a") < col("b"), col("a") < 5] - new_table = daft_table.filter(exprs) + new_table = daft_recordbatch.filter(exprs) assert len(new_table) == 4 assert new_table.column_names() == ["a", "b"] result = new_table.to_pydict() @@ -23,7 +23,7 @@ def test_table_filter_all_pass() -> None: assert result["b"] == [5, 6, 7, 8] exprs = [lit(True), lit(True)] - new_table = daft_table.filter(exprs) + new_table = daft_recordbatch.filter(exprs) assert len(new_table) == 4 assert new_table.column_names() == ["a", "b"] result = new_table.to_pydict() @@ -33,12 +33,12 @@ def test_table_filter_all_pass() -> None: def test_table_filter_some_pass() -> None: pa_table = pa.Table.from_pydict({"a": [1, 2, 3, 4], "b": [5, 6, 7, 8]}) - daft_table = MicroPartition.from_arrow(pa_table) - assert len(daft_table) == 4 - assert daft_table.column_names() == ["a", "b"] + daft_recordbatch = MicroPartition.from_arrow(pa_table) + assert len(daft_recordbatch) == 4 + assert daft_recordbatch.column_names() == ["a", "b"] exprs = [((col("a") * 4) < col("b")) | (col("b") == 8)] - new_table = daft_table.filter(exprs) + new_table = daft_recordbatch.filter(exprs) assert len(new_table) == 2 assert new_table.column_names() == ["a", "b"] result = new_table.to_pydict() @@ -46,7 +46,7 @@ def test_table_filter_some_pass() -> None: assert result["b"] == [5, 8] exprs = [(col("b") / col("a")) >= 3] - new_table = daft_table.filter(exprs) + new_table = daft_recordbatch.filter(exprs) assert len(new_table) == 2 assert new_table.column_names() == ["a", "b"] result = new_table.to_pydict() @@ -56,12 +56,12 @@ def test_table_filter_some_pass() -> None: def test_table_filter_none_pass() -> None: pa_table = pa.Table.from_pydict({"a": [1, 2, 3, 4], "b": [5, 6, 7, 8]}) - daft_table = MicroPartition.from_arrow(pa_table) - assert len(daft_table) == 4 - assert daft_table.column_names() == ["a", "b"] + daft_recordbatch = MicroPartition.from_arrow(pa_table) + assert len(daft_recordbatch) == 4 + assert daft_recordbatch.column_names() == ["a", "b"] exprs = [col("a") < col("b"), col("a") > 5] - new_table = daft_table.filter(exprs) + new_table = daft_recordbatch.filter(exprs) assert len(new_table) == 0 assert new_table.column_names() == ["a", "b"] result = new_table.to_pydict() @@ -69,7 +69,7 @@ def test_table_filter_none_pass() -> None: assert result["b"] == [] exprs = [col("a") < col("b"), lit(False)] - new_table = daft_table.filter(exprs) + new_table = daft_recordbatch.filter(exprs) assert len(new_table) == 0 assert new_table.column_names() == ["a", "b"] result = new_table.to_pydict() @@ -79,12 +79,12 @@ def test_table_filter_none_pass() -> None: def test_table_filter_with_null_filter() -> None: pa_table = pa.Table.from_pydict({"a": [1, 2, 3, 4], "b": [5, 6, 7, 8]}) - daft_table = MicroPartition.from_arrow(pa_table) - assert len(daft_table) == 4 - assert daft_table.column_names() == ["a", "b"] + daft_recordbatch = MicroPartition.from_arrow(pa_table) + assert len(daft_recordbatch) == 4 + assert daft_recordbatch.column_names() == ["a", "b"] exprs = [lit(None).cast(DataType.bool())] - new_table = daft_table.filter(exprs) + new_table = daft_recordbatch.filter(exprs) assert len(new_table) == 0 assert new_table.column_names() == ["a", "b"] result = new_table.to_pydict() @@ -94,14 +94,14 @@ def test_table_filter_with_null_filter() -> None: def test_table_filter_bad_expression() -> None: pa_table = pa.Table.from_pydict({"a": [1, 2, 3, 4], "b": [5, 6, 7, 8]}) - daft_table = MicroPartition.from_arrow(pa_table) - assert len(daft_table) == 4 - assert daft_table.column_names() == ["a", "b"] + daft_recordbatch = MicroPartition.from_arrow(pa_table) + assert len(daft_recordbatch) == 4 + assert daft_recordbatch.column_names() == ["a", "b"] exprs = [col("a") + 1] with pytest.raises(ValueError, match="Boolean Series"): - daft_table.filter(exprs) + daft_recordbatch.filter(exprs) def test_table_filter_with_dates() -> None: @@ -114,12 +114,12 @@ def date_maker(d): days = list(map(date_maker, [5, 4, 1, None, 2, None])) pa_table = pa.Table.from_pydict({"days": days, "enum": [0, 1, 2, 3, 4, 5]}) - daft_table = MicroPartition.from_arrow(pa_table) - assert len(daft_table) == 6 - assert daft_table.column_names() == ["days", "enum"] + daft_recordbatch = MicroPartition.from_arrow(pa_table) + assert len(daft_recordbatch) == 6 + assert daft_recordbatch.column_names() == ["days", "enum"] exprs = [(col("days") > date(2023, 1, 2)) & (col("enum") > 0)] - new_table = daft_table.filter(exprs) + new_table = daft_recordbatch.filter(exprs) assert len(new_table) == 1 assert new_table.column_names() == ["days", "enum"] result = new_table.to_pydict() @@ -137,12 +137,12 @@ def date_maker(d): days = list(map(date_maker, [3, 28, None, 9, 18, None])) pa_table = pa.Table.from_pydict({"days": days, "enum": [0, 1, 2, 3, 4, 5]}) - daft_table = MicroPartition.from_arrow(pa_table) - assert len(daft_table) == 6 - assert daft_table.column_names() == ["days", "enum"] + daft_recordbatch = MicroPartition.from_arrow(pa_table) + assert len(daft_recordbatch) == 6 + assert daft_recordbatch.column_names() == ["days", "enum"] exprs = [col("days").dt.day() > 15] - new_table = daft_table.filter(exprs) + new_table = daft_recordbatch.filter(exprs) assert len(new_table) == 2 assert new_table.column_names() == ["days", "enum"] result = new_table.to_pydict() @@ -160,12 +160,12 @@ def date_maker(m): days = list(map(date_maker, [2, 6, None, 4, 11, None])) pa_table = pa.Table.from_pydict({"days": days, "enum": [0, 1, 2, 3, 4, 5]}) - daft_table = MicroPartition.from_arrow(pa_table) - assert len(daft_table) == 6 - assert daft_table.column_names() == ["days", "enum"] + daft_recordbatch = MicroPartition.from_arrow(pa_table) + assert len(daft_recordbatch) == 6 + assert daft_recordbatch.column_names() == ["days", "enum"] exprs = [col("days").dt.month() > 5] - new_table = daft_table.filter(exprs) + new_table = daft_recordbatch.filter(exprs) assert len(new_table) == 2 assert new_table.column_names() == ["days", "enum"] result = new_table.to_pydict() @@ -183,12 +183,12 @@ def date_maker(y): days = list(map(date_maker, [5, 4000, 1, None, 2022, None])) pa_table = pa.Table.from_pydict({"days": days, "enum": [0, 1, 2, 3, 4, 5]}) - daft_table = MicroPartition.from_arrow(pa_table) - assert len(daft_table) == 6 - assert daft_table.column_names() == ["days", "enum"] + daft_recordbatch = MicroPartition.from_arrow(pa_table) + assert len(daft_recordbatch) == 6 + assert daft_recordbatch.column_names() == ["days", "enum"] exprs = [col("days").dt.year() > 2000] - new_table = daft_table.filter(exprs) + new_table = daft_recordbatch.filter(exprs) assert len(new_table) == 2 assert new_table.column_names() == ["days", "enum"] result = new_table.to_pydict() @@ -207,12 +207,12 @@ def date_maker(d): # 04/03/2023 is a Monday. days = list(map(date_maker, [8, 5, None, 15, 12, None])) pa_table = pa.Table.from_pydict({"days": days, "enum": [0, 1, 2, 3, 4, 5]}) - daft_table = MicroPartition.from_arrow(pa_table) - assert len(daft_table) == 6 - assert daft_table.column_names() == ["days", "enum"] + daft_recordbatch = MicroPartition.from_arrow(pa_table) + assert len(daft_recordbatch) == 6 + assert daft_recordbatch.column_names() == ["days", "enum"] exprs = [col("days").dt.day_of_week() == 2] - new_table = daft_table.filter(exprs) + new_table = daft_recordbatch.filter(exprs) assert len(new_table) == 2 assert new_table.column_names() == ["days", "enum"] result = new_table.to_pydict() diff --git a/tests/table/test_from_py.py b/tests/recordbatch/test_from_py.py similarity index 86% rename from tests/table/test_from_py.py rename to tests/recordbatch/test_from_py.py index fd5f8cd0ae..872b04ce4b 100644 --- a/tests/table/test_from_py.py +++ b/tests/recordbatch/test_from_py.py @@ -11,8 +11,8 @@ import daft from daft import DataType, TimeUnit +from daft.recordbatch import MicroPartition from daft.series import Series -from daft.table import MicroPartition from daft.utils import pyarrow_supports_fixed_shape_tensor from tests.conftest import get_tests_daft_runner_name @@ -256,66 +256,66 @@ def test_from_pandas_roundtrip() -> None: def test_from_pydict_list() -> None: - daft_table = MicroPartition.from_pydict({"a": [1, 2, 3]}) - assert "a" in daft_table.column_names() - assert daft_table.to_arrow()["a"].combine_chunks() == pa.array([1, 2, 3], type=pa.int64()) + daft_recordbatch = MicroPartition.from_pydict({"a": [1, 2, 3]}) + assert "a" in daft_recordbatch.column_names() + assert daft_recordbatch.to_arrow()["a"].combine_chunks() == pa.array([1, 2, 3], type=pa.int64()) def test_from_pydict_np() -> None: - daft_table = MicroPartition.from_pydict({"a": np.array([1, 2, 3], dtype=np.int64)}) - assert "a" in daft_table.column_names() - assert daft_table.to_arrow()["a"].combine_chunks() == pa.array([1, 2, 3], type=pa.int64()) + daft_recordbatch = MicroPartition.from_pydict({"a": np.array([1, 2, 3], dtype=np.int64)}) + assert "a" in daft_recordbatch.column_names() + assert daft_recordbatch.to_arrow()["a"].combine_chunks() == pa.array([1, 2, 3], type=pa.int64()) def test_from_pydict_arrow() -> None: - daft_table = MicroPartition.from_pydict({"a": pa.array([1, 2, 3], type=pa.int8())}) - assert "a" in daft_table.column_names() - assert daft_table.to_arrow()["a"].combine_chunks() == pa.array([1, 2, 3], type=pa.int8()) + daft_recordbatch = MicroPartition.from_pydict({"a": pa.array([1, 2, 3], type=pa.int8())}) + assert "a" in daft_recordbatch.column_names() + assert daft_recordbatch.to_arrow()["a"].combine_chunks() == pa.array([1, 2, 3], type=pa.int8()) @pytest.mark.parametrize("list_type", [pa.list_, pa.large_list]) def test_from_pydict_arrow_list_array(list_type) -> None: arrow_arr = pa.array([["a", "b"], ["c"], None, [None, "d", "e"]], list_type(pa.string())) - daft_table = MicroPartition.from_pydict({"a": arrow_arr}) - assert "a" in daft_table.column_names() + daft_recordbatch = MicroPartition.from_pydict({"a": arrow_arr}) + assert "a" in daft_recordbatch.column_names() # Perform expected Daft cast, where the outer list array is cast to a large list array # (if the outer list array wasn't already a large list in the first place), and # the inner string array is cast to a large string array. expected = arrow_arr.cast(pa.large_list(pa.large_string())) - assert daft_table.to_arrow()["a"].combine_chunks() == expected + assert daft_recordbatch.to_arrow()["a"].combine_chunks() == expected def test_from_pydict_arrow_fixed_size_list_array() -> None: data = [["a", "b"], ["c", "d"], None, [None, "e"]] arrow_arr = pa.array(data, pa.list_(pa.string(), 2)) - daft_table = MicroPartition.from_pydict({"a": arrow_arr}) - assert "a" in daft_table.column_names() + daft_recordbatch = MicroPartition.from_pydict({"a": arrow_arr}) + assert "a" in daft_recordbatch.column_names() # Perform expected Daft cast, where the inner string array is cast to a large string array. expected = pa.array(data, type=pa.list_(pa.large_string(), 2)) - assert daft_table.to_arrow()["a"].combine_chunks() == expected + assert daft_recordbatch.to_arrow()["a"].combine_chunks() == expected def test_from_pydict_arrow_map_array() -> None: data = [[(1, 2.0), (3, 4.0)], None, [(5, 6.0), (7, 8.0)]] arrow_arr = pa.array(data, pa.map_(pa.int64(), pa.float64())) - daft_table = MicroPartition.from_pydict({"a": arrow_arr}) - assert "a" in daft_table.column_names() + daft_recordbatch = MicroPartition.from_pydict({"a": arrow_arr}) + assert "a" in daft_recordbatch.column_names() # Perform expected Daft cast, where the inner string and int arrays are cast to large string and int arrays. expected = arrow_arr.cast(pa.map_(pa.int64(), pa.float64())) - assert daft_table.to_arrow()["a"].combine_chunks() == expected - assert daft_table.to_pydict()["a"] == data + assert daft_recordbatch.to_arrow()["a"].combine_chunks() == expected + assert daft_recordbatch.to_pydict()["a"] == data def test_from_pydict_arrow_struct_array() -> None: data = [{"a": "foo", "b": "bar"}, {"b": "baz", "c": "quux"}] arrow_arr = pa.array(data) - daft_table = MicroPartition.from_pydict({"a": arrow_arr}) - assert "a" in daft_table.column_names() + daft_recordbatch = MicroPartition.from_pydict({"a": arrow_arr}) + assert "a" in daft_recordbatch.column_names() # Perform expected Daft cast, where the inner string array is cast to a large string array. expected = pa.array( data, type=pa.struct([("a", pa.large_string()), ("b", pa.large_string()), ("c", pa.large_string())]) ) - assert daft_table.to_arrow()["a"].combine_chunks() == expected + assert daft_recordbatch.to_arrow()["a"].combine_chunks() == expected @pytest.mark.skipif( @@ -327,11 +327,11 @@ def test_from_pydict_arrow_extension_array(uuid_ext_type) -> None: pydata[2] = None storage = pa.array(pydata) arrow_arr = pa.ExtensionArray.from_storage(uuid_ext_type, storage) - daft_table = MicroPartition.from_pydict({"a": arrow_arr}) - assert "a" in daft_table.column_names() + daft_recordbatch = MicroPartition.from_pydict({"a": arrow_arr}) + assert "a" in daft_recordbatch.column_names() # Although Daft will internally represent the binary storage array as a large_binary array, # it should be cast back to the ingress extension type. - result = daft_table.to_arrow()["a"] + result = daft_recordbatch.to_arrow()["a"] assert result.type == uuid_ext_type assert result.to_pylist() == arrow_arr.to_pylist() @@ -340,15 +340,15 @@ def test_from_pydict_arrow_deeply_nested() -> None: # Test a struct of lists of struct of lists of strings. data = [{"a": [{"b": ["foo", "bar"]}]}, {"a": [{"b": ["baz", "quux"]}]}] arrow_arr = pa.array(data) - daft_table = MicroPartition.from_pydict({"a": arrow_arr}) - assert "a" in daft_table.column_names() + daft_recordbatch = MicroPartition.from_pydict({"a": arrow_arr}) + assert "a" in daft_recordbatch.column_names() # Perform the expected Daft cast, where each list array is cast to a large list array and # the string array is cast to a large string array. expected = pa.array( data, type=pa.struct([("a", pa.large_list(pa.field("item", pa.struct([("b", pa.large_list(pa.large_string()))]))))]), ) - assert daft_table.to_arrow()["a"].combine_chunks() == expected + assert daft_recordbatch.to_arrow()["a"].combine_chunks() == expected @pytest.mark.parametrize( @@ -374,11 +374,11 @@ def test_from_pydict_arrow_deeply_nested() -> None: def test_from_pydict_arrow_with_nulls_roundtrip(data, out_dtype, chunked) -> None: if chunked: data = pa.chunked_array(data) - daft_table = MicroPartition.from_pydict({"a": data}) - assert "a" in daft_table.column_names() + daft_recordbatch = MicroPartition.from_pydict({"a": data}) + assert "a" in daft_recordbatch.column_names() if chunked: data = data.combine_chunks() - assert daft_table.to_arrow()["a"].combine_chunks() == pac.cast(data, out_dtype) + assert daft_recordbatch.to_arrow()["a"].combine_chunks() == pac.cast(data, out_dtype) @pytest.mark.parametrize( @@ -424,17 +424,17 @@ def test_from_pydict_arrow_sliced_roundtrip(data, out_dtype, chunked, slice_) -> sliced_data = data.slice(offset, length) if chunked: sliced_data = pa.chunked_array(sliced_data) - daft_table = MicroPartition.from_pydict({"a": sliced_data}) - assert "a" in daft_table.column_names() + daft_recordbatch = MicroPartition.from_pydict({"a": sliced_data}) + assert "a" in daft_recordbatch.column_names() if chunked: sliced_data = sliced_data.combine_chunks() - assert daft_table.to_arrow()["a"].combine_chunks() == pac.cast(sliced_data, out_dtype) + assert daft_recordbatch.to_arrow()["a"].combine_chunks() == pac.cast(sliced_data, out_dtype) def test_from_pydict_series() -> None: - daft_table = MicroPartition.from_pydict({"a": Series.from_arrow(pa.array([1, 2, 3], type=pa.int8()))}) - assert "a" in daft_table.column_names() - assert daft_table.to_arrow()["a"].combine_chunks() == pa.array([1, 2, 3], type=pa.int8()) + daft_recordbatch = MicroPartition.from_pydict({"a": Series.from_arrow(pa.array([1, 2, 3], type=pa.int8()))}) + assert "a" in daft_recordbatch.column_names() + assert daft_recordbatch.to_arrow()["a"].combine_chunks() == pa.array([1, 2, 3], type=pa.int8()) @pytest.mark.parametrize( @@ -477,54 +477,54 @@ def test_from_arrow_sliced_roundtrip(data, out_dtype, slice_) -> None: offset, end = slice_ length = end - offset sliced_data = data.slice(offset, length) - daft_table = MicroPartition.from_arrow(pa.table({"a": sliced_data})) - assert "a" in daft_table.column_names() - assert daft_table.to_arrow()["a"].combine_chunks() == pac.cast(sliced_data, out_dtype) + daft_recordbatch = MicroPartition.from_arrow(pa.table({"a": sliced_data})) + assert "a" in daft_recordbatch.column_names() + assert daft_recordbatch.to_arrow()["a"].combine_chunks() == pac.cast(sliced_data, out_dtype) @pytest.mark.parametrize("list_type", [pa.list_, pa.large_list]) def test_from_arrow_list_array(list_type) -> None: arrow_arr = pa.array([["a", "b"], ["c"], None, [None, "d", "e"]], list_type(pa.string())) - daft_table = MicroPartition.from_arrow(pa.table({"a": arrow_arr})) - assert "a" in daft_table.column_names() + daft_recordbatch = MicroPartition.from_arrow(pa.table({"a": arrow_arr})) + assert "a" in daft_recordbatch.column_names() # Perform expected Daft cast, where the outer list array is cast to a large list array # (if the outer list array wasn't already a large list in the first place), and # the inner string array is cast to a large string array. expected = arrow_arr.cast(pa.large_list(pa.large_string())) - assert daft_table.to_arrow()["a"].combine_chunks() == expected + assert daft_recordbatch.to_arrow()["a"].combine_chunks() == expected def test_from_arrow_fixed_size_list_array() -> None: data = [["a", "b"], ["c", "d"], None, [None, "e"]] arrow_arr = pa.array(data, pa.list_(pa.string(), 2)) - daft_table = MicroPartition.from_arrow(pa.table({"a": arrow_arr})) - assert "a" in daft_table.column_names() + daft_recordbatch = MicroPartition.from_arrow(pa.table({"a": arrow_arr})) + assert "a" in daft_recordbatch.column_names() # Perform expected Daft cast, where the inner string array is cast to a large string array. expected = pa.array(data, type=pa.list_(pa.large_string(), 2)) - assert daft_table.to_arrow()["a"].combine_chunks() == expected + assert daft_recordbatch.to_arrow()["a"].combine_chunks() == expected def test_from_arrow_struct_array() -> None: data = [{"a": "foo", "b": "bar"}, {"b": "baz", "c": "quux"}] arrow_arr = pa.array(data) - daft_table = MicroPartition.from_arrow(pa.table({"a": arrow_arr})) - assert "a" in daft_table.column_names() + daft_recordbatch = MicroPartition.from_arrow(pa.table({"a": arrow_arr})) + assert "a" in daft_recordbatch.column_names() # Perform expected Daft cast, where the inner string array is cast to a large string array. expected = pa.array( data, type=pa.struct([("a", pa.large_string()), ("b", pa.large_string()), ("c", pa.large_string())]) ) - assert daft_table.to_arrow()["a"].combine_chunks() == expected + assert daft_recordbatch.to_arrow()["a"].combine_chunks() == expected def test_from_arrow_map_array() -> None: data = [[(1.0, 1), (2.0, 2)], [(3.0, 3), (4.0, 4)]] arrow_arr = pa.array(data, pa.map_(pa.float32(), pa.int32())) - daft_table = MicroPartition.from_arrow(pa.table({"a": arrow_arr})) - assert "a" in daft_table.column_names() + daft_recordbatch = MicroPartition.from_arrow(pa.table({"a": arrow_arr})) + assert "a" in daft_recordbatch.column_names() # Perform expected Daft cast, where the inner string and int arrays are cast to large string and int arrays. expected = arrow_arr.cast(pa.map_(pa.float32(), pa.int32())) - assert daft_table.to_arrow()["a"].combine_chunks() == expected - assert daft_table.to_pydict()["a"] == data + assert daft_recordbatch.to_arrow()["a"].combine_chunks() == expected + assert daft_recordbatch.to_pydict()["a"] == data @pytest.mark.skipif( @@ -536,11 +536,11 @@ def test_from_arrow_extension_array(uuid_ext_type) -> None: pydata[2] = None storage = pa.array(pydata) arrow_arr = pa.ExtensionArray.from_storage(uuid_ext_type, storage) - daft_table = MicroPartition.from_arrow(pa.table({"a": arrow_arr})) - assert "a" in daft_table.column_names() + daft_recordbatch = MicroPartition.from_arrow(pa.table({"a": arrow_arr})) + assert "a" in daft_recordbatch.column_names() # Although Daft will internally represent the binary storage array as a large_binary array, # it should be cast back to the ingress extension type. - result = daft_table.to_arrow()["a"] + result = daft_recordbatch.to_arrow()["a"] assert result.type == uuid_ext_type assert result.to_pylist() == arrow_arr.to_pylist() @@ -549,8 +549,8 @@ def test_from_arrow_deeply_nested() -> None: # Test a struct of lists of struct of lists of strings. data = [{"a": [{"b": ["foo", "bar"]}]}, {"a": [{"b": ["baz", "quux"]}]}] arrow_arr = pa.array(data) - daft_table = MicroPartition.from_arrow(pa.table({"a": arrow_arr})) - assert "a" in daft_table.column_names() + daft_recordbatch = MicroPartition.from_arrow(pa.table({"a": arrow_arr})) + assert "a" in daft_recordbatch.column_names() # Perform the expected Daft cast, where each list array is cast to a large list array and # the string array is cast to a large string array. expected = pa.array( @@ -567,7 +567,7 @@ def test_from_arrow_deeply_nested() -> None: ), ) - assert daft_table.to_arrow()["a"].combine_chunks() == expected + assert daft_recordbatch.to_arrow()["a"].combine_chunks() == expected def test_from_pydict_bad_input() -> None: diff --git a/tests/table/test_hash.py b/tests/recordbatch/test_hash.py similarity index 100% rename from tests/table/test_hash.py rename to tests/recordbatch/test_hash.py diff --git a/tests/table/test_head.py b/tests/recordbatch/test_head.py similarity index 64% rename from tests/table/test_head.py rename to tests/recordbatch/test_head.py index 883cb6aaf6..716876fd92 100644 --- a/tests/table/test_head.py +++ b/tests/recordbatch/test_head.py @@ -3,24 +3,24 @@ import pyarrow as pa import pytest -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition def test_table_head() -> None: pa_table = pa.Table.from_pydict({"a": [1, 2, 3, 4], "b": [5, 6, 7, 8]}) - daft_table = MicroPartition.from_arrow(pa_table) - assert len(daft_table) == 4 - assert daft_table.column_names() == ["a", "b"] + daft_recordbatch = MicroPartition.from_arrow(pa_table) + assert len(daft_recordbatch) == 4 + assert daft_recordbatch.column_names() == ["a", "b"] # subslice - headed = daft_table.head(3) + headed = daft_recordbatch.head(3) assert len(headed) == 3 assert headed.column_names() == ["a", "b"] pa_headed = headed.to_arrow() assert pa_table[:3] == pa_headed # overslice - headed = daft_table.head(5) + headed = daft_recordbatch.head(5) assert len(headed) == 4 assert headed.column_names() == ["a", "b"] pa_headed = headed.to_arrow() @@ -28,4 +28,4 @@ def test_table_head() -> None: # negative slice with pytest.raises(ValueError, match="negative number"): - headed = daft_table.head(-1) + headed = daft_recordbatch.head(-1) diff --git a/tests/table/test_if_else.py b/tests/recordbatch/test_if_else.py similarity index 63% rename from tests/table/test_if_else.py rename to tests/recordbatch/test_if_else.py index 8f27ec88b9..8da3fefe6e 100644 --- a/tests/table/test_if_else.py +++ b/tests/recordbatch/test_if_else.py @@ -4,7 +4,7 @@ from daft.expressions import col from daft.expressions.expressions import lit -from daft.table.micropartition import MicroPartition +from daft.recordbatch.micropartition import MicroPartition @pytest.mark.parametrize( @@ -19,9 +19,11 @@ ], ) def test_table_expr_if_else(predicate, if_true, if_false, expected) -> None: - daft_table = MicroPartition.from_pydict({"predicate": predicate, "if_true": if_true, "if_false": if_false}) - daft_table = daft_table.eval_expression_list([col("predicate").if_else(col("if_true"), col("if_false"))]) - pydict = daft_table.to_pydict() + daft_recordbatch = MicroPartition.from_pydict({"predicate": predicate, "if_true": if_true, "if_false": if_false}) + daft_recordbatch = daft_recordbatch.eval_expression_list( + [col("predicate").if_else(col("if_true"), col("if_false"))] + ) + pydict = daft_recordbatch.to_pydict() assert pydict["if_true"] == expected @@ -34,8 +36,8 @@ def test_table_expr_if_else(predicate, if_true, if_false, expected) -> None: ], ) def test_table_expr_if_else_literal_predicate(if_else_expr) -> None: - daft_table = MicroPartition.from_pydict({"struct": [{"key": "value"}]}) - daft_table = daft_table.eval_expression_list([if_else_expr]) - pydict = daft_table.to_pydict() + daft_recordbatch = MicroPartition.from_pydict({"struct": [{"key": "value"}]}) + daft_recordbatch = daft_recordbatch.eval_expression_list([if_else_expr]) + pydict = daft_recordbatch.to_pydict() assert pydict == {if_else_expr.name(): ["value"]} diff --git a/tests/table/test_is_in.py b/tests/recordbatch/test_is_in.py similarity index 78% rename from tests/table/test_is_in.py rename to tests/recordbatch/test_is_in.py index a3ed42b850..1786f1626d 100644 --- a/tests/table/test_is_in.py +++ b/tests/recordbatch/test_is_in.py @@ -8,8 +8,8 @@ import pytest from daft import col +from daft.recordbatch import MicroPartition from daft.series import Series -from daft.table import MicroPartition class CustomClass: @@ -62,9 +62,9 @@ def __eq__(self, other): ], ) def test_table_expr_is_in_same_types(input, items, expected) -> None: - daft_table = MicroPartition.from_pydict({"input": input}) - daft_table = daft_table.eval_expression_list([col("input").is_in(items)]) - pydict = daft_table.to_pydict() + daft_recordbatch = MicroPartition.from_pydict({"input": input}) + daft_recordbatch = daft_recordbatch.eval_expression_list([col("input").is_in(items)]) + pydict = daft_recordbatch.to_pydict() assert pydict["input"] == expected @@ -103,14 +103,14 @@ def test_table_expr_is_in_same_types(input, items, expected) -> None: ], ) def test_table_expr_is_in_different_types_castable(input, items, expected) -> None: - daft_table = MicroPartition.from_pydict({"input": input}) + daft_recordbatch = MicroPartition.from_pydict({"input": input}) if expected is None: with pytest.raises(ValueError, match="Cannot perform comparison on types:"): - daft_table = daft_table.eval_expression_list([col("input").is_in(items)]) + daft_recordbatch = daft_recordbatch.eval_expression_list([col("input").is_in(items)]) else: - daft_table = daft_table.eval_expression_list([col("input").is_in(items)]) - pydict = daft_table.to_pydict() + daft_recordbatch = daft_recordbatch.eval_expression_list([col("input").is_in(items)]) + pydict = daft_recordbatch.to_pydict() assert pydict["input"] == expected @@ -126,9 +126,9 @@ def test_table_expr_is_in_different_types_castable(input, items, expected) -> No ], ) def test_table_expr_is_in_items_is_none(input, items, expected) -> None: - daft_table = MicroPartition.from_pydict({"input": input}) - daft_table = daft_table.eval_expression_list([col("input").is_in(items)]) - pydict = daft_table.to_pydict() + daft_recordbatch = MicroPartition.from_pydict({"input": input}) + daft_recordbatch = daft_recordbatch.eval_expression_list([col("input").is_in(items)]) + pydict = daft_recordbatch.to_pydict() assert pydict["input"] == expected @@ -143,31 +143,31 @@ def test_table_expr_is_in_items_is_none(input, items, expected) -> None: ], ) def test_table_expr_is_in_different_input_types(input, items, expected) -> None: - daft_table = MicroPartition.from_pydict({"input": input}) - daft_table = daft_table.eval_expression_list([col("input").is_in(items)]) - pydict = daft_table.to_pydict() + daft_recordbatch = MicroPartition.from_pydict({"input": input}) + daft_recordbatch = daft_recordbatch.eval_expression_list([col("input").is_in(items)]) + pydict = daft_recordbatch.to_pydict() assert pydict["input"] == expected def test_table_expr_is_in_with_another_df_column() -> None: - daft_table = MicroPartition.from_pydict({"input": [1, 2, 3, 4], "items": [3, 4, 5, 6]}) - daft_table = daft_table.eval_expression_list([col("input").is_in(col("items"))]) - pydict = daft_table.to_pydict() + daft_recordbatch = MicroPartition.from_pydict({"input": [1, 2, 3, 4], "items": [3, 4, 5, 6]}) + daft_recordbatch = daft_recordbatch.eval_expression_list([col("input").is_in(col("items"))]) + pydict = daft_recordbatch.to_pydict() assert pydict["input"] == [False, False, True, True] def test_table_expr_is_in_empty_items() -> None: - daft_table = MicroPartition.from_pydict({"input": [1, 2, 3, 4]}) - daft_table = daft_table.eval_expression_list([col("input").is_in([])]) - pydict = daft_table.to_pydict() + daft_recordbatch = MicroPartition.from_pydict({"input": [1, 2, 3, 4]}) + daft_recordbatch = daft_recordbatch.eval_expression_list([col("input").is_in([])]) + pydict = daft_recordbatch.to_pydict() assert pydict["input"] == [False, False, False, False] def test_table_expr_is_in_items_invalid_input() -> None: - daft_table = MicroPartition.from_pydict({"input": [1, 2, 3, 4]}) + daft_recordbatch = MicroPartition.from_pydict({"input": [1, 2, 3, 4]}) with pytest.raises(ValueError, match="Creating a Series from data of type"): - daft_table.eval_expression_list([col("input").is_in(1)]) + daft_recordbatch.eval_expression_list([col("input").is_in(1)]) diff --git a/tests/table/test_joins.py b/tests/recordbatch/test_joins.py similarity index 99% rename from tests/table/test_joins.py rename to tests/recordbatch/test_joins.py index 81c305026a..3b756af2f6 100644 --- a/tests/table/test_joins.py +++ b/tests/recordbatch/test_joins.py @@ -8,8 +8,8 @@ from daft.daft import JoinType from daft.datatype import DataType from daft.expressions import col +from daft.recordbatch import MicroPartition from daft.series import Series -from daft.table import MicroPartition daft_int_types = [ DataType.int8(), diff --git a/tests/table/test_micropartition.py b/tests/recordbatch/test_micropartition.py similarity index 92% rename from tests/table/test_micropartition.py rename to tests/recordbatch/test_micropartition.py index 1afacc8415..949f67dcbe 100644 --- a/tests/table/test_micropartition.py +++ b/tests/recordbatch/test_micropartition.py @@ -6,7 +6,7 @@ import pytest from daft.logical.schema import Schema -from daft.table.micropartition import MicroPartition +from daft.recordbatch.micropartition import MicroPartition @pytest.mark.parametrize( diff --git a/tests/table/test_minhash.py b/tests/recordbatch/test_minhash.py similarity index 100% rename from tests/table/test_minhash.py rename to tests/recordbatch/test_minhash.py diff --git a/tests/table/test_monotonically_increasing_id.py b/tests/recordbatch/test_monotonically_increasing_id.py similarity index 84% rename from tests/table/test_monotonically_increasing_id.py rename to tests/recordbatch/test_monotonically_increasing_id.py index 9220f69790..a46384372f 100644 --- a/tests/table/test_monotonically_increasing_id.py +++ b/tests/recordbatch/test_monotonically_increasing_id.py @@ -1,8 +1,8 @@ from __future__ import annotations from daft.datatype import DataType -from daft.table.micropartition import MicroPartition -from daft.table.table import Table +from daft.recordbatch.micropartition import MicroPartition +from daft.recordbatch.recordbatch import RecordBatch def test_monotonically_increasing_id() -> None: @@ -28,9 +28,9 @@ def test_monotonically_increasing_id_empty_table() -> None: def test_monotonically_increasing_id_multiple_tables_in_micropartition() -> None: - table1 = Table.from_pydict({"a": [1, 2, 3]}) - table2 = Table.from_pydict({"a": [4, 5, 6]}) - table3 = Table.from_pydict({"a": [7, 8, 9]}) + table1 = RecordBatch.from_pydict({"a": [1, 2, 3]}) + table2 = RecordBatch.from_pydict({"a": [4, 5, 6]}) + table3 = RecordBatch.from_pydict({"a": [7, 8, 9]}) table = MicroPartition._from_tables([table1, table2, table3]) table = table.add_monotonically_increasing_id(0, "id") diff --git a/tests/table/test_partitioning.py b/tests/recordbatch/test_partitioning.py similarity index 94% rename from tests/table/test_partitioning.py rename to tests/recordbatch/test_partitioning.py index 1ca60634cc..83e8ee6cf1 100644 --- a/tests/table/test_partitioning.py +++ b/tests/recordbatch/test_partitioning.py @@ -9,7 +9,7 @@ from daft.datatype import DataType from daft.expressions import col from daft.logical.schema import Schema -from daft.table import MicroPartition, Table +from daft.recordbatch import MicroPartition, RecordBatch daft_int_types = [ DataType.int8(), @@ -67,7 +67,7 @@ def test_partitioning_micropartitions_hash(mp) -> None: ], ) def test_partitioning_micropartitions_range_empty(mp) -> None: - boundaries = Table.from_pydict({"a": np.linspace(0, 10, 3)[1:]}).eval_expression_list( + boundaries = RecordBatch.from_pydict({"a": np.linspace(0, 10, 3)[1:]}).eval_expression_list( [col("a").cast(DataType.int64())] ) split_tables = mp.partition_by_range([col("a")], boundaries, [True]) @@ -85,7 +85,7 @@ def test_partitioning_micropartitions_range_empty(mp) -> None: ], ) def test_partitioning_micropartitions_range_boundaries_empty(mp) -> None: - boundaries = Table.from_pydict({"a": [], "b": []}).eval_expression_list([col("a").cast(DataType.int64())]) + boundaries = RecordBatch.from_pydict({"a": [], "b": []}).eval_expression_list([col("a").cast(DataType.int64())]) split_tables = mp.partition_by_range([col("a"), col("b")], boundaries, [False, False]) assert len(split_tables) == 1 assert split_tables[0].to_pydict() == {"a": [], "b": []} @@ -105,7 +105,7 @@ def test_partitioning_micropartitions_range_boundaries_empty(mp) -> None: ], ) def test_partitioning_micropartitions_range(mp) -> None: - boundaries = Table.from_pydict({"a": np.linspace(0, 5, 3)[1:]}).eval_expression_list( + boundaries = RecordBatch.from_pydict({"a": np.linspace(0, 5, 3)[1:]}).eval_expression_list( [col("a").cast(DataType.int64())] ) split_tables = mp.partition_by_range([col("a")], boundaries, [True]) @@ -251,7 +251,7 @@ def test_table_partition_by_range_single_column(size, k, desc) -> None: if desc: input_boundaries = input_boundaries[::-1] - boundaries = Table.from_pydict({"x": input_boundaries}).eval_expression_list( + boundaries = RecordBatch.from_pydict({"x": input_boundaries}).eval_expression_list( [col("x").cast(table.get_column("x").datatype())] ) @@ -286,7 +286,7 @@ def test_table_partition_by_range_multi_column(size, k, desc) -> None: if desc: input_boundaries = input_boundaries[::-1] - boundaries = Table.from_pydict({"x": np.ones(k - 1), "y": input_boundaries}).eval_expression_list( + boundaries = RecordBatch.from_pydict({"x": np.ones(k - 1), "y": input_boundaries}).eval_expression_list( [col("x").cast(table.get_column("x").datatype()), col("y").cast(table.get_column("y").datatype())] ) @@ -310,7 +310,7 @@ def test_table_partition_by_range_multi_column(size, k, desc) -> None: def test_table_partition_by_range_multi_column_string() -> None: table = MicroPartition.from_pydict({"x": ["a", "c", "a", "c"], "y": ["1", "2", "3", "4"]}) - boundaries = Table.from_pydict({"x": ["b"], "y": ["1"]}) + boundaries = RecordBatch.from_pydict({"x": ["b"], "y": ["1"]}) split_tables = table.partition_by_range([col("x"), col("y")], boundaries, [False, False]) assert len(split_tables) == 2 @@ -326,7 +326,7 @@ def test_table_partition_by_range_multi_column_string() -> None: def test_table_partition_by_range_input() -> None: data = {"x": [1, 2, 3], "b": [0, 1, 2]} table_cls = MicroPartition.from_pydict(data) - boundaries = Table.from_pydict(data) + boundaries = RecordBatch.from_pydict(data) with pytest.raises(ValueError, match="Schema Mismatch"): table_cls.partition_by_range([col("x")], boundaries, [False]) diff --git a/tests/table/test_pivot.py b/tests/recordbatch/test_pivot.py similarity index 64% rename from tests/table/test_pivot.py rename to tests/recordbatch/test_pivot.py index 53a32ba9ef..734573cff0 100644 --- a/tests/table/test_pivot.py +++ b/tests/recordbatch/test_pivot.py @@ -3,23 +3,23 @@ import pytest from daft.expressions.expressions import col -from daft.table.micropartition import MicroPartition +from daft.recordbatch.micropartition import MicroPartition def test_pivot_empty_table() -> None: - daft_table = MicroPartition.from_pydict( + daft_recordbatch = MicroPartition.from_pydict( { "group": [], "pivot": [], "value": [], } ) - daft_table = daft_table.pivot([col("group")], col("pivot"), col("value"), []) + daft_recordbatch = daft_recordbatch.pivot([col("group")], col("pivot"), col("value"), []) expected = { "group": [], } - assert daft_table.to_pydict() == expected + assert daft_recordbatch.to_pydict() == expected @pytest.mark.parametrize( @@ -43,42 +43,42 @@ def test_pivot_empty_table() -> None: ], ) def test_pivot_multipartition(input: MicroPartition) -> None: - daft_table = input.pivot([col("group")], col("pivot"), col("value"), ["1", "2"]) + daft_recordbatch = input.pivot([col("group")], col("pivot"), col("value"), ["1", "2"]) expected = { "group": ["A", "B"], "1": [3, 1], "2": [2, 4], } - assert daft_table.to_pydict() == expected + assert daft_recordbatch.to_pydict() == expected def test_pivot_column_names_subset() -> None: - daft_table = MicroPartition.from_pydict( + daft_recordbatch = MicroPartition.from_pydict( { "group": ["A", "A", "B"], "pivot": [1, 2, 1], "value": [1, 2, 3], } ) - daft_table = daft_table.pivot([col("group")], col("pivot"), col("value"), ["1"]) + daft_recordbatch = daft_recordbatch.pivot([col("group")], col("pivot"), col("value"), ["1"]) expected = { "group": ["A", "B"], "1": [1, 3], } - assert daft_table.to_pydict() == expected + assert daft_recordbatch.to_pydict() == expected def test_pivot_column_names_superset() -> None: - daft_table = MicroPartition.from_pydict( + daft_recordbatch = MicroPartition.from_pydict( { "group": ["A", "A", "B"], "pivot": [1, 2, 1], "value": [1, 2, 3], } ) - daft_table = daft_table.pivot([col("group")], col("pivot"), col("value"), ["1", "2", "3"]) + daft_recordbatch = daft_recordbatch.pivot([col("group")], col("pivot"), col("value"), ["1", "2", "3"]) expected = { "group": ["A", "B"], @@ -86,40 +86,40 @@ def test_pivot_column_names_superset() -> None: "2": [2, None], "3": [None, None], } - assert daft_table.to_pydict() == expected + assert daft_recordbatch.to_pydict() == expected def test_pivot_nulls_in_group() -> None: - daft_table = MicroPartition.from_pydict( + daft_recordbatch = MicroPartition.from_pydict( { "group": ["A", None, "B", None], "pivot": [1, 1, 2, 2], "value": [1, 2, 3, 4], } ) - daft_table = daft_table.pivot([col("group")], col("pivot"), col("value"), ["1", "2"]) + daft_recordbatch = daft_recordbatch.pivot([col("group")], col("pivot"), col("value"), ["1", "2"]) expected = { "group": ["A", "B", None], "1": [1, None, 2], "2": [None, 3, 4], } - assert daft_table.to_pydict() == expected + assert daft_recordbatch.to_pydict() == expected def test_pivot_nulls_in_pivot() -> None: - daft_table = MicroPartition.from_pydict( + daft_recordbatch = MicroPartition.from_pydict( { "group": ["A", "A", "B", "B"], "pivot": [1, None, 1, None], "value": [1, 2, 3, 4], } ) - daft_table = daft_table.pivot([col("group")], col("pivot"), col("value"), ["1", "None"]) + daft_recordbatch = daft_recordbatch.pivot([col("group")], col("pivot"), col("value"), ["1", "None"]) expected = { "group": ["A", "B"], "1": [1, 3], "None": [2, 4], } - assert daft_table.to_pydict() == expected + assert daft_recordbatch.to_pydict() == expected diff --git a/tests/table/test_size_bytes.py b/tests/recordbatch/test_size_bytes.py similarity index 97% rename from tests/table/test_size_bytes.py rename to tests/recordbatch/test_size_bytes.py index 94109ff63e..85d58703f4 100644 --- a/tests/table/test_size_bytes.py +++ b/tests/recordbatch/test_size_bytes.py @@ -6,7 +6,7 @@ from daft import DataType, col from daft.logical.schema import Schema -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition @pytest.mark.parametrize( diff --git a/tests/table/test_sorting.py b/tests/recordbatch/test_sorting.py similarity index 53% rename from tests/table/test_sorting.py rename to tests/recordbatch/test_sorting.py index 0f9e635685..130956509b 100644 --- a/tests/table/test_sorting.py +++ b/tests/recordbatch/test_sorting.py @@ -8,9 +8,9 @@ from daft import col from daft.logical.schema import Schema +from daft.recordbatch import MicroPartition from daft.series import Series -from daft.table import MicroPartition -from tests.table import daft_numeric_types, daft_string_types +from tests.recordbatch import daft_numeric_types, daft_string_types @pytest.mark.parametrize( @@ -55,20 +55,24 @@ def test_table_single_col_sorting(sort_dtype, value_dtype, first_col) -> None: argsort_order = Series.from_pylist([3, 2, 1, 4, 0]) - daft_table = MicroPartition.from_arrow(pa_table) + daft_recordbatch = MicroPartition.from_arrow(pa_table) if first_col: - daft_table = daft_table.eval_expression_list([col("a").cast(sort_dtype), col("b").cast(value_dtype)]) + daft_recordbatch = daft_recordbatch.eval_expression_list( + [col("a").cast(sort_dtype), col("b").cast(value_dtype)] + ) else: - daft_table = daft_table.eval_expression_list([col("b").cast(value_dtype), col("a").cast(sort_dtype)]) + daft_recordbatch = daft_recordbatch.eval_expression_list( + [col("b").cast(value_dtype), col("a").cast(sort_dtype)] + ) - assert len(daft_table) == 5 + assert len(daft_recordbatch) == 5 if first_col: - assert daft_table.column_names() == ["a", "b"] + assert daft_recordbatch.column_names() == ["a", "b"] else: - assert daft_table.column_names() == ["b", "a"] + assert daft_recordbatch.column_names() == ["b", "a"] - sorted_table = daft_table.sort([col("a")]) + sorted_table = daft_recordbatch.sort([col("a")]) assert len(sorted_table) == 5 @@ -77,17 +81,17 @@ def test_table_single_col_sorting(sort_dtype, value_dtype, first_col) -> None: else: assert sorted_table.column_names() == ["b", "a"] - assert sorted_table.get_column("a").datatype() == daft_table.get_column("a").datatype() - assert sorted_table.get_column("b").datatype() == daft_table.get_column("b").datatype() + assert sorted_table.get_column("a").datatype() == daft_recordbatch.get_column("a").datatype() + assert sorted_table.get_column("b").datatype() == daft_recordbatch.get_column("b").datatype() - assert sorted_table.get_column("a").to_pylist() == daft_table.get_column("a").take(argsort_order).to_pylist() - assert sorted_table.get_column("b").to_pylist() == daft_table.get_column("b").take(argsort_order).to_pylist() + assert sorted_table.get_column("a").to_pylist() == daft_recordbatch.get_column("a").take(argsort_order).to_pylist() + assert sorted_table.get_column("b").to_pylist() == daft_recordbatch.get_column("b").take(argsort_order).to_pylist() - assert daft_table.argsort([col("a")]).to_pylist() == argsort_order.to_pylist() + assert daft_recordbatch.argsort([col("a")]).to_pylist() == argsort_order.to_pylist() # Descending - sorted_table = daft_table.sort([col("a")], descending=True) + sorted_table = daft_recordbatch.sort([col("a")], descending=True) assert len(sorted_table) == 5 if first_col: @@ -95,13 +99,19 @@ def test_table_single_col_sorting(sort_dtype, value_dtype, first_col) -> None: else: assert sorted_table.column_names() == ["b", "a"] - assert sorted_table.get_column("a").datatype() == daft_table.get_column("a").datatype() - assert sorted_table.get_column("b").datatype() == daft_table.get_column("b").datatype() + assert sorted_table.get_column("a").datatype() == daft_recordbatch.get_column("a").datatype() + assert sorted_table.get_column("b").datatype() == daft_recordbatch.get_column("b").datatype() - assert sorted_table.get_column("a").to_pylist() == daft_table.get_column("a").take(argsort_order).to_pylist()[::-1] - assert sorted_table.get_column("b").to_pylist() == daft_table.get_column("b").take(argsort_order).to_pylist()[::-1] + assert ( + sorted_table.get_column("a").to_pylist() + == daft_recordbatch.get_column("a").take(argsort_order).to_pylist()[::-1] + ) + assert ( + sorted_table.get_column("b").to_pylist() + == daft_recordbatch.get_column("b").take(argsort_order).to_pylist()[::-1] + ) - assert daft_table.argsort([col("a")], descending=True).to_pylist() == argsort_order.to_pylist()[::-1] + assert daft_recordbatch.argsort([col("a")], descending=True).to_pylist() == argsort_order.to_pylist()[::-1] @pytest.mark.parametrize( @@ -127,45 +137,52 @@ def test_table_multiple_col_sorting(sort_dtype, value_dtype, data) -> None: argsort_order = Series.from_pylist(expected) - daft_table = MicroPartition.from_arrow(pa_table) + daft_recordbatch = MicroPartition.from_arrow(pa_table) - daft_table = daft_table.eval_expression_list([col("a").cast(sort_dtype), col("b").cast(value_dtype)]) + daft_recordbatch = daft_recordbatch.eval_expression_list([col("a").cast(sort_dtype), col("b").cast(value_dtype)]) - assert len(daft_table) == 5 - assert daft_table.column_names() == ["a", "b"] + assert len(daft_recordbatch) == 5 + assert daft_recordbatch.column_names() == ["a", "b"] - sorted_table = daft_table.sort([col("a"), col("b")], descending=[a_desc, b_desc]) + sorted_table = daft_recordbatch.sort([col("a"), col("b")], descending=[a_desc, b_desc]) assert len(sorted_table) == 5 assert sorted_table.column_names() == ["a", "b"] - assert sorted_table.get_column("a").datatype() == daft_table.get_column("a").datatype() - assert sorted_table.get_column("b").datatype() == daft_table.get_column("b").datatype() + assert sorted_table.get_column("a").datatype() == daft_recordbatch.get_column("a").datatype() + assert sorted_table.get_column("b").datatype() == daft_recordbatch.get_column("b").datatype() - assert sorted_table.get_column("a").to_pylist() == daft_table.get_column("a").take(argsort_order).to_pylist() - assert sorted_table.get_column("b").to_pylist() == daft_table.get_column("b").take(argsort_order).to_pylist() + assert sorted_table.get_column("a").to_pylist() == daft_recordbatch.get_column("a").take(argsort_order).to_pylist() + assert sorted_table.get_column("b").to_pylist() == daft_recordbatch.get_column("b").take(argsort_order).to_pylist() assert ( - daft_table.argsort([col("a"), col("b")], descending=[a_desc, b_desc]).to_pylist() == argsort_order.to_pylist() + daft_recordbatch.argsort([col("a"), col("b")], descending=[a_desc, b_desc]).to_pylist() + == argsort_order.to_pylist() ) # Descending - sorted_table = daft_table.sort([col("a"), col("b")], descending=[not a_desc, not b_desc]) + sorted_table = daft_recordbatch.sort([col("a"), col("b")], descending=[not a_desc, not b_desc]) assert len(sorted_table) == 5 assert sorted_table.column_names() == ["a", "b"] - assert sorted_table.get_column("a").datatype() == daft_table.get_column("a").datatype() - assert sorted_table.get_column("b").datatype() == daft_table.get_column("b").datatype() + assert sorted_table.get_column("a").datatype() == daft_recordbatch.get_column("a").datatype() + assert sorted_table.get_column("b").datatype() == daft_recordbatch.get_column("b").datatype() - assert sorted_table.get_column("a").to_pylist() == daft_table.get_column("a").take(argsort_order).to_pylist()[::-1] - assert sorted_table.get_column("b").to_pylist() == daft_table.get_column("b").take(argsort_order).to_pylist()[::-1] + assert ( + sorted_table.get_column("a").to_pylist() + == daft_recordbatch.get_column("a").take(argsort_order).to_pylist()[::-1] + ) + assert ( + sorted_table.get_column("b").to_pylist() + == daft_recordbatch.get_column("b").take(argsort_order).to_pylist()[::-1] + ) assert ( - daft_table.argsort([col("a"), col("b")], descending=[not a_desc, not b_desc]).to_pylist() + daft_recordbatch.argsort([col("a"), col("b")], descending=[not a_desc, not b_desc]).to_pylist() == argsort_order.to_pylist()[::-1] ) @@ -192,42 +209,49 @@ def test_table_multiple_col_sorting_binary(data) -> None: argsort_order = Series.from_pylist(expected) - daft_table = MicroPartition.from_arrow(pa_table) - assert len(daft_table) == 5 - assert daft_table.column_names() == ["a", "b"] + daft_recordbatch = MicroPartition.from_arrow(pa_table) + assert len(daft_recordbatch) == 5 + assert daft_recordbatch.column_names() == ["a", "b"] - sorted_table = daft_table.sort([col("a"), col("b")], descending=[a_desc, b_desc]) + sorted_table = daft_recordbatch.sort([col("a"), col("b")], descending=[a_desc, b_desc]) assert len(sorted_table) == 5 assert sorted_table.column_names() == ["a", "b"] - assert sorted_table.get_column("a").datatype() == daft_table.get_column("a").datatype() - assert sorted_table.get_column("b").datatype() == daft_table.get_column("b").datatype() + assert sorted_table.get_column("a").datatype() == daft_recordbatch.get_column("a").datatype() + assert sorted_table.get_column("b").datatype() == daft_recordbatch.get_column("b").datatype() - assert sorted_table.get_column("a").to_pylist() == daft_table.get_column("a").take(argsort_order).to_pylist() - assert sorted_table.get_column("b").to_pylist() == daft_table.get_column("b").take(argsort_order).to_pylist() + assert sorted_table.get_column("a").to_pylist() == daft_recordbatch.get_column("a").take(argsort_order).to_pylist() + assert sorted_table.get_column("b").to_pylist() == daft_recordbatch.get_column("b").take(argsort_order).to_pylist() assert ( - daft_table.argsort([col("a"), col("b")], descending=[a_desc, b_desc]).to_pylist() == argsort_order.to_pylist() + daft_recordbatch.argsort([col("a"), col("b")], descending=[a_desc, b_desc]).to_pylist() + == argsort_order.to_pylist() ) # Descending - sorted_table = daft_table.sort([col("a"), col("b")], descending=[not a_desc, not b_desc]) + sorted_table = daft_recordbatch.sort([col("a"), col("b")], descending=[not a_desc, not b_desc]) assert len(sorted_table) == 5 assert sorted_table.column_names() == ["a", "b"] - assert sorted_table.get_column("a").datatype() == daft_table.get_column("a").datatype() - assert sorted_table.get_column("b").datatype() == daft_table.get_column("b").datatype() + assert sorted_table.get_column("a").datatype() == daft_recordbatch.get_column("a").datatype() + assert sorted_table.get_column("b").datatype() == daft_recordbatch.get_column("b").datatype() - assert sorted_table.get_column("a").to_pylist() == daft_table.get_column("a").take(argsort_order).to_pylist()[::-1] - assert sorted_table.get_column("b").to_pylist() == daft_table.get_column("b").take(argsort_order).to_pylist()[::-1] + assert ( + sorted_table.get_column("a").to_pylist() + == daft_recordbatch.get_column("a").take(argsort_order).to_pylist()[::-1] + ) + assert ( + sorted_table.get_column("b").to_pylist() + == daft_recordbatch.get_column("b").take(argsort_order).to_pylist()[::-1] + ) assert ( - daft_table.argsort([col("a"), col("b")], descending=[not a_desc, not b_desc]).to_pylist() + daft_recordbatch.argsort([col("a"), col("b")], descending=[not a_desc, not b_desc]).to_pylist() == argsort_order.to_pylist()[::-1] ) @@ -251,45 +275,52 @@ def test_table_boolean_multiple_col_sorting(second_dtype, data) -> None: pa_table = pa.Table.from_pydict({"a": a, "b": b}) argsort_order = Series.from_pylist(expected) - daft_table = MicroPartition.from_arrow(pa_table) + daft_recordbatch = MicroPartition.from_arrow(pa_table) - daft_table = daft_table.eval_expression_list([col("a"), col("b").cast(second_dtype)]) + daft_recordbatch = daft_recordbatch.eval_expression_list([col("a"), col("b").cast(second_dtype)]) - assert len(daft_table) == 5 - assert daft_table.column_names() == ["a", "b"] + assert len(daft_recordbatch) == 5 + assert daft_recordbatch.column_names() == ["a", "b"] - sorted_table = daft_table.sort([col("a"), col("b")], descending=[a_desc, b_desc]) + sorted_table = daft_recordbatch.sort([col("a"), col("b")], descending=[a_desc, b_desc]) assert len(sorted_table) == 5 assert sorted_table.column_names() == ["a", "b"] - assert sorted_table.get_column("a").datatype() == daft_table.get_column("a").datatype() - assert sorted_table.get_column("b").datatype() == daft_table.get_column("b").datatype() + assert sorted_table.get_column("a").datatype() == daft_recordbatch.get_column("a").datatype() + assert sorted_table.get_column("b").datatype() == daft_recordbatch.get_column("b").datatype() - assert sorted_table.get_column("a").to_pylist() == daft_table.get_column("a").take(argsort_order).to_pylist() - assert sorted_table.get_column("b").to_pylist() == daft_table.get_column("b").take(argsort_order).to_pylist() + assert sorted_table.get_column("a").to_pylist() == daft_recordbatch.get_column("a").take(argsort_order).to_pylist() + assert sorted_table.get_column("b").to_pylist() == daft_recordbatch.get_column("b").take(argsort_order).to_pylist() assert ( - daft_table.argsort([col("a"), col("b")], descending=[a_desc, b_desc]).to_pylist() == argsort_order.to_pylist() + daft_recordbatch.argsort([col("a"), col("b")], descending=[a_desc, b_desc]).to_pylist() + == argsort_order.to_pylist() ) # Descending - sorted_table = daft_table.sort([col("a"), col("b")], descending=[not a_desc, not b_desc]) + sorted_table = daft_recordbatch.sort([col("a"), col("b")], descending=[not a_desc, not b_desc]) assert len(sorted_table) == 5 assert sorted_table.column_names() == ["a", "b"] - assert sorted_table.get_column("a").datatype() == daft_table.get_column("a").datatype() - assert sorted_table.get_column("b").datatype() == daft_table.get_column("b").datatype() + assert sorted_table.get_column("a").datatype() == daft_recordbatch.get_column("a").datatype() + assert sorted_table.get_column("b").datatype() == daft_recordbatch.get_column("b").datatype() - assert sorted_table.get_column("a").to_pylist() == daft_table.get_column("a").take(argsort_order).to_pylist()[::-1] - assert sorted_table.get_column("b").to_pylist() == daft_table.get_column("b").take(argsort_order).to_pylist()[::-1] + assert ( + sorted_table.get_column("a").to_pylist() + == daft_recordbatch.get_column("a").take(argsort_order).to_pylist()[::-1] + ) + assert ( + sorted_table.get_column("b").to_pylist() + == daft_recordbatch.get_column("b").take(argsort_order).to_pylist()[::-1] + ) assert ( - daft_table.argsort([col("a"), col("b")], descending=[not a_desc, not b_desc]).to_pylist() + daft_recordbatch.argsort([col("a"), col("b")], descending=[not a_desc, not b_desc]).to_pylist() == argsort_order.to_pylist()[::-1] ) @@ -298,12 +329,12 @@ def test_table_sample() -> None: pa_table = pa.Table.from_pydict({"a": [1, 2, 3, 4], "b": [5, 6, 7, 8]}) source_pairs = {(1, 5), (2, 6), (3, 7), (4, 8)} - daft_table = MicroPartition.from_arrow(pa_table) - assert len(daft_table) == 4 - assert daft_table.column_names() == ["a", "b"] + daft_recordbatch = MicroPartition.from_arrow(pa_table) + assert len(daft_recordbatch) == 4 + assert daft_recordbatch.column_names() == ["a", "b"] # subsample - sampled = daft_table.sample(size=3) + sampled = daft_recordbatch.sample(size=3) assert len(sampled) == 3 assert sampled.column_names() == ["a", "b"] assert all( @@ -311,7 +342,7 @@ def test_table_sample() -> None: ) # oversample - sampled = daft_table.sample(size=4) + sampled = daft_recordbatch.sample(size=4) assert len(sampled) == 4 assert sampled.column_names() == ["a", "b"] assert all( @@ -320,23 +351,23 @@ def test_table_sample() -> None: # negative sample with pytest.raises(ValueError, match="negative size"): - daft_table.sample(size=-1) + daft_recordbatch.sample(size=-1) # fraction > 1.0 with pytest.raises(ValueError, match="fraction greater than 1.0"): - daft_table.sample(fraction=1.1) + daft_recordbatch.sample(fraction=1.1) # fraction < 0.0 with pytest.raises(ValueError, match="negative fraction"): - daft_table.sample(fraction=-0.1) + daft_recordbatch.sample(fraction=-0.1) # size and fraction with pytest.raises(ValueError, match="Must specify either `fraction` or `size`"): - daft_table.sample(size=1, fraction=0.5) + daft_recordbatch.sample(size=1, fraction=0.5) # no arguments with pytest.raises(ValueError, match="Must specify either `fraction` or `size`"): - daft_table.sample() + daft_recordbatch.sample() @pytest.mark.parametrize("size, k", itertools.product([0, 1, 10, 33, 100, 101], [0, 1, 2, 3, 100, 101, 200])) @@ -345,12 +376,12 @@ def test_table_quantiles(size, k) -> None: second = 2 * first - daft_table = MicroPartition.from_pydict({"a": first, "b": second}) - assert len(daft_table) == size - assert daft_table.column_names() == ["a", "b"] + daft_recordbatch = MicroPartition.from_pydict({"a": first, "b": second}) + assert len(daft_recordbatch) == size + assert daft_recordbatch.column_names() == ["a", "b"] # sub - quantiles = daft_table.quantiles(k) + quantiles = daft_recordbatch.quantiles(k) if size > 0: assert len(quantiles) == max(k - 1, 0) @@ -377,14 +408,14 @@ def test_table_quantiles_bad_input() -> None: pa_table = pa.Table.from_pydict({"a": first, "b": second}) - daft_table = MicroPartition.from_arrow(pa_table) + daft_recordbatch = MicroPartition.from_arrow(pa_table) with pytest.raises(ValueError, match="negative number"): - daft_table.quantiles(-1) + daft_recordbatch.quantiles(-1) def test_string_table_sorting(): - daft_table = MicroPartition.from_pydict( + daft_recordbatch = MicroPartition.from_pydict( { "firstname": [ "bob", @@ -398,7 +429,7 @@ def test_string_table_sorting(): "lastname": ["a", "a", "a", "bond", None, None, "a"], } ) - sorted_table = daft_table.sort([col("firstname"), col("lastname")]) + sorted_table = daft_recordbatch.sort([col("firstname"), col("lastname")]) assert sorted_table.to_pydict() == { "firstname": ["alice", "alice", "bob", "bob", "eve", None, None], "lastname": ["a", "a", "a", None, "a", "bond", None], diff --git a/tests/table/test_table_aggs.py b/tests/recordbatch/test_table_aggs.py similarity index 81% rename from tests/table/test_table_aggs.py rename to tests/recordbatch/test_table_aggs.py index c56cca2cdb..b35e4346a4 100644 --- a/tests/table/test_table_aggs.py +++ b/tests/recordbatch/test_table_aggs.py @@ -9,9 +9,9 @@ from daft import DataType, col, from_pydict, utils from daft.logical.schema import Schema +from daft.recordbatch import MicroPartition from daft.series import Series -from daft.table import MicroPartition -from tests.table import ( +from tests.recordbatch import ( daft_comparable_types, daft_floating_types, daft_nonnull_types, @@ -77,29 +77,29 @@ def test_table_count(idx_dtype, case) -> None: input, expected = case if idx_dtype == DataType.date(): input = [datetime.date(2020 + x, 1 + x, 1 + x) if x is not None else None for x in input] - daft_table = MicroPartition.from_pydict({"input": input}) - daft_table = daft_table.eval_expression_list([col("input").cast(idx_dtype)]) - daft_table = daft_table.eval_expression_list([col("input").alias("count").count()]) + daft_recordbatch = MicroPartition.from_pydict({"input": input}) + daft_recordbatch = daft_recordbatch.eval_expression_list([col("input").cast(idx_dtype)]) + daft_recordbatch = daft_recordbatch.eval_expression_list([col("input").alias("count").count()]) - res = daft_table.to_pydict() + res = daft_recordbatch.to_pydict() assert res == expected @pytest.mark.parametrize("length", [0, 1, 10]) def test_table_count_nulltype(length) -> None: - daft_table = MicroPartition.from_pydict({"input": [None] * length}) - daft_table = daft_table.eval_expression_list([col("input").cast(DataType.null())]) - daft_table = daft_table.eval_expression_list([col("input").alias("count").count()]) + daft_recordbatch = MicroPartition.from_pydict({"input": [None] * length}) + daft_recordbatch = daft_recordbatch.eval_expression_list([col("input").cast(DataType.null())]) + daft_recordbatch = daft_recordbatch.eval_expression_list([col("input").alias("count").count()]) - res = daft_table.to_pydict()["count"] + res = daft_recordbatch.to_pydict()["count"] assert res == [0] def test_table_count_pyobject() -> None: - daft_table = MicroPartition.from_pydict({"objs": [object(), object(), None, object(), None]}) - daft_table = daft_table.eval_expression_list([col("objs").alias("count").count()]) + daft_recordbatch = MicroPartition.from_pydict({"objs": [object(), object(), None, object(), None]}) + daft_recordbatch = daft_recordbatch.eval_expression_list([col("objs").alias("count").count()]) - res = daft_table.to_pydict()["count"] + res = daft_recordbatch.to_pydict()["count"] assert res == [3] @@ -119,16 +119,16 @@ def test_table_count_pyobject() -> None: ) def test_table_minmax_numerics(idx_dtype, case) -> None: input, expected = case - daft_table = MicroPartition.from_pydict({"input": input}) - daft_table = daft_table.eval_expression_list([col("input").cast(idx_dtype)]) - daft_table = daft_table.eval_expression_list( + daft_recordbatch = MicroPartition.from_pydict({"input": input}) + daft_recordbatch = daft_recordbatch.eval_expression_list([col("input").cast(idx_dtype)]) + daft_recordbatch = daft_recordbatch.eval_expression_list( [ col("input").alias("min").min(), col("input").alias("max").max(), ] ) - res = daft_table.to_pydict() + res = daft_recordbatch.to_pydict() assert res == expected @@ -146,16 +146,16 @@ def test_table_minmax_numerics(idx_dtype, case) -> None: @pytest.mark.parametrize("case", test_table_minmax_string_cases, ids=[f"{_}" for _ in test_table_minmax_string_cases]) def test_table_minmax_string(idx_dtype, case) -> None: input, expected = case - daft_table = MicroPartition.from_pydict({"input": input}) - daft_table = daft_table.eval_expression_list([col("input").cast(idx_dtype)]) - daft_table = daft_table.eval_expression_list( + daft_recordbatch = MicroPartition.from_pydict({"input": input}) + daft_recordbatch = daft_recordbatch.eval_expression_list([col("input").cast(idx_dtype)]) + daft_recordbatch = daft_recordbatch.eval_expression_list( [ col("input").alias("min").min(), col("input").alias("max").max(), ] ) - res = daft_table.to_pydict() + res = daft_recordbatch.to_pydict() assert res == expected @@ -171,16 +171,16 @@ def test_table_minmax_string(idx_dtype, case) -> None: @pytest.mark.parametrize("case", test_table_minmax_bool_cases, ids=[f"{_}" for _ in test_table_minmax_bool_cases]) def test_table_minmax_bool(case) -> None: input, expected = case - daft_table = MicroPartition.from_pydict({"input": input}) - daft_table = daft_table.eval_expression_list([col("input").cast(DataType.bool())]) - daft_table = daft_table.eval_expression_list( + daft_recordbatch = MicroPartition.from_pydict({"input": input}) + daft_recordbatch = daft_recordbatch.eval_expression_list([col("input").cast(DataType.bool())]) + daft_recordbatch = daft_recordbatch.eval_expression_list( [ col("input").alias("min").min(), col("input").alias("max").max(), ] ) - res = daft_table.to_pydict() + res = daft_recordbatch.to_pydict() assert res == expected @@ -198,15 +198,15 @@ def test_table_minmax_bool(case) -> None: @pytest.mark.parametrize("type", [pa.binary(), pa.binary(1)]) def test_table_minmax_binary(case, type) -> None: input, expected = case - daft_table = MicroPartition.from_arrow(pa.table({"input": pa.array(input, type=type)})) - daft_table = daft_table.eval_expression_list( + daft_recordbatch = MicroPartition.from_arrow(pa.table({"input": pa.array(input, type=type)})) + daft_recordbatch = daft_recordbatch.eval_expression_list( [ col("input").alias("min").min(), col("input").alias("max").max(), ] ) - res = daft_table.to_pydict() + res = daft_recordbatch.to_pydict() assert res == expected @@ -224,38 +224,38 @@ def test_table_minmax_binary(case, type) -> None: @pytest.mark.parametrize("case", test_table_sum_mean_cases, ids=[f"{_}" for _ in test_table_sum_mean_cases]) def test_table_sum_mean(idx_dtype, case) -> None: input, expected = case - daft_table = MicroPartition.from_pydict({"input": input}) - daft_table = daft_table.eval_expression_list([col("input").cast(idx_dtype)]) - daft_table = daft_table.eval_expression_list( + daft_recordbatch = MicroPartition.from_pydict({"input": input}) + daft_recordbatch = daft_recordbatch.eval_expression_list([col("input").cast(idx_dtype)]) + daft_recordbatch = daft_recordbatch.eval_expression_list( [ col("input").alias("sum").sum(), col("input").alias("mean").mean(), ] ) - res = daft_table.to_pydict() + res = daft_recordbatch.to_pydict() assert res == expected @pytest.mark.parametrize("nptype", [np.uint8, np.uint16, np.uint32, np.int8, np.int16, np.int32]) def test_table_sum_upcast(nptype) -> None: """Tests correctness, including type upcasting, of sum aggregations.""" - daft_table = MicroPartition.from_pydict( + daft_recordbatch = MicroPartition.from_pydict( { "maxes": np.full(128, fill_value=np.iinfo(nptype).max, dtype=nptype), "mins": np.full(128, fill_value=np.iinfo(nptype).min, dtype=nptype), } ) - daft_table = daft_table.eval_expression_list([col("maxes").sum(), col("mins").sum()]) - pydict = daft_table.to_pydict() + daft_recordbatch = daft_recordbatch.eval_expression_list([col("maxes").sum(), col("mins").sum()]) + pydict = daft_recordbatch.to_pydict() assert pydict["maxes"] == [128 * np.iinfo(nptype).max] assert pydict["mins"] == [128 * np.iinfo(nptype).min] def test_table_sum_badtype() -> None: - daft_table = MicroPartition.from_pydict({"a": ["str1", "str2"]}) + daft_recordbatch = MicroPartition.from_pydict({"a": ["str1", "str2"]}) with pytest.raises(ValueError): - daft_table = daft_table.eval_expression_list([col("a").sum()]) + daft_recordbatch = daft_recordbatch.eval_expression_list([col("a").sum()]) test_micropartition_any_value_cases = [ @@ -311,14 +311,14 @@ def test_micropartition_any_value(mp, expected_nulls, expected_no_nulls): @pytest.mark.parametrize("case,expected_nulls,expected_no_nulls", test_table_any_value_cases) def test_table_any_value(case, expected_nulls, expected_no_nulls): - daft_table = MicroPartition.from_pydict(case) + daft_recordbatch = MicroPartition.from_pydict(case) - any_values = daft_table.agg([col("a").any_value(False)], group_by=[col("b")]).to_pydict() + any_values = daft_recordbatch.agg([col("a").any_value(False)], group_by=[col("b")]).to_pydict() assert len(any_values["b"]) == len(expected_nulls) for k, v in zip(any_values["b"], any_values["a"]): assert expected_nulls[k] or v is not None - any_values = daft_table.agg([col("a").any_value(True)], group_by=[col("b")]).to_pydict() + any_values = daft_recordbatch.agg([col("a").any_value(True)], group_by=[col("b")]).to_pydict() assert len(any_values["b"]) == len(expected_no_nulls) for k, v in zip(any_values["b"], any_values["a"]): assert expected_no_nulls[k] or v is not None @@ -355,8 +355,8 @@ def test_table_any_value(case, expected_nulls, expected_no_nulls): def test_table_agg_global(case) -> None: """Test that global aggregation works at the API layer.""" input, expected = case - daft_table = MicroPartition.from_pydict({"input": input}) - daft_table = daft_table.agg( + daft_recordbatch = MicroPartition.from_pydict({"input": input}) + daft_recordbatch = daft_recordbatch.agg( [ col("input").cast(DataType.int32()).alias("count").count(), col("input").cast(DataType.int32()).alias("sum").sum(), @@ -367,7 +367,7 @@ def test_table_agg_global(case) -> None: ] ) - result = daft_table.to_pydict() + result = daft_recordbatch.to_pydict() for key, value in expected.items(): assert result[key] == value @@ -381,12 +381,12 @@ def test_table_agg_global(case) -> None: ) def test_table_agg_groupby_empty(groups_and_aggs) -> None: groups, aggs = groups_and_aggs - daft_table = MicroPartition.from_pydict({"col_A": [], "col_B": []}) - daft_table = daft_table.agg( + daft_recordbatch = MicroPartition.from_pydict({"col_A": [], "col_B": []}) + daft_recordbatch = daft_recordbatch.agg( [col(a).count() for a in aggs], [col(g).cast(DataType.int32()) for g in groups], ) - res = daft_table.to_pydict() + res = daft_recordbatch.to_pydict() assert res == {"col_A": [], "col_B": []} @@ -444,30 +444,30 @@ def test_table_agg_groupby(case) -> None: ("Alice", None), ("Alice", None), ] - daft_table = MicroPartition.from_pydict( + daft_recordbatch = MicroPartition.from_pydict( { "name": [_[0] for _ in values], "cookies": [_[1] for _ in values], } ) - daft_table = daft_table.agg( + daft_recordbatch = daft_recordbatch.agg( [aggexpr for aggexpr in case["aggs"]], [col(group) for group in case["groups"]], ) - assert set(utils.freeze(utils.pydict_to_rows(daft_table.to_pydict()))) == set( + assert set(utils.freeze(utils.pydict_to_rows(daft_recordbatch.to_pydict()))) == set( utils.freeze(utils.pydict_to_rows(case["expected"])) ) @pytest.mark.parametrize("dtype", daft_comparable_types, ids=[f"{_}" for _ in daft_comparable_types]) def test_groupby_all_nulls(dtype) -> None: - daft_table = MicroPartition.from_pydict( + daft_recordbatch = MicroPartition.from_pydict( { "group": Series.from_pylist([None, None, None]).cast(dtype), "cookies": [1, 2, 3], } ) - result_table = daft_table.agg([col("cookies").sum()], group_by=[col("group")]) + result_table = daft_recordbatch.agg([col("cookies").sum()], group_by=[col("group")]) assert result_table.to_pydict() == {"group": [None], "cookies": [6]} @@ -477,13 +477,13 @@ def test_groupby_all_nulls(dtype) -> None: ids=[f"{_}" for _ in daft_numeric_types + daft_string_types + [DataType.bool()]], ) def test_groupby_numeric_string_bool_some_nulls(dtype) -> None: - daft_table = MicroPartition.from_pydict( + daft_recordbatch = MicroPartition.from_pydict( { "group": Series.from_pylist([1, 1, None]).cast(dtype), "cookies": [2, 2, 3], } ) - result_table = daft_table.agg([col("cookies").sum()], group_by=[col("group")]) + result_table = daft_recordbatch.agg([col("cookies").sum()], group_by=[col("group")]) expected_table = MicroPartition.from_pydict( { "group": Series.from_pylist([1, None]).cast(dtype), @@ -502,13 +502,13 @@ def test_groupby_numeric_string_bool_some_nulls(dtype) -> None: ids=[f"{_}" for _ in daft_numeric_types + daft_string_types + [DataType.bool()]], ) def test_groupby_numeric_string_bool_no_nulls(dtype) -> None: - daft_table = MicroPartition.from_pydict( + daft_recordbatch = MicroPartition.from_pydict( { "group": Series.from_pylist([1, 0, 1, 0]).cast(dtype), "cookies": [1, 2, 2, 3], } ) - result_table = daft_table.agg([col("cookies").sum()], group_by=[col("group")]) + result_table = daft_recordbatch.agg([col("cookies").sum()], group_by=[col("group")]) expected_table = MicroPartition.from_pydict( { "group": Series.from_pylist([0, 1]).cast(dtype), @@ -530,13 +530,13 @@ def test_groupby_numeric_string_bool_no_nulls(dtype) -> None: ], ) def test_groupby_binary_bool_some_nulls(type, agg, expected) -> None: - daft_table = MicroPartition.from_pydict( + daft_recordbatch = MicroPartition.from_pydict( { "group": Series.from_arrow(pa.array([b"1", b"1", None, None], type=type)), "cookies": Series.from_arrow(pa.array([b"1", b"2", b"3", b"4"], type=type)), } ) - result_table = daft_table.agg([agg], group_by=[col("group")]) + result_table = daft_recordbatch.agg([agg], group_by=[col("group")]) expected_table = MicroPartition.from_pydict( { "group": Series.from_pylist([b"1", None]), @@ -558,13 +558,13 @@ def test_groupby_binary_bool_some_nulls(type, agg, expected) -> None: ], ) def test_groupby_binary_no_nulls(type, agg, expected) -> None: - daft_table = MicroPartition.from_pydict( + daft_recordbatch = MicroPartition.from_pydict( { "group": Series.from_arrow(pa.array([b"1", b"0", b"1", b"0"], type=type)), "cookies": Series.from_arrow(pa.array([b"1", b"2", b"3", b"4"], type=type)), } ) - result_table = daft_table.agg([agg], group_by=[col("group")]) + result_table = daft_recordbatch.agg([agg], group_by=[col("group")]) expected_table = MicroPartition.from_pydict( { "group": Series.from_pylist([b"0", b"1"]), @@ -586,7 +586,7 @@ def test_groupby_floats_nan(dtype) -> None: NAN = float("nan") INF = float("inf") - daft_table = MicroPartition.from_pydict( + daft_recordbatch = MicroPartition.from_pydict( { "group": Series.from_pylist([None, 1.0, NAN, 5 * NAN, -1 * NAN, -NAN, 1.0, None, INF, -INF, INF]).cast( dtype @@ -594,7 +594,7 @@ def test_groupby_floats_nan(dtype) -> None: "cookies": [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10], } ) - result_table = daft_table.agg([col("cookies").count()], group_by=[col("group")]) + result_table = daft_recordbatch.agg([col("cookies").count()], group_by=[col("group")]) expected_table = MicroPartition.from_pydict( { "group": Series.from_pylist([None, 1.0, NAN, -INF, INF]).cast(dtype), @@ -610,7 +610,7 @@ def test_groupby_floats_nan(dtype) -> None: def test_groupby_timestamp() -> None: - daft_table = MicroPartition.from_pydict( + daft_recordbatch = MicroPartition.from_pydict( { "group": Series.from_pylist( [ @@ -628,7 +628,7 @@ def test_groupby_timestamp() -> None: "value": [1, 1, 1, 2, 2, 2, 3, 3, 3], } ) - result_table = daft_table.agg([col("value").sum()], group_by=[col("group").dt.truncate("1 hour")]) + result_table = daft_recordbatch.agg([col("value").sum()], group_by=[col("group").dt.truncate("1 hour")]) expected_table = MicroPartition.from_pydict( { "group": Series.from_pylist( @@ -654,11 +654,11 @@ def test_global_list_aggs(dtype) -> None: input = [None, 0, 1, 2, None, 4] if dtype == DataType.date(): input = [datetime.date(2020 + x, 1 + x, 1 + x) if x is not None else None for x in input] - daft_table = MicroPartition.from_pydict({"input": input}) - daft_table = daft_table.eval_expression_list([col("input").cast(dtype)]) - result = daft_table.eval_expression_list([col("input").alias("list").agg_list()]) + daft_recordbatch = MicroPartition.from_pydict({"input": input}) + daft_recordbatch = daft_recordbatch.eval_expression_list([col("input").cast(dtype)]) + result = daft_recordbatch.eval_expression_list([col("input").alias("list").agg_list()]) assert result.get_column("list").datatype() == DataType.list(dtype) - assert result.to_pydict() == {"list": [daft_table.to_pydict()["input"]]} + assert result.to_pydict() == {"list": [daft_recordbatch.to_pydict()["input"]]} def test_global_pyobj_list_aggs() -> None: @@ -705,12 +705,14 @@ def test_grouped_list_aggs(dtype) -> None: if dtype == DataType.date(): input = [datetime.date(2020 + x, 1 + x, 1 + x) if x is not None else None for x in input] - daft_table = MicroPartition.from_pydict({"groups": groups, "input": input}) - daft_table = daft_table.eval_expression_list([col("groups"), col("input").cast(dtype)]) - result = daft_table.agg([col("input").alias("list").agg_list()], group_by=[col("groups")]).sort([col("groups")]) + daft_recordbatch = MicroPartition.from_pydict({"groups": groups, "input": input}) + daft_recordbatch = daft_recordbatch.eval_expression_list([col("groups"), col("input").cast(dtype)]) + result = daft_recordbatch.agg([col("input").alias("list").agg_list()], group_by=[col("groups")]).sort( + [col("groups")] + ) assert result.get_column("list").datatype() == DataType.list(dtype) - input_as_dtype = daft_table.get_column("input").to_pylist() + input_as_dtype = daft_recordbatch.get_column("input").to_pylist() expected_groups = [[input_as_dtype[i] for i in group] for group in expected_idx] assert result.to_pydict() == {"groups": [1, 2, None], "list": expected_groups} @@ -721,8 +723,10 @@ def test_grouped_pyobj_list_aggs() -> None: input = [None, object(), object(), object(), None, object()] expected_idx = [[1, 3], [4, 5], [0, 2]] - daft_table = MicroPartition.from_pydict({"groups": groups, "input": input}) - result = daft_table.agg([col("input").alias("list").agg_list()], group_by=[col("groups")]).sort([col("groups")]) + daft_recordbatch = MicroPartition.from_pydict({"groups": groups, "input": input}) + result = daft_recordbatch.agg([col("input").alias("list").agg_list()], group_by=[col("groups")]).sort( + [col("groups")] + ) expected_groups = [[input[i] for i in group] for group in expected_idx] assert result.to_pydict() == {"groups": [1, 2, None], "list": expected_groups} @@ -732,12 +736,14 @@ def test_grouped_list_list_aggs() -> None: input = [[1], [2, 3, 4], [5, None], None, [], [8, 9]] expected_idx = [[1, 3], [4, 5], [0, 2]] - daft_table = MicroPartition.from_pydict({"groups": groups, "input": input}) - daft_table = daft_table.eval_expression_list([col("groups"), col("input")]) - result = daft_table.agg([col("input").alias("list").agg_list()], group_by=[col("groups")]).sort([col("groups")]) + daft_recordbatch = MicroPartition.from_pydict({"groups": groups, "input": input}) + daft_recordbatch = daft_recordbatch.eval_expression_list([col("groups"), col("input")]) + result = daft_recordbatch.agg([col("input").alias("list").agg_list()], group_by=[col("groups")]).sort( + [col("groups")] + ) assert result.get_column("list").datatype() == DataType.list(DataType.list(DataType.int64())) - input_as_dtype = daft_table.get_column("input").to_pylist() + input_as_dtype = daft_recordbatch.get_column("input").to_pylist() expected_groups = [[input_as_dtype[i] for i in group] for group in expected_idx] assert result.to_pydict() == {"groups": [1, 2, None], "list": expected_groups} @@ -750,12 +756,14 @@ def test_grouped_fixed_size_list_list_aggs() -> None: ) expected_idx = [[1, 3], [4, 5], [0, 2]] - daft_table = MicroPartition.from_pydict({"groups": groups, "input": input}) - daft_table = daft_table.eval_expression_list([col("groups"), col("input")]) - result = daft_table.agg([col("input").alias("list").agg_list()], group_by=[col("groups")]).sort([col("groups")]) + daft_recordbatch = MicroPartition.from_pydict({"groups": groups, "input": input}) + daft_recordbatch = daft_recordbatch.eval_expression_list([col("groups"), col("input")]) + result = daft_recordbatch.agg([col("input").alias("list").agg_list()], group_by=[col("groups")]).sort( + [col("groups")] + ) assert result.get_column("list").datatype() == DataType.list(DataType.fixed_size_list(DataType.int64(), 2)) - input_as_dtype = daft_table.get_column("input").to_pylist() + input_as_dtype = daft_recordbatch.get_column("input").to_pylist() expected_groups = [[input_as_dtype[i] for i in group] for group in expected_idx] assert result.to_pydict() == {"groups": [1, 2, None], "list": expected_groups} @@ -766,27 +774,29 @@ def test_grouped_struct_list_aggs() -> None: input = [{"x": 1, "y": 2}, {"x": 3, "y": 4}, {"x": 5, "y": None}, None, {"x": 6, "y": 7}, {"x": 8, "y": 9}] expected_idx = [[1, 3], [4, 5], [0, 2]] - daft_table = MicroPartition.from_pydict({"groups": groups, "input": input}) - daft_table = daft_table.eval_expression_list([col("groups"), col("input")]) - result = daft_table.agg([col("input").alias("list").agg_list()], group_by=[col("groups")]).sort([col("groups")]) + daft_recordbatch = MicroPartition.from_pydict({"groups": groups, "input": input}) + daft_recordbatch = daft_recordbatch.eval_expression_list([col("groups"), col("input")]) + result = daft_recordbatch.agg([col("input").alias("list").agg_list()], group_by=[col("groups")]).sort( + [col("groups")] + ) assert result.get_column("list").datatype() == DataType.list( DataType.struct({"x": DataType.int64(), "y": DataType.int64()}) ) - input_as_dtype = daft_table.get_column("input").to_pylist() + input_as_dtype = daft_recordbatch.get_column("input").to_pylist() expected_groups = [[input_as_dtype[i] for i in group] for group in expected_idx] assert result.to_pydict() == {"groups": [1, 2, None], "list": expected_groups} def test_list_aggs_empty() -> None: - daft_table = MicroPartition.from_pydict({"col_A": [], "col_B": []}) - daft_table = daft_table.agg( + daft_recordbatch = MicroPartition.from_pydict({"col_A": [], "col_B": []}) + daft_recordbatch = daft_recordbatch.agg( [col("col_A").cast(DataType.int32()).alias("list").agg_list()], group_by=[col("col_B")], ) - assert daft_table.get_column("list").datatype() == DataType.list(DataType.int32()) - res = daft_table.to_pydict() + assert daft_recordbatch.get_column("list").datatype() == DataType.list(DataType.int32()) + res = daft_recordbatch.to_pydict() assert res == {"col_B": [], "list": []} @@ -805,13 +815,13 @@ def test_global_concat_aggs(dtype, with_null) -> None: if with_null: input += [None] - daft_table = MicroPartition.from_pydict({"input": input}).eval_expression_list( + daft_recordbatch = MicroPartition.from_pydict({"input": input}).eval_expression_list( [col("input").cast(DataType.list(dtype))] ) - concated = daft_table.agg([col("input").alias("concat").agg_concat()]) + concated = daft_recordbatch.agg([col("input").alias("concat").agg_concat()]) assert concated.get_column("concat").datatype() == DataType.list(dtype) - input_as_dtype = daft_table.get_column("input").to_pylist() + input_as_dtype = daft_recordbatch.get_column("input").to_pylist() # We should ignore Null Array elements when performing the concat agg expected = [[val[0] for val in input_as_dtype if val is not None]] assert concated.to_pydict() == {"concat": expected} @@ -843,15 +853,15 @@ def test_grouped_concat_aggs(dtype) -> None: input = [[x] for x in input] + [None] groups = [1, 2, 3, 4, 5, 6, 7] - daft_table = MicroPartition.from_pydict({"groups": groups, "input": input}).eval_expression_list( + daft_recordbatch = MicroPartition.from_pydict({"groups": groups, "input": input}).eval_expression_list( [col("groups"), col("input").cast(DataType.list(dtype))] ) - concat_grouped = daft_table.agg([col("input").alias("concat").agg_concat()], group_by=[col("groups") % 2]).sort( - [col("groups")] - ) + concat_grouped = daft_recordbatch.agg( + [col("input").alias("concat").agg_concat()], group_by=[col("groups") % 2] + ).sort([col("groups")]) assert concat_grouped.get_column("concat").datatype() == DataType.list(dtype) - input_as_dtype = daft_table.get_column("input").to_pylist() + input_as_dtype = daft_recordbatch.get_column("input").to_pylist() # We should ignore Null Array elements when performing the concat agg expected_groups = [ [input_as_dtype[i][0] for i in group if input_as_dtype[i] is not None] for group in [[1, 3, 5], [0, 2, 4, 6]] @@ -884,14 +894,14 @@ def test_grouped_concat_aggs_pyobj() -> None: def test_concat_aggs_empty() -> None: - daft_table = MicroPartition.from_pydict({"col_A": [], "col_B": []}) - daft_table = daft_table.agg( + daft_recordbatch = MicroPartition.from_pydict({"col_A": [], "col_B": []}) + daft_recordbatch = daft_recordbatch.agg( [col("col_A").cast(DataType.list(DataType.int32())).alias("concat").agg_concat()], group_by=[col("col_B")], ) - assert daft_table.get_column("concat").datatype() == DataType.list(DataType.int32()) - res = daft_table.to_pydict() + assert daft_recordbatch.get_column("concat").datatype() == DataType.list(DataType.int32()) + res = daft_recordbatch.to_pydict() assert res == {"col_B": [], "concat": []} diff --git a/tests/table/test_take.py b/tests/recordbatch/test_take.py similarity index 78% rename from tests/table/test_take.py rename to tests/recordbatch/test_take.py index d38f1e7359..dd7bd1d817 100644 --- a/tests/table/test_take.py +++ b/tests/recordbatch/test_take.py @@ -7,9 +7,9 @@ from daft import col from daft.logical.schema import Schema +from daft.recordbatch import MicroPartition from daft.series import Series -from daft.table import MicroPartition -from tests.table import daft_int_types, daft_numeric_types +from tests.recordbatch import daft_int_types, daft_numeric_types @pytest.mark.parametrize( @@ -73,15 +73,15 @@ def test_micropartitions_take(mp: MicroPartition) -> None: @pytest.mark.parametrize("data_dtype, idx_dtype", itertools.product(daft_numeric_types, daft_int_types)) def test_table_take_numeric(data_dtype, idx_dtype) -> None: pa_table = pa.Table.from_pydict({"a": [1, 2, 3, 4], "b": [5, 6, 7, 8]}) - daft_table = MicroPartition.from_arrow(pa_table) - daft_table = daft_table.eval_expression_list([col("a").cast(data_dtype), col("b")]) + daft_recordbatch = MicroPartition.from_arrow(pa_table) + daft_recordbatch = daft_recordbatch.eval_expression_list([col("a").cast(data_dtype), col("b")]) - assert len(daft_table) == 4 - assert daft_table.column_names() == ["a", "b"] + assert len(daft_recordbatch) == 4 + assert daft_recordbatch.column_names() == ["a", "b"] indices = Series.from_pylist([0, 1]).cast(idx_dtype) - taken = daft_table.take(indices) + taken = daft_recordbatch.take(indices) assert len(taken) == 2 assert taken.column_names() == ["a", "b"] @@ -89,7 +89,7 @@ def test_table_take_numeric(data_dtype, idx_dtype) -> None: indices = Series.from_pylist([3, 2]).cast(idx_dtype) - taken = daft_table.take(indices) + taken = daft_recordbatch.take(indices) assert len(taken) == 2 assert taken.column_names() == ["a", "b"] @@ -97,7 +97,7 @@ def test_table_take_numeric(data_dtype, idx_dtype) -> None: indices = Series.from_pylist([3, 2, 2, 2, 3]).cast(idx_dtype) - taken = daft_table.take(indices) + taken = daft_recordbatch.take(indices) assert len(taken) == 5 assert taken.column_names() == ["a", "b"] @@ -107,13 +107,13 @@ def test_table_take_numeric(data_dtype, idx_dtype) -> None: @pytest.mark.parametrize("idx_dtype", daft_int_types) def test_table_take_str(idx_dtype) -> None: pa_table = pa.Table.from_pydict({"a": ["1", "2", "3", "4"], "b": ["5", "6", "7", "8"]}) - daft_table = MicroPartition.from_arrow(pa_table) - assert len(daft_table) == 4 - assert daft_table.column_names() == ["a", "b"] + daft_recordbatch = MicroPartition.from_arrow(pa_table) + assert len(daft_recordbatch) == 4 + assert daft_recordbatch.column_names() == ["a", "b"] indices = Series.from_pylist([0, 1]).cast(idx_dtype) - taken = daft_table.take(indices) + taken = daft_recordbatch.take(indices) assert len(taken) == 2 assert taken.column_names() == ["a", "b"] @@ -121,7 +121,7 @@ def test_table_take_str(idx_dtype) -> None: indices = Series.from_pylist([3, 2]).cast(idx_dtype) - taken = daft_table.take(indices) + taken = daft_recordbatch.take(indices) assert len(taken) == 2 assert taken.column_names() == ["a", "b"] @@ -129,7 +129,7 @@ def test_table_take_str(idx_dtype) -> None: indices = Series.from_pylist([3, 2, 2, 2, 3]).cast(idx_dtype) - taken = daft_table.take(indices) + taken = daft_recordbatch.take(indices) assert len(taken) == 5 assert taken.column_names() == ["a", "b"] @@ -139,13 +139,13 @@ def test_table_take_str(idx_dtype) -> None: @pytest.mark.parametrize("idx_dtype", daft_int_types) def test_table_take_bool(idx_dtype) -> None: pa_table = pa.Table.from_pydict({"a": [False, True, False, True], "b": [True, False, True, False]}) - daft_table = MicroPartition.from_arrow(pa_table) - assert len(daft_table) == 4 - assert daft_table.column_names() == ["a", "b"] + daft_recordbatch = MicroPartition.from_arrow(pa_table) + assert len(daft_recordbatch) == 4 + assert daft_recordbatch.column_names() == ["a", "b"] indices = Series.from_pylist([0, 1]).cast(idx_dtype) - taken = daft_table.take(indices) + taken = daft_recordbatch.take(indices) assert len(taken) == 2 assert taken.column_names() == ["a", "b"] @@ -153,7 +153,7 @@ def test_table_take_bool(idx_dtype) -> None: indices = Series.from_pylist([3, 2]).cast(idx_dtype) - taken = daft_table.take(indices) + taken = daft_recordbatch.take(indices) assert len(taken) == 2 assert taken.column_names() == ["a", "b"] @@ -161,7 +161,7 @@ def test_table_take_bool(idx_dtype) -> None: indices = Series.from_pylist([3, 2, 2, 2, 3]).cast(idx_dtype) - taken = daft_table.take(indices) + taken = daft_recordbatch.take(indices) assert len(taken) == 5 assert taken.column_names() == ["a", "b"] @@ -171,13 +171,13 @@ def test_table_take_bool(idx_dtype) -> None: @pytest.mark.parametrize("idx_dtype", daft_int_types) def test_table_take_null(idx_dtype) -> None: pa_table = pa.Table.from_pydict({"a": [None, None, None, None], "b": [None, None, None, None]}) - daft_table = MicroPartition.from_arrow(pa_table) - assert len(daft_table) == 4 - assert daft_table.column_names() == ["a", "b"] + daft_recordbatch = MicroPartition.from_arrow(pa_table) + assert len(daft_recordbatch) == 4 + assert daft_recordbatch.column_names() == ["a", "b"] indices = Series.from_pylist([0, 1]).cast(idx_dtype) - taken = daft_table.take(indices) + taken = daft_recordbatch.take(indices) assert len(taken) == 2 assert taken.column_names() == ["a", "b"] @@ -186,13 +186,13 @@ def test_table_take_null(idx_dtype) -> None: def test_table_take_pyobject() -> None: objects = [object(), None, object(), object()] - daft_table = MicroPartition.from_pydict({"objs": objects}) - assert len(daft_table) == 4 - assert daft_table.column_names() == ["objs"] + daft_recordbatch = MicroPartition.from_pydict({"objs": objects}) + assert len(daft_recordbatch) == 4 + assert daft_recordbatch.column_names() == ["objs"] indices = Series.from_pylist([0, 1]) - taken = daft_table.take(indices) + taken = daft_recordbatch.take(indices) assert len(taken) == 2 assert taken.column_names() == ["objs"] @@ -200,7 +200,7 @@ def test_table_take_pyobject() -> None: indices = Series.from_pylist([3, 2]) - taken = daft_table.take(indices) + taken = daft_recordbatch.take(indices) assert len(taken) == 2 assert taken.column_names() == ["objs"] @@ -208,7 +208,7 @@ def test_table_take_pyobject() -> None: indices = Series.from_pylist([3, 2, 2, 2, 3]) - taken = daft_table.take(indices) + taken = daft_recordbatch.take(indices) assert len(taken) == 5 assert taken.column_names() == ["objs"] @@ -223,13 +223,13 @@ def test_table_take_fixed_size_list(idx_dtype) -> None: "b": pa.array([[4, 5], [6, None], None, [None, None]], type=pa.list_(pa.int64(), 2)), } ) - daft_table = MicroPartition.from_arrow(pa_table) - assert len(daft_table) == 4 - assert daft_table.column_names() == ["a", "b"] + daft_recordbatch = MicroPartition.from_arrow(pa_table) + assert len(daft_recordbatch) == 4 + assert daft_recordbatch.column_names() == ["a", "b"] indices = Series.from_pylist([0, 1]).cast(idx_dtype) - taken = daft_table.take(indices) + taken = daft_recordbatch.take(indices) assert len(taken) == 2 assert taken.column_names() == ["a", "b"] @@ -237,7 +237,7 @@ def test_table_take_fixed_size_list(idx_dtype) -> None: indices = Series.from_pylist([3, 2]).cast(idx_dtype) - taken = daft_table.take(indices) + taken = daft_recordbatch.take(indices) assert len(taken) == 2 assert taken.column_names() == ["a", "b"] @@ -245,7 +245,7 @@ def test_table_take_fixed_size_list(idx_dtype) -> None: indices = Series.from_pylist([3, 2, 2, 2, 3]).cast(idx_dtype) - taken = daft_table.take(indices) + taken = daft_recordbatch.take(indices) assert len(taken) == 5 assert taken.column_names() == ["a", "b"] diff --git a/tests/table/test_tokenize.py b/tests/recordbatch/test_tokenize.py similarity index 100% rename from tests/table/test_tokenize.py rename to tests/recordbatch/test_tokenize.py diff --git a/tests/table/utf8/__init__.py b/tests/recordbatch/utf8/__init__.py similarity index 100% rename from tests/table/utf8/__init__.py rename to tests/recordbatch/utf8/__init__.py diff --git a/tests/table/utf8/test_capitalize.py b/tests/recordbatch/utf8/test_capitalize.py similarity index 88% rename from tests/table/utf8/test_capitalize.py rename to tests/recordbatch/utf8/test_capitalize.py index b3aef00482..0280842d49 100644 --- a/tests/table/utf8/test_capitalize.py +++ b/tests/recordbatch/utf8/test_capitalize.py @@ -1,7 +1,7 @@ from __future__ import annotations from daft.expressions import col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition def test_utf8_capitalize(): diff --git a/tests/table/utf8/test_compares.py b/tests/recordbatch/utf8/test_compares.py similarity index 96% rename from tests/table/utf8/test_compares.py rename to tests/recordbatch/utf8/test_compares.py index 9f0ca99145..624f969d03 100644 --- a/tests/table/utf8/test_compares.py +++ b/tests/recordbatch/utf8/test_compares.py @@ -3,7 +3,7 @@ import pytest from daft.expressions import col, lit -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition ENDSWITH_DATA = ["x_foo", "y_foo", "z_bar"] STARTSWITH_DATA = ["foo_x", "foo_y", "bar_z"] diff --git a/tests/table/utf8/test_count_matches.py b/tests/recordbatch/utf8/test_count_matches.py similarity index 97% rename from tests/table/utf8/test_count_matches.py rename to tests/recordbatch/utf8/test_count_matches.py index 9c2ac4183e..1463579e4f 100644 --- a/tests/table/utf8/test_count_matches.py +++ b/tests/recordbatch/utf8/test_count_matches.py @@ -5,7 +5,7 @@ import pytest from daft.expressions import col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition def py_count_matches(s, pat, whole_words, case_sensitive): diff --git a/tests/table/utf8/test_extract.py b/tests/recordbatch/utf8/test_extract.py similarity index 94% rename from tests/table/utf8/test_extract.py rename to tests/recordbatch/utf8/test_extract.py index 673010d91e..2771ddc88c 100644 --- a/tests/table/utf8/test_extract.py +++ b/tests/recordbatch/utf8/test_extract.py @@ -3,7 +3,7 @@ import pytest from daft.expressions import col, lit -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition REGEX = r"^\d" diff --git a/tests/table/utf8/test_extract_all.py b/tests/recordbatch/utf8/test_extract_all.py similarity index 95% rename from tests/table/utf8/test_extract_all.py rename to tests/recordbatch/utf8/test_extract_all.py index d9375ae1df..3d69ab755b 100644 --- a/tests/table/utf8/test_extract_all.py +++ b/tests/recordbatch/utf8/test_extract_all.py @@ -3,7 +3,7 @@ import pytest from daft.expressions import col, lit -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition REGEX = r"\d+" diff --git a/tests/table/utf8/test_find.py b/tests/recordbatch/utf8/test_find.py similarity index 93% rename from tests/table/utf8/test_find.py rename to tests/recordbatch/utf8/test_find.py index fccbc382aa..a84b0c44df 100644 --- a/tests/table/utf8/test_find.py +++ b/tests/recordbatch/utf8/test_find.py @@ -3,7 +3,7 @@ import pytest from daft.expressions import col, lit -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition @pytest.mark.parametrize( diff --git a/tests/table/utf8/test_ilike.py b/tests/recordbatch/utf8/test_ilike.py similarity index 88% rename from tests/table/utf8/test_ilike.py rename to tests/recordbatch/utf8/test_ilike.py index 5ade93ec30..413a68563b 100644 --- a/tests/table/utf8/test_ilike.py +++ b/tests/recordbatch/utf8/test_ilike.py @@ -1,7 +1,7 @@ from __future__ import annotations from daft.expressions import col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition def test_utf8_ilike(): diff --git a/tests/table/utf8/test_left.py b/tests/recordbatch/utf8/test_left.py similarity index 87% rename from tests/table/utf8/test_left.py rename to tests/recordbatch/utf8/test_left.py index 764ec24b77..c3c2e8e41a 100644 --- a/tests/table/utf8/test_left.py +++ b/tests/recordbatch/utf8/test_left.py @@ -1,7 +1,7 @@ from __future__ import annotations from daft.expressions import col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition def test_utf8_left(): diff --git a/tests/table/utf8/test_length.py b/tests/recordbatch/utf8/test_length.py similarity index 88% rename from tests/table/utf8/test_length.py rename to tests/recordbatch/utf8/test_length.py index c96815ed64..9996a5e21b 100644 --- a/tests/table/utf8/test_length.py +++ b/tests/recordbatch/utf8/test_length.py @@ -1,7 +1,7 @@ from __future__ import annotations from daft.expressions import col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition def test_utf8_length(): diff --git a/tests/table/utf8/test_like.py b/tests/recordbatch/utf8/test_like.py similarity index 88% rename from tests/table/utf8/test_like.py rename to tests/recordbatch/utf8/test_like.py index ee8fb98160..ef054a86e4 100644 --- a/tests/table/utf8/test_like.py +++ b/tests/recordbatch/utf8/test_like.py @@ -1,7 +1,7 @@ from __future__ import annotations from daft.expressions import col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition def test_utf8_like(): diff --git a/tests/table/utf8/test_lower.py b/tests/recordbatch/utf8/test_lower.py similarity index 87% rename from tests/table/utf8/test_lower.py rename to tests/recordbatch/utf8/test_lower.py index 6ed7d6b050..87d955b647 100644 --- a/tests/table/utf8/test_lower.py +++ b/tests/recordbatch/utf8/test_lower.py @@ -1,7 +1,7 @@ from __future__ import annotations from daft.expressions import col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition def test_utf8_lower(): diff --git a/tests/table/utf8/test_lpad.py b/tests/recordbatch/utf8/test_lpad.py similarity index 94% rename from tests/table/utf8/test_lpad.py rename to tests/recordbatch/utf8/test_lpad.py index e08cb51efe..5b75da6f18 100644 --- a/tests/table/utf8/test_lpad.py +++ b/tests/recordbatch/utf8/test_lpad.py @@ -3,7 +3,7 @@ import pytest from daft.expressions import col, lit -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition @pytest.mark.parametrize( diff --git a/tests/table/utf8/test_lstrip.py b/tests/recordbatch/utf8/test_lstrip.py similarity index 88% rename from tests/table/utf8/test_lstrip.py rename to tests/recordbatch/utf8/test_lstrip.py index e515131fa1..1abbe3b03e 100644 --- a/tests/table/utf8/test_lstrip.py +++ b/tests/recordbatch/utf8/test_lstrip.py @@ -1,7 +1,7 @@ from __future__ import annotations from daft.expressions import col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition def test_utf8_lstrip(): diff --git a/tests/table/utf8/test_match.py b/tests/recordbatch/utf8/test_match.py similarity index 94% rename from tests/table/utf8/test_match.py rename to tests/recordbatch/utf8/test_match.py index 95b38c71c7..b63aa6f4b2 100644 --- a/tests/table/utf8/test_match.py +++ b/tests/recordbatch/utf8/test_match.py @@ -3,7 +3,7 @@ import pytest from daft.expressions import col, lit -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition REGEX = r"^\d+$" # match only digits diff --git a/tests/table/utf8/test_normalize.py b/tests/recordbatch/utf8/test_normalize.py similarity index 98% rename from tests/table/utf8/test_normalize.py rename to tests/recordbatch/utf8/test_normalize.py index ad204cee47..d244e7b9c9 100644 --- a/tests/table/utf8/test_normalize.py +++ b/tests/recordbatch/utf8/test_normalize.py @@ -7,7 +7,7 @@ import pytest from daft.expressions import col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition # source: RedPajama diff --git a/tests/table/utf8/test_repeat.py b/tests/recordbatch/utf8/test_repeat.py similarity index 88% rename from tests/table/utf8/test_repeat.py rename to tests/recordbatch/utf8/test_repeat.py index 52aad1a4c5..97a3bf1ccb 100644 --- a/tests/table/utf8/test_repeat.py +++ b/tests/recordbatch/utf8/test_repeat.py @@ -1,7 +1,7 @@ from __future__ import annotations from daft.expressions import col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition def test_utf8_repeat(): diff --git a/tests/table/utf8/test_replace.py b/tests/recordbatch/utf8/test_replace.py similarity index 96% rename from tests/table/utf8/test_replace.py rename to tests/recordbatch/utf8/test_replace.py index fa5330604e..fe7c8be84d 100644 --- a/tests/table/utf8/test_replace.py +++ b/tests/recordbatch/utf8/test_replace.py @@ -4,7 +4,7 @@ import daft from daft.expressions import col, lit -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition @pytest.mark.parametrize( diff --git a/tests/table/utf8/test_reverse.py b/tests/recordbatch/utf8/test_reverse.py similarity index 87% rename from tests/table/utf8/test_reverse.py rename to tests/recordbatch/utf8/test_reverse.py index f8c9547146..726088b20f 100644 --- a/tests/table/utf8/test_reverse.py +++ b/tests/recordbatch/utf8/test_reverse.py @@ -1,7 +1,7 @@ from __future__ import annotations from daft.expressions import col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition def test_utf8_reverse(): diff --git a/tests/table/utf8/test_right.py b/tests/recordbatch/utf8/test_right.py similarity index 87% rename from tests/table/utf8/test_right.py rename to tests/recordbatch/utf8/test_right.py index cd211e6da4..db6c6cd711 100644 --- a/tests/table/utf8/test_right.py +++ b/tests/recordbatch/utf8/test_right.py @@ -1,7 +1,7 @@ from __future__ import annotations from daft.expressions import col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition def test_utf8_right(): diff --git a/tests/table/utf8/test_rpad.py b/tests/recordbatch/utf8/test_rpad.py similarity index 94% rename from tests/table/utf8/test_rpad.py rename to tests/recordbatch/utf8/test_rpad.py index c1851b853d..ac27af85ef 100644 --- a/tests/table/utf8/test_rpad.py +++ b/tests/recordbatch/utf8/test_rpad.py @@ -3,7 +3,7 @@ import pytest from daft.expressions import col, lit -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition @pytest.mark.parametrize( diff --git a/tests/table/utf8/test_rstrip.py b/tests/recordbatch/utf8/test_rstrip.py similarity index 88% rename from tests/table/utf8/test_rstrip.py rename to tests/recordbatch/utf8/test_rstrip.py index db7b21218a..4c4598843f 100644 --- a/tests/table/utf8/test_rstrip.py +++ b/tests/recordbatch/utf8/test_rstrip.py @@ -1,7 +1,7 @@ from __future__ import annotations from daft.expressions import col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition def test_utf8_rstrip(): diff --git a/tests/table/utf8/test_split.py b/tests/recordbatch/utf8/test_split.py similarity index 97% rename from tests/table/utf8/test_split.py rename to tests/recordbatch/utf8/test_split.py index fc70ea17ca..60658ea838 100644 --- a/tests/table/utf8/test_split.py +++ b/tests/recordbatch/utf8/test_split.py @@ -3,7 +3,7 @@ import pytest from daft.expressions import col, lit -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition @pytest.mark.parametrize( diff --git a/tests/table/utf8/test_substr.py b/tests/recordbatch/utf8/test_substr.py similarity index 88% rename from tests/table/utf8/test_substr.py rename to tests/recordbatch/utf8/test_substr.py index 9989e9bd86..ff78c24f5f 100644 --- a/tests/table/utf8/test_substr.py +++ b/tests/recordbatch/utf8/test_substr.py @@ -1,7 +1,7 @@ from __future__ import annotations from daft.expressions import col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition def test_utf8_substr(): diff --git a/tests/table/utf8/test_to_date.py b/tests/recordbatch/utf8/test_to_date.py similarity index 92% rename from tests/table/utf8/test_to_date.py rename to tests/recordbatch/utf8/test_to_date.py index 8a62519437..685313c71c 100644 --- a/tests/table/utf8/test_to_date.py +++ b/tests/recordbatch/utf8/test_to_date.py @@ -3,7 +3,7 @@ import datetime from daft.expressions import col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition def test_utf8_to_date(): diff --git a/tests/table/utf8/test_to_datetime.py b/tests/recordbatch/utf8/test_to_datetime.py similarity index 91% rename from tests/table/utf8/test_to_datetime.py rename to tests/recordbatch/utf8/test_to_datetime.py index 794856dc63..89212bb1a9 100644 --- a/tests/table/utf8/test_to_datetime.py +++ b/tests/recordbatch/utf8/test_to_datetime.py @@ -1,7 +1,7 @@ import datetime from daft.expressions import col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition def test_utf8_to_datetime(): diff --git a/tests/table/utf8/test_upper.py b/tests/recordbatch/utf8/test_upper.py similarity index 88% rename from tests/table/utf8/test_upper.py rename to tests/recordbatch/utf8/test_upper.py index 812afdf7d3..6f085d52ef 100644 --- a/tests/table/utf8/test_upper.py +++ b/tests/recordbatch/utf8/test_upper.py @@ -1,7 +1,7 @@ from __future__ import annotations from daft.expressions import col -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition def test_utf8_upper(): diff --git a/tests/test_schema.py b/tests/test_schema.py index 174970a5db..4fa7b57697 100644 --- a/tests/test_schema.py +++ b/tests/test_schema.py @@ -8,7 +8,7 @@ from daft.datatype import DataType from daft.expressions import ExpressionsProjection, col from daft.logical.schema import Schema -from daft.table import MicroPartition +from daft.recordbatch import MicroPartition DATA = { "int": ([1, 2, None], DataType.int64()), diff --git a/tests/test_size_estimations.py b/tests/test_size_estimations.py index 38f9ca6454..19cb24296f 100644 --- a/tests/test_size_estimations.py +++ b/tests/test_size_estimations.py @@ -3,7 +3,7 @@ import pytest from daft.daft import testing as native_testing_utils -from daft.table.micropartition import MicroPartition +from daft.recordbatch.micropartition import MicroPartition pytest.skip(allow_module_level=True, reason="Skipping because these tests don't currently pass") diff --git a/tests/utils.py b/tests/utils.py index 836758c0bd..5cac7fd6d7 100644 --- a/tests/utils.py +++ b/tests/utils.py @@ -4,7 +4,7 @@ import pyarrow as pa -from daft.table import Table +from daft.recordbatch import RecordBatch ANSI_ESCAPE = re.compile(r"\x1B(?:[@-Z\\-_]|\[[0-?]*[ -/]*[@-~])") @@ -18,5 +18,5 @@ def sort_arrow_table(tbl: pa.Table, *sort_by: str): def assert_pyarrow_tables_equal(from_daft: pa.Table, expected: pa.Table): # Do a round-trip with Daft in order to cast pyarrow dtypes to Daft's supported Arrow dtypes (e.g. string -> large_string). - expected = Table.from_arrow(expected).to_arrow() + expected = RecordBatch.from_arrow(expected).to_arrow() assert from_daft == expected, f"from_daft = {from_daft}\n\nexpected = {expected}"