From e857c43c817d29ee60df205a37200a7da4a2f514 Mon Sep 17 00:00:00 2001 From: Jay Chia Date: Thu, 30 Nov 2023 12:53:38 -0800 Subject: [PATCH 1/3] [FEAT] Use DAFT_MICROPARTITIONS=1 by default --- daft/io/common.py | 2 +- daft/table/__init__.py | 2 +- tests/dataframe/test_creation.py | 2 +- tests/integration/io/parquet/test_reads_s3_minio.py | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/daft/io/common.py b/daft/io/common.py index 6ab6989f94..45ceb00ee6 100644 --- a/daft/io/common.py +++ b/daft/io/common.py @@ -48,7 +48,7 @@ def _get_tabular_files_scan( ### FEATURE_FLAG: $DAFT_MICROPARTITIONS # # This environment variable will make Daft use the new "v2 scans" and MicroPartitions when building Daft logical plans - if os.getenv("DAFT_MICROPARTITIONS", "0") == "1": + if os.getenv("DAFT_MICROPARTITIONS", "1") == "1": scan_op: ScanOperatorHandle if isinstance(path, list): scan_op = ScanOperatorHandle.glob_scan( diff --git a/daft/table/__init__.py b/daft/table/__init__.py index 9669259fdd..65009d3666 100644 --- a/daft/table/__init__.py +++ b/daft/table/__init__.py @@ -10,7 +10,7 @@ # Use $DAFT_MICROPARTITIONS envvar as a feature flag to turn on MicroPartitions LegacyTable = Table -if os.getenv("DAFT_MICROPARTITIONS", "0") == "1": +if os.getenv("DAFT_MICROPARTITIONS", "1") == "1": Table = _MicroPartition # type: ignore diff --git a/tests/dataframe/test_creation.py b/tests/dataframe/test_creation.py index 65005a7457..b9411cb7b7 100644 --- a/tests/dataframe/test_creation.py +++ b/tests/dataframe/test_creation.py @@ -401,7 +401,7 @@ def test_create_dataframe_csv_generate_headers(valid_data: list[dict[str, float] cnames = ( [f"column_{i}" for i in range(1, 6)] - if use_native_downloader or os.environ.get("DAFT_MICROPARTITIONS", "0") == "1" + if use_native_downloader or os.environ.get("DAFT_MICROPARTITIONS", "1") == "1" else [f"f{i}" for i in range(5)] ) df = daft.read_csv(fname, has_headers=False, use_native_downloader=use_native_downloader) diff --git a/tests/integration/io/parquet/test_reads_s3_minio.py b/tests/integration/io/parquet/test_reads_s3_minio.py index 59b4e00979..44ae5e3633 100644 --- a/tests/integration/io/parquet/test_reads_s3_minio.py +++ b/tests/integration/io/parquet/test_reads_s3_minio.py @@ -38,7 +38,7 @@ def test_minio_parquet_read_no_files(minio_io_config): msg = ( "Glob path had no matches:" - if os.getenv("DAFT_MICROPARTITIONS") == "1" + if os.getenv("DAFT_MICROPARTITIONS", "1") == "1" else "No files found at s3://data-engineering-prod/foo/\\*\\*.parquet" ) with pytest.raises(FileNotFoundError, match=msg): From 7c27f78c716d9ec733d2fee04f413aa7d0f36708 Mon Sep 17 00:00:00 2001 From: Jay Chia Date: Thu, 30 Nov 2023 13:18:26 -0800 Subject: [PATCH 2/3] Rename `Table -> MicroPartition` --- benchmarking/parquet/conftest.py | 2 +- daft/dataframe/dataframe.py | 14 +-- daft/dataframe/preview.py | 4 +- daft/execution/execution_step.py | 104 +++++++++--------- daft/execution/rust_physical_plan_shim.py | 12 +- daft/filesystem.py | 4 +- daft/io/file_path.py | 4 +- daft/logical/map_partition_ops.py | 6 +- daft/runners/partitioning.py | 8 +- daft/runners/pyrunner.py | 36 +++--- daft/runners/ray_runner.py | 56 ++++++---- daft/runners/runner.py | 8 +- daft/table/__init__.py | 15 ++- daft/table/micropartition.py | 2 +- daft/table/schema_inference.py | 4 +- daft/table/table_io.py | 44 ++++---- tests/benchmarks/test_filter.py | 3 +- tests/benchmarks/test_if_else.py | 3 +- tests/benchmarks/test_take.py | 3 +- tests/conftest.py | 3 +- tests/expressions/test_expressions.py | 4 +- .../test_expressions_projection.py | 8 +- tests/expressions/test_udf.py | 16 +-- tests/expressions/typing/conftest.py | 6 +- .../io/parquet/test_reads_local_fixtures.py | 6 +- .../io/parquet/test_reads_public_data.py | 36 +++--- tests/table/image/test_crop.py | 12 +- tests/table/list/test_list_join.py | 10 +- tests/table/list/test_list_lengths.py | 4 +- tests/table/table_io/test_csv.py | 18 +-- tests/table/table_io/test_json.py | 8 +- tests/table/table_io/test_parquet.py | 22 ++-- tests/table/table_io/test_read_time_cast.py | 12 +- tests/table/test_blackbox_kernels.py | 6 +- tests/table/test_broadcasts.py | 6 +- tests/table/test_concat.py | 56 +++++----- tests/table/test_eval.py | 20 ++-- tests/table/test_explodes.py | 18 +-- tests/table/test_filter.py | 26 +++-- tests/table/test_from_py.py | 54 ++++----- tests/table/test_head.py | 4 +- tests/table/test_joins.py | 50 ++++----- tests/table/test_partitioning.py | 70 ++++++------ tests/table/test_size_bytes.py | 24 ++-- tests/table/test_sorting.py | 32 +++--- tests/table/test_table_aggs.py | 90 +++++++-------- tests/table/test_take.py | 30 ++--- tests/table/utf8/test_compares.py | 4 +- tests/table/utf8/test_length.py | 4 +- tests/table/utf8/test_split.py | 4 +- tests/test_schema.py | 14 +-- 51 files changed, 525 insertions(+), 484 deletions(-) diff --git a/benchmarking/parquet/conftest.py b/benchmarking/parquet/conftest.py index e43bcc4487..bea8b2a99d 100644 --- a/benchmarking/parquet/conftest.py +++ b/benchmarking/parquet/conftest.py @@ -41,7 +41,7 @@ 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.Table.read_parquet(path, columns=columns) + tbl = daft.table.MicroPartition.read_parquet(path, columns=columns) return tbl.to_arrow() diff --git a/daft/dataframe/dataframe.py b/daft/dataframe/dataframe.py index 444ceb6081..91ddbe17b8 100644 --- a/daft/dataframe/dataframe.py +++ b/daft/dataframe/dataframe.py @@ -33,7 +33,7 @@ from daft.logical.builder import LogicalPlanBuilder from daft.runners.partitioning import PartitionCacheEntry, PartitionSet from daft.runners.pyrunner import LocalPartitionSet -from daft.table import Table +from daft.table import MicroPartition from daft.viz import DataFrameDisplay if TYPE_CHECKING: @@ -184,7 +184,7 @@ def __iter__(self) -> Iterator[Dict[str, Any]]: yield row @DataframePublicAPI - def iter_partitions(self) -> Iterator[Union[Table, "RayObjectRef"]]: + def iter_partitions(self) -> Iterator[Union[MicroPartition, "RayObjectRef"]]: """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) @@ -236,7 +236,7 @@ def _from_pydict(cls, data: Dict[str, InputListType]) -> "DataFrame": f"Expected all columns to be of the same length, but received columns with lengths: {column_lengths}" ) - data_vpartition = Table.from_pydict(data) + data_vpartition = MicroPartition.from_pydict(data) return cls._from_tables(data_vpartition) @classmethod @@ -244,7 +244,7 @@ def _from_arrow(cls, data: Union["pa.Table", List["pa.Table"]]) -> "DataFrame": """Creates a DataFrame from a pyarrow Table.""" if not isinstance(data, list): data = [data] - data_vpartitions = [Table.from_arrow(table) for table in data] + data_vpartitions = [MicroPartition.from_arrow(table) for table in data] return cls._from_tables(*data_vpartitions) @classmethod @@ -252,11 +252,11 @@ def _from_pandas(cls, data: Union["pd.DataFrame", List["pd.DataFrame"]]) -> "Dat """Creates a Daft DataFrame from a pandas DataFrame.""" if not isinstance(data, list): data = [data] - data_vpartitions = [Table.from_pandas(df) for df in data] + data_vpartitions = [MicroPartition.from_pandas(df) for df in data] return cls._from_tables(*data_vpartitions) @classmethod - def _from_tables(cls, *parts: Table) -> "DataFrame": + def _from_tables(cls, *parts: MicroPartition) -> "DataFrame": """Creates a Daft DataFrame from a single Table. Args: @@ -1036,7 +1036,7 @@ def _construct_show_display(self, n: int) -> "DataFrameDisplay": if seen >= n: break - preview_partition = Table.concat(tables) + preview_partition = MicroPartition.concat(tables) if len(preview_partition) > n: preview_partition = preview_partition.slice(0, n) elif len(preview_partition) < n: diff --git a/daft/dataframe/preview.py b/daft/dataframe/preview.py index 20962f7ccb..ec8129c066 100644 --- a/daft/dataframe/preview.py +++ b/daft/dataframe/preview.py @@ -2,12 +2,12 @@ from dataclasses import dataclass -from daft.table import Table +from daft.table import MicroPartition @dataclass(frozen=True) class DataFramePreview: """A class containing all the metadata/data required to preview a dataframe.""" - preview_partition: Table | None + preview_partition: MicroPartition | None dataframe_num_rows: int | None diff --git a/daft/execution/execution_step.py b/daft/execution/execution_step.py index a41ec19da4..59c613f360 100644 --- a/daft/execution/execution_step.py +++ b/daft/execution/execution_step.py @@ -33,7 +33,7 @@ TableParseCSVOptions, TableReadOptions, ) -from daft.table import Table, table_io +from daft.table import MicroPartition, table_io ID_GEN = itertools.count() @@ -196,7 +196,7 @@ def partition_metadata(self) -> PartitionMetadata: """ return self.result().metadata() - def vpartition(self) -> Table: + def vpartition(self) -> MicroPartition: """Get the raw vPartition of the result.""" return self.result().vpartition() @@ -241,7 +241,7 @@ def partition_metadatas(self) -> list[PartitionMetadata]: assert self._results is not None return [result.metadata() for result in self._results] - def vpartition(self, index: int) -> Table: + def vpartition(self, index: int) -> MicroPartition: """Get the raw vPartition of the result.""" assert self._results is not None return self._results[index].vpartition() @@ -262,7 +262,7 @@ class Instruction(Protocol): To accomodate these, instructions are typed as list[Table] -> list[Table]. """ - def run(self, inputs: list[Table]) -> list[Table]: + def run(self, inputs: list[MicroPartition]) -> list[MicroPartition]: """Run the Instruction over the input partitions. Note: Dispatching a descriptively named helper here will aid profiling. @@ -295,10 +295,10 @@ class ReadFile(SingleOutputInstruction): columns_to_read: list[str] | None file_format_config: FileFormatConfig - def run(self, inputs: list[Table]) -> list[Table]: + def run(self, inputs: list[MicroPartition]) -> list[MicroPartition]: return self._read_file(inputs) - def _read_file(self, inputs: list[Table]) -> list[Table]: + def _read_file(self, inputs: list[MicroPartition]) -> list[MicroPartition]: assert len(inputs) == 1 [filepaths_partition] = inputs partition = self._handle_tabular_files_scan( @@ -323,8 +323,8 @@ def run_partial_metadata(self, input_metadatas: list[PartialPartitionMetadata]) def _handle_tabular_files_scan( self, - filepaths_partition: Table, - ) -> Table: + filepaths_partition: MicroPartition, + ) -> MicroPartition: data = filepaths_partition.to_pydict() filepaths = data["path"] @@ -341,7 +341,7 @@ def _handle_tabular_files_scan( format_config = self.file_format_config.config if file_format == FileFormat.Csv: assert isinstance(format_config, CsvSourceConfig) - table = Table.concat( + table = MicroPartition.concat( [ table_io.read_csv( file=fp, @@ -364,7 +364,7 @@ def _handle_tabular_files_scan( ) elif file_format == FileFormat.Json: assert isinstance(format_config, JsonSourceConfig) - table = Table.concat( + table = MicroPartition.concat( [ table_io.read_json( file=fp, @@ -377,7 +377,7 @@ def _handle_tabular_files_scan( ) elif file_format == FileFormat.Parquet: assert isinstance(format_config, ParquetSourceConfig) - table = Table.concat( + table = MicroPartition.concat( [ table_io.read_parquet( file=fp, @@ -411,10 +411,10 @@ class WriteFile(SingleOutputInstruction): partition_cols: ExpressionsProjection | None io_config: IOConfig | None - def run(self, inputs: list[Table]) -> list[Table]: + def run(self, inputs: list[MicroPartition]) -> list[MicroPartition]: return self._write_file(inputs) - def _write_file(self, inputs: list[Table]) -> list[Table]: + def _write_file(self, inputs: list[MicroPartition]) -> list[MicroPartition]: [input] = inputs partition = self._handle_file_write( input=input, @@ -430,7 +430,7 @@ def run_partial_metadata(self, input_metadatas: list[PartialPartitionMetadata]) ) ] - def _handle_file_write(self, input: Table) -> Table: + def _handle_file_write(self, input: MicroPartition) -> MicroPartition: if self.file_format == FileFormat.Parquet: file_names = table_io.write_parquet( input, @@ -453,7 +453,7 @@ def _handle_file_write(self, input: Table) -> Table: ) assert len(self.schema) == 1 - return Table.from_pydict( + return MicroPartition.from_pydict( { self.schema.column_names()[0]: file_names, } @@ -464,10 +464,10 @@ def _handle_file_write(self, input: Table) -> Table: class Filter(SingleOutputInstruction): predicate: ExpressionsProjection - def run(self, inputs: list[Table]) -> list[Table]: + def run(self, inputs: list[MicroPartition]) -> list[MicroPartition]: return self._filter(inputs) - def _filter(self, inputs: list[Table]) -> list[Table]: + def _filter(self, inputs: list[MicroPartition]) -> list[MicroPartition]: [input] = inputs return [input.filter(self.predicate)] @@ -485,10 +485,10 @@ def run_partial_metadata(self, input_metadatas: list[PartialPartitionMetadata]) class Project(SingleOutputInstruction): projection: ExpressionsProjection - def run(self, inputs: list[Table]) -> list[Table]: + def run(self, inputs: list[MicroPartition]) -> list[MicroPartition]: return self._project(inputs) - def _project(self, inputs: list[Table]) -> list[Table]: + def _project(self, inputs: list[MicroPartition]) -> list[MicroPartition]: [input] = inputs return [input.eval_expression_list(self.projection)] @@ -506,12 +506,12 @@ def run_partial_metadata(self, input_metadatas: list[PartialPartitionMetadata]) class LocalCount(SingleOutputInstruction): schema: Schema - def run(self, inputs: list[Table]) -> list[Table]: + def run(self, inputs: list[MicroPartition]) -> list[MicroPartition]: return self._count(inputs) - def _count(self, inputs: list[Table]) -> list[Table]: + def _count(self, inputs: list[MicroPartition]) -> list[MicroPartition]: [input] = inputs - partition = Table.from_pydict({"count": [len(input)]}) + partition = MicroPartition.from_pydict({"count": [len(input)]}) assert partition.schema() == self.schema return [partition] @@ -528,10 +528,10 @@ def run_partial_metadata(self, input_metadatas: list[PartialPartitionMetadata]) class LocalLimit(SingleOutputInstruction): limit: int - def run(self, inputs: list[Table]) -> list[Table]: + def run(self, inputs: list[MicroPartition]) -> list[MicroPartition]: return self._limit(inputs) - def _limit(self, inputs: list[Table]) -> list[Table]: + def _limit(self, inputs: list[MicroPartition]) -> list[MicroPartition]: [input] = inputs return [input.head(self.limit)] @@ -554,10 +554,10 @@ class GlobalLimit(LocalLimit): class MapPartition(SingleOutputInstruction): map_op: MapPartitionOp - def run(self, inputs: list[Table]) -> list[Table]: + def run(self, inputs: list[MicroPartition]) -> list[MicroPartition]: return self._map_partition(inputs) - def _map_partition(self, inputs: list[Table]) -> list[Table]: + def _map_partition(self, inputs: list[MicroPartition]) -> list[MicroPartition]: [input] = inputs return [self.map_op.run(input)] @@ -576,10 +576,10 @@ class Sample(SingleOutputInstruction): sort_by: ExpressionsProjection num_samples: int = 20 - def run(self, inputs: list[Table]) -> list[Table]: + def run(self, inputs: list[MicroPartition]) -> list[MicroPartition]: return self._sample(inputs) - def _sample(self, inputs: list[Table]) -> list[Table]: + def _sample(self, inputs: list[MicroPartition]) -> list[MicroPartition]: [input] = inputs result = ( input.sample(self.num_samples) @@ -603,10 +603,10 @@ class Aggregate(SingleOutputInstruction): to_agg: list[Expression] group_by: ExpressionsProjection | None - def run(self, inputs: list[Table]) -> list[Table]: + def run(self, inputs: list[MicroPartition]) -> list[MicroPartition]: return self._aggregate(inputs) - def _aggregate(self, inputs: list[Table]) -> list[Table]: + def _aggregate(self, inputs: list[MicroPartition]) -> list[MicroPartition]: [input] = inputs return [input.agg(self.to_agg, self.group_by)] @@ -626,10 +626,10 @@ class Join(SingleOutputInstruction): right_on: ExpressionsProjection how: JoinType - def run(self, inputs: list[Table]) -> list[Table]: + def run(self, inputs: list[MicroPartition]) -> list[MicroPartition]: return self._join(inputs) - def _join(self, inputs: list[Table]) -> list[Table]: + def _join(self, inputs: list[MicroPartition]) -> list[MicroPartition]: [left, right] = inputs result = left.join( right, @@ -655,11 +655,11 @@ class ReduceInstruction(SingleOutputInstruction): @dataclass(frozen=True) class ReduceMerge(ReduceInstruction): - def run(self, inputs: list[Table]) -> list[Table]: + def run(self, inputs: list[MicroPartition]) -> list[MicroPartition]: return self._reduce_merge(inputs) - def _reduce_merge(self, inputs: list[Table]) -> list[Table]: - return [Table.concat(inputs)] + def _reduce_merge(self, inputs: list[MicroPartition]) -> list[MicroPartition]: + return [MicroPartition.concat(inputs)] def run_partial_metadata(self, input_metadatas: list[PartialPartitionMetadata]) -> list[PartialPartitionMetadata]: input_rows = [_.num_rows for _ in input_metadatas] @@ -677,11 +677,11 @@ class ReduceMergeAndSort(ReduceInstruction): sort_by: ExpressionsProjection descending: list[bool] - def run(self, inputs: list[Table]) -> list[Table]: + def run(self, inputs: list[MicroPartition]) -> list[MicroPartition]: return self._reduce_merge_and_sort(inputs) - def _reduce_merge_and_sort(self, inputs: list[Table]) -> list[Table]: - partition = Table.concat(inputs).sort(self.sort_by, descending=self.descending) + def _reduce_merge_and_sort(self, inputs: list[MicroPartition]) -> list[MicroPartition]: + partition = MicroPartition.concat(inputs).sort(self.sort_by, descending=self.descending) return [partition] def run_partial_metadata(self, input_metadatas: list[PartialPartitionMetadata]) -> list[PartialPartitionMetadata]: @@ -701,11 +701,11 @@ class ReduceToQuantiles(ReduceInstruction): sort_by: ExpressionsProjection descending: list[bool] - def run(self, inputs: list[Table]) -> list[Table]: + def run(self, inputs: list[MicroPartition]) -> list[MicroPartition]: return self._reduce_to_quantiles(inputs) - def _reduce_to_quantiles(self, inputs: list[Table]) -> list[Table]: - merged = Table.concat(inputs) + def _reduce_to_quantiles(self, inputs: list[MicroPartition]) -> list[MicroPartition]: + merged = MicroPartition.concat(inputs) # Skip evaluation of expressions by converting to Column Expression, since evaluation was done in Sample merged_sorted = merged.sort(self.sort_by.to_column_expressions(), descending=self.descending) @@ -744,10 +744,10 @@ def num_outputs(self) -> int: class FanoutRandom(FanoutInstruction): seed: int - def run(self, inputs: list[Table]) -> list[Table]: + def run(self, inputs: list[MicroPartition]) -> list[MicroPartition]: return self._fanout_random(inputs) - def _fanout_random(self, inputs: list[Table]) -> list[Table]: + def _fanout_random(self, inputs: list[MicroPartition]) -> list[MicroPartition]: [input] = inputs return input.partition_by_random(num_partitions=self._num_outputs, seed=self.seed) @@ -756,10 +756,10 @@ def _fanout_random(self, inputs: list[Table]) -> list[Table]: class FanoutHash(FanoutInstruction): partition_by: ExpressionsProjection - def run(self, inputs: list[Table]) -> list[Table]: + def run(self, inputs: list[MicroPartition]) -> list[MicroPartition]: return self._fanout_hash(inputs) - def _fanout_hash(self, inputs: list[Table]) -> list[Table]: + def _fanout_hash(self, inputs: list[MicroPartition]) -> list[MicroPartition]: [input] = inputs return input.partition_by_hash(self.partition_by, num_partitions=self._num_outputs) @@ -769,23 +769,23 @@ class FanoutRange(FanoutInstruction, Generic[PartitionT]): sort_by: ExpressionsProjection descending: list[bool] - def run(self, inputs: list[Table]) -> list[Table]: + def run(self, inputs: list[MicroPartition]) -> list[MicroPartition]: return self._fanout_range(inputs) - def _fanout_range(self, inputs: list[Table]) -> list[Table]: + def _fanout_range(self, inputs: list[MicroPartition]) -> list[MicroPartition]: [boundaries, input] = inputs if self._num_outputs == 1: return [input] - boundaries = boundaries.to_table() - partitioned_tables = input.partition_by_range(self.sort_by, boundaries, self.descending) + table_boundaries = boundaries.to_table() + partitioned_tables = input.partition_by_range(self.sort_by, table_boundaries, self.descending) # Pad the partitioned_tables with empty tables if fewer than self._num_outputs were returned # This can happen when all values are null or empty, which leads to an empty `boundaries` input assert len(partitioned_tables) >= 1, "Should have at least one returned table" schema = partitioned_tables[0].schema() partitioned_tables = partitioned_tables + [ - Table.empty(schema=schema) for _ in range(self._num_outputs - len(partitioned_tables)) + MicroPartition.empty(schema=schema) for _ in range(self._num_outputs - len(partitioned_tables)) ] return partitioned_tables @@ -795,10 +795,10 @@ def _fanout_range(self, inputs: list[Table]) -> list[Table]: class FanoutSlices(FanoutInstruction): slices: list[tuple[int, int]] # start inclusive, end exclusive - def run(self, inputs: list[Table]) -> list[Table]: + def run(self, inputs: list[MicroPartition]) -> list[MicroPartition]: return self._multislice(inputs) - def _multislice(self, inputs: list[Table]) -> list[Table]: + def _multislice(self, inputs: list[MicroPartition]) -> list[MicroPartition]: [input] = inputs results = [] diff --git a/daft/execution/rust_physical_plan_shim.py b/daft/execution/rust_physical_plan_shim.py index aab94c339e..4e3644523d 100644 --- a/daft/execution/rust_physical_plan_shim.py +++ b/daft/execution/rust_physical_plan_shim.py @@ -20,7 +20,7 @@ from daft.logical.map_partition_ops import MapPartitionOp from daft.logical.schema import Schema from daft.runners.partitioning import PartialPartitionMetadata, PartitionT -from daft.table import Table +from daft.table import MicroPartition def scan_with_tasks( @@ -46,12 +46,12 @@ def scan_with_tasks( class ScanWithTask(execution_step.SingleOutputInstruction): scan_task: ScanTask - def run(self, inputs: list[Table]) -> list[Table]: + def run(self, inputs: list[MicroPartition]) -> list[MicroPartition]: return self._scan(inputs) - def _scan(self, inputs: list[Table]) -> list[Table]: + def _scan(self, inputs: list[MicroPartition]) -> list[MicroPartition]: assert len(inputs) == 0 - return [Table._from_scan_task(self.scan_task)] + return [MicroPartition._from_scan_task(self.scan_task)] def run_partial_metadata(self, input_metadatas: list[PartialPartitionMetadata]) -> list[PartialPartitionMetadata]: assert len(input_metadatas) == 0 @@ -74,7 +74,7 @@ def tabular_scan( is_ray_runner: bool, ) -> physical_plan.InProgressPhysicalPlan[PartitionT]: # TODO(Clark): Fix this Ray runner hack. - part = Table._from_pytable(file_info_table) + part = MicroPartition._from_pytable(file_info_table) if is_ray_runner: import ray @@ -114,7 +114,7 @@ def __init__(self, explode_columns: ExpressionsProjection) -> None: def get_output_schema(self) -> Schema: raise NotImplementedError("Output schema shouldn't be needed at execution time") - def run(self, input_partition: Table) -> Table: + def run(self, input_partition: MicroPartition) -> MicroPartition: return input_partition.explode(self.explode_columns) diff --git a/daft/filesystem.py b/daft/filesystem.py index bd6cb15910..da246a4a8c 100644 --- a/daft/filesystem.py +++ b/daft/filesystem.py @@ -19,7 +19,7 @@ from pyarrow.fs import _resolve_filesystem_and_path as pafs_resolve_filesystem_and_path from daft.daft import FileFormat, FileInfos, IOConfig, io_glob -from daft.table import Table +from daft.table import MicroPartition logger = logging.getLogger(__name__) @@ -305,7 +305,7 @@ def glob_path_with_stats( # Set number of rows if available. if file_format is not None and file_format == FileFormat.Parquet: - parquet_statistics = Table.read_parquet_statistics( + parquet_statistics = MicroPartition.read_parquet_statistics( list(filepaths_to_infos.keys()), io_config=io_config, ).to_pydict() diff --git a/daft/io/file_path.py b/daft/io/file_path.py index 2cfd3388c3..079fa9ef5a 100644 --- a/daft/io/file_path.py +++ b/daft/io/file_path.py @@ -9,7 +9,7 @@ from daft.dataframe import DataFrame from daft.logical.builder import LogicalPlanBuilder from daft.runners.pyrunner import LocalPartitionSet -from daft.table import Table +from daft.table import MicroPartition @PublicAPI @@ -45,7 +45,7 @@ def from_glob_path(path: str, io_config: Optional[IOConfig] = None) -> DataFrame context = get_context() runner_io = context.runner().runner_io() file_infos = runner_io.glob_paths_details([path], io_config=io_config) - file_infos_table = Table._from_pytable(file_infos.to_table()) + file_infos_table = MicroPartition._from_pytable(file_infos.to_table()) partition = LocalPartitionSet({0: file_infos_table}) cache_entry = context.runner().put_partition_set_into_cache(partition) builder = LogicalPlanBuilder.from_in_memory_scan( diff --git a/daft/logical/map_partition_ops.py b/daft/logical/map_partition_ops.py index ef93fb7e77..46819a27bc 100644 --- a/daft/logical/map_partition_ops.py +++ b/daft/logical/map_partition_ops.py @@ -4,7 +4,7 @@ from daft.expressions import ExpressionsProjection from daft.logical.schema import Schema -from daft.table import Table +from daft.table import MicroPartition class MapPartitionOp: @@ -13,7 +13,7 @@ def get_output_schema(self) -> Schema: """Returns the output schema after running this MapPartitionOp""" @abstractmethod - def run(self, input_partition: Table) -> Table: + def run(self, input_partition: MicroPartition) -> MicroPartition: """Runs this MapPartitionOp on the supplied vPartition""" @@ -39,5 +39,5 @@ def __init__(self, input_schema: Schema, explode_columns: ExpressionsProjection) def get_output_schema(self) -> Schema: return self.output_schema - def run(self, input_partition: Table) -> Table: + def run(self, input_partition: MicroPartition) -> MicroPartition: return input_partition.explode(self.explode_columns) diff --git a/daft/runners/partitioning.py b/daft/runners/partitioning.py index e11c3b1f72..ec6c04085c 100644 --- a/daft/runners/partitioning.py +++ b/daft/runners/partitioning.py @@ -11,7 +11,7 @@ from daft.datatype import TimeUnit from daft.logical.schema import Schema -from daft.table import Table +from daft.table import MicroPartition if sys.version_info < (3, 8): pass @@ -82,7 +82,7 @@ class PartitionMetadata(PartialPartitionMetadata): size_bytes: int | None @classmethod - def from_table(cls, table: Table) -> PartitionMetadata: + def from_table(cls, table: MicroPartition) -> PartitionMetadata: return PartitionMetadata( num_rows=len(table), size_bytes=table.size_bytes(), @@ -104,7 +104,7 @@ def partition(self) -> PartitionT: ... @abstractmethod - def vpartition(self) -> Table: + def vpartition(self) -> MicroPartition: """Get the vPartition of this result.""" ... @@ -127,7 +127,7 @@ def _noop(self, _: PartitionT) -> None: class PartitionSet(Generic[PartitionT]): - def _get_merged_vpartition(self) -> Table: + def _get_merged_vpartition(self) -> MicroPartition: raise NotImplementedError() def to_pydict(self) -> dict[str, list[Any]]: diff --git a/daft/runners/pyrunner.py b/daft/runners/pyrunner.py index 7c87ea82c8..7af9b7d9e0 100644 --- a/daft/runners/pyrunner.py +++ b/daft/runners/pyrunner.py @@ -32,28 +32,28 @@ from daft.runners.profiler import profiler from daft.runners.progress_bar import ProgressBar from daft.runners.runner import Runner -from daft.table import Table +from daft.table import MicroPartition logger = logging.getLogger(__name__) @dataclass -class LocalPartitionSet(PartitionSet[Table]): - _partitions: dict[PartID, Table] +class LocalPartitionSet(PartitionSet[MicroPartition]): + _partitions: dict[PartID, MicroPartition] - def items(self) -> list[tuple[PartID, Table]]: + def items(self) -> list[tuple[PartID, MicroPartition]]: return sorted(self._partitions.items()) - def _get_merged_vpartition(self) -> Table: + def _get_merged_vpartition(self) -> MicroPartition: ids_and_partitions = self.items() assert ids_and_partitions[0][0] == 0 assert ids_and_partitions[-1][0] + 1 == len(ids_and_partitions) - return Table.concat([part for id, part in ids_and_partitions]) + return MicroPartition.concat([part for id, part in ids_and_partitions]) - def get_partition(self, idx: PartID) -> Table: + def get_partition(self, idx: PartID) -> MicroPartition: return self._partitions[idx] - def set_partition(self, idx: PartID, part: MaterializedResult[Table]) -> None: + def set_partition(self, idx: PartID, part: MaterializedResult[MicroPartition]) -> None: self._partitions[idx] = part.partition() def delete_partition(self, idx: PartID) -> None: @@ -103,7 +103,7 @@ def get_schema_from_first_filepath( return runner_io.sample_schema(file_infos[0].file_path, file_format_config, storage_config) -class PyRunner(Runner[Table]): +class PyRunner(Runner[MicroPartition]): def __init__(self, use_thread_pool: bool | None) -> None: super().__init__() self._use_thread_pool: bool = use_thread_pool if use_thread_pool is not None else True @@ -147,12 +147,14 @@ def run_iter( results_gen = self._physical_plan_to_partitions(tasks) yield from results_gen - def run_iter_tables(self, builder: LogicalPlanBuilder, results_buffer_size: int | None = None) -> Iterator[Table]: + def run_iter_tables( + self, builder: LogicalPlanBuilder, results_buffer_size: int | None = None + ) -> Iterator[MicroPartition]: for result in self.run_iter(builder, results_buffer_size=results_buffer_size): yield result.partition() def _physical_plan_to_partitions( - self, plan: physical_plan.MaterializedPhysicalPlan[Table] + self, plan: physical_plan.MaterializedPhysicalPlan[MicroPartition] ) -> Iterator[PyMaterializedResult]: inflight_tasks: dict[str, PartitionTask] = dict() inflight_tasks_resources: dict[str, ResourceRequest] = dict() @@ -269,7 +271,7 @@ def _can_admit_task(self, resource_request: ResourceRequest, inflight_resources: return all((cpus_okay, gpus_okay, memory_okay)) @staticmethod - def build_partitions(instruction_stack: list[Instruction], *inputs: Table) -> list[Table]: + def build_partitions(instruction_stack: list[Instruction], *inputs: MicroPartition) -> list[MicroPartition]: partitions = list(inputs) for instruction in instruction_stack: partitions = instruction.run(partitions) @@ -278,13 +280,13 @@ def build_partitions(instruction_stack: list[Instruction], *inputs: Table) -> li @dataclass(frozen=True) -class PyMaterializedResult(MaterializedResult[Table]): - _partition: Table +class PyMaterializedResult(MaterializedResult[MicroPartition]): + _partition: MicroPartition - def partition(self) -> Table: + def partition(self) -> MicroPartition: return self._partition - def vpartition(self) -> Table: + def vpartition(self) -> MicroPartition: return self._partition def metadata(self) -> PartitionMetadata: @@ -293,5 +295,5 @@ def metadata(self) -> PartitionMetadata: def cancel(self) -> None: return None - def _noop(self, _: Table) -> None: + def _noop(self, _: MicroPartition) -> None: return None diff --git a/daft/runners/ray_runner.py b/daft/runners/ray_runner.py index 6401af3dc7..4e457caf38 100644 --- a/daft/runners/ray_runner.py +++ b/daft/runners/ray_runner.py @@ -53,7 +53,7 @@ from daft.runners.profiler import profiler from daft.runners.pyrunner import LocalPartitionSet from daft.runners.runner import Runner -from daft.table import Table +from daft.table import MicroPartition if TYPE_CHECKING: import dask @@ -77,7 +77,7 @@ def _glob_path_into_file_infos( paths: list[str], file_format_config: FileFormatConfig | None, io_config: IOConfig | None, -) -> Table: +) -> MicroPartition: file_infos = FileInfos() file_format = file_format_config.file_format() if file_format_config is not None else None for path in paths: @@ -86,11 +86,11 @@ def _glob_path_into_file_infos( raise FileNotFoundError(f"No files found at {path}") file_infos.extend(path_file_infos) - return Table._from_pytable(file_infos.to_table()) + return MicroPartition._from_pytable(file_infos.to_table()) @ray.remote -def _make_ray_block_from_vpartition(partition: Table) -> RayDatasetBlock: +def _make_ray_block_from_vpartition(partition: MicroPartition) -> RayDatasetBlock: try: return partition.to_arrow(cast_tensors_to_ray_tensor_dtype=True) except pa.ArrowInvalid: @@ -98,13 +98,17 @@ def _make_ray_block_from_vpartition(partition: Table) -> RayDatasetBlock: @ray.remote -def _make_daft_partition_from_ray_dataset_blocks(ray_dataset_block: pa.Table, daft_schema: Schema) -> Table: - return Table.from_arrow(ray_dataset_block) +def _make_daft_partition_from_ray_dataset_blocks( + ray_dataset_block: pa.MicroPartition, daft_schema: Schema +) -> MicroPartition: + return MicroPartition.from_arrow(ray_dataset_block) @ray.remote(num_returns=2) -def _make_daft_partition_from_dask_dataframe_partitions(dask_df_partition: pd.DataFrame) -> tuple[Table, pa.Schema]: - vpart = Table.from_pandas(dask_df_partition) +def _make_daft_partition_from_dask_dataframe_partitions( + dask_df_partition: pd.DataFrame, +) -> tuple[MicroPartition, pa.Schema]: + vpart = MicroPartition.from_pandas(dask_df_partition) return vpart, vpart.schema() @@ -126,7 +130,7 @@ def sample_schema_from_filepath( file_format_config: FileFormatConfig, storage_config: StorageConfig, ) -> Schema: - """Ray remote function to run schema sampling on top of a Table containing a single filepath""" + """Ray remote function to run schema sampling on top of a MicroPartition containing a single filepath""" # Currently just samples the Schema from the first file return runner_io.sample_schema(first_file_path, file_format_config, storage_config) @@ -138,12 +142,12 @@ class RayPartitionSet(PartitionSet[ray.ObjectRef]): def items(self) -> list[tuple[PartID, ray.ObjectRef]]: return [(pid, result.partition()) for pid, result in sorted(self._results.items())] - def _get_merged_vpartition(self) -> Table: + def _get_merged_vpartition(self) -> MicroPartition: ids_and_partitions = self.items() assert ids_and_partitions[0][0] == 0 assert ids_and_partitions[-1][0] + 1 == len(ids_and_partitions) all_partitions = ray.get([part for id, part in ids_and_partitions]) - return Table.concat(all_partitions) + return MicroPartition.concat(all_partitions) def to_ray_dataset(self) -> RayDataset: if not _RAY_FROM_ARROW_REFS_AVAILABLE: @@ -167,7 +171,7 @@ def to_dask_dataframe( dask.config.set(scheduler=ray_dask_get) @dask.delayed - def _make_dask_dataframe_partition_from_vpartition(partition: Table) -> pd.DataFrame: + def _make_dask_dataframe_partition_from_vpartition(partition: MicroPartition) -> pd.DataFrame: return partition.to_pandas() ddf_parts = [ @@ -306,7 +310,9 @@ def _get_ray_task_options(resource_request: ResourceRequest) -> dict[str, Any]: return options -def build_partitions(instruction_stack: list[Instruction], *inputs: Table) -> list[list[PartitionMetadata] | Table]: +def build_partitions( + instruction_stack: list[Instruction], *inputs: MicroPartition +) -> list[list[PartitionMetadata] | MicroPartition]: partitions = list(inputs) for instruction in instruction_stack: partitions = instruction.run(partitions) @@ -321,32 +327,38 @@ def build_partitions(instruction_stack: list[Instruction], *inputs: Table) -> li @ray.remote def single_partition_pipeline( - instruction_stack: list[Instruction], *inputs: Table -) -> list[list[PartitionMetadata] | Table]: + instruction_stack: list[Instruction], *inputs: MicroPartition +) -> list[list[PartitionMetadata] | MicroPartition]: return build_partitions(instruction_stack, *inputs) @ray.remote -def fanout_pipeline(instruction_stack: list[Instruction], *inputs: Table) -> list[list[PartitionMetadata] | Table]: +def fanout_pipeline( + instruction_stack: list[Instruction], *inputs: MicroPartition +) -> list[list[PartitionMetadata] | MicroPartition]: return build_partitions(instruction_stack, *inputs) @ray.remote(scheduling_strategy="SPREAD") -def reduce_pipeline(instruction_stack: list[Instruction], inputs: list) -> list[list[PartitionMetadata] | Table]: +def reduce_pipeline( + instruction_stack: list[Instruction], inputs: list +) -> list[list[PartitionMetadata] | MicroPartition]: import ray return build_partitions(instruction_stack, *ray.get(inputs)) @ray.remote(scheduling_strategy="SPREAD") -def reduce_and_fanout(instruction_stack: list[Instruction], inputs: list) -> list[list[PartitionMetadata] | Table]: +def reduce_and_fanout( + instruction_stack: list[Instruction], inputs: list +) -> list[list[PartitionMetadata] | MicroPartition]: import ray return build_partitions(instruction_stack, *ray.get(inputs)) @ray.remote -def get_meta(partition: Table) -> PartitionMetadata: +def get_meta(partition: MicroPartition) -> PartitionMetadata: return PartitionMetadata.from_table(partition) @@ -712,7 +724,9 @@ def run_iter( else: self.scheduler.stop_plan(result_uuid) - def run_iter_tables(self, builder: LogicalPlanBuilder, results_buffer_size: int | None = None) -> Iterator[Table]: + def run_iter_tables( + self, builder: LogicalPlanBuilder, results_buffer_size: int | None = None + ) -> Iterator[MicroPartition]: for result in self.run_iter(builder, results_buffer_size=results_buffer_size): yield ray.get(result.partition()) @@ -747,7 +761,7 @@ class RayMaterializedResult(MaterializedResult[ray.ObjectRef]): def partition(self) -> ray.ObjectRef: return self._partition - def vpartition(self) -> Table: + def vpartition(self) -> MicroPartition: return ray.get(self._partition) def metadata(self) -> PartitionMetadata: diff --git a/daft/runners/runner.py b/daft/runners/runner.py index e969ce5ebc..e844bdb97c 100644 --- a/daft/runners/runner.py +++ b/daft/runners/runner.py @@ -12,7 +12,7 @@ PartitionT, ) from daft.runners.runner_io import RunnerIO -from daft.table import Table +from daft.table import MicroPartition class Runner(Generic[PartitionT]): @@ -47,8 +47,10 @@ def run_iter( ... @abstractmethod - def run_iter_tables(self, builder: LogicalPlanBuilder, results_buffer_size: int | None = None) -> Iterator[Table]: - """Similar to run_iter(), but always dereference and yield Table objects. + def run_iter_tables( + self, builder: LogicalPlanBuilder, results_buffer_size: int | None = None + ) -> Iterator[MicroPartition]: + """Similar to run_iter(), but always dereference and yield MicroPartition objects. Args: builder: the builder for the LogicalPlan that is to be executed diff --git a/daft/table/__init__.py b/daft/table/__init__.py index 65009d3666..73a557aba9 100644 --- a/daft/table/__init__.py +++ b/daft/table/__init__.py @@ -2,16 +2,19 @@ import os -from .table import Table, read_parquet_into_pyarrow, read_parquet_into_pyarrow_bulk +from .table import Table as _Table +from .table import read_parquet_into_pyarrow, read_parquet_into_pyarrow_bulk # Need to import after `.table` due to circular dep issue otherwise from .micropartition import MicroPartition as _MicroPartition # isort:skip -# Use $DAFT_MICROPARTITIONS envvar as a feature flag to turn on MicroPartitions -LegacyTable = Table -if os.getenv("DAFT_MICROPARTITIONS", "1") == "1": - Table = _MicroPartition # type: ignore +LegacyTable = _Table +MicroPartition = _MicroPartition +# Use $DAFT_MICROPARTITIONS envvar as a feature flag to turn off MicroPartitions +if os.getenv("DAFT_MICROPARTITIONS", "1") != "1": + MicroPartition = LegacyTable # type: ignore -__all__ = ["Table", "LegacyTable", "read_parquet_into_pyarrow", "read_parquet_into_pyarrow_bulk"] + +__all__ = ["MicroPartition", "LegacyTable", "read_parquet_into_pyarrow", "read_parquet_into_pyarrow_bulk"] diff --git a/daft/table/micropartition.py b/daft/table/micropartition.py index c8bb398cd0..f931fda2bd 100644 --- a/daft/table/micropartition.py +++ b/daft/table/micropartition.py @@ -19,7 +19,7 @@ from daft.expressions import Expression, ExpressionsProjection from daft.logical.schema import Schema from daft.series import Series -from daft.table import Table +from daft.table.table import Table if TYPE_CHECKING: import pandas as pd diff --git a/daft/table/schema_inference.py b/daft/table/schema_inference.py index 9b8d43094a..dce4ca4f18 100644 --- a/daft/table/schema_inference.py +++ b/daft/table/schema_inference.py @@ -16,7 +16,7 @@ from daft.filesystem import _resolve_paths_and_filesystem from daft.logical.schema import Schema from daft.runners.partitioning import TableParseCSVOptions -from daft.table import Table +from daft.table import MicroPartition from daft.table.table_io import FileInput, _open_stream @@ -96,7 +96,7 @@ def from_json( with _open_stream(file, io_config) as f: table = pajson.read_json(f) - return Table.from_arrow(table).schema() + return MicroPartition.from_arrow(table).schema() def from_parquet( diff --git a/daft/table/table_io.py b/daft/table/table_io.py index 5ce668012b..82f48b733c 100644 --- a/daft/table/table_io.py +++ b/daft/table/table_io.py @@ -29,7 +29,7 @@ TableParseParquetOptions, TableReadOptions, ) -from daft.table import Table +from daft.table import MicroPartition FileInput = Union[pathlib.Path, str, IO[bytes]] @@ -50,14 +50,14 @@ def _open_stream( yield file -def _cast_table_to_schema(table: Table, read_options: TableReadOptions, schema: Schema) -> pa.Table: - """Performs a cast of a Daft Table to the requested Schema/Data. This is required because: +def _cast_table_to_schema(table: MicroPartition, read_options: TableReadOptions, schema: Schema) -> pa.Table: + """Performs a cast of a Daft MicroPartition to the requested Schema/Data. This is required because: 1. Data read from the datasource may have types that do not match the inferred global schema 2. Data read from the datasource may have columns that are out-of-order with the inferred schema 3. We may need only a subset of columns, or differently-ordered columns, in `read_options` - This helper function takes care of all that, ensuring that the resulting Table has all column types matching + This helper function takes care of all that, ensuring that the resulting MicroPartition has all column types matching their corresponding dtype in `schema`, and column ordering/inclusion matches `read_options.column_names` (if provided). """ pruned_schema = schema @@ -75,8 +75,8 @@ def read_json( schema: Schema, storage_config: StorageConfig | None = None, read_options: TableReadOptions = TableReadOptions(), -) -> Table: - """Reads a Table from a JSON file +) -> MicroPartition: + """Reads a MicroPartition from a JSON file Args: file (str | IO): either a file-like object or a string file path (potentially prefixed with a protocol such as "s3://") @@ -85,7 +85,7 @@ def read_json( read_options (TableReadOptions, optional): Options for reading the file Returns: - Table: Parsed Table from JSON + MicroPartition: Parsed MicroPartition from JSON """ io_config = None if storage_config is not None: @@ -103,7 +103,7 @@ def read_json( if read_options.num_rows is not None: table = table[: read_options.num_rows] - return _cast_table_to_schema(Table.from_arrow(table), read_options=read_options, schema=schema) + return _cast_table_to_schema(MicroPartition.from_arrow(table), read_options=read_options, schema=schema) def read_parquet( @@ -112,8 +112,8 @@ def read_parquet( storage_config: StorageConfig | None = None, read_options: TableReadOptions = TableReadOptions(), parquet_options: TableParseParquetOptions = TableParseParquetOptions(), -) -> Table: - """Reads a Table from a Parquet file +) -> MicroPartition: + """Reads a MicroPartition from a Parquet file Args: file (str | IO): either a file-like object or a string file path (potentially prefixed with a protocol such as "s3://") @@ -122,7 +122,7 @@ def read_parquet( read_options (TableReadOptions, optional): Options for reading the file Returns: - Table: Parsed Table from Parquet + MicroPartition: Parsed MicroPartition from Parquet """ io_config = None if storage_config is not None: @@ -131,7 +131,7 @@ def read_parquet( assert isinstance( file, (str, pathlib.Path) ), "Native downloader only works on string inputs to read_parquet" - tbl = Table.read_parquet( + tbl = MicroPartition.read_parquet( str(file), columns=read_options.column_names, num_rows=read_options.num_rows, @@ -178,7 +178,7 @@ def read_parquet( coerce_int96_timestamp_unit=str(parquet_options.coerce_int96_timestamp_unit), ) - return _cast_table_to_schema(Table.from_arrow(table), read_options=read_options, schema=schema) + return _cast_table_to_schema(MicroPartition.from_arrow(table), read_options=read_options, schema=schema) class PACSVStreamHelper: @@ -205,8 +205,8 @@ def read_csv( storage_config: StorageConfig | None = None, csv_options: TableParseCSVOptions = TableParseCSVOptions(), read_options: TableReadOptions = TableReadOptions(), -) -> Table: - """Reads a Table from a CSV file +) -> MicroPartition: + """Reads a MicroPartition from a CSV file Args: file (str | IO): either a file-like object or a string file path (potentially prefixed with a protocol such as "s3://") @@ -217,7 +217,7 @@ def read_csv( read_options (TableReadOptions, optional): Options for reading the file Returns: - Table: Parsed Table from CSV + MicroPartition: Parsed MicroPartition from CSV """ io_config = None if storage_config is not None: @@ -242,7 +242,7 @@ def read_csv( comment=csv_options.comment, ) csv_read_options = CsvReadOptions(buffer_size=csv_options.buffer_size, chunk_size=csv_options.chunk_size) - tbl = Table.read_csv( + tbl = MicroPartition.read_csv( str(file), convert_options=csv_convert_options, parse_options=csv_parse_options, @@ -308,18 +308,18 @@ def read_csv( if pa_schema is None: pa_schema = pa.schema([]) - daft_table = Table.from_arrow_record_batches(pa_batches, pa_schema) + daft_table = MicroPartition.from_arrow_record_batches(pa_batches, pa_schema) assert len(daft_table) <= read_options.num_rows else: pa_table = pacsv_stream.read_all() - daft_table = Table.from_arrow(pa_table) + daft_table = MicroPartition.from_arrow(pa_table) return _cast_table_to_schema(daft_table, read_options=read_options, schema=schema) def write_csv( - table: Table, + table: MicroPartition, path: str | pathlib.Path, compression: str | None = None, partition_cols: ExpressionsProjection | None = None, @@ -336,7 +336,7 @@ def write_csv( def write_parquet( - table: Table, + table: MicroPartition, path: str | pathlib.Path, compression: str | None = None, partition_cols: ExpressionsProjection | None = None, @@ -353,7 +353,7 @@ def write_parquet( def _to_file( - table: Table, + table: MicroPartition, file_format: str, path: str | pathlib.Path, partition_cols: ExpressionsProjection | None = None, diff --git a/tests/benchmarks/test_filter.py b/tests/benchmarks/test_filter.py index 6db6cf39b8..f676adea35 100644 --- a/tests/benchmarks/test_filter.py +++ b/tests/benchmarks/test_filter.py @@ -4,6 +4,7 @@ import daft from daft import DataFrame +from daft.table import MicroPartition NUM_ROWS = 1_000_000 @@ -123,7 +124,7 @@ def generate_list_int64_keep_none() -> tuple[dict, daft.Expression, list]: def test_filter(test_data_generator, benchmark) -> None: """If_else between NUM_ROWS values""" data, expected = test_data_generator() - table = daft.table.Table.from_pydict(data) + table = MicroPartition.from_pydict(data) def bench_filter() -> DataFrame: return table.filter([daft.col("mask")]) diff --git a/tests/benchmarks/test_if_else.py b/tests/benchmarks/test_if_else.py index 4e7bcc4e89..2efcfaea4a 100644 --- a/tests/benchmarks/test_if_else.py +++ b/tests/benchmarks/test_if_else.py @@ -6,6 +6,7 @@ import daft from daft import DataFrame +from daft.table import MicroPartition NUM_ROWS = 1_000_000 @@ -110,7 +111,7 @@ def generate_list_params() -> tuple[dict, daft.Expression, list]: def test_if_else(test_data_generator, benchmark) -> None: """If_else between NUM_ROWS values""" data, expr, expected = test_data_generator() - table = daft.table.Table.from_pydict(data) + table = MicroPartition.from_pydict(data) def bench_if_else() -> DataFrame: return table.eval_expression_list([expr.alias("result")]) diff --git a/tests/benchmarks/test_take.py b/tests/benchmarks/test_take.py index 9d931820e9..009e82745c 100644 --- a/tests/benchmarks/test_take.py +++ b/tests/benchmarks/test_take.py @@ -4,6 +4,7 @@ import daft from daft import DataFrame, Series +from daft.table import MicroPartition NUM_ROWS = 10_000_000 @@ -97,7 +98,7 @@ def generate_list_int64_take_reversed() -> tuple[dict, daft.Expression, list]: def test_take(test_data_generator, benchmark) -> None: """If_else between NUM_ROWS values""" data, idx, expected = test_data_generator() - table = daft.table.Table.from_pydict(data) + table = MicroPartition.from_pydict(data) def bench_take() -> DataFrame: return table.take(idx) diff --git a/tests/conftest.py b/tests/conftest.py index bdc7e6d9c5..dfb396c053 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -7,6 +7,7 @@ import pytest import daft +from daft.table import MicroPartition def pytest_configure(config): @@ -78,7 +79,7 @@ def _make_df( import pyarrow.parquet as papq name = str(uuid.uuid4()) - daft_table = daft.table.Table.from_arrow(pa_table) + daft_table = MicroPartition.from_arrow(pa_table) partitioned_tables = ( daft_table.partition_by_random(repartition, 0) if len(repartition_columns) == 0 diff --git a/tests/expressions/test_expressions.py b/tests/expressions/test_expressions.py index c133aa3552..8dac113fc2 100644 --- a/tests/expressions/test_expressions.py +++ b/tests/expressions/test_expressions.py @@ -9,7 +9,7 @@ from daft.datatype import DataType, TimeUnit from daft.expressions import col, lit from daft.expressions.testing import expr_structurally_equal -from daft.table import Table +from daft.table import MicroPartition @pytest.mark.parametrize( @@ -31,7 +31,7 @@ def test_make_lit(data, expected_dtype) -> None: l = lit(data) assert l.name() == "literal" - empty_table = Table.empty() + empty_table = MicroPartition.empty() lit_table = empty_table.eval_expression_list([l]) series = lit_table.get_column("literal") assert series.datatype() == expected_dtype diff --git a/tests/expressions/test_expressions_projection.py b/tests/expressions/test_expressions_projection.py index 1c2332a637..853514d802 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 Table +from daft.table import MicroPartition def test_expressions_projection_error_dup_name(): @@ -156,7 +156,7 @@ def test_expressions_projection_indexing(): def test_resolve_schema(): - tbl = Table.from_pydict( + tbl = MicroPartition.from_pydict( { "foo": [1, 2, 3], } @@ -167,7 +167,7 @@ def test_resolve_schema(): def test_resolve_schema_invalid_type(): - tbl = Table.from_pydict( + tbl = MicroPartition.from_pydict( { "foo": ["a", "b", "c"], } @@ -178,7 +178,7 @@ def test_resolve_schema_invalid_type(): def test_resolve_schema_missing_col(): - tbl = Table.from_pydict( + tbl = MicroPartition.from_pydict( { "foo": ["a", "b", "c"], } diff --git a/tests/expressions/test_udf.py b/tests/expressions/test_udf.py index fb6a37a115..ff5463c2d1 100644 --- a/tests/expressions/test_udf.py +++ b/tests/expressions/test_udf.py @@ -8,12 +8,12 @@ from daft.expressions import Expression from daft.expressions.testing import expr_structurally_equal from daft.series import Series -from daft.table import Table +from daft.table import MicroPartition from daft.udf import udf def test_udf(): - table = Table.from_pydict({"a": ["foo", "bar", "baz"]}) + table = MicroPartition.from_pydict({"a": ["foo", "bar", "baz"]}) @udf(return_dtype=DataType.string()) def repeat_n(data, n): @@ -29,7 +29,7 @@ def repeat_n(data, n): def test_class_udf(): - table = Table.from_pydict({"a": ["foo", "bar", "baz"]}) + table = MicroPartition.from_pydict({"a": ["foo", "bar", "baz"]}) @udf(return_dtype=DataType.string()) class RepeatN: @@ -49,7 +49,7 @@ def __call__(self, data): def test_udf_kwargs(): - table = Table.from_pydict({"a": ["foo", "bar", "baz"]}) + table = MicroPartition.from_pydict({"a": ["foo", "bar", "baz"]}) @udf(return_dtype=DataType.string()) def repeat_n(*, data=None, n=2): @@ -66,7 +66,7 @@ def repeat_n(*, data=None, n=2): @pytest.mark.parametrize("container", [Series, list, np.ndarray]) def test_udf_return_containers(container): - table = Table.from_pydict({"a": ["foo", "bar", "baz"]}) + table = MicroPartition.from_pydict({"a": ["foo", "bar", "baz"]}) @udf(return_dtype=DataType.string()) def identity(data): @@ -84,7 +84,7 @@ def identity(data): def test_udf_error(): - table = Table.from_pydict({"a": ["foo", "bar", "baz"]}) + table = MicroPartition.from_pydict({"a": ["foo", "bar", "baz"]}) @udf(return_dtype=DataType.string()) def throw_value_err(x): @@ -122,7 +122,7 @@ def full_udf(e_arg, val, kwarg_val=None, kwarg_ex=None): def test_class_udf_initialization_error(): - table = Table.from_pydict({"a": ["foo", "bar", "baz"]}) + table = MicroPartition.from_pydict({"a": ["foo", "bar", "baz"]}) @udf(return_dtype=DataType.string()) class IdentityWithInitError: @@ -152,7 +152,7 @@ def np_udf(x): return [np.ones((3, 3)) * i for i in x.to_pylist()] expr = np_udf(col("x")) - table = Table.from_pydict({"x": [0, 1, 2]}) + table = MicroPartition.from_pydict({"x": [0, 1, 2]}) result = table.eval_expression_list([expr]) assert len(result.to_pydict()["x"]) == 3 for i in range(3): diff --git a/tests/expressions/typing/conftest.py b/tests/expressions/typing/conftest.py index 268d115925..20831fc32b 100644 --- a/tests/expressions/typing/conftest.py +++ b/tests/expressions/typing/conftest.py @@ -16,7 +16,7 @@ from daft.datatype import DataType from daft.expressions import Expression, ExpressionsProjection from daft.series import Series -from daft.table import Table +from daft.table import MicroPartition ALL_DTYPES = [ (DataType.int8(), pa.array([1, 2, None], type=pa.int8())), @@ -93,11 +93,11 @@ def assert_typing_resolve_vs_runtime_behavior( Args: data: data to test against (generated using one of the provided fixtures, `{unary, binary}_data_fixture`) - expr (Expression): Expression used to run the kernel in a Table (use `.name()` of the generated data to refer to columns) + expr (Expression): Expression used to run the kernel in a MicroPartition (use `.name()` of the generated data to refer to columns) run_kernel (Callable): A lambda that will run the kernel directly on the generated Series' without going through the Expressions API resolvable (bool): Whether this kernel should be valid, given the datatypes of the generated Series' """ - table = Table.from_pydict({s.name(): s for s in data}) + table = MicroPartition.from_pydict({s.name(): s for s in data}) projection = ExpressionsProjection([expr.alias("result")]) if resolvable: # Check that schema resolution and Series runtime return the same datatype diff --git a/tests/integration/io/parquet/test_reads_local_fixtures.py b/tests/integration/io/parquet/test_reads_local_fixtures.py index 0802160054..bb36125c20 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 Table +from daft.table import MicroPartition BUCKETS = ["head-retries-parquet-bucket", "get-retries-parquet-bucket"] @@ -16,7 +16,7 @@ def test_non_retryable_errors(retry_server_s3_config, status_code: tuple[int, st status_code, status_code_str = status_code data_path = f"s3://{bucket}/{status_code}/{status_code_str}/1/{uuid.uuid4()}" with pytest.raises((FileNotFoundError, ValueError)): - Table.read_parquet(data_path, io_config=retry_server_s3_config) + MicroPartition.read_parquet(data_path, io_config=retry_server_s3_config) @pytest.mark.integration() @@ -48,4 +48,4 @@ def test_retryable_errors(retry_server_s3_config, status_code: tuple[int, str], status_code, status_code_str = status_code data_path = f"s3://{bucket}/{status_code}/{status_code_str}/{NUM_ERRORS}/{uuid.uuid4()}" - Table.read_parquet(data_path, io_config=retry_server_s3_config) + MicroPartition.read_parquet(data_path, io_config=retry_server_s3_config) diff --git a/tests/integration/io/parquet/test_reads_public_data.py b/tests/integration/io/parquet/test_reads_public_data.py index 638235a6f2..0b1dc21071 100644 --- a/tests/integration/io/parquet/test_reads_public_data.py +++ b/tests/integration/io/parquet/test_reads_public_data.py @@ -8,7 +8,7 @@ import daft from daft.filesystem import get_filesystem, get_protocol_from_path -from daft.table import LegacyTable, Table +from daft.table import LegacyTable, MicroPartition def get_filesystem_from_path(path: str, **kwargs) -> fsspec.AbstractFileSystem: @@ -216,8 +216,10 @@ def read_parquet_with_pyarrow(path) -> pa.Table: ) def test_parquet_read_table(parquet_file, public_storage_io_config, multithreaded_io): _, url = parquet_file - daft_native_read = Table.read_parquet(url, io_config=public_storage_io_config, multithreaded_io=multithreaded_io) - pa_read = Table.from_arrow(read_parquet_with_pyarrow(url)) + daft_native_read = MicroPartition.read_parquet( + url, io_config=public_storage_io_config, multithreaded_io=multithreaded_io + ) + pa_read = MicroPartition.from_arrow(read_parquet_with_pyarrow(url)) assert daft_native_read.schema() == pa_read.schema() pd.testing.assert_frame_equal(daft_native_read.to_pandas(), pa_read.to_pandas()) @@ -245,20 +247,22 @@ def test_parquet_read_table_into_pyarrow(parquet_file, public_storage_io_config, ) def test_parquet_read_table_bulk(parquet_file, public_storage_io_config, multithreaded_io): _, url = parquet_file - daft_native_reads = Table.read_parquet_bulk( + daft_native_reads = MicroPartition.read_parquet_bulk( [url] * 2, io_config=public_storage_io_config, multithreaded_io=multithreaded_io ) - pa_read = Table.from_arrow(read_parquet_with_pyarrow(url)) + pa_read = MicroPartition.from_arrow(read_parquet_with_pyarrow(url)) # Legacy Table returns a list[Table] - if Table == LegacyTable: + if MicroPartition == LegacyTable: 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()) # MicroPartitions returns a MicroPartition else: assert daft_native_reads.schema() == pa_read.schema() - pd.testing.assert_frame_equal(daft_native_reads.to_pandas(), Table.concat([pa_read, pa_read]).to_pandas()) + pd.testing.assert_frame_equal( + daft_native_reads.to_pandas(), MicroPartition.concat([pa_read, pa_read]).to_pandas() + ) @pytest.mark.integration() @@ -282,7 +286,7 @@ def test_parquet_into_pyarrow_bulk(parquet_file, public_storage_io_config, multi def test_parquet_read_df(parquet_file, public_storage_io_config): _, url = parquet_file daft_native_read = daft.read_parquet(url, io_config=public_storage_io_config) - pa_read = Table.from_arrow(read_parquet_with_pyarrow(url)) + pa_read = MicroPartition.from_arrow(read_parquet_with_pyarrow(url)) assert daft_native_read.schema() == pa_read.schema() pd.testing.assert_frame_equal(daft_native_read.to_pandas(), pa_read.to_pandas()) @@ -294,21 +298,21 @@ def test_parquet_read_df(parquet_file, public_storage_io_config): ) def test_row_groups_selection(public_storage_io_config, multithreaded_io): url = "s3://daft-public-data/test_fixtures/parquet-dev/mvp.parquet" - all_rows = Table.read_parquet(url, io_config=public_storage_io_config, multithreaded_io=multithreaded_io) + all_rows = MicroPartition.read_parquet(url, io_config=public_storage_io_config, multithreaded_io=multithreaded_io) assert len(all_rows) == 100 - first = Table.read_parquet( + first = MicroPartition.read_parquet( url, io_config=public_storage_io_config, multithreaded_io=multithreaded_io, row_groups=[0] ) assert len(first) == 10 assert all_rows.to_arrow()[:10] == first.to_arrow() - fifth = Table.read_parquet( + fifth = MicroPartition.read_parquet( url, io_config=public_storage_io_config, multithreaded_io=multithreaded_io, row_groups=[5] ) assert len(fifth) == 10 assert all_rows.to_arrow()[50:60] == fifth.to_arrow() - repeated = Table.read_parquet( + repeated = MicroPartition.read_parquet( url, io_config=public_storage_io_config, multithreaded_io=multithreaded_io, row_groups=[1, 1, 1] ) assert len(repeated) == 30 @@ -316,7 +320,7 @@ def test_row_groups_selection(public_storage_io_config, multithreaded_io): assert all_rows.to_arrow()[10:20] == repeated.to_arrow()[10:20] assert all_rows.to_arrow()[10:20] == repeated.to_arrow()[20:] - out_of_order = Table.read_parquet( + out_of_order = MicroPartition.read_parquet( url, io_config=public_storage_io_config, multithreaded_io=multithreaded_io, row_groups=[1, 0] ) assert len(out_of_order) == 20 @@ -333,8 +337,8 @@ 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 Table == LegacyTable: - first, *rest = Table.read_parquet_bulk( + if MicroPartition == LegacyTable: + first, *rest = MicroPartition.read_parquet_bulk( url, io_config=public_storage_io_config, multithreaded_io=multithreaded_io, row_groups_per_path=row_groups ) assert len(first) == 100 @@ -344,7 +348,7 @@ def test_row_groups_selection_bulk(public_storage_io_config, multithreaded_io): assert len(t) == 10 assert first.to_arrow()[i * 10 : (i + 1) * 10] == t.to_arrow() else: - mp = Table.read_parquet_bulk( + mp = MicroPartition.read_parquet_bulk( url, io_config=public_storage_io_config, multithreaded_io=multithreaded_io, row_groups_per_path=row_groups ) assert len(mp) == 100 + ( diff --git a/tests/table/image/test_crop.py b/tests/table/image/test_crop.py index 824320a1e9..71e8801f9d 100644 --- a/tests/table/image/test_crop.py +++ b/tests/table/image/test_crop.py @@ -5,7 +5,7 @@ import pytest import daft -from daft.table import Table +from daft.table import MicroPartition MODES = ["L", "LA", "RGB", "RGBA"] MODE_TO_NP_DTYPE = { @@ -79,7 +79,7 @@ def mixed_shape_data_fixture(request): def test_image_crop_mixed_shape_same_mode(mixed_shape_data_fixture): - table = Table.from_pydict({"images": mixed_shape_data_fixture}) + table = MicroPartition.from_pydict({"images": mixed_shape_data_fixture}) result = table.eval_expression_list([daft.col("images").image.crop((1, 1, 1, 1))]) result = result.to_pydict() @@ -93,7 +93,7 @@ def crop(arr): def test_image_crop_mixed_shape_same_mode_crop_col(mixed_shape_data_fixture): # TODO(jay): Need to fix nested casts -- for now we workaround by creating a nested uint32 bboxes = pa.array([[1, 1, 1, 1], None, None], type=pa.list_(pa.uint32())) - table = Table.from_pydict( + table = MicroPartition.from_pydict( { "images": mixed_shape_data_fixture, "bboxes": bboxes, @@ -110,7 +110,7 @@ def crop(arr): def test_image_crop_fixed_shape_same_mode(fixed_shape_data_fixture): - table = Table.from_pydict({"images": fixed_shape_data_fixture}) + table = MicroPartition.from_pydict({"images": fixed_shape_data_fixture}) result = table.eval_expression_list([daft.col("images").image.crop((1, 1, 1, 1))]) result = result.to_pydict() @@ -124,7 +124,7 @@ def crop(arr): def test_image_crop_fixed_shape_same_mode_crop_col(fixed_shape_data_fixture): # TODO(jay): Need to fix nested casts -- for now we workaround by creating a nested uint32 bboxes = pa.array([[1, 1, 1, 1], None, None], type=pa.list_(pa.uint32())) - table = Table.from_pydict( + table = MicroPartition.from_pydict( { "images": fixed_shape_data_fixture, "bboxes": bboxes, @@ -141,7 +141,7 @@ def crop(arr): def test_bad_expr_input(): - table = Table.from_pydict({"x": [1, 2, 3], "y": ["a", "b", "c"]}) + table = MicroPartition.from_pydict({"x": [1, 2, 3], "y": ["a", "b", "c"]}) # Test bad Expression calls with pytest.raises(ValueError): diff --git a/tests/table/list/test_list_join.py b/tests/table/list/test_list_join.py index 18ea5c827c..8cc489ec1a 100644 --- a/tests/table/list/test_list_join.py +++ b/tests/table/list/test_list_join.py @@ -3,17 +3,17 @@ import pytest from daft.expressions import col -from daft.table import Table +from daft.table import MicroPartition def test_list_join(): - table = Table.from_pydict({"col": [None, [], ["a"], [None], ["a", "a"], ["a", None], ["a", None, "a"]]}) + table = MicroPartition.from_pydict({"col": [None, [], ["a"], [None], ["a", "a"], ["a", None], ["a", None, "a"]]}) result = table.eval_expression_list([col("col").list.join(",")]) assert result.to_pydict() == {"col": [None, "", "a", "", "a,a", "a,", "a,,a"]} def test_list_join_other_col(): - table = Table.from_pydict( + table = MicroPartition.from_pydict( { "col": [None, [], ["a"], [None], ["a", "a"], ["a", None], ["a", None, "a"]], "delimiter": ["1", "2", "3", "4", "5", "6", "7"], @@ -24,10 +24,10 @@ def test_list_join_other_col(): def test_list_join_bad_type(): - table = Table.from_pydict({"col": [1, 2, 3]}) + table = MicroPartition.from_pydict({"col": [1, 2, 3]}) with pytest.raises(ValueError): table.eval_expression_list([col("col").list.join(",")]) - table = Table.from_pydict({"col": [[1, 2, 3], [4, 5, 6], []]}) + table = MicroPartition.from_pydict({"col": [[1, 2, 3], [4, 5, 6], []]}) with pytest.raises(ValueError): table.eval_expression_list([col("col").list.join(",")]) diff --git a/tests/table/list/test_list_lengths.py b/tests/table/list/test_list_lengths.py index 45b36cb06b..a3520908dc 100644 --- a/tests/table/list/test_list_lengths.py +++ b/tests/table/list/test_list_lengths.py @@ -1,10 +1,10 @@ from __future__ import annotations from daft.expressions import col -from daft.table import Table +from daft.table import MicroPartition def test_list_lengths(): - table = Table.from_pydict({"col": [None, [], ["a"], [None], ["a", "a"], ["a", None], ["a", None, "a"]]}) + table = MicroPartition.from_pydict({"col": [None, [], ["a"], [None], ["a", "a"], ["a", None], ["a", None, "a"]]}) result = table.eval_expression_list([col("col").list.lengths()]) assert result.to_pydict() == {"col": [None, 0, 1, 1, 2, 2, 3]} diff --git a/tests/table/table_io/test_csv.py b/tests/table/table_io/test_csv.py index 167c316ef4..d45ca88103 100644 --- a/tests/table/table_io/test_csv.py +++ b/tests/table/table_io/test_csv.py @@ -14,7 +14,7 @@ from daft.logical.schema import Schema from daft.runners.partitioning import TableParseCSVOptions, TableReadOptions from daft.series import ARROW_VERSION -from daft.table import Table, schema_inference, table_io +from daft.table import MicroPartition, schema_inference, table_io def storage_config_from_use_native_downloader(use_native_downloader: bool) -> StorageConfig: @@ -143,7 +143,7 @@ def test_csv_read_data(data, expected_data_series, use_native_downloader): schema = Schema._from_field_name_and_types( [("id", DataType.int64()), ("data", expected_data_series.datatype())] ) - expected = Table.from_pydict( + expected = MicroPartition.from_pydict( { "id": [1, 2, 3], "data": expected_data_series, @@ -166,7 +166,7 @@ def test_csv_read_data_csv_limit_rows(use_native_downloader): storage_config = storage_config_from_use_native_downloader(use_native_downloader) schema = Schema._from_field_name_and_types([("id", DataType.int64()), ("data", DataType.int64())]) - expected = Table.from_pydict( + expected = MicroPartition.from_pydict( { "id": [1, 2], "data": [1, 2], @@ -194,7 +194,7 @@ def test_csv_read_data_csv_select_columns(use_native_downloader): storage_config = storage_config_from_use_native_downloader(use_native_downloader) schema = Schema._from_field_name_and_types([("id", DataType.int64()), ("data", DataType.int64())]) - expected = Table.from_pydict( + expected = MicroPartition.from_pydict( { "data": [1, 2, None], } @@ -222,7 +222,7 @@ def test_csv_read_data_csv_custom_delimiter(use_native_downloader): storage_config = storage_config_from_use_native_downloader(use_native_downloader) schema = Schema._from_field_name_and_types([("id", DataType.int64()), ("data", DataType.int64())]) - expected = Table.from_pydict( + expected = MicroPartition.from_pydict( { "id": [1, 2, 3], "data": [1, 2, None], @@ -250,7 +250,7 @@ def test_csv_read_data_csv_no_header(use_native_downloader): storage_config = storage_config_from_use_native_downloader(use_native_downloader) schema = Schema._from_field_name_and_types([("id", DataType.int64()), ("data", DataType.int64())]) - expected = Table.from_pydict( + expected = MicroPartition.from_pydict( { "id": [1, 2, 3], "data": [1, 2, None], @@ -279,7 +279,7 @@ def test_csv_read_data_csv_custom_quote(use_native_downloader): storage_config = storage_config_from_use_native_downloader(use_native_downloader) schema = Schema._from_field_name_and_types([("id", DataType.int64()), ("data", DataType.string())]) - expected = Table.from_pydict( + expected = MicroPartition.from_pydict( { "id": [1, 2, 3], "data": ["aa", "aa", "aa"], @@ -313,7 +313,7 @@ def test_csv_read_data_custom_escape(use_native_downloader): storage_config = storage_config_from_use_native_downloader(use_native_downloader) schema = Schema._from_field_name_and_types([("id", DataType.int64()), ("data", DataType.string())]) - expected = Table.from_pydict( + expected = MicroPartition.from_pydict( { "id": [1, 2, 3], "data": ['a"a"a', "aa", "aa"], @@ -345,7 +345,7 @@ def test_csv_read_data_custom_comment(use_native_downloader): storage_config = storage_config_from_use_native_downloader(use_native_downloader) schema = Schema._from_field_name_and_types([("id", DataType.int64()), ("data", DataType.string())]) - expected = Table.from_pydict( + expected = MicroPartition.from_pydict( { "id": [1, 3], "data": ["aa", "aa"], diff --git a/tests/table/table_io/test_json.py b/tests/table/table_io/test_json.py index d8ebdfea0a..74989eae1f 100644 --- a/tests/table/table_io/test_json.py +++ b/tests/table/table_io/test_json.py @@ -11,7 +11,7 @@ from daft.datatype import DataType from daft.logical.schema import Schema from daft.runners.partitioning import TableReadOptions -from daft.table import Table, schema_inference, table_io +from daft.table import MicroPartition, schema_inference, table_io def test_read_input(tmpdir): @@ -94,7 +94,7 @@ def test_json_read_data(data, expected_data_series): ) schema = Schema._from_field_name_and_types([("id", DataType.int64()), ("data", expected_data_series.datatype())]) - expected = Table.from_pydict( + expected = MicroPartition.from_pydict( { "id": [1, 2, 3], "data": expected_data_series, @@ -113,7 +113,7 @@ def test_json_read_data_limit_rows(): ) schema = Schema._from_field_name_and_types([("id", DataType.int64()), ("data", DataType.int64())]) - expected = Table.from_pydict( + expected = MicroPartition.from_pydict( { "id": [1, 2], "data": [1, 2], @@ -132,7 +132,7 @@ def test_json_read_data_select_columns(): ) schema = Schema._from_field_name_and_types([("id", DataType.int64()), ("data", DataType.int64())]) - expected = Table.from_pydict( + expected = MicroPartition.from_pydict( { "data": [1, 2, None], } diff --git a/tests/table/table_io/test_parquet.py b/tests/table/table_io/test_parquet.py index b617b5beae..cff4c41791 100644 --- a/tests/table/table_io/test_parquet.py +++ b/tests/table/table_io/test_parquet.py @@ -16,7 +16,7 @@ from daft.logical.schema import Schema from daft.runners.partitioning import TableParseParquetOptions, TableReadOptions from daft.table import ( - Table, + MicroPartition, read_parquet_into_pyarrow, read_parquet_into_pyarrow_bulk, schema_inference, @@ -98,7 +98,7 @@ def test_parquet_infer_schema(data, expected_dtype, use_native_downloader): def test_parquet_read_empty(use_native_downloader): 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 = Table.from_pydict({"foo": pa.array([], type=pa.int64())}) + expected = MicroPartition.from_pydict({"foo": pa.array([], type=pa.int64())}) storage_config = storage_config_from_use_native_downloader(use_native_downloader) table = table_io.read_parquet(f, schema, storage_config=storage_config) assert table.to_arrow() == expected.to_arrow(), f"Expected:\n{expected}\n\nReceived:\n{table}" @@ -134,7 +134,7 @@ def test_parquet_read_data(data, expected_data_series, use_native_downloader): schema = Schema._from_field_name_and_types( [("id", DataType.int64()), ("data", expected_data_series.datatype())] ) - expected = Table.from_pydict( + expected = MicroPartition.from_pydict( { "id": [1, 2, 3], "data": expected_data_series, @@ -158,7 +158,7 @@ def test_parquet_read_data_limit_rows(row_group_size, use_native_downloader): row_group_size=row_group_size, ) as f: schema = Schema._from_field_name_and_types([("id", DataType.int64()), ("data", DataType.int64())]) - expected = Table.from_pydict( + expected = MicroPartition.from_pydict( { "id": [1, 2], "data": [1, 2], @@ -173,8 +173,8 @@ def test_parquet_read_data_limit_rows(row_group_size, use_native_downloader): def test_parquet_read_data_multi_row_groups(): path = "tests/assets/parquet-data/mvp.parquet" - table = Table.read_parquet(path) - expected = Table.from_arrow(papq.read_table(path)) + table = MicroPartition.read_parquet(path) + expected = MicroPartition.from_arrow(papq.read_table(path)) assert table.to_arrow() == expected.to_arrow(), f"Expected:\n{expected}\n\nReceived:\n{table}" @@ -189,7 +189,7 @@ def test_parquet_read_data_select_columns(use_native_downloader): ) ) as f: schema = Schema._from_field_name_and_types([("id", DataType.int64()), ("data", DataType.int64())]) - expected = Table.from_pydict( + expected = MicroPartition.from_pydict( { "data": [1, 2, None], } @@ -234,7 +234,7 @@ def test_parquet_read_int96_timestamps(use_deprecated_int96_timestamps, use_nati papq_write_table_kwargs=papq_write_table_kwargs, ) as f: schema = Schema._from_field_name_and_types(schema) - expected = Table.from_pydict(data) + expected = MicroPartition.from_pydict(data) storage_config = storage_config_from_use_native_downloader(use_native_downloader) table = table_io.read_parquet( f, @@ -272,7 +272,7 @@ def test_parquet_read_int96_timestamps_overflow(coerce_to, use_native_downloader papq_write_table_kwargs=papq_write_table_kwargs, ) as f: schema = Schema._from_field_name_and_types(schema) - expected = Table.from_pydict(data) + expected = MicroPartition.from_pydict(data) storage_config = storage_config_from_use_native_downloader(use_native_downloader) table = table_io.read_parquet( f, @@ -331,7 +331,7 @@ def test_read_too_small_parquet_file(tmpdir, n_bytes): for _ in range(n_bytes): f.write(b"0") with pytest.raises(ValueError, match="smaller than the minimum size of 12 bytes"): - Table.read_parquet(file_path.as_posix()) + MicroPartition.read_parquet(file_path.as_posix()) def test_read_empty_parquet_file_with_table(tmpdir): @@ -340,7 +340,7 @@ def test_read_empty_parquet_file_with_table(tmpdir): tab = pa.table({"x": pa.array([], type=pa.int64())}) with open(file_path, "wb") as f: papq.write_table(tab, file_path.as_posix()) - read_back = Table.read_parquet(file_path.as_posix()).to_arrow() + read_back = MicroPartition.read_parquet(file_path.as_posix()).to_arrow() assert tab == read_back diff --git a/tests/table/table_io/test_read_time_cast.py b/tests/table/table_io/test_read_time_cast.py index f2aa7f10b2..65f00305dd 100644 --- a/tests/table/table_io/test_read_time_cast.py +++ b/tests/table/table_io/test_read_time_cast.py @@ -6,7 +6,7 @@ import daft from daft import DataType from daft.logical.schema import Schema -from daft.table import Table, table_io +from daft.table import MicroPartition, table_io from tests.table.table_io.test_parquet import _parquet_write_helper @@ -17,26 +17,28 @@ ( pa.Table.from_pydict({"foo": pa.array([1, 2, 3], type=pa.int64())}), Schema._from_field_name_and_types([("foo", DataType.int8())]), - Table.from_pydict({"foo": daft.Series.from_arrow(pa.array([1, 2, 3], type=pa.int8()))}), + MicroPartition.from_pydict({"foo": daft.Series.from_arrow(pa.array([1, 2, 3], type=pa.int8()))}), ), # Test what happens if a cast should occur, but fails at runtime (in this case, a potentially bad cast from utf8->int64) ( pa.Table.from_pydict({"foo": pa.array(["1", "2", "FAIL"], type=pa.string())}), Schema._from_field_name_and_types([("foo", DataType.int64())]), # NOTE: cast failures will become a Null value - Table.from_pydict({"foo": daft.Series.from_arrow(pa.array([1, 2, None], type=pa.int64()))}), + MicroPartition.from_pydict({"foo": daft.Series.from_arrow(pa.array([1, 2, None], type=pa.int64()))}), ), # Test reordering of columns ( pa.Table.from_pydict({"foo": pa.array([1, 2, 3]), "bar": pa.array([1, 2, 3])}), Schema._from_field_name_and_types([("bar", DataType.int64()), ("foo", DataType.int64())]), - Table.from_pydict({"bar": pa.array([1, 2, 3]), "foo": pa.array([1, 2, 3])}), + MicroPartition.from_pydict({"bar": pa.array([1, 2, 3]), "foo": pa.array([1, 2, 3])}), ), # Test automatic insertion of null values for missing column ( pa.Table.from_pydict({"foo": pa.array([1, 2, 3])}), Schema._from_field_name_and_types([("bar", DataType.int64()), ("foo", DataType.int64())]), - Table.from_pydict({"bar": pa.array([None, None, None], type=pa.int64()), "foo": pa.array([1, 2, 3])}), + MicroPartition.from_pydict( + {"bar": pa.array([None, None, None], type=pa.int64()), "foo": pa.array([1, 2, 3])} + ), ), ], ) diff --git a/tests/table/test_blackbox_kernels.py b/tests/table/test_blackbox_kernels.py index c02bdbbdaa..2e40e897f2 100644 --- a/tests/table/test_blackbox_kernels.py +++ b/tests/table/test_blackbox_kernels.py @@ -1,12 +1,12 @@ from __future__ import annotations from daft.expressions import col -from daft.table import Table +from daft.table import MicroPartition def test_pyobjects_blackbox_kernels() -> None: objects = [object(), None, object()] - table = Table.from_pydict({"keys": [0, 1, 2], "objs": objects}) + table = MicroPartition.from_pydict({"keys": [0, 1, 2], "objs": objects}) # Head. assert table.head(2).to_pydict()["objs"] == objects[:2] # Filter. @@ -16,7 +16,7 @@ def test_pyobjects_blackbox_kernels() -> None: def test_nested_blackbox_kernels() -> None: structs = [{"a": 1, "b": 2}, None, {"a": 3}] lists = [[1, 2], None, [3]] - table = Table.from_pydict({"keys": [0, 1, 2], "structs": structs, "lists": lists}) + table = MicroPartition.from_pydict({"keys": [0, 1, 2], "structs": structs, "lists": lists}) # pyarrow fills in implicit field-internal Nones on a .to_pylist() conversion. structs[2]["b"] = None # Head. diff --git a/tests/table/test_broadcasts.py b/tests/table/test_broadcasts.py index 561e9c0c25..127a8d6a43 100644 --- a/tests/table/test_broadcasts.py +++ b/tests/table/test_broadcasts.py @@ -4,19 +4,19 @@ import daft from daft.expressions import col, lit -from daft.table import Table +from daft.table import MicroPartition @pytest.mark.parametrize("data", [1, "a", True, b"Y", 0.5, None, [1, 2, 3], object()]) def test_broadcast(data): - table = Table.from_pydict({"x": [1, 2, 3]}) + table = MicroPartition.from_pydict({"x": [1, 2, 3]}) new_table = table.eval_expression_list([col("x"), lit(data)]) assert new_table.to_pydict() == {"x": [1, 2, 3], "literal": [data for _ in range(3)]} def test_broadcast_fixed_size_list(): data = [1, 2, 3] - table = Table.from_pydict({"x": [1, 2, 3]}) + table = MicroPartition.from_pydict({"x": [1, 2, 3]}) new_table = table.eval_expression_list( [col("x"), lit(data).cast(daft.DataType.fixed_size_list(daft.DataType.int64(), 3))] ) diff --git a/tests/table/test_concat.py b/tests/table/test_concat.py index 86a452c1a8..4b6f3399a4 100644 --- a/tests/table/test_concat.py +++ b/tests/table/test_concat.py @@ -2,67 +2,67 @@ import pytest -from daft.table import Table +from daft.table import MicroPartition def test_table_concat() -> None: objs1 = [None, object(), object()] objs2 = [object(), None, object()] tables = [ - Table.from_pydict({"x": [1, 2, 3], "y": ["a", "b", "c"], "z": objs1}), - Table.from_pydict({"x": [4, 5, 6], "y": ["d", "e", "f"], "z": objs2}), + MicroPartition.from_pydict({"x": [1, 2, 3], "y": ["a", "b", "c"], "z": objs1}), + MicroPartition.from_pydict({"x": [4, 5, 6], "y": ["d", "e", "f"], "z": objs2}), ] - result = Table.concat(tables) + result = MicroPartition.concat(tables) assert result.to_pydict() == {"x": [1, 2, 3, 4, 5, 6], "y": ["a", "b", "c", "d", "e", "f"], "z": objs1 + objs2} tables = [ - Table.from_pydict({"x": [], "y": []}), - Table.from_pydict({"x": [], "y": []}), + MicroPartition.from_pydict({"x": [], "y": []}), + MicroPartition.from_pydict({"x": [], "y": []}), ] - result = Table.concat(tables) + result = MicroPartition.concat(tables) assert result.to_pydict() == {"x": [], "y": []} def test_table_concat_bad_input() -> None: - mix_types_table = [Table.from_pydict({"x": [1, 2, 3]}), []] - with pytest.raises(TypeError, match=f"Expected a {Table.__name__} for concat"): - Table.concat(mix_types_table) + mix_types_table = [MicroPartition.from_pydict({"x": [1, 2, 3]}), []] + with pytest.raises(TypeError, match=f"Expected a {MicroPartition.__name__} for concat"): + MicroPartition.concat(mix_types_table) - with pytest.raises(ValueError, match=f"Need at least 1 {Table.__name__}"): - Table.concat([]) + with pytest.raises(ValueError, match=f"Need at least 1 {MicroPartition.__name__}"): + MicroPartition.concat([]) def test_table_concat_schema_mismatch() -> None: mix_types_table = [ - Table.from_pydict({"x": [1, 2, 3]}), - Table.from_pydict({"y": [1, 2, 3]}), + MicroPartition.from_pydict({"x": [1, 2, 3]}), + MicroPartition.from_pydict({"y": [1, 2, 3]}), ] - with pytest.raises(ValueError, match=f"{Table.__name__} concat requires all schemas to match"): - Table.concat(mix_types_table) + with pytest.raises(ValueError, match=f"{MicroPartition.__name__} concat requires all schemas to match"): + MicroPartition.concat(mix_types_table) mix_types_table = [ - Table.from_pydict({"x": [1, 2, 3]}), - Table.from_pydict({"x": [1.0, 2.0, 3.0]}), + MicroPartition.from_pydict({"x": [1, 2, 3]}), + MicroPartition.from_pydict({"x": [1.0, 2.0, 3.0]}), ] - with pytest.raises(ValueError, match=f"{Table.__name__} concat requires all schemas to match"): - Table.concat(mix_types_table) + with pytest.raises(ValueError, match=f"{MicroPartition.__name__} concat requires all schemas to match"): + MicroPartition.concat(mix_types_table) mix_types_table = [ - Table.from_pydict({"x": [1, 2, 3]}), - Table.from_pydict({"x": [object(), object(), object()]}), + MicroPartition.from_pydict({"x": [1, 2, 3]}), + MicroPartition.from_pydict({"x": [object(), object(), object()]}), ] - with pytest.raises(ValueError, match=f"{Table.__name__} concat requires all schemas to match"): - Table.concat(mix_types_table) + with pytest.raises(ValueError, match=f"{MicroPartition.__name__} concat requires all schemas to match"): + MicroPartition.concat(mix_types_table) mix_types_table = [ - Table.from_pydict({"x": [1, 2, 3]}), - Table.from_pydict({"x": [1, 2, 3], "y": [2, 3, 4]}), + MicroPartition.from_pydict({"x": [1, 2, 3]}), + MicroPartition.from_pydict({"x": [1, 2, 3], "y": [2, 3, 4]}), ] - with pytest.raises(ValueError, match=f"{Table.__name__} concat requires all schemas to match"): - Table.concat(mix_types_table) + with pytest.raises(ValueError, match=f"{MicroPartition.__name__} concat requires all schemas to match"): + MicroPartition.concat(mix_types_table) diff --git a/tests/table/test_eval.py b/tests/table/test_eval.py index 3f6a383556..fb0b7e0de4 100644 --- a/tests/table/test_eval.py +++ b/tests/table/test_eval.py @@ -7,13 +7,13 @@ import pytest from daft import DataType, col -from daft.table import Table +from daft.table import MicroPartition from tests.table import daft_numeric_types def test_table_eval_expressions() -> None: pa_table = pa.Table.from_pydict({"a": [1, 2, 3, 4], "b": [5, 6, 7, 8]}) - daft_table = Table.from_arrow(pa_table) + daft_table = MicroPartition.from_arrow(pa_table) assert len(daft_table) == 4 assert daft_table.column_names() == ["a", "b"] @@ -28,7 +28,7 @@ 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 = Table.from_arrow(pa_table) + daft_table = MicroPartition.from_arrow(pa_table) assert len(daft_table) == 4 assert daft_table.column_names() == ["a", "b"] @@ -48,7 +48,7 @@ def test_table_eval_expressions_conflict() -> None: ) def test_table_expr_not(input, expr, expected) -> None: """Test logical not expression.""" - daft_table = Table.from_pydict({"input": input}) + daft_table = MicroPartition.from_pydict({"input": input}) daft_table = daft_table.eval_expression_list([expr]) pydict = daft_table.to_pydict() @@ -56,7 +56,7 @@ def test_table_expr_not(input, expr, expected) -> None: def test_table_expr_not_wrong() -> None: - daft_table = Table.from_pydict({"input": [None, 0, 1]}) + daft_table = MicroPartition.from_pydict({"input": [None, 0, 1]}) with pytest.raises(ValueError): daft_table = daft_table.eval_expression_list([~col("input")]) @@ -73,7 +73,7 @@ def test_table_expr_not_wrong() -> None: ) def test_table_expr_is_null(input, expected) -> None: """Test logical not expression.""" - daft_table = Table.from_pydict({"input": input}) + daft_table = MicroPartition.from_pydict({"input": input}) daft_table = daft_table.eval_expression_list([col("input").is_null()]) pydict = daft_table.to_pydict() @@ -89,7 +89,7 @@ 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 = Table.from_arrow(pa_table) + daft_table = MicroPartition.from_arrow(pa_table) daft_table = daft_table.eval_expression_list( [op(col("a").cast(data_dtype), col("b").cast(data_dtype)).alias("result")] ) @@ -105,7 +105,7 @@ 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 = Table.from_arrow(pa_table) + daft_table = MicroPartition.from_arrow(pa_table) daft_table = daft_table.eval_expression_list( [op(col("a").cast(data_dtype), col("b").cast(data_dtype)).alias("result")] ) @@ -119,7 +119,7 @@ def test_table_numeric_expressions_with_nulls(data_dtype, op) -> None: def test_table_numeric_abs() -> None: - table = Table.from_pydict({"a": [None, -1.0, 0, 2, 3, None], "b": [-1, -2, 3, 4, None, None]}) + table = MicroPartition.from_pydict({"a": [None, -1.0, 0, 2, 3, None], "b": [-1, -2, 3, 4, None, None]}) abs_table = table.eval_expression_list([abs(col("a")), col("b").abs()]) @@ -132,7 +132,7 @@ def test_table_numeric_abs() -> None: def test_table_abs_bad_input() -> None: - table = Table.from_pydict({"a": ["a", "b", "c"]}) + table = MicroPartition.from_pydict({"a": ["a", "b", "c"]}) with pytest.raises(ValueError, match="Expected input to abs to be numeric"): table.eval_expression_list([abs(col("a"))]) diff --git a/tests/table/test_explodes.py b/tests/table/test_explodes.py index 69eea2e77a..9565c6cc1f 100644 --- a/tests/table/test_explodes.py +++ b/tests/table/test_explodes.py @@ -5,7 +5,7 @@ from daft.expressions import col from daft.series import Series -from daft.table import Table +from daft.table import MicroPartition TEST_DATA = [ Series.from_arrow(pa.array([[1, 2], [3, 4], None, []], type=pa.list_(pa.int64()))), @@ -19,7 +19,7 @@ TEST_DATA, ) def test_explode(data): - table = Table.from_pydict({"nested": data, "sidecar": ["a", "b", "c", "d"]}) + table = MicroPartition.from_pydict({"nested": data, "sidecar": ["a", "b", "c", "d"]}) table = table.explode([col("nested")._explode()]) assert table.column_names() == ["nested", "sidecar"] assert table.to_pydict() == {"nested": [1, 2, 3, 4, None, None], "sidecar": ["a", "a", "b", "b", "c", "d"]} @@ -30,7 +30,7 @@ def test_explode(data): TEST_DATA, ) def test_explode_flipped(data): - table = Table.from_pydict({"sidecar": ["a", "b", "c", "d"], "nested": data}) + table = MicroPartition.from_pydict({"sidecar": ["a", "b", "c", "d"], "nested": data}) table = table.explode([col("nested")._explode()]) assert table.column_names() == ["sidecar", "nested"] assert table.to_pydict() == {"nested": [1, 2, 3, 4, None, None], "sidecar": ["a", "a", "b", "b", "c", "d"]} @@ -41,7 +41,7 @@ def test_explode_flipped(data): TEST_DATA, ) def test_explode_multiple_cols(data): - table = Table.from_pydict({"nested": data, "nested2": data, "sidecar": ["a", "b", "c", "d"]}) + table = MicroPartition.from_pydict({"nested": data, "nested2": data, "sidecar": ["a", "b", "c", "d"]}) table = table.explode([col("nested")._explode(), col("nested2")._explode()]) assert table.column_names() == ["nested", "nested2", "sidecar"] assert table.to_pydict() == { @@ -54,7 +54,7 @@ def test_explode_multiple_cols(data): def test_explode_multiple_cols_mixed_types(): data1 = pa.array([[1, 2], [3, 4], None, None], type=pa.list_(pa.int64())) data2 = pa.array([[1, 2], [3, 4], None, None], type=pa.list_(pa.int64(), list_size=2)) - table = Table.from_pydict({"nested": data1, "nested2": data2, "sidecar": ["a", "b", "c", "d"]}) + table = MicroPartition.from_pydict({"nested": data1, "nested2": data2, "sidecar": ["a", "b", "c", "d"]}) table = table.explode([col("nested")._explode(), col("nested2")._explode()]) assert table.to_pydict() == { "nested": [1, 2, 3, 4, None, None], @@ -64,7 +64,7 @@ def test_explode_multiple_cols_mixed_types(): def test_explode_bad_multiple_cols(): - table = Table.from_pydict( + table = MicroPartition.from_pydict( { "nested": [[1, 2, 3], [4], None, None], "nested2": [[1, 2], [3, 4], None, None], @@ -80,7 +80,7 @@ def test_explode_bad_multiple_cols(): TEST_DATA, ) def test_explode_multiple_cols_with_alias(data): - table = Table.from_pydict({"nested": data, "nested2": data, "sidecar": ["a", "b", "c", "d"]}) + table = MicroPartition.from_pydict({"nested": data, "nested2": data, "sidecar": ["a", "b", "c", "d"]}) table = table.explode([col("nested").alias("nested3")._explode(), col("nested2")._explode()]) assert table.column_names() == ["nested", "nested2", "sidecar", "nested3"] data_py = data.to_pylist() @@ -97,12 +97,12 @@ def test_explode_multiple_cols_with_alias(data): TEST_DATA, ) def test_explode_eval_expr(data): - table = Table.from_pydict({"nested": data}) + table = MicroPartition.from_pydict({"nested": data}) table = table.eval_expression_list([col("nested")._explode()]) assert table.to_pydict() == {"nested": [1, 2, 3, 4, None, None]} def test_explode_bad_col_type(): - table = Table.from_pydict({"a": [1, 2, 3]}) + table = MicroPartition.from_pydict({"a": [1, 2, 3]}) with pytest.raises(ValueError, match="to be a List Type, but is"): table = table.explode([col("a")._explode()]) diff --git a/tests/table/test_filter.py b/tests/table/test_filter.py index a5724e9164..afaf1afb60 100644 --- a/tests/table/test_filter.py +++ b/tests/table/test_filter.py @@ -5,12 +5,12 @@ import pytest from daft import col, lit -from daft.table import Table +from daft.table 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 = Table.from_arrow(pa_table) + daft_table = MicroPartition.from_arrow(pa_table) assert len(daft_table) == 4 assert daft_table.column_names() == ["a", "b"] @@ -33,7 +33,7 @@ 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 = Table.from_arrow(pa_table) + daft_table = MicroPartition.from_arrow(pa_table) assert len(daft_table) == 4 assert daft_table.column_names() == ["a", "b"] @@ -56,7 +56,7 @@ 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 = Table.from_arrow(pa_table) + daft_table = MicroPartition.from_arrow(pa_table) assert len(daft_table) == 4 assert daft_table.column_names() == ["a", "b"] @@ -79,7 +79,7 @@ def test_table_filter_none_pass() -> 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 = Table.from_arrow(pa_table) + daft_table = MicroPartition.from_arrow(pa_table) assert len(daft_table) == 4 assert daft_table.column_names() == ["a", "b"] @@ -99,7 +99,7 @@ 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 = Table.from_arrow(pa_table) + daft_table = MicroPartition.from_arrow(pa_table) assert len(daft_table) == 6 assert daft_table.column_names() == ["days", "enum"] @@ -122,7 +122,7 @@ 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 = Table.from_arrow(pa_table) + daft_table = MicroPartition.from_arrow(pa_table) assert len(daft_table) == 6 assert daft_table.column_names() == ["days", "enum"] @@ -145,7 +145,7 @@ 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 = Table.from_arrow(pa_table) + daft_table = MicroPartition.from_arrow(pa_table) assert len(daft_table) == 6 assert daft_table.column_names() == ["days", "enum"] @@ -168,7 +168,7 @@ 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 = Table.from_arrow(pa_table) + daft_table = MicroPartition.from_arrow(pa_table) assert len(daft_table) == 6 assert daft_table.column_names() == ["days", "enum"] @@ -192,7 +192,7 @@ 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 = Table.from_arrow(pa_table) + daft_table = MicroPartition.from_arrow(pa_table) assert len(daft_table) == 6 assert daft_table.column_names() == ["days", "enum"] @@ -206,13 +206,15 @@ def date_maker(d): def test_table_float_is_nan() -> None: - table = Table.from_pydict({"a": [1.0, np.nan, 3.0, None, float("nan")]}) + table = MicroPartition.from_pydict({"a": [1.0, np.nan, 3.0, None, float("nan")]}) result_table = table.eval_expression_list([col("a").float.is_nan()]) # Note that null entries are _not_ treated as float NaNs. assert result_table.to_pydict() == {"a": [False, True, False, None, True]} def test_table_if_else() -> None: - table = Table.from_arrow(pa.Table.from_pydict({"ones": [1, 1, 1], "zeros": [0, 0, 0], "pred": [True, False, None]})) + table = MicroPartition.from_arrow( + pa.Table.from_pydict({"ones": [1, 1, 1], "zeros": [0, 0, 0], "pred": [True, False, None]}) + ) result_table = table.eval_expression_list([col("pred").if_else(col("ones"), col("zeros"))]) assert result_table.to_pydict() == {"ones": [1, 0, None]} diff --git a/tests/table/test_from_py.py b/tests/table/test_from_py.py index 265de512a6..936d3d5607 100644 --- a/tests/table/test_from_py.py +++ b/tests/table/test_from_py.py @@ -12,7 +12,7 @@ from daft import DataType, TimeUnit from daft.context import get_context from daft.series import Series -from daft.table import Table +from daft.table import MicroPartition from daft.utils import pyarrow_supports_fixed_shape_tensor ARROW_VERSION = tuple(int(s) for s in pa.__version__.split(".") if s.isnumeric()) @@ -160,7 +160,7 @@ def _with_uuid_ext_type(uuid_ext_type) -> tuple[dict, dict]: def test_from_pydict_roundtrip() -> None: - table = Table.from_pydict(PYTHON_TYPE_ARRAYS) + table = MicroPartition.from_pydict(PYTHON_TYPE_ARRAYS) assert len(table) == 2 assert set(table.column_names()) == set(PYTHON_TYPE_ARRAYS.keys()) for field in table.schema(): @@ -178,7 +178,7 @@ def test_from_pydict_roundtrip() -> None: def test_from_pydict_arrow_roundtrip(uuid_ext_type) -> None: arrow_roundtrip_types, arrow_type_arrays = _with_uuid_ext_type(uuid_ext_type) - table = Table.from_pydict(arrow_type_arrays) + table = MicroPartition.from_pydict(arrow_type_arrays) assert len(table) == 2 assert set(table.column_names()) == set(arrow_type_arrays.keys()) for field in table.schema(): @@ -190,7 +190,7 @@ def test_from_pydict_arrow_roundtrip(uuid_ext_type) -> None: def test_from_arrow_roundtrip(uuid_ext_type) -> None: arrow_roundtrip_types, arrow_type_arrays = _with_uuid_ext_type(uuid_ext_type) pa_table = pa.table(arrow_type_arrays) - table = Table.from_arrow(pa_table) + table = MicroPartition.from_arrow(pa_table) assert len(table) == 2 assert set(table.column_names()) == set(arrow_type_arrays.keys()) for field in table.schema(): @@ -201,7 +201,7 @@ def test_from_arrow_roundtrip(uuid_ext_type) -> None: def test_from_pandas_roundtrip() -> None: df = pd.DataFrame(PYTHON_TYPE_ARRAYS) - table = Table.from_pandas(df) + table = MicroPartition.from_pandas(df) assert len(table) == 2 assert set(table.column_names()) == set(PYTHON_TYPE_ARRAYS.keys()) for field in table.schema(): @@ -214,19 +214,19 @@ def test_from_pandas_roundtrip() -> None: def test_from_pydict_list() -> None: - daft_table = Table.from_pydict({"a": [1, 2, 3]}) + 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()) def test_from_pydict_np() -> None: - daft_table = Table.from_pydict({"a": np.array([1, 2, 3], dtype=np.int64)}) + 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()) def test_from_pydict_arrow() -> None: - daft_table = Table.from_pydict({"a": pa.array([1, 2, 3], type=pa.int8())}) + 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()) @@ -234,7 +234,7 @@ def test_from_pydict_arrow() -> None: @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 = Table.from_pydict({"a": arrow_arr}) + daft_table = MicroPartition.from_pydict({"a": arrow_arr}) assert "a" in daft_table.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 @@ -246,7 +246,7 @@ def test_from_pydict_arrow_list_array(list_type) -> None: 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 = Table.from_pydict({"a": arrow_arr}) + daft_table = MicroPartition.from_pydict({"a": arrow_arr}) assert "a" in daft_table.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)) @@ -256,7 +256,7 @@ def test_from_pydict_arrow_fixed_size_list_array() -> None: def test_from_pydict_arrow_struct_array() -> None: data = [{"a": "foo", "b": "bar"}, {"b": "baz", "c": "quux"}] arrow_arr = pa.array(data) - daft_table = Table.from_pydict({"a": arrow_arr}) + daft_table = MicroPartition.from_pydict({"a": arrow_arr}) assert "a" in daft_table.column_names() # Perform expected Daft cast, where the inner string array is cast to a large string array. expected = pa.array( @@ -274,7 +274,7 @@ 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 = Table.from_pydict({"a": arrow_arr}) + daft_table = MicroPartition.from_pydict({"a": arrow_arr}) assert "a" in daft_table.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. @@ -287,7 +287,7 @@ 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 = Table.from_pydict({"a": arrow_arr}) + daft_table = MicroPartition.from_pydict({"a": arrow_arr}) assert "a" in daft_table.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. @@ -316,7 +316,7 @@ 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 = Table.from_pydict({"a": data}) + daft_table = MicroPartition.from_pydict({"a": data}) assert "a" in daft_table.column_names() if chunked: data = data.combine_chunks() @@ -356,7 +356,7 @@ 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 = Table.from_pydict({"a": sliced_data}) + daft_table = MicroPartition.from_pydict({"a": sliced_data}) assert "a" in daft_table.column_names() if chunked: sliced_data = sliced_data.combine_chunks() @@ -364,7 +364,7 @@ def test_from_pydict_arrow_sliced_roundtrip(data, out_dtype, chunked, slice_) -> def test_from_pydict_series() -> None: - daft_table = Table.from_pydict({"a": Series.from_arrow(pa.array([1, 2, 3], type=pa.int8()))}) + 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()) @@ -399,7 +399,7 @@ 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 = Table.from_arrow(pa.table({"a": sliced_data})) + 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) @@ -407,7 +407,7 @@ def test_from_arrow_sliced_roundtrip(data, out_dtype, slice_) -> None: @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 = Table.from_arrow(pa.table({"a": arrow_arr})) + daft_table = MicroPartition.from_arrow(pa.table({"a": arrow_arr})) assert "a" in daft_table.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 @@ -419,7 +419,7 @@ def test_from_arrow_list_array(list_type) -> None: 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 = Table.from_arrow(pa.table({"a": arrow_arr})) + daft_table = MicroPartition.from_arrow(pa.table({"a": arrow_arr})) assert "a" in daft_table.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)) @@ -429,7 +429,7 @@ def test_from_arrow_fixed_size_list_array() -> None: def test_from_arrow_struct_array() -> None: data = [{"a": "foo", "b": "bar"}, {"b": "baz", "c": "quux"}] arrow_arr = pa.array(data) - daft_table = Table.from_arrow(pa.table({"a": arrow_arr})) + daft_table = MicroPartition.from_arrow(pa.table({"a": arrow_arr})) assert "a" in daft_table.column_names() # Perform expected Daft cast, where the inner string array is cast to a large string array. expected = pa.array( @@ -447,7 +447,7 @@ 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 = Table.from_arrow(pa.table({"a": arrow_arr})) + daft_table = MicroPartition.from_arrow(pa.table({"a": arrow_arr})) assert "a" in daft_table.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. @@ -460,7 +460,7 @@ 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 = Table.from_arrow(pa.table({"a": arrow_arr})) + daft_table = MicroPartition.from_arrow(pa.table({"a": arrow_arr})) assert "a" in daft_table.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. @@ -483,12 +483,12 @@ def test_from_arrow_deeply_nested() -> None: def test_from_pydict_bad_input() -> None: with pytest.raises(ValueError, match="Mismatch in Series lengths"): - Table.from_pydict({"a": [1, 2, 3, 4], "b": [5, 6, 7]}) + MicroPartition.from_pydict({"a": [1, 2, 3, 4], "b": [5, 6, 7]}) def test_pyobjects_roundtrip() -> None: o0, o1 = object(), object() - table = Table.from_pydict({"objs": [o0, o1, None]}) + table = MicroPartition.from_pydict({"objs": [o0, o1, None]}) objs = table.to_pydict()["objs"] assert objs[0] is o0 assert objs[1] is o1 @@ -500,7 +500,7 @@ def test_nested_list_dates(levels: int) -> None: data = [datetime.date.today(), datetime.date.today()] for _ in range(levels): data = [data, data] - table = Table.from_pydict({"item": data}) + table = MicroPartition.from_pydict({"item": data}) back_again = table.get_column("item") dtype = back_again.datatype() @@ -532,7 +532,7 @@ def test_nested_fixed_size_list_dates(levels: int) -> None: expected_arrow_type = pa.list_(expected_arrow_type, 2) pa_data = pa.array(data, type=expected_arrow_type) - table = Table.from_pydict({"data": pa_data}) + table = MicroPartition.from_pydict({"data": pa_data}) back_again = table.get_column("data") dtype = back_again.datatype() @@ -549,7 +549,7 @@ def test_nested_struct_dates(levels: int) -> None: data = {"data": data} data = [data] - table = Table.from_pydict({"data": data}) + table = MicroPartition.from_pydict({"data": data}) back_again = table.get_column("data") dtype = back_again.datatype() expected_dtype = DataType.date() diff --git a/tests/table/test_head.py b/tests/table/test_head.py index b9acbc9912..883cb6aaf6 100644 --- a/tests/table/test_head.py +++ b/tests/table/test_head.py @@ -3,12 +3,12 @@ import pyarrow as pa import pytest -from daft.table import Table +from daft.table 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 = Table.from_arrow(pa_table) + daft_table = MicroPartition.from_arrow(pa_table) assert len(daft_table) == 4 assert daft_table.column_names() == ["a", "b"] diff --git a/tests/table/test_joins.py b/tests/table/test_joins.py index b12c15eb3c..e770340572 100644 --- a/tests/table/test_joins.py +++ b/tests/table/test_joins.py @@ -9,7 +9,7 @@ from daft.datatype import DataType from daft.expressions import col from daft.series import Series -from daft.table import Table +from daft.table import MicroPartition daft_int_types = [ DataType.int8(), @@ -42,10 +42,10 @@ ) def test_table_join_single_column(dtype, data) -> None: l, r, expected_pairs = data - left_table = Table.from_pydict({"x": l, "x_ind": list(range(len(l)))}).eval_expression_list( + left_table = MicroPartition.from_pydict({"x": l, "x_ind": list(range(len(l)))}).eval_expression_list( [col("x").cast(dtype), col("x_ind")] ) - right_table = Table.from_pydict({"y": r, "y_ind": list(range(len(r)))}) + right_table = MicroPartition.from_pydict({"y": r, "y_ind": list(range(len(r)))}) result_table = left_table.join(right_table, left_on=[col("x")], right_on=[col("y")], how=JoinType.Inner) assert result_table.column_names() == ["x", "x_ind", "y", "y_ind"] @@ -77,8 +77,8 @@ def test_table_join_single_column(dtype, data) -> None: def test_table_join_mismatch_column() -> None: - left_table = Table.from_pydict({"x": [1, 2, 3, 4], "y": [2, 3, 4, 5]}) - right_table = Table.from_pydict({"a": [1, 2, 3, 4], "b": [2, 3, 4, 5]}) + left_table = MicroPartition.from_pydict({"x": [1, 2, 3, 4], "y": [2, 3, 4, 5]}) + right_table = MicroPartition.from_pydict({"a": [1, 2, 3, 4], "b": [2, 3, 4, 5]}) with pytest.raises(ValueError, match="Mismatch of number of join keys"): left_table.join(right_table, left_on=[col("x"), col("y")], right_on=[col("a")]) @@ -100,10 +100,10 @@ def test_table_join_mismatch_column() -> None: ) def test_table_join_multicolumn_empty_result(left, right) -> None: """Various multicol joins that should all produce an empty result.""" - left_table = Table.from_pydict(left).eval_expression_list( + left_table = MicroPartition.from_pydict(left).eval_expression_list( [col("a").cast(DataType.string()), col("b").cast(DataType.int32())] ) - right_table = Table.from_pydict(right).eval_expression_list( + right_table = MicroPartition.from_pydict(right).eval_expression_list( [col("x").cast(DataType.string()), col("y").cast(DataType.int32())] ) @@ -117,14 +117,14 @@ def test_table_join_multicolumn_nocross() -> None: Input has duplicate join values and overlapping single-column values, but there should only be two correct matches, both not cross. """ - left_table = Table.from_pydict( + left_table = MicroPartition.from_pydict( { "a": ["apple", "apple", "banana", "banana", "carrot"], "b": [1, 2, 2, 2, 3], "c": [1, 2, 3, 4, 5], } ) - right_table = Table.from_pydict( + right_table = MicroPartition.from_pydict( { "x": ["banana", "carrot", "apple", "banana", "apple", "durian"], "y": [1, 3, 2, 1, 3, 6], @@ -146,14 +146,14 @@ def test_table_join_multicolumn_nocross() -> None: def test_table_join_multicolumn_cross() -> None: """A multicol join that should produce a cross product and a non-cross product.""" - left_table = Table.from_pydict( + left_table = MicroPartition.from_pydict( { "a": ["apple", "apple", "banana", "banana", "banana"], "b": [1, 0, 1, 1, 1], "c": [1, 2, 3, 4, 5], } ) - right_table = Table.from_pydict( + right_table = MicroPartition.from_pydict( { "x": ["apple", "apple", "banana", "banana", "banana"], "y": [1, 0, 1, 1, 0], @@ -179,14 +179,14 @@ def test_table_join_multicolumn_cross() -> None: def test_table_join_multicolumn_all_nulls() -> None: - left_table = Table.from_pydict( + left_table = MicroPartition.from_pydict( { "a": Series.from_pylist([None, None, None]).cast(DataType.int64()), "b": Series.from_pylist([None, None, None]).cast(DataType.string()), "c": [1, 2, 3], } ) - right_table = Table.from_pydict( + right_table = MicroPartition.from_pydict( { "x": Series.from_pylist([None, None, None]).cast(DataType.int64()), "y": Series.from_pylist([None, None, None]).cast(DataType.string()), @@ -199,16 +199,16 @@ def test_table_join_multicolumn_all_nulls() -> None: def test_table_join_no_columns() -> None: - left_table = Table.from_pydict({"x": [1, 2, 3, 4], "y": [2, 3, 4, 5]}) - right_table = Table.from_pydict({"a": [1, 2, 3, 4], "b": [2, 3, 4, 5]}) + left_table = MicroPartition.from_pydict({"x": [1, 2, 3, 4], "y": [2, 3, 4, 5]}) + right_table = MicroPartition.from_pydict({"a": [1, 2, 3, 4], "b": [2, 3, 4, 5]}) with pytest.raises(ValueError, match="No columns were passed in to join on"): left_table.join(right_table, left_on=[], right_on=[]) def test_table_join_single_column_name_conflicts() -> None: - left_table = Table.from_pydict({"x": [0, 1, 2, 3], "y": [2, 3, 4, 5]}) - right_table = Table.from_pydict({"x": [3, 2, 1, 0], "y": [6, 7, 8, 9]}) + left_table = MicroPartition.from_pydict({"x": [0, 1, 2, 3], "y": [2, 3, 4, 5]}) + right_table = MicroPartition.from_pydict({"x": [3, 2, 1, 0], "y": [6, 7, 8, 9]}) result_table = left_table.join(right_table, left_on=[col("x")], right_on=[col("x")]) assert result_table.column_names() == ["x", "y", "right.y"] @@ -219,8 +219,8 @@ def test_table_join_single_column_name_conflicts() -> None: def test_table_join_single_column_name_conflicts_different_named_join() -> None: - left_table = Table.from_pydict({"x": [0, 1, 2, 3], "y": [2, 3, 4, 5]}) - right_table = Table.from_pydict({"y": [3, 2, 1, 0], "x": [6, 7, 8, 9]}) + left_table = MicroPartition.from_pydict({"x": [0, 1, 2, 3], "y": [2, 3, 4, 5]}) + right_table = MicroPartition.from_pydict({"y": [3, 2, 1, 0], "x": [6, 7, 8, 9]}) result_table = left_table.join(right_table, left_on=[col("x")], right_on=[col("y")]) @@ -234,8 +234,8 @@ def test_table_join_single_column_name_conflicts_different_named_join() -> None: def test_table_join_single_column_name_multiple_conflicts() -> None: - left_table = Table.from_pydict({"x": [0, 1, 2, 3], "y": [2, 3, 4, 5], "right.y": [6, 7, 8, 9]}) - right_table = Table.from_pydict({"x": [3, 2, 1, 0], "y": [10, 11, 12, 13]}) + left_table = MicroPartition.from_pydict({"x": [0, 1, 2, 3], "y": [2, 3, 4, 5], "right.y": [6, 7, 8, 9]}) + right_table = MicroPartition.from_pydict({"x": [3, 2, 1, 0], "y": [10, 11, 12, 13]}) result_table = left_table.join(right_table, left_on=[col("x")], right_on=[col("x")]) assert result_table.column_names() == ["x", "y", "right.y", "right.right.y"] @@ -247,8 +247,8 @@ def test_table_join_single_column_name_multiple_conflicts() -> None: def test_table_join_single_column_name_boolean() -> None: - left_table = Table.from_pydict({"x": [False, True, None], "y": [0, 1, 2]}) - right_table = Table.from_pydict({"x": [None, True, False, None], "y": [0, 1, 2, 3]}) + left_table = MicroPartition.from_pydict({"x": [False, True, None], "y": [0, 1, 2]}) + right_table = MicroPartition.from_pydict({"x": [None, True, False, None], "y": [0, 1, 2, 3]}) result_table = left_table.join(right_table, left_on=[col("x")], right_on=[col("x")]) assert result_table.column_names() == ["x", "y", "right.y"] @@ -258,8 +258,8 @@ def test_table_join_single_column_name_boolean() -> None: def test_table_join_single_column_name_null() -> None: - left_table = Table.from_pydict({"x": [None, None, None], "y": [0, 1, 2]}) - right_table = Table.from_pydict({"x": [None, None, None, None], "y": [0, 1, 2, 3]}) + left_table = MicroPartition.from_pydict({"x": [None, None, None], "y": [0, 1, 2]}) + right_table = MicroPartition.from_pydict({"x": [None, None, None, None], "y": [0, 1, 2, 3]}) result_table = left_table.join(right_table, left_on=[col("x")], right_on=[col("x")]) assert result_table.column_names() == ["x", "y", "right.y"] diff --git a/tests/table/test_partitioning.py b/tests/table/test_partitioning.py index be933ce1bc..96a7e93c76 100644 --- a/tests/table/test_partitioning.py +++ b/tests/table/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 LegacyTable, Table +from daft.table import LegacyTable, MicroPartition daft_int_types = [ DataType.int8(), @@ -29,8 +29,8 @@ @pytest.mark.parametrize( "mp", [ - Table.from_pydict({"a": pa.array([], type=pa.int64())}), # 1 empty table - Table.empty(Schema.from_pyarrow_schema(pa.schema({"a": pa.int64()}))), # No tables + MicroPartition.from_pydict({"a": pa.array([], type=pa.int64())}), # 1 empty table + MicroPartition.empty(Schema.from_pyarrow_schema(pa.schema({"a": pa.int64()}))), # No tables ], ) def test_partitioning_micropartitions_hash_empty(mp) -> None: @@ -42,12 +42,12 @@ def test_partitioning_micropartitions_hash_empty(mp) -> None: @pytest.mark.parametrize( "mp", [ - Table.from_pydict({"a": [1, 3, 2, 4]}), # 1 table - Table.concat( + MicroPartition.from_pydict({"a": [1, 3, 2, 4]}), # 1 table + MicroPartition.concat( [ - Table.from_pydict({"a": np.array([]).astype(np.int64)}), - Table.from_pydict({"a": [1]}), - Table.from_pydict({"a": [3, 2, 4]}), + MicroPartition.from_pydict({"a": np.array([]).astype(np.int64)}), + MicroPartition.from_pydict({"a": [1]}), + MicroPartition.from_pydict({"a": [3, 2, 4]}), ] ), # 3 tables ], @@ -62,8 +62,8 @@ def test_partitioning_micropartitions_hash(mp) -> None: @pytest.mark.parametrize( "mp", [ - Table.from_pydict({"a": pa.array([], type=pa.int64())}), # 1 empty table - Table.empty(Schema.from_pyarrow_schema(pa.schema({"a": pa.int64()}))), # No tables + MicroPartition.from_pydict({"a": pa.array([], type=pa.int64())}), # 1 empty table + MicroPartition.empty(Schema.from_pyarrow_schema(pa.schema({"a": pa.int64()}))), # No tables ], ) def test_partitioning_micropartitions_range_empty(mp) -> None: @@ -78,8 +78,10 @@ def test_partitioning_micropartitions_range_empty(mp) -> None: @pytest.mark.parametrize( "mp", [ - Table.from_pydict({"a": pa.array([], type=pa.int64()), "b": pa.array([], type=pa.string())}), # 1 empty table - Table.empty(Schema.from_pyarrow_schema(pa.schema({"a": pa.int64(), "b": pa.string()}))), # No tables + MicroPartition.from_pydict( + {"a": pa.array([], type=pa.int64()), "b": pa.array([], type=pa.string())} + ), # 1 empty table + MicroPartition.empty(Schema.from_pyarrow_schema(pa.schema({"a": pa.int64(), "b": pa.string()}))), # No tables ], ) def test_partitioning_micropartitions_range_boundaries_empty(mp) -> None: @@ -92,12 +94,12 @@ def test_partitioning_micropartitions_range_boundaries_empty(mp) -> None: @pytest.mark.parametrize( "mp", [ - Table.from_pydict({"a": [1, 3, 2, 4]}), # 1 table - Table.concat( + MicroPartition.from_pydict({"a": [1, 3, 2, 4]}), # 1 table + MicroPartition.concat( [ - Table.from_pydict({"a": np.array([]).astype(np.int64)}), - Table.from_pydict({"a": [1]}), - Table.from_pydict({"a": [3, 2, 4]}), + MicroPartition.from_pydict({"a": np.array([]).astype(np.int64)}), + MicroPartition.from_pydict({"a": [1]}), + MicroPartition.from_pydict({"a": [3, 2, 4]}), ] ), # 3 tables ], @@ -115,8 +117,8 @@ def test_partitioning_micropartitions_range(mp) -> None: @pytest.mark.parametrize( "mp", [ - Table.from_pydict({"a": pa.array([], type=pa.int64())}), # 1 empty table - Table.empty(Schema.from_pyarrow_schema(pa.schema({"a": pa.int64()}))), # No tables + MicroPartition.from_pydict({"a": pa.array([], type=pa.int64())}), # 1 empty table + MicroPartition.empty(Schema.from_pyarrow_schema(pa.schema({"a": pa.int64()}))), # No tables ], ) def test_partitioning_micropartitions_random_empty(mp) -> None: @@ -128,12 +130,12 @@ def test_partitioning_micropartitions_random_empty(mp) -> None: @pytest.mark.parametrize( "mp", [ - Table.from_pydict({"a": [1, 3, 2, 4]}), # 1 table - Table.concat( + MicroPartition.from_pydict({"a": [1, 3, 2, 4]}), # 1 table + MicroPartition.concat( [ - Table.from_pydict({"a": np.array([]).astype(np.int64)}), - Table.from_pydict({"a": [1]}), - Table.from_pydict({"a": [3, 2, 4]}), + MicroPartition.from_pydict({"a": np.array([]).astype(np.int64)}), + MicroPartition.from_pydict({"a": [1]}), + MicroPartition.from_pydict({"a": [3, 2, 4]}), ] ), # 3 tables ], @@ -149,7 +151,7 @@ def test_partitioning_micropartitions_random(mp) -> None: "size, k, dtype", itertools.product([0, 1, 10, 33, 100], [1, 2, 3, 10, 40], daft_numeric_types + daft_string_types) ) def test_table_partition_by_hash_single_col(size, k, dtype) -> None: - table = Table.from_pydict( + table = MicroPartition.from_pydict( {"x": [i % k for i in range(size)], "x_ind": [i for i in range(size)]} ).eval_expression_list([col("x").cast(dtype), col("x_ind")]) split_tables = table.partition_by_hash([col("x")], k) @@ -168,7 +170,9 @@ def test_table_partition_by_hash_single_col(size, k, dtype) -> None: "size, k, dtype", itertools.product([0, 1, 10, 33, 100], [1, 2, 3, 10, 40], daft_numeric_types + daft_string_types) ) def test_table_partition_by_hash_two_col(size, k, dtype) -> None: - table = Table.from_pydict({"x": [i for i in range(size)], "x_ind": [i for i in range(size)]}).eval_expression_list( + table = MicroPartition.from_pydict( + {"x": [i for i in range(size)], "x_ind": [i for i in range(size)]} + ).eval_expression_list( [ (col("x").cast(DataType.int8()) % k).cast(dtype), (col("x").cast(DataType.int8()) % (k + 1)).alias("y"), @@ -191,7 +195,7 @@ def test_table_partition_by_hash_two_col(size, k, dtype) -> None: @pytest.mark.parametrize("size, k", itertools.product([0, 1, 10, 33, 100], [1, 2, 3, 10, 40])) def test_table_partition_by_random(size, k) -> None: - table = Table.from_pydict({"x": [i for i in range(size)]}) + table = MicroPartition.from_pydict({"x": [i for i in range(size)]}) split_tables = table.partition_by_random(k, 0) seen_so_far = set() @@ -214,7 +218,7 @@ def test_table_partition_by_random(size, k) -> None: def test_table_partition_by_hash_bad_input() -> None: - table = Table.from_pydict({"x": [1, 2, 3], "b": [0, 1, 2]}) + table = MicroPartition.from_pydict({"x": [1, 2, 3], "b": [0, 1, 2]}) with pytest.raises(ValueError, match="negative number"): table.partition_by_hash([col("x")], -1) @@ -224,7 +228,7 @@ def test_table_partition_by_hash_bad_input() -> None: def test_table_partition_by_random_bad_input() -> None: - table = Table.from_pydict({"x": [1, 2, 3], "b": [0, 1, 2]}) + table = MicroPartition.from_pydict({"x": [1, 2, 3], "b": [0, 1, 2]}) with pytest.raises(ValueError, match="negative number"): table.partition_by_random(10, -1) @@ -238,7 +242,7 @@ def test_table_partition_by_random_bad_input() -> None: @pytest.mark.parametrize("size, k, desc", itertools.product([0, 1, 10, 33, 100], [1, 2, 3, 10, 40], [False, True])) def test_table_partition_by_range_single_column(size, k, desc) -> None: - table = Table.from_pydict({"x": np.arange(size, dtype=np.float64()), "x_ind": list(range(size))}) + table = MicroPartition.from_pydict({"x": np.arange(size, dtype=np.float64()), "x_ind": list(range(size))}) original_boundaries = np.linspace(0, size, k) @@ -274,7 +278,7 @@ def test_table_partition_by_range_multi_column(size, k, desc) -> None: x = np.ones(size) y = np.arange(size, dtype=np.float64()) - table = Table.from_pydict({"x": x, "y": y}) + table = MicroPartition.from_pydict({"x": x, "y": y}) original_boundaries = np.linspace(0, size, k) @@ -305,7 +309,7 @@ def test_table_partition_by_range_multi_column(size, k, desc) -> None: def test_table_partition_by_range_multi_column_string() -> None: - table = Table.from_pydict({"x": ["a", "c", "a", "c"], "y": ["1", "2", "3", "4"]}) + table = MicroPartition.from_pydict({"x": ["a", "c", "a", "c"], "y": ["1", "2", "3", "4"]}) boundaries = LegacyTable.from_pydict({"x": ["b"], "y": ["1"]}) split_tables = table.partition_by_range([col("x"), col("y")], boundaries, [False, False]) assert len(split_tables) == 2 @@ -321,7 +325,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 = Table.from_pydict(data) + table_cls = MicroPartition.from_pydict(data) boundaries = LegacyTable.from_pydict(data) with pytest.raises(ValueError, match="Schema Mismatch"): diff --git a/tests/table/test_size_bytes.py b/tests/table/test_size_bytes.py index fde5088439..f3c27b5a48 100644 --- a/tests/table/test_size_bytes.py +++ b/tests/table/test_size_bytes.py @@ -6,14 +6,14 @@ from daft import DataType, col from daft.logical.schema import Schema -from daft.table import Table +from daft.table import MicroPartition @pytest.mark.parametrize( "mp", [ - Table.from_pydict({"a": pa.array([], type=pa.int64())}), # 1 empty table - Table.empty(Schema.from_pyarrow_schema(pa.schema({"a": pa.int64()}))), # No tables + MicroPartition.from_pydict({"a": pa.array([], type=pa.int64())}), # 1 empty table + MicroPartition.empty(Schema.from_pyarrow_schema(pa.schema({"a": pa.int64()}))), # No tables ], ) def test_micropartitions_size_bytes_empty(mp) -> None: @@ -23,12 +23,12 @@ def test_micropartitions_size_bytes_empty(mp) -> None: @pytest.mark.parametrize( "mp", [ - Table.from_pydict({"a": [1, 3, 2, 4]}), # 1 table - Table.concat( + MicroPartition.from_pydict({"a": [1, 3, 2, 4]}), # 1 table + MicroPartition.concat( [ - Table.from_pydict({"a": np.array([]).astype(np.int64)}), - Table.from_pydict({"a": [1]}), - Table.from_pydict({"a": [3, 2, 4]}), + MicroPartition.from_pydict({"a": np.array([]).astype(np.int64)}), + MicroPartition.from_pydict({"a": [1]}), + MicroPartition.from_pydict({"a": [3, 2, 4]}), ] ), # 3 tables ], @@ -39,9 +39,9 @@ def test_micropartitions_size_bytes(mp) -> None: def test_table_size_bytes() -> None: - data = Table.from_pydict({"a": [1, 2, 3, 4, None], "b": [False, True, False, True, None]}).eval_expression_list( - [col("a").cast(DataType.int64()), col("b")] - ) + data = MicroPartition.from_pydict( + {"a": [1, 2, 3, 4, None], "b": [False, True, False, True, None]} + ).eval_expression_list([col("a").cast(DataType.int64()), col("b")]) assert data.size_bytes() == (5 * 8 + 1) + (1 + 1) @@ -50,5 +50,5 @@ def test_table_size_bytes_pyobj(length) -> None: import pickle size_per_obj = len(pickle.dumps(object())) - data = Table.from_pydict({"a": [object()] * length}) + data = MicroPartition.from_pydict({"a": [object()] * length}) assert data.size_bytes() == size_per_obj * length diff --git a/tests/table/test_sorting.py b/tests/table/test_sorting.py index 8b4a53444f..b7078213fc 100644 --- a/tests/table/test_sorting.py +++ b/tests/table/test_sorting.py @@ -10,15 +10,15 @@ from daft.datatype import DataType from daft.logical.schema import Schema from daft.series import Series -from daft.table import Table +from daft.table import MicroPartition from tests.table import daft_numeric_types, daft_string_types @pytest.mark.parametrize( "mp", [ - Table.from_pydict({"a": pa.array([], type=pa.int64())}), # 1 empty table - Table.empty(Schema.from_pyarrow_schema(pa.schema({"a": pa.int64()}))), # No tables + MicroPartition.from_pydict({"a": pa.array([], type=pa.int64())}), # 1 empty table + MicroPartition.empty(Schema.from_pyarrow_schema(pa.schema({"a": pa.int64()}))), # No tables ], ) def test_micropartitions_sort_empty(mp) -> None: @@ -30,12 +30,12 @@ def test_micropartitions_sort_empty(mp) -> None: @pytest.mark.parametrize( "mp", [ - Table.from_pydict({"a": [1, 3, 2, 4]}), # 1 table - Table.concat( + MicroPartition.from_pydict({"a": [1, 3, 2, 4]}), # 1 table + MicroPartition.concat( [ - Table.from_pydict({"a": np.array([]).astype(np.int64)}), - Table.from_pydict({"a": [1]}), - Table.from_pydict({"a": [3, 2, 4]}), + MicroPartition.from_pydict({"a": np.array([]).astype(np.int64)}), + MicroPartition.from_pydict({"a": [1]}), + MicroPartition.from_pydict({"a": [3, 2, 4]}), ] ), # 3 tables ], @@ -56,7 +56,7 @@ 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 = Table.from_arrow(pa_table) + daft_table = 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)]) @@ -128,7 +128,7 @@ def test_table_multiple_col_sorting(sort_dtype, value_dtype, data) -> None: argsort_order = Series.from_pylist(expected) - daft_table = Table.from_arrow(pa_table) + daft_table = MicroPartition.from_arrow(pa_table) daft_table = daft_table.eval_expression_list([col("a").cast(sort_dtype), col("b").cast(value_dtype)]) @@ -193,7 +193,7 @@ def test_table_multiple_col_sorting_binary(data) -> None: argsort_order = Series.from_pylist(expected) - daft_table = Table.from_arrow(pa_table) + daft_table = MicroPartition.from_arrow(pa_table) daft_table = daft_table.eval_expression_list([col("a").cast(DataType.binary()), col("b").cast(DataType.binary())]) assert len(daft_table) == 5 assert daft_table.column_names() == ["a", "b"] @@ -253,7 +253,7 @@ 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 = Table.from_arrow(pa_table) + daft_table = MicroPartition.from_arrow(pa_table) daft_table = daft_table.eval_expression_list([col("a"), col("b").cast(second_dtype)]) @@ -300,7 +300,7 @@ 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 = Table.from_arrow(pa_table) + daft_table = MicroPartition.from_arrow(pa_table) assert len(daft_table) == 4 assert daft_table.column_names() == ["a", "b"] @@ -331,7 +331,7 @@ def test_table_quantiles(size, k) -> None: second = 2 * first - daft_table = Table.from_pydict({"a": first, "b": second}) + daft_table = MicroPartition.from_pydict({"a": first, "b": second}) assert len(daft_table) == size assert daft_table.column_names() == ["a", "b"] @@ -363,14 +363,14 @@ def test_table_quantiles_bad_input() -> None: pa_table = pa.Table.from_pydict({"a": first, "b": second}) - daft_table = Table.from_arrow(pa_table) + daft_table = MicroPartition.from_arrow(pa_table) with pytest.raises(ValueError, match="negative number"): daft_table.quantiles(-1) def test_string_table_sorting(): - daft_table = Table.from_pydict( + daft_table = MicroPartition.from_pydict( { "firstname": [ "bob", diff --git a/tests/table/test_table_aggs.py b/tests/table/test_table_aggs.py index 41c60c4ff3..b10bd28534 100644 --- a/tests/table/test_table_aggs.py +++ b/tests/table/test_table_aggs.py @@ -10,7 +10,7 @@ from daft import DataType, col, utils from daft.logical.schema import Schema from daft.series import Series -from daft.table import Table +from daft.table import MicroPartition from tests.table import ( daft_comparable_types, daft_floating_types, @@ -32,8 +32,10 @@ @pytest.mark.parametrize( "mp", [ - Table.from_pydict({"a": pa.array([], type=pa.int64()), "b": pa.array([], type=pa.string())}), # 1 empty table - Table.empty(Schema.from_pyarrow_schema(pa.schema({"a": pa.int64(), "b": pa.string()}))), # No tables + MicroPartition.from_pydict( + {"a": pa.array([], type=pa.int64()), "b": pa.array([], type=pa.string())} + ), # 1 empty table + MicroPartition.empty(Schema.from_pyarrow_schema(pa.schema({"a": pa.int64(), "b": pa.string()}))), # No tables ], ) def test_multipartition_count_empty(mp): @@ -49,12 +51,12 @@ def test_multipartition_count_empty(mp): @pytest.mark.parametrize( "mp", [ - Table.from_pydict({"a": [1, None, 3, None], "b": ["a", "a", "b", "b"]}), # 1 table - Table.concat( + MicroPartition.from_pydict({"a": [1, None, 3, None], "b": ["a", "a", "b", "b"]}), # 1 table + MicroPartition.concat( [ - Table.from_pydict({"a": np.array([]).astype(np.int64), "b": pa.array([], type=pa.string())}), - Table.from_pydict({"a": [1], "b": ["a"]}), - Table.from_pydict({"a": [None, 3, None], "b": ["a", "b", "b"]}), + MicroPartition.from_pydict({"a": np.array([]).astype(np.int64), "b": pa.array([], type=pa.string())}), + MicroPartition.from_pydict({"a": [1], "b": ["a"]}), + MicroPartition.from_pydict({"a": [None, 3, None], "b": ["a", "b", "b"]}), ] ), # 3 tables ], @@ -75,7 +77,7 @@ 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 = Table.from_pydict({"input": 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()]) @@ -85,7 +87,7 @@ def test_table_count(idx_dtype, case) -> None: @pytest.mark.parametrize("length", [0, 1, 10]) def test_table_count_nulltype(length) -> None: - daft_table = Table.from_pydict({"input": [None] * length}) + 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()]) @@ -94,7 +96,7 @@ def test_table_count_nulltype(length) -> None: def test_table_count_pyobject() -> None: - daft_table = Table.from_pydict({"objs": [object(), object(), None, object(), None]}) + daft_table = MicroPartition.from_pydict({"objs": [object(), object(), None, object(), None]}) daft_table = daft_table.eval_expression_list([col("objs").alias("count")._count()]) res = daft_table.to_pydict()["count"] @@ -117,7 +119,7 @@ def test_table_count_pyobject() -> None: ) def test_table_minmax_numerics(idx_dtype, case) -> None: input, expected = case - daft_table = Table.from_pydict({"input": 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( [ @@ -144,7 +146,7 @@ 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 = Table.from_pydict({"input": 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( [ @@ -169,7 +171,7 @@ 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 = Table.from_pydict({"input": input}) + 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( [ @@ -196,7 +198,7 @@ def test_table_minmax_bool(case) -> 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 = Table.from_pydict({"input": 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( [ @@ -212,7 +214,7 @@ def test_table_sum_mean(idx_dtype, case) -> None: @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 = Table.from_pydict( + daft_table = 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), @@ -225,7 +227,7 @@ def test_table_sum_upcast(nptype) -> None: def test_table_sum_badtype() -> None: - daft_table = Table.from_pydict({"a": ["str1", "str2"]}) + daft_table = MicroPartition.from_pydict({"a": ["str1", "str2"]}) with pytest.raises(ValueError): daft_table = daft_table.eval_expression_list([col("a")._sum()]) @@ -261,7 +263,7 @@ def test_table_sum_badtype() -> None: def test_table_agg_global(case) -> None: """Test that global aggregation works at the API layer.""" input, expected = case - daft_table = Table.from_pydict({"input": input}) + daft_table = MicroPartition.from_pydict({"input": input}) daft_table = daft_table.agg( [ col("input").cast(DataType.int32()).alias("count")._count(), @@ -287,7 +289,7 @@ def test_table_agg_global(case) -> None: ) def test_table_agg_groupby_empty(groups_and_aggs) -> None: groups, aggs = groups_and_aggs - daft_table = Table.from_pydict({"col_A": [], "col_B": []}) + daft_table = MicroPartition.from_pydict({"col_A": [], "col_B": []}) daft_table = daft_table.agg( [col(a)._count() for a in aggs], [col(g).cast(DataType.int32()) for g in groups], @@ -350,7 +352,7 @@ def test_table_agg_groupby(case) -> None: ("Alice", None), ("Alice", None), ] - daft_table = Table.from_pydict( + daft_table = MicroPartition.from_pydict( { "name": [_[0] for _ in values], "cookies": [_[1] for _ in values], @@ -367,7 +369,7 @@ def test_table_agg_groupby(case) -> None: @pytest.mark.parametrize("dtype", daft_comparable_types, ids=[f"{_}" for _ in daft_comparable_types]) def test_groupby_all_nulls(dtype) -> None: - daft_table = Table.from_pydict( + daft_table = MicroPartition.from_pydict( { "group": Series.from_pylist([None, None, None]).cast(dtype), "cookies": [1, 2, 3], @@ -383,14 +385,14 @@ 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 = Table.from_pydict( + daft_table = 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")]) - expected_table = Table.from_pydict( + expected_table = MicroPartition.from_pydict( { "group": Series.from_pylist([1, None]).cast(dtype), "cookies": [4, 3], @@ -408,14 +410,14 @@ 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 = Table.from_pydict( + daft_table = 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")]) - expected_table = Table.from_pydict( + expected_table = MicroPartition.from_pydict( { "group": Series.from_pylist([0, 1]).cast(dtype), "cookies": [5, 3], @@ -428,14 +430,14 @@ def test_groupby_numeric_string_bool_no_nulls(dtype) -> None: def test_groupby_binary_bool_some_nulls() -> None: - daft_table = Table.from_pydict( + daft_table = MicroPartition.from_pydict( { "group": Series.from_pylist([b"1", b"1", None]), "cookies": [2, 2, 3], } ) result_table = daft_table.agg([col("cookies")._sum()], group_by=[col("group")]) - expected_table = Table.from_pydict( + expected_table = MicroPartition.from_pydict( { "group": Series.from_pylist([b"1", None]), "cookies": [4, 3], @@ -448,14 +450,14 @@ def test_groupby_binary_bool_some_nulls() -> None: def test_groupby_binary_no_nulls() -> None: - daft_table = Table.from_pydict( + daft_table = MicroPartition.from_pydict( { "group": Series.from_pylist([b"1", b"0", b"1", b"0"]), "cookies": [1, 2, 2, 3], } ) result_table = daft_table.agg([col("cookies")._sum()], group_by=[col("group")]) - expected_table = Table.from_pydict( + expected_table = MicroPartition.from_pydict( { "group": Series.from_pylist([b"0", b"1"]), "cookies": [5, 3], @@ -468,14 +470,14 @@ def test_groupby_binary_no_nulls() -> None: def test_groupby_binary_no_nulls_max() -> None: - daft_table = Table.from_pydict( + daft_table = MicroPartition.from_pydict( { "group": Series.from_pylist([b"1", b"0", b"1", b"0"]), "cookies": [b"1", b"2", b"2", b"3"], } ) result_table = daft_table.agg([col("cookies")._max()], group_by=[col("group")]) - expected_table = Table.from_pydict( + expected_table = MicroPartition.from_pydict( { "group": Series.from_pylist([b"0", b"1"]), "cookies": [b"3", "2"], @@ -496,7 +498,7 @@ def test_groupby_floats_nan(dtype) -> None: NAN = float("nan") INF = float("inf") - daft_table = Table.from_pydict( + daft_table = MicroPartition.from_pydict( { "group": Series.from_pylist([None, 1.0, NAN, 5 * NAN, -1 * NAN, -NAN, 1.0, None, INF, -INF, INF]).cast( dtype @@ -505,7 +507,7 @@ def test_groupby_floats_nan(dtype) -> None: } ) result_table = daft_table.agg([col("cookies")._count()], group_by=[col("group")]) - expected_table = Table.from_pydict( + expected_table = MicroPartition.from_pydict( { "group": Series.from_pylist([None, 1.0, NAN, -INF, INF]).cast(dtype), "cookies": [2, 2, 4, 1, 2], @@ -526,7 +528,7 @@ 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 = Table.from_pydict({"input": 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()]) assert result.get_column("list").datatype() == DataType.list(dtype) @@ -535,7 +537,7 @@ def test_global_list_aggs(dtype) -> None: def test_global_pyobj_list_aggs() -> None: input = [object(), object(), object()] - table = Table.from_pydict({"input": input}) + table = MicroPartition.from_pydict({"input": input}) result = table.eval_expression_list([col("input").alias("list")._agg_list()]) assert result.get_column("list").datatype() == DataType.python() assert result.to_pydict()["list"][0] == input @@ -551,7 +553,7 @@ 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 = Table.from_pydict({"groups": groups, "input": 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")]) assert result.get_column("list").datatype() == DataType.list(dtype) @@ -567,7 +569,7 @@ def test_grouped_pyobj_list_aggs() -> None: input = [None, object(), object(), object(), None, object()] expected_idx = [[1, 3], [4, 5], [0, 2]] - daft_table = Table.from_pydict({"groups": groups, "input": input}) + 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")]) expected_groups = [[input[i] for i in group] for group in expected_idx] assert result.to_pydict() == {"groups": [1, 2, None], "list": expected_groups} @@ -575,7 +577,7 @@ def test_grouped_pyobj_list_aggs() -> None: def test_list_aggs_empty() -> None: - daft_table = Table.from_pydict({"col_A": [], "col_B": []}) + daft_table = MicroPartition.from_pydict({"col_A": [], "col_B": []}) daft_table = daft_table.agg( [col("col_A").cast(DataType.int32()).alias("list")._agg_list()], group_by=[col("col_B")], @@ -600,7 +602,9 @@ def test_global_concat_aggs(dtype, with_null) -> None: if with_null: input += [None] - daft_table = Table.from_pydict({"input": input}).eval_expression_list([col("input").cast(DataType.list(dtype))]) + daft_table = MicroPartition.from_pydict({"input": input}).eval_expression_list( + [col("input").cast(DataType.list(dtype))] + ) concated = daft_table.agg([col("input").alias("concat")._agg_concat()]) assert concated.get_column("concat").datatype() == DataType.list(dtype) @@ -619,7 +623,7 @@ def test_global_concat_aggs_pyobj() -> None: [expected[3], expected[4]], ] - table = Table.from_pydict({"input": input}) + table = MicroPartition.from_pydict({"input": input}) concatted = table.agg([col("input").alias("concat")._agg_concat()]) assert concatted.get_column("concat").datatype() == DataType.python() assert concatted.to_pydict()["concat"] == [expected] @@ -636,7 +640,7 @@ def test_grouped_concat_aggs(dtype) -> None: input = [[x] for x in input] + [None] groups = [1, 2, 3, 4, 5, 6, 7] - daft_table = Table.from_pydict({"groups": groups, "input": input}).eval_expression_list( + daft_table = 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( @@ -662,7 +666,7 @@ def test_grouped_concat_aggs_pyobj() -> None: [None, objects[3]], ] - table = Table.from_pydict({"input": input, "groups": [1, 2, 3, 3, 4]}) + table = MicroPartition.from_pydict({"input": input, "groups": [1, 2, 3, 3, 4]}) concatted = table.agg([col("input").alias("concat")._agg_concat()], group_by=[col("groups")]).sort([col("groups")]) assert concatted.get_column("concat").datatype() == DataType.python() assert concatted.to_pydict() == { @@ -678,7 +682,7 @@ def test_grouped_concat_aggs_pyobj() -> None: def test_concat_aggs_empty() -> None: - daft_table = Table.from_pydict({"col_A": [], "col_B": []}) + daft_table = MicroPartition.from_pydict({"col_A": [], "col_B": []}) daft_table = daft_table.agg( [col("col_A").cast(DataType.list(DataType.int32())).alias("concat")._agg_concat()], group_by=[col("col_B")], diff --git a/tests/table/test_take.py b/tests/table/test_take.py index d5538d64ab..d38f1e7359 100644 --- a/tests/table/test_take.py +++ b/tests/table/test_take.py @@ -8,15 +8,15 @@ from daft import col from daft.logical.schema import Schema from daft.series import Series -from daft.table import Table +from daft.table import MicroPartition from tests.table import daft_int_types, daft_numeric_types @pytest.mark.parametrize( "mp", [ - Table.from_pydict({"a": pa.array([], type=pa.int64())}), # 1 empty table - Table.empty(Schema.from_pyarrow_schema(pa.schema({"a": pa.int64()}))), # No tables + MicroPartition.from_pydict({"a": pa.array([], type=pa.int64())}), # 1 empty table + MicroPartition.empty(Schema.from_pyarrow_schema(pa.schema({"a": pa.int64()}))), # No tables ], ) def test_micropartitions_take_empty(mp) -> None: @@ -37,17 +37,17 @@ def test_micropartitions_take_empty(mp) -> None: @pytest.mark.parametrize( "mp", [ - Table.from_pydict({"a": [1, 2, 3, 4]}), # 1 table - Table.concat( + MicroPartition.from_pydict({"a": [1, 2, 3, 4]}), # 1 table + MicroPartition.concat( [ - Table.from_pydict({"a": pa.array([], type=pa.int64())}), - Table.from_pydict({"a": [1]}), - Table.from_pydict({"a": [2, 3, 4]}), + MicroPartition.from_pydict({"a": pa.array([], type=pa.int64())}), + MicroPartition.from_pydict({"a": [1]}), + MicroPartition.from_pydict({"a": [2, 3, 4]}), ] ), # 3 tables ], ) -def test_micropartitions_take(mp: Table) -> None: +def test_micropartitions_take(mp: MicroPartition) -> None: assert mp.column_names() == ["a"] assert len(mp) == 4 @@ -73,7 +73,7 @@ def test_micropartitions_take(mp: Table) -> 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 = Table.from_arrow(pa_table) + daft_table = MicroPartition.from_arrow(pa_table) daft_table = daft_table.eval_expression_list([col("a").cast(data_dtype), col("b")]) assert len(daft_table) == 4 @@ -107,7 +107,7 @@ 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 = Table.from_arrow(pa_table) + daft_table = MicroPartition.from_arrow(pa_table) assert len(daft_table) == 4 assert daft_table.column_names() == ["a", "b"] @@ -139,7 +139,7 @@ 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 = Table.from_arrow(pa_table) + daft_table = MicroPartition.from_arrow(pa_table) assert len(daft_table) == 4 assert daft_table.column_names() == ["a", "b"] @@ -171,7 +171,7 @@ 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 = Table.from_arrow(pa_table) + daft_table = MicroPartition.from_arrow(pa_table) assert len(daft_table) == 4 assert daft_table.column_names() == ["a", "b"] @@ -186,7 +186,7 @@ def test_table_take_null(idx_dtype) -> None: def test_table_take_pyobject() -> None: objects = [object(), None, object(), object()] - daft_table = Table.from_pydict({"objs": objects}) + daft_table = MicroPartition.from_pydict({"objs": objects}) assert len(daft_table) == 4 assert daft_table.column_names() == ["objs"] @@ -223,7 +223,7 @@ 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 = Table.from_arrow(pa_table) + daft_table = MicroPartition.from_arrow(pa_table) assert len(daft_table) == 4 assert daft_table.column_names() == ["a", "b"] diff --git a/tests/table/utf8/test_compares.py b/tests/table/utf8/test_compares.py index 88567fd29e..9f0ca99145 100644 --- a/tests/table/utf8/test_compares.py +++ b/tests/table/utf8/test_compares.py @@ -3,7 +3,7 @@ import pytest from daft.expressions import col, lit -from daft.table import Table +from daft.table import MicroPartition ENDSWITH_DATA = ["x_foo", "y_foo", "z_bar"] STARTSWITH_DATA = ["foo_x", "foo_y", "bar_z"] @@ -25,6 +25,6 @@ ], ) def test_utf8_substrs(expr, data): - table = Table.from_pydict({"col": data, "emptystrings": ["", "", ""]}) + table = MicroPartition.from_pydict({"col": data, "emptystrings": ["", "", ""]}) result = table.eval_expression_list([expr]) assert result.to_pydict() == {"col": [True, True, False]} diff --git a/tests/table/utf8/test_length.py b/tests/table/utf8/test_length.py index 872eec597e..10e00e065d 100644 --- a/tests/table/utf8/test_length.py +++ b/tests/table/utf8/test_length.py @@ -1,10 +1,10 @@ from __future__ import annotations from daft.expressions import col -from daft.table import Table +from daft.table import MicroPartition def test_utf8_length(): - table = Table.from_pydict({"col": ["foo", None, "barbaz", "quux"]}) + table = MicroPartition.from_pydict({"col": ["foo", None, "barbaz", "quux"]}) result = table.eval_expression_list([col("col").str.length()]) assert result.to_pydict() == {"col": [3, None, 6, 4]} diff --git a/tests/table/utf8/test_split.py b/tests/table/utf8/test_split.py index 0da7735b1d..ac7535c0b7 100644 --- a/tests/table/utf8/test_split.py +++ b/tests/table/utf8/test_split.py @@ -3,7 +3,7 @@ import pytest from daft.expressions import col, lit -from daft.table import Table +from daft.table import MicroPartition @pytest.mark.parametrize( @@ -23,6 +23,6 @@ ], ) def test_series_utf8_split_broadcast_pattern(expr, data, expected) -> None: - table = Table.from_pydict({"col": data, "emptystrings": ["", "", "", ""]}) + table = MicroPartition.from_pydict({"col": data, "emptystrings": ["", "", "", ""]}) result = table.eval_expression_list([expr]) assert result.to_pydict() == {"col": expected} diff --git a/tests/test_schema.py b/tests/test_schema.py index 234df3637c..b58e95d5bf 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 Table +from daft.table import MicroPartition DATA = { "int": ([1, 2, None], DataType.int64()), @@ -17,7 +17,7 @@ "bool": ([True, True, None], DataType.bool()), } -TABLE = Table.from_pydict({k: data for k, (data, _) in DATA.items()}) +TABLE = MicroPartition.from_pydict({k: data for k, (data, _) in DATA.items()}) EXPECTED_TYPES = {k: t for k, (_, t) in DATA.items()} from tests.utils import ANSI_ESCAPE @@ -48,13 +48,13 @@ def test_schema_iter(): def test_schema_eq(): - t1, t2 = Table.from_pydict({k: data for k, (data, _) in DATA.items()}), Table.from_pydict( + t1, t2 = MicroPartition.from_pydict({k: data for k, (data, _) in DATA.items()}), MicroPartition.from_pydict( {k: data for k, (data, _) in DATA.items()} ) s1, s2 = t1.schema(), t2.schema() assert s1 == s2 - t_empty = Table.empty() + t_empty = MicroPartition.empty() assert s1 != t_empty.schema() @@ -94,7 +94,7 @@ def test_union(): schema.union(schema) new_data = {f"{k}_": d for k, (d, _) in DATA.items()} - new_table = Table.from_pydict(new_data) + new_table = MicroPartition.from_pydict(new_data) unioned_schema = schema.union(new_table.schema()) assert unioned_schema.column_names() == list(DATA.keys()) + list(new_data.keys()) @@ -122,7 +122,7 @@ def test_field_pickling(): def test_schema_pickling(): - t1, t2 = Table.from_pydict({k: data for k, (data, _) in DATA.items()}), Table.from_pydict( + t1, t2 = MicroPartition.from_pydict({k: data for k, (data, _) in DATA.items()}), MicroPartition.from_pydict( {k: data for k, (data, _) in DATA.items()} ) @@ -132,7 +132,7 @@ def test_schema_pickling(): assert s1 == s2 - t_empty = Table.empty() + t_empty = MicroPartition.empty() assert s1 != t_empty.schema() t_empty_schema_copy = copy.deepcopy(t_empty.schema()) assert t_empty.schema() == t_empty_schema_copy From 43ed1d7220c681a2c9287a4bd47e0f5debbb8a71 Mon Sep 17 00:00:00 2001 From: Jay Chia Date: Thu, 30 Nov 2023 13:29:59 -0800 Subject: [PATCH 3/3] Rename LegacyTable -> Table --- benchmarking/parquet/conftest.py | 2 +- daft/table/__init__.py | 8 +++----- .../io/parquet/test_reads_public_data.py | 6 +++--- tests/table/test_partitioning.py | 16 ++++++++-------- 4 files changed, 15 insertions(+), 17 deletions(-) diff --git a/benchmarking/parquet/conftest.py b/benchmarking/parquet/conftest.py index bea8b2a99d..978d0cf8a9 100644 --- a/benchmarking/parquet/conftest.py +++ b/benchmarking/parquet/conftest.py @@ -76,7 +76,7 @@ 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.LegacyTable.read_parquet_bulk(paths, columns=columns) + tables = daft.table.Table.read_parquet_bulk(paths, columns=columns) return [t.to_arrow() for t in tables] diff --git a/daft/table/__init__.py b/daft/table/__init__.py index 73a557aba9..5b66e7b616 100644 --- a/daft/table/__init__.py +++ b/daft/table/__init__.py @@ -2,19 +2,17 @@ import os -from .table import Table as _Table -from .table import read_parquet_into_pyarrow, read_parquet_into_pyarrow_bulk +from .table import Table, read_parquet_into_pyarrow, read_parquet_into_pyarrow_bulk # Need to import after `.table` due to circular dep issue otherwise from .micropartition import MicroPartition as _MicroPartition # isort:skip -LegacyTable = _Table MicroPartition = _MicroPartition # Use $DAFT_MICROPARTITIONS envvar as a feature flag to turn off MicroPartitions if os.getenv("DAFT_MICROPARTITIONS", "1") != "1": - MicroPartition = LegacyTable # type: ignore + MicroPartition = Table # type: ignore -__all__ = ["MicroPartition", "LegacyTable", "read_parquet_into_pyarrow", "read_parquet_into_pyarrow_bulk"] +__all__ = ["MicroPartition", "Table", "read_parquet_into_pyarrow", "read_parquet_into_pyarrow_bulk"] diff --git a/tests/integration/io/parquet/test_reads_public_data.py b/tests/integration/io/parquet/test_reads_public_data.py index 0b1dc21071..d7c155857b 100644 --- a/tests/integration/io/parquet/test_reads_public_data.py +++ b/tests/integration/io/parquet/test_reads_public_data.py @@ -8,7 +8,7 @@ import daft from daft.filesystem import get_filesystem, get_protocol_from_path -from daft.table import LegacyTable, MicroPartition +from daft.table import MicroPartition, Table def get_filesystem_from_path(path: str, **kwargs) -> fsspec.AbstractFileSystem: @@ -253,7 +253,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 == LegacyTable: + if MicroPartition == Table: 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()) @@ -337,7 +337,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 == LegacyTable: + if MicroPartition == Table: first, *rest = MicroPartition.read_parquet_bulk( url, io_config=public_storage_io_config, multithreaded_io=multithreaded_io, row_groups_per_path=row_groups ) diff --git a/tests/table/test_partitioning.py b/tests/table/test_partitioning.py index 96a7e93c76..1ca60634cc 100644 --- a/tests/table/test_partitioning.py +++ b/tests/table/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 LegacyTable, MicroPartition +from daft.table import MicroPartition, Table 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 = LegacyTable.from_pydict({"a": np.linspace(0, 10, 3)[1:]}).eval_expression_list( + boundaries = Table.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 = LegacyTable.from_pydict({"a": [], "b": []}).eval_expression_list([col("a").cast(DataType.int64())]) + boundaries = Table.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 = LegacyTable.from_pydict({"a": np.linspace(0, 5, 3)[1:]}).eval_expression_list( + boundaries = Table.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 = LegacyTable.from_pydict({"x": input_boundaries}).eval_expression_list( + boundaries = Table.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 = LegacyTable.from_pydict({"x": np.ones(k - 1), "y": input_boundaries}).eval_expression_list( + boundaries = Table.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 = LegacyTable.from_pydict({"x": ["b"], "y": ["1"]}) + boundaries = Table.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 = LegacyTable.from_pydict(data) + boundaries = Table.from_pydict(data) with pytest.raises(ValueError, match="Schema Mismatch"): table_cls.partition_by_range([col("x")], boundaries, [False])