-
Notifications
You must be signed in to change notification settings - Fork 128
feat: expose DataFrame.write_table #1264
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Open
timsaucer
wants to merge
8
commits into
apache:main
Choose a base branch
from
timsaucer:feat/dataframe_write
base: main
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
+324
−35
Open
Changes from 5 commits
Commits
Show all changes
8 commits
Select commit
Hold shift + click to select a range
819451f
Initial commit for dataframe write_table
timsaucer a8ccbcd
Add dataframe writer options and docstring
timsaucer d416a68
add csv write unit test
timsaucer 6720ee5
add docstrings
timsaucer 0d191f6
more testing around writer options
timsaucer 4b87153
Minor docstring change
timsaucer d92d73e
Format docstring so it renders better
timsaucer 33188a2
whitespace
timsaucer File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -39,10 +39,13 @@ | |
from typing_extensions import deprecated # Python 3.12 | ||
|
||
from datafusion._internal import DataFrame as DataFrameInternal | ||
from datafusion._internal import DataFrameWriteOptions as DataFrameWriteOptionsInternal | ||
from datafusion._internal import InsertOp as InsertOpInternal | ||
from datafusion._internal import ParquetColumnOptions as ParquetColumnOptionsInternal | ||
from datafusion._internal import ParquetWriterOptions as ParquetWriterOptionsInternal | ||
from datafusion.expr import ( | ||
Expr, | ||
SortExpr, | ||
SortKey, | ||
ensure_expr, | ||
ensure_expr_list, | ||
|
@@ -925,21 +928,31 @@ def except_all(self, other: DataFrame) -> DataFrame: | |
""" | ||
return DataFrame(self.df.except_all(other.df)) | ||
|
||
def write_csv(self, path: str | pathlib.Path, with_header: bool = False) -> None: | ||
def write_csv( | ||
self, | ||
path: str | pathlib.Path, | ||
with_header: bool = False, | ||
write_options: DataFrameWriteOptions | None = None, | ||
) -> None: | ||
"""Execute the :py:class:`DataFrame` and write the results to a CSV file. | ||
|
||
Args: | ||
path: Path of the CSV file to write. | ||
with_header: If true, output the CSV header row. | ||
write_options: Options that impact how the DataFrame is written. | ||
""" | ||
self.df.write_csv(str(path), with_header) | ||
raw_write_options = ( | ||
write_options._raw_write_options if write_options is not None else None | ||
) | ||
self.df.write_csv(str(path), with_header, raw_write_options) | ||
|
||
@overload | ||
def write_parquet( | ||
self, | ||
path: str | pathlib.Path, | ||
compression: str, | ||
compression_level: int | None = None, | ||
write_options: DataFrameWriteOptions | None = None, | ||
) -> None: ... | ||
|
||
@overload | ||
|
@@ -948,6 +961,7 @@ def write_parquet( | |
path: str | pathlib.Path, | ||
compression: Compression = Compression.ZSTD, | ||
compression_level: int | None = None, | ||
write_options: DataFrameWriteOptions | None = None, | ||
) -> None: ... | ||
|
||
@overload | ||
|
@@ -956,16 +970,20 @@ def write_parquet( | |
path: str | pathlib.Path, | ||
compression: ParquetWriterOptions, | ||
compression_level: None = None, | ||
write_options: DataFrameWriteOptions | None = None, | ||
) -> None: ... | ||
|
||
def write_parquet( | ||
self, | ||
path: str | pathlib.Path, | ||
compression: Union[str, Compression, ParquetWriterOptions] = Compression.ZSTD, | ||
compression_level: int | None = None, | ||
write_options: DataFrameWriteOptions | None = None, | ||
) -> None: | ||
"""Execute the :py:class:`DataFrame` and write the results to a Parquet file. | ||
|
||
LZO compression is not yet implemented in arrow-rs and is therefore excluded. | ||
|
||
Args: | ||
path: Path of the Parquet file to write. | ||
compression: Compression type to use. Default is "ZSTD". | ||
|
@@ -977,10 +995,10 @@ def write_parquet( | |
- "lz4": LZ4 compression. | ||
- "lz4_raw": LZ4_RAW compression. | ||
- "zstd": Zstandard compression. | ||
Note: LZO is not yet implemented in arrow-rs and is therefore excluded. | ||
compression_level: Compression level to use. For ZSTD, the | ||
recommended range is 1 to 22, with the default being 4. Higher levels | ||
provide better compression but slower speed. | ||
write_options: Options that impact how the DataFrame is written. | ||
""" | ||
if isinstance(compression, ParquetWriterOptions): | ||
if compression_level is not None: | ||
|
@@ -998,10 +1016,21 @@ def write_parquet( | |
): | ||
compression_level = compression.get_default_level() | ||
|
||
self.df.write_parquet(str(path), compression.value, compression_level) | ||
raw_write_options = ( | ||
write_options._raw_write_options if write_options is not None else None | ||
) | ||
self.df.write_parquet( | ||
str(path), | ||
compression.value, | ||
compression_level, | ||
raw_write_options, | ||
) | ||
|
||
def write_parquet_with_options( | ||
self, path: str | pathlib.Path, options: ParquetWriterOptions | ||
self, | ||
path: str | pathlib.Path, | ||
options: ParquetWriterOptions, | ||
write_options: DataFrameWriteOptions | None = None, | ||
) -> None: | ||
"""Execute the :py:class:`DataFrame` and write the results to a Parquet file. | ||
|
||
|
@@ -1010,6 +1039,7 @@ def write_parquet_with_options( | |
Args: | ||
path: Path of the Parquet file to write. | ||
options: Sets the writer parquet options (see `ParquetWriterOptions`). | ||
write_options: Options that impact how the DataFrame is written. | ||
""" | ||
options_internal = ParquetWriterOptionsInternal( | ||
options.data_pagesize_limit, | ||
|
@@ -1046,19 +1076,45 @@ def write_parquet_with_options( | |
bloom_filter_ndv=opts.bloom_filter_ndv, | ||
) | ||
|
||
raw_write_options = ( | ||
write_options._raw_write_options if write_options is not None else None | ||
) | ||
self.df.write_parquet_with_options( | ||
str(path), | ||
options_internal, | ||
column_specific_options_internal, | ||
raw_write_options, | ||
) | ||
|
||
def write_json(self, path: str | pathlib.Path) -> None: | ||
def write_json( | ||
self, | ||
path: str | pathlib.Path, | ||
write_options: DataFrameWriteOptions | None = None, | ||
) -> None: | ||
"""Execute the :py:class:`DataFrame` and write the results to a JSON file. | ||
|
||
Args: | ||
path: Path of the JSON file to write. | ||
write_options: Options that impact how the DataFrame is written. | ||
""" | ||
raw_write_options = ( | ||
write_options._raw_write_options if write_options is not None else None | ||
) | ||
self.df.write_json(str(path), write_options=raw_write_options) | ||
|
||
def write_table( | ||
self, table_name: str, write_options: DataFrameWriteOptions | None = None | ||
) -> None: | ||
"""Execute the :py:class:`DataFrame` and write the results to a table. | ||
|
||
The table must be registered with the session to perform this operation. | ||
Not all table providers support writing operations. See the individual | ||
implementations for details. | ||
""" | ||
self.df.write_json(str(path)) | ||
raw_write_options = ( | ||
write_options._raw_write_options if write_options is not None else None | ||
) | ||
self.df.write_table(table_name, raw_write_options) | ||
|
||
def to_arrow_table(self) -> pa.Table: | ||
"""Execute the :py:class:`DataFrame` and convert it into an Arrow Table. | ||
|
@@ -1206,3 +1262,48 @@ def fill_null(self, value: Any, subset: list[str] | None = None) -> DataFrame: | |
- For columns not in subset, the original column is kept unchanged | ||
""" | ||
return DataFrame(self.df.fill_null(value, subset)) | ||
|
||
|
||
class InsertOp(Enum): | ||
"""Insert operation mode. | ||
|
||
These modes are used by the table writing feature to define how record | ||
batches should be written to a table. | ||
""" | ||
|
||
APPEND = InsertOpInternal.APPEND | ||
"""Appends new rows to the existing table without modifying any existing rows.""" | ||
|
||
REPLACE = InsertOpInternal.REPLACE | ||
"""Replace existing rows that collide with the inserted rows. | ||
|
||
Replacement is typically based on a unique key or primary key. | ||
""" | ||
|
||
OVERWRITE = InsertOpInternal.OVERWRITE | ||
"""Overwrites all existing rows in the table with the new rows.""" | ||
|
||
|
||
class DataFrameWriteOptions: | ||
"""Writer options for DataFrame. | ||
|
||
There is no guarantee the table provider supports all writer options. | ||
See the individual implementation and documentation for details. | ||
""" | ||
|
||
def __init__( | ||
self, | ||
insert_operation: InsertOp | None = None, | ||
single_file_output: bool = False, | ||
partition_by: str | Sequence[str] | None = None, | ||
sort_by: Expr | SortExpr | Sequence[Expr] | Sequence[SortExpr] | None = None, | ||
) -> None: | ||
"""Instantiate writer options for DataFrame.""" | ||
if isinstance(partition_by, str): | ||
partition_by = [partition_by] | ||
|
||
sort_by_raw = sort_list_to_raw_sort_list(sort_by) | ||
|
||
self._raw_write_options = DataFrameWriteOptionsInternal( | ||
insert_operation, single_file_output, partition_by, sort_by_raw | ||
) | ||
Comment on lines
+1310
to
+1312
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think you can't pass insert_operation directly here. eg this test will fail def test_dataframe_write_options_accepts_insert_op() -> None:
"""DataFrameWriteOptions should accept InsertOp enums."""
try:
DataFrameWriteOptions(insert_operation=InsertOp.REPLACE)
except TypeError as exc:
pytest.fail(f"DataFrameWriteOptions rejected InsertOp: {exc}") |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.