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: add support for pyarrow.ExtensionType #2885

Merged
merged 3 commits into from
Sep 17, 2024
Merged
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
2 changes: 2 additions & 0 deletions python/deltalake/schema.py
Original file line number Diff line number Diff line change
Expand Up @@ -95,6 +95,8 @@ def dtype_to_delta_dtype(dtype: pa.DataType) -> pa.DataType:
return pa.timestamp("us", "UTC")
elif type(dtype) is pa.FixedSizeBinaryType:
return pa.binary()
elif isinstance(dtype, pa.ExtensionType):
return dtype.storage_type
try:
return dtype_map[dtype]
except KeyError:
Expand Down
1 change: 1 addition & 0 deletions python/stubs/pyarrow/__init__.pyi
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ FixedSizeListType: Any
LargeListViewType: Any
ListViewType: Any
FixedSizeBinaryType: Any
ExtensionType: Any
schema: Any
map_: Any
list_: Any
Expand Down
25 changes: 25 additions & 0 deletions python/tests/test_schema.py
Original file line number Diff line number Diff line change
Expand Up @@ -223,6 +223,26 @@ def test_delta_schema():
assert schema_without_metadata == Schema.from_pyarrow(pa_schema)


def _generate_test_type():
class UuidType(pa.ExtensionType):
def __init__(self):
pa.ExtensionType.__init__(self, pa.binary(16), "my_package.uuid")

def __arrow_ext_serialize__(self):
# since we don't have a parameterized type, we don't need extra
# metadata to be deserialized
return b""

@classmethod
def __arrow_ext_deserialize__(self, storage_type, serialized):
# return an instance of this subclass given the serialized
# metadata.
return UuidType()

pa.register_extension_type(UuidType())
return UuidType()


def _generate_test_tuples():
test_tuples = [
(
Expand Down Expand Up @@ -515,6 +535,11 @@ def _generate_test_tuples():
),
ArrowSchemaConversionMode.NORMAL,
),
(
pa.schema([("uuid", _generate_test_type())]),
pa.schema([("uuid", pa.binary(16))]),
ArrowSchemaConversionMode.NORMAL,
),
]

return test_tuples
Expand Down
Loading