Skip to content
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

feat: extend write_deltalake to accept Deltalake schema #1879

Closed
wants to merge 3 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
13 changes: 10 additions & 3 deletions python/deltalake/writer.py
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
)
from urllib.parse import unquote

from deltalake import Schema
from deltalake.fs import DeltaStorageHandler

from ._util import encode_partition_value
Expand Down Expand Up @@ -142,7 +143,7 @@ def write_deltalake(
RecordBatchReader,
],
*,
schema: Optional[pa.Schema] = None,
schema: Optional[Union[pa.Schema, Schema]] = None,
partition_by: Optional[Union[List[str], str]] = None,
filesystem: Optional[pa_fs.FileSystem] = None,
mode: Literal["error", "append", "overwrite", "ignore"] = "error",
Expand Down Expand Up @@ -244,6 +245,9 @@ def write_deltalake(
if isinstance(partition_by, str):
partition_by = [partition_by]

if isinstance(schema, Schema):
schema = schema.to_pyarrow()

if isinstance(data, RecordBatchReader):
data = convert_pyarrow_recordbatchreader(data, large_dtypes)
elif isinstance(data, pa.RecordBatch):
Expand Down Expand Up @@ -336,16 +340,19 @@ def _large_to_normal_dtype(dtype: pa.DataType) -> pa.DataType:
return dtype

if partition_by:
table_schema: pa.Schema = schema
if PYARROW_MAJOR_VERSION < 12:
partition_schema = pa.schema(
[
pa.field(name, _large_to_normal_dtype(schema.field(name).type))
pa.field(
name, _large_to_normal_dtype(table_schema.field(name).type)
)
for name in partition_by
]
)
else:
partition_schema = pa.schema(
[schema.field(name) for name in partition_by]
[table_schema.field(name) for name in partition_by]
)
partitioning = ds.partitioning(partition_schema, flavor="hive")
else:
Expand Down
10 changes: 9 additions & 1 deletion python/tests/test_writer.py
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
from pyarrow.dataset import ParquetFileFormat, ParquetReadOptions
from pyarrow.lib import RecordBatchReader

from deltalake import DeltaTable, write_deltalake
from deltalake import DeltaTable, Schema, write_deltalake
from deltalake.exceptions import CommitFailedError, DeltaError, DeltaProtocolError
from deltalake.table import ProtocolVersions
from deltalake.writer import try_get_table_and_table_uri
Expand Down Expand Up @@ -1176,3 +1176,11 @@ def test_float_values(tmp_path: pathlib.Path):
assert actions["min"].field("x2")[0].as_py() is None
assert actions["max"].field("x2")[0].as_py() == 1.0
assert actions["null_count"].field("x2")[0].as_py() == 1


def test_with_deltalake_schema(tmp_path: pathlib.Path, sample_data: pa.Table):
write_deltalake(
tmp_path, sample_data, schema=Schema.from_pyarrow(sample_data.schema)
)
delta_table = DeltaTable(tmp_path)
assert delta_table.schema().to_pyarrow() == sample_data.schema