|
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 |
|
49 | 50 | from pyiceberg.table import TableProperties |
50 | 51 | from pyiceberg.table.refs import MAIN_BRANCH |
51 | 52 | from pyiceberg.table.sorting import SortDirection, SortField, SortOrder |
52 | | -from pyiceberg.transforms import DayTransform, HourTransform, IdentityTransform |
| 53 | +from pyiceberg.transforms import DayTransform, HourTransform, IdentityTransform, Transform |
53 | 54 | from pyiceberg.types import ( |
54 | 55 | DateType, |
55 | 56 | DecimalType, |
|
59 | 60 | LongType, |
60 | 61 | NestedField, |
61 | 62 | StringType, |
| 63 | + UUIDType, |
62 | 64 | ) |
63 | 65 | from utils import _create_table |
64 | 66 |
|
@@ -1286,7 +1288,7 @@ def test_table_write_schema_with_valid_upcast( |
1286 | 1288 | pa.field("list", pa.list_(pa.int64()), nullable=False), |
1287 | 1289 | pa.field("map", pa.map_(pa.string(), pa.int64()), nullable=False), |
1288 | 1290 | pa.field("double", pa.float64(), nullable=True), # can support upcasting float to double |
1289 | | - pa.field("uuid", pa.binary(length=16), nullable=True), # can UUID is read as fixed length binary of length 16 |
| 1291 | + pa.field("uuid", pa.uuid(), nullable=True), |
1290 | 1292 | ) |
1291 | 1293 | ) |
1292 | 1294 | ) |
@@ -1858,6 +1860,59 @@ def test_read_write_decimals(session_catalog: Catalog) -> None: |
1858 | 1860 | assert tbl.scan().to_arrow() == arrow_table |
1859 | 1861 |
|
1860 | 1862 |
|
| 1863 | +@pytest.mark.integration |
| 1864 | +@pytest.mark.parametrize( |
| 1865 | + "transform", |
| 1866 | + [ |
| 1867 | + IdentityTransform(), |
| 1868 | + # Bucket is disabled because of an issue in Iceberg Java: |
| 1869 | + # https://github.com/apache/iceberg/pull/13324 |
| 1870 | + # BucketTransform(32) |
| 1871 | + ], |
| 1872 | +) |
| 1873 | +def test_uuid_partitioning(session_catalog: Catalog, spark: SparkSession, transform: Transform) -> None: # type: ignore |
| 1874 | + identifier = f"default.test_uuid_partitioning_{str(transform).replace('[32]', '')}" |
| 1875 | + |
| 1876 | + schema = Schema(NestedField(field_id=1, name="uuid", field_type=UUIDType(), required=True)) |
| 1877 | + |
| 1878 | + try: |
| 1879 | + session_catalog.drop_table(identifier=identifier) |
| 1880 | + except NoSuchTableError: |
| 1881 | + pass |
| 1882 | + |
| 1883 | + partition_spec = PartitionSpec(PartitionField(source_id=1, field_id=1000, transform=transform, name="uuid_identity")) |
| 1884 | + |
| 1885 | + import pyarrow as pa |
| 1886 | + |
| 1887 | + arr_table = pa.Table.from_pydict( |
| 1888 | + { |
| 1889 | + "uuid": [ |
| 1890 | + uuid.UUID("00000000-0000-0000-0000-000000000000").bytes, |
| 1891 | + uuid.UUID("11111111-1111-1111-1111-111111111111").bytes, |
| 1892 | + ], |
| 1893 | + }, |
| 1894 | + schema=pa.schema( |
| 1895 | + [ |
| 1896 | + # Uuid not yet supported, so we have to stick with `binary(16)` |
| 1897 | + # https://github.com/apache/arrow/issues/46468 |
| 1898 | + pa.field("uuid", pa.binary(16), nullable=False), |
| 1899 | + ] |
| 1900 | + ), |
| 1901 | + ) |
| 1902 | + |
| 1903 | + tbl = session_catalog.create_table( |
| 1904 | + identifier=identifier, |
| 1905 | + schema=schema, |
| 1906 | + partition_spec=partition_spec, |
| 1907 | + ) |
| 1908 | + |
| 1909 | + tbl.append(arr_table) |
| 1910 | + |
| 1911 | + lhs = [r[0] for r in spark.table(identifier).collect()] |
| 1912 | + rhs = [str(u.as_py()) for u in tbl.scan().to_arrow()["uuid"].combine_chunks()] |
| 1913 | + assert lhs == rhs |
| 1914 | + |
| 1915 | + |
1861 | 1916 | @pytest.mark.integration |
1862 | 1917 | def test_avro_compression_codecs(session_catalog: Catalog, arrow_table_with_null: pa.Table) -> None: |
1863 | 1918 | identifier = "default.test_avro_compression_codecs" |
|
0 commit comments