|
19 | 19 | import os |
20 | 20 | import random |
21 | 21 | import time |
| 22 | +import uuid |
22 | 23 | from datetime import date, datetime, timedelta |
23 | 24 | from decimal import Decimal |
24 | 25 | from pathlib import Path |
|
48 | 49 | from pyiceberg.schema import Schema |
49 | 50 | from pyiceberg.table import TableProperties |
50 | 51 | from pyiceberg.table.sorting import SortDirection, SortField, SortOrder |
51 | | -from pyiceberg.transforms import DayTransform, HourTransform, IdentityTransform |
| 52 | +from pyiceberg.transforms import DayTransform, HourTransform, IdentityTransform, BucketTransform, Transform |
52 | 53 | from pyiceberg.types import ( |
53 | 54 | DateType, |
54 | 55 | DecimalType, |
|
58 | 59 | LongType, |
59 | 60 | NestedField, |
60 | 61 | StringType, |
| 62 | + UUIDType, |
61 | 63 | ) |
62 | 64 | from utils import _create_table |
63 | 65 |
|
@@ -1841,3 +1843,50 @@ def test_read_write_decimals(session_catalog: Catalog) -> None: |
1841 | 1843 | tbl.append(arrow_table) |
1842 | 1844 |
|
1843 | 1845 | assert tbl.scan().to_arrow() == arrow_table |
| 1846 | + |
| 1847 | + |
| 1848 | +@pytest.mark.integration |
| 1849 | +@pytest.mark.parametrize("transform", [IdentityTransform(), BucketTransform(32)]) |
| 1850 | +def test_uuid_partitioning(session_catalog: Catalog, spark: SparkSession, transform: Transform) -> None: |
| 1851 | + identifier = f"default.test_uuid_partitioning_{str(transform).replace('[32]', '')}" |
| 1852 | + |
| 1853 | + schema = Schema(NestedField(field_id=1, name="uuid", field_type=UUIDType(), required=True)) |
| 1854 | + |
| 1855 | + try: |
| 1856 | + session_catalog.drop_table(identifier=identifier) |
| 1857 | + except NoSuchTableError: |
| 1858 | + pass |
| 1859 | + |
| 1860 | + partition_spec = PartitionSpec( |
| 1861 | + PartitionField(source_id=1, field_id=1000, transform=transform, name="uuid_identity") |
| 1862 | + ) |
| 1863 | + |
| 1864 | + import pyarrow as pa |
| 1865 | + |
| 1866 | + arr_table = pa.Table.from_pydict( |
| 1867 | + { |
| 1868 | + "uuid": [ |
| 1869 | + uuid.UUID("00000000-0000-0000-0000-000000000000").bytes, |
| 1870 | + uuid.UUID("11111111-1111-1111-1111-111111111111").bytes, |
| 1871 | + ], |
| 1872 | + }, |
| 1873 | + schema=pa.schema( |
| 1874 | + [ |
| 1875 | + # Uuid not yet supported, so we have to stick with `binary(16)` |
| 1876 | + # https://github.com/apache/arrow/issues/46468 |
| 1877 | + pa.field("uuid", pa.binary(16), nullable=False), |
| 1878 | + ] |
| 1879 | + ), |
| 1880 | + ) |
| 1881 | + |
| 1882 | + tbl = session_catalog.create_table( |
| 1883 | + identifier=identifier, |
| 1884 | + schema=schema, |
| 1885 | + partition_spec=partition_spec, |
| 1886 | + ) |
| 1887 | + |
| 1888 | + tbl.append(arr_table) |
| 1889 | + |
| 1890 | + lhs = [r[0] for r in spark.table(identifier).collect()] |
| 1891 | + rhs = [str(u.as_py()) for u in tbl.scan().to_arrow()["uuid"].combine_chunks()] |
| 1892 | + assert lhs == rhs |
0 commit comments