3939 from typing_extensions import deprecated # Python 3.12
4040
4141from datafusion ._internal import DataFrame as DataFrameInternal
42+ from datafusion ._internal import DataFrameWriteOptions as DataFrameWriteOptionsInternal
43+ from datafusion ._internal import InsertOp as InsertOpInternal
4244from datafusion ._internal import ParquetColumnOptions as ParquetColumnOptionsInternal
4345from datafusion ._internal import ParquetWriterOptions as ParquetWriterOptionsInternal
4446from datafusion .expr import (
4547 Expr ,
48+ SortExpr ,
4649 SortKey ,
4750 ensure_expr ,
4851 ensure_expr_list ,
@@ -939,21 +942,31 @@ def except_all(self, other: DataFrame) -> DataFrame:
939942 """
940943 return DataFrame (self .df .except_all (other .df ))
941944
942- def write_csv (self , path : str | pathlib .Path , with_header : bool = False ) -> None :
945+ def write_csv (
946+ self ,
947+ path : str | pathlib .Path ,
948+ with_header : bool = False ,
949+ write_options : DataFrameWriteOptions | None = None ,
950+ ) -> None :
943951 """Execute the :py:class:`DataFrame` and write the results to a CSV file.
944952
945953 Args:
946954 path: Path of the CSV file to write.
947955 with_header: If true, output the CSV header row.
956+ write_options: Options that impact how the DataFrame is written.
948957 """
949- self .df .write_csv (str (path ), with_header )
958+ raw_write_options = (
959+ write_options ._raw_write_options if write_options is not None else None
960+ )
961+ self .df .write_csv (str (path ), with_header , raw_write_options )
950962
951963 @overload
952964 def write_parquet (
953965 self ,
954966 path : str | pathlib .Path ,
955967 compression : str ,
956968 compression_level : int | None = None ,
969+ write_options : DataFrameWriteOptions | None = None ,
957970 ) -> None : ...
958971
959972 @overload
@@ -962,6 +975,7 @@ def write_parquet(
962975 path : str | pathlib .Path ,
963976 compression : Compression = Compression .ZSTD ,
964977 compression_level : int | None = None ,
978+ write_options : DataFrameWriteOptions | None = None ,
965979 ) -> None : ...
966980
967981 @overload
@@ -970,31 +984,38 @@ def write_parquet(
970984 path : str | pathlib .Path ,
971985 compression : ParquetWriterOptions ,
972986 compression_level : None = None ,
987+ write_options : DataFrameWriteOptions | None = None ,
973988 ) -> None : ...
974989
975990 def write_parquet (
976991 self ,
977992 path : str | pathlib .Path ,
978993 compression : Union [str , Compression , ParquetWriterOptions ] = Compression .ZSTD ,
979994 compression_level : int | None = None ,
995+ write_options : DataFrameWriteOptions | None = None ,
980996 ) -> None :
981997 """Execute the :py:class:`DataFrame` and write the results to a Parquet file.
982998
999+ Available compression types are:
1000+
1001+ - "uncompressed": No compression.
1002+ - "snappy": Snappy compression.
1003+ - "gzip": Gzip compression.
1004+ - "brotli": Brotli compression.
1005+ - "lz4": LZ4 compression.
1006+ - "lz4_raw": LZ4_RAW compression.
1007+ - "zstd": Zstandard compression.
1008+
1009+ LZO compression is not yet implemented in arrow-rs and is therefore
1010+ excluded.
1011+
9831012 Args:
9841013 path: Path of the Parquet file to write.
9851014 compression: Compression type to use. Default is "ZSTD".
986- Available compression types are:
987- - "uncompressed": No compression.
988- - "snappy": Snappy compression.
989- - "gzip": Gzip compression.
990- - "brotli": Brotli compression.
991- - "lz4": LZ4 compression.
992- - "lz4_raw": LZ4_RAW compression.
993- - "zstd": Zstandard compression.
994- Note: LZO is not yet implemented in arrow-rs and is therefore excluded.
9951015 compression_level: Compression level to use. For ZSTD, the
9961016 recommended range is 1 to 22, with the default being 4. Higher levels
9971017 provide better compression but slower speed.
1018+ write_options: Options that impact how the DataFrame is written.
9981019 """
9991020 if isinstance (compression , ParquetWriterOptions ):
10001021 if compression_level is not None :
@@ -1012,10 +1033,21 @@ def write_parquet(
10121033 ):
10131034 compression_level = compression .get_default_level ()
10141035
1015- self .df .write_parquet (str (path ), compression .value , compression_level )
1036+ raw_write_options = (
1037+ write_options ._raw_write_options if write_options is not None else None
1038+ )
1039+ self .df .write_parquet (
1040+ str (path ),
1041+ compression .value ,
1042+ compression_level ,
1043+ raw_write_options ,
1044+ )
10161045
10171046 def write_parquet_with_options (
1018- self , path : str | pathlib .Path , options : ParquetWriterOptions
1047+ self ,
1048+ path : str | pathlib .Path ,
1049+ options : ParquetWriterOptions ,
1050+ write_options : DataFrameWriteOptions | None = None ,
10191051 ) -> None :
10201052 """Execute the :py:class:`DataFrame` and write the results to a Parquet file.
10211053
@@ -1024,6 +1056,7 @@ def write_parquet_with_options(
10241056 Args:
10251057 path: Path of the Parquet file to write.
10261058 options: Sets the writer parquet options (see `ParquetWriterOptions`).
1059+ write_options: Options that impact how the DataFrame is written.
10271060 """
10281061 options_internal = ParquetWriterOptionsInternal (
10291062 options .data_pagesize_limit ,
@@ -1060,19 +1093,45 @@ def write_parquet_with_options(
10601093 bloom_filter_ndv = opts .bloom_filter_ndv ,
10611094 )
10621095
1096+ raw_write_options = (
1097+ write_options ._raw_write_options if write_options is not None else None
1098+ )
10631099 self .df .write_parquet_with_options (
10641100 str (path ),
10651101 options_internal ,
10661102 column_specific_options_internal ,
1103+ raw_write_options ,
10671104 )
10681105
1069- def write_json (self , path : str | pathlib .Path ) -> None :
1106+ def write_json (
1107+ self ,
1108+ path : str | pathlib .Path ,
1109+ write_options : DataFrameWriteOptions | None = None ,
1110+ ) -> None :
10701111 """Execute the :py:class:`DataFrame` and write the results to a JSON file.
10711112
10721113 Args:
10731114 path: Path of the JSON file to write.
1115+ write_options: Options that impact how the DataFrame is written.
1116+ """
1117+ raw_write_options = (
1118+ write_options ._raw_write_options if write_options is not None else None
1119+ )
1120+ self .df .write_json (str (path ), write_options = raw_write_options )
1121+
1122+ def write_table (
1123+ self , table_name : str , write_options : DataFrameWriteOptions | None = None
1124+ ) -> None :
1125+ """Execute the :py:class:`DataFrame` and write the results to a table.
1126+
1127+ The table must be registered with the session to perform this operation.
1128+ Not all table providers support writing operations. See the individual
1129+ implementations for details.
10741130 """
1075- self .df .write_json (str (path ))
1131+ raw_write_options = (
1132+ write_options ._raw_write_options if write_options is not None else None
1133+ )
1134+ self .df .write_table (table_name , raw_write_options )
10761135
10771136 def to_arrow_table (self ) -> pa .Table :
10781137 """Execute the :py:class:`DataFrame` and convert it into an Arrow Table.
@@ -1220,3 +1279,49 @@ def fill_null(self, value: Any, subset: list[str] | None = None) -> DataFrame:
12201279 - For columns not in subset, the original column is kept unchanged
12211280 """
12221281 return DataFrame (self .df .fill_null (value , subset ))
1282+
1283+
1284+ class InsertOp (Enum ):
1285+ """Insert operation mode.
1286+
1287+ These modes are used by the table writing feature to define how record
1288+ batches should be written to a table.
1289+ """
1290+
1291+ APPEND = InsertOpInternal .APPEND
1292+ """Appends new rows to the existing table without modifying any existing rows."""
1293+
1294+ REPLACE = InsertOpInternal .REPLACE
1295+ """Replace existing rows that collide with the inserted rows.
1296+
1297+ Replacement is typically based on a unique key or primary key.
1298+ """
1299+
1300+ OVERWRITE = InsertOpInternal .OVERWRITE
1301+ """Overwrites all existing rows in the table with the new rows."""
1302+
1303+
1304+ class DataFrameWriteOptions :
1305+ """Writer options for DataFrame.
1306+
1307+ There is no guarantee the table provider supports all writer options.
1308+ See the individual implementation and documentation for details.
1309+ """
1310+
1311+ def __init__ (
1312+ self ,
1313+ insert_operation : InsertOp | None = None ,
1314+ single_file_output : bool = False ,
1315+ partition_by : str | Sequence [str ] | None = None ,
1316+ sort_by : Expr | SortExpr | Sequence [Expr ] | Sequence [SortExpr ] | None = None ,
1317+ ) -> None :
1318+ """Instantiate writer options for DataFrame."""
1319+ if isinstance (partition_by , str ):
1320+ partition_by = [partition_by ]
1321+
1322+ sort_by_raw = sort_list_to_raw_sort_list (sort_by )
1323+ insert_op = insert_operation .value if insert_operation is not None else None
1324+
1325+ self ._raw_write_options = DataFrameWriteOptionsInternal (
1326+ insert_op , single_file_output , partition_by , sort_by_raw
1327+ )
0 commit comments