|
15 | 15 | # specific language governing permissions and limitations |
16 | 16 | # under the License. |
17 | 17 | # pylint:disable=redefined-outer-name |
| 18 | +import uuid |
18 | 19 | from datetime import date, datetime, timedelta, timezone |
19 | 20 | from decimal import Decimal |
20 | 21 | from typing import Any, List |
|
50 | 51 | StringType, |
51 | 52 | TimestampType, |
52 | 53 | TimestamptzType, |
| 54 | + UUIDType, |
53 | 55 | ) |
54 | 56 |
|
55 | | -# @pytest.fixture(scope="session") |
56 | | -# def session_catalog() -> Catalog: |
57 | | -# return load_catalog( |
58 | | -# "local", |
59 | | -# **{ |
60 | | -# "type": "rest", |
61 | | -# "uri": "http://localhost:8181", |
62 | | -# "s3.endpoint": "http://localhost:9000", |
63 | | -# "s3.access-key-id": "admin", |
64 | | -# "s3.secret-access-key": "password", |
65 | | -# }, |
66 | | -# ) |
67 | | - |
68 | | - |
69 | | -# @pytest.fixture(scope="session") |
70 | | -# def spark() -> SparkSession: |
71 | | -# import importlib.metadata |
72 | | -# import os |
73 | | - |
74 | | -# spark_version = ".".join(importlib.metadata.version("pyspark").split(".")[:2]) |
75 | | -# scala_version = "2.12" |
76 | | -# iceberg_version = "1.4.3" |
77 | | - |
78 | | -# os.environ["PYSPARK_SUBMIT_ARGS"] = ( |
79 | | -# f"--packages org.apache.iceberg:iceberg-spark-runtime-{spark_version}_{scala_version}:{iceberg_version}," |
80 | | -# f"org.apache.iceberg:iceberg-aws-bundle:{iceberg_version} pyspark-shell" |
81 | | -# ) |
82 | | -# os.environ["AWS_REGION"] = "us-east-1" |
83 | | -# os.environ["AWS_ACCESS_KEY_ID"] = "admin" |
84 | | -# os.environ["AWS_SECRET_ACCESS_KEY"] = "password" |
85 | | - |
86 | | -# spark = ( |
87 | | -# SparkSession.builder.appName("PyIceberg integration test") |
88 | | -# .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") |
89 | | -# .config("spark.sql.catalog.integration", "org.apache.iceberg.spark.SparkCatalog") |
90 | | -# .config("spark.sql.catalog.integration.catalog-impl", "org.apache.iceberg.rest.RESTCatalog") |
91 | | -# .config("spark.sql.catalog.integration.uri", "http://localhost:8181") |
92 | | -# .config("spark.sql.catalog.integration.io-impl", "org.apache.iceberg.aws.s3.S3FileIO") |
93 | | -# .config("spark.sql.catalog.integration.warehouse", "s3://warehouse/wh/") |
94 | | -# .config("spark.sql.catalog.integration.s3.endpoint", "http://localhost:9000") |
95 | | -# .config("spark.sql.catalog.integration.s3.path-style-access", "true") |
96 | | -# .config("spark.sql.defaultCatalog", "integration") |
97 | | -# .getOrCreate() |
98 | | -# ) |
99 | | - |
100 | | -# return spark |
101 | | - |
102 | | - |
103 | 57 | TABLE_SCHEMA = Schema( |
104 | 58 | NestedField(field_id=1, name="boolean_field", field_type=BooleanType(), required=False), |
105 | 59 | NestedField(field_id=2, name="string_field", field_type=StringType(), required=False), |
|
112 | 66 | NestedField(field_id=9, name="timestamptz_field", field_type=TimestamptzType(), required=False), |
113 | 67 | NestedField(field_id=10, name="date_field", field_type=DateType(), required=False), |
114 | 68 | # NestedField(field_id=11, name="time", field_type=TimeType(), required=False), |
115 | | - # NestedField(field_id=12, name="uuid", field_type=UuidType(), required=False), |
116 | 69 | NestedField(field_id=11, name="binary_field", field_type=BinaryType(), required=False), |
117 | 70 | NestedField(field_id=12, name="fixed_field", field_type=FixedType(16), required=False), |
118 | | - NestedField(field_id=13, name="decimal", field_type=DecimalType(5, 2), required=False), |
| 71 | + NestedField(field_id=13, name="decimal_field", field_type=DecimalType(5, 2), required=False), |
| 72 | + NestedField(field_id=14, name="uuid_field", field_type=UUIDType(), required=False), |
119 | 73 | ) |
120 | 74 |
|
121 | 75 |
|
|
353 | 307 | (CAST('2023-01-01' AS DATE), 'Associated string value for date 2023-01-01') |
354 | 308 | """, |
355 | 309 | ), |
| 310 | + ( |
| 311 | + [PartitionField(source_id=14, field_id=1001, transform=IdentityTransform(), name="uuid_field")], |
| 312 | + [uuid.UUID("f47ac10b-58cc-4372-a567-0e02b2c3d479")], |
| 313 | + Record(uuid_field="f47ac10b-58cc-4372-a567-0e02b2c3d479"), |
| 314 | + "uuid_field=f47ac10b-58cc-4372-a567-0e02b2c3d479", |
| 315 | + f"""CREATE TABLE {identifier} ( |
| 316 | + uuid_field string, |
| 317 | + string_field string |
| 318 | + ) |
| 319 | + USING iceberg |
| 320 | + PARTITIONED BY ( |
| 321 | + identity(uuid_field) |
| 322 | + ) |
| 323 | + """, |
| 324 | + f"""INSERT INTO {identifier} |
| 325 | + VALUES |
| 326 | + ('f47ac10b-58cc-4372-a567-0e02b2c3d479', 'Associated string value for UUID f47ac10b-58cc-4372-a567-0e02b2c3d479') |
| 327 | + """, |
| 328 | + ), |
356 | 329 | ( |
357 | 330 | [PartitionField(source_id=11, field_id=1001, transform=IdentityTransform(), name="binary_field")], |
358 | 331 | [b'example'], |
@@ -770,6 +743,7 @@ def test_partition_key( |
770 | 743 | partition_spec=spec, |
771 | 744 | schema=TABLE_SCHEMA, |
772 | 745 | ) |
| 746 | + |
773 | 747 | # key.partition is used to write the metadata in DataFile, ManifestFile and all above layers |
774 | 748 | assert key.partition == expected_partition_record |
775 | 749 | # key.to_path() generates the hive partitioning part of the to-write parquet file path |
|
0 commit comments