Skip to content

Commit

Permalink
Merge branch 'main' into bug/iam-assume-role
Browse files Browse the repository at this point in the history
  • Loading branch information
rtyler authored Sep 17, 2024
2 parents 9e2d000 + 546a344 commit 9846bac
Show file tree
Hide file tree
Showing 4 changed files with 30 additions and 2 deletions.
4 changes: 2 additions & 2 deletions docs/integrations/delta-lake-dagster.md
Original file line number Diff line number Diff line change
Expand Up @@ -212,7 +212,7 @@ def iris_dataset() -> pd.DataFrame:
```

## Using Delta Lake and Dagster with Polars
To read and write data to Delta Lake using pandas, use the `DeltaLakePolarsIOManager()`.
To read and write data to Delta Lake using polars, use the `DeltaLakePolarsIOManager()`.

You will need to install it using:

Expand All @@ -223,7 +223,7 @@ pip install dagster-deltalake-polars
In your `Definitions` object, change the `io_manager` to `DeltaLakePolarsIOManager()`:

```
from dagster_polars import DeltaLakePolarsIOManager
from dagster_deltalake_polars import DeltaLakePolarsIOManager
defs = Definitions(
assets=all_assets,
Expand Down
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

0 comments on commit 9846bac

Please sign in to comment.