|
16 | 16 | # under the License. |
17 | 17 | from __future__ import annotations |
18 | 18 |
|
| 19 | +import uuid |
19 | 20 | from abc import ABC, abstractmethod |
| 21 | +from dataclasses import dataclass |
| 22 | +from datetime import date, datetime |
20 | 23 | from functools import cached_property, singledispatch |
21 | | -from typing import Any, Dict, Generic, List, Optional, Tuple, TypeVar |
| 24 | +from typing import ( |
| 25 | + Any, |
| 26 | + Dict, |
| 27 | + Generic, |
| 28 | + List, |
| 29 | + Optional, |
| 30 | + Tuple, |
| 31 | + TypeVar, |
| 32 | +) |
| 33 | +from urllib.parse import quote |
22 | 34 |
|
23 | 35 | from pydantic import ( |
24 | 36 | BeforeValidator, |
|
41 | 53 | YearTransform, |
42 | 54 | parse_transform, |
43 | 55 | ) |
44 | | -from pyiceberg.typedef import IcebergBaseModel |
45 | | -from pyiceberg.types import NestedField, StructType |
| 56 | +from pyiceberg.typedef import IcebergBaseModel, Record |
| 57 | +from pyiceberg.types import ( |
| 58 | + DateType, |
| 59 | + IcebergType, |
| 60 | + NestedField, |
| 61 | + PrimitiveType, |
| 62 | + StructType, |
| 63 | + TimestampType, |
| 64 | + TimestamptzType, |
| 65 | + UUIDType, |
| 66 | +) |
| 67 | +from pyiceberg.utils.datetime import date_to_days, datetime_to_micros |
46 | 68 |
|
47 | 69 | INITIAL_PARTITION_SPEC_ID = 0 |
48 | 70 | PARTITION_FIELD_ID_START: int = 1000 |
@@ -199,6 +221,23 @@ def partition_type(self, schema: Schema) -> StructType: |
199 | 221 | nested_fields.append(NestedField(field.field_id, field.name, result_type, required=False)) |
200 | 222 | return StructType(*nested_fields) |
201 | 223 |
|
| 224 | + def partition_to_path(self, data: Record, schema: Schema) -> str: |
| 225 | + partition_type = self.partition_type(schema) |
| 226 | + field_types = partition_type.fields |
| 227 | + |
| 228 | + field_strs = [] |
| 229 | + value_strs = [] |
| 230 | + for pos, value in enumerate(data.record_fields()): |
| 231 | + partition_field = self.fields[pos] |
| 232 | + value_str = partition_field.transform.to_human_string(field_types[pos].field_type, value=value) |
| 233 | + |
| 234 | + value_str = quote(value_str, safe='') |
| 235 | + value_strs.append(value_str) |
| 236 | + field_strs.append(partition_field.name) |
| 237 | + |
| 238 | + path = "/".join([field_str + "=" + value_str for field_str, value_str in zip(field_strs, value_strs)]) |
| 239 | + return path |
| 240 | + |
202 | 241 |
|
203 | 242 | UNPARTITIONED_PARTITION_SPEC = PartitionSpec(spec_id=0) |
204 | 243 |
|
@@ -326,3 +365,59 @@ def _visit_partition_field(schema: Schema, field: PartitionField, visitor: Parti |
326 | 365 | return visitor.unknown(field.field_id, source_name, field.source_id, repr(transform)) |
327 | 366 | else: |
328 | 367 | raise ValueError(f"Unknown transform {transform}") |
| 368 | + |
| 369 | + |
| 370 | +@dataclass(frozen=True) |
| 371 | +class PartitionFieldValue: |
| 372 | + field: PartitionField |
| 373 | + value: Any |
| 374 | + |
| 375 | + |
| 376 | +@dataclass(frozen=True) |
| 377 | +class PartitionKey: |
| 378 | + raw_partition_field_values: List[PartitionFieldValue] |
| 379 | + partition_spec: PartitionSpec |
| 380 | + schema: Schema |
| 381 | + |
| 382 | + @cached_property |
| 383 | + def partition(self) -> Record: # partition key transformed with iceberg internal representation as input |
| 384 | + iceberg_typed_key_values = {} |
| 385 | + for raw_partition_field_value in self.raw_partition_field_values: |
| 386 | + partition_fields = self.partition_spec.source_id_to_fields_map[raw_partition_field_value.field.source_id] |
| 387 | + if len(partition_fields) != 1: |
| 388 | + raise ValueError("partition_fields must contain exactly one field.") |
| 389 | + partition_field = partition_fields[0] |
| 390 | + iceberg_type = self.schema.find_field(name_or_id=raw_partition_field_value.field.source_id).field_type |
| 391 | + iceberg_typed_value = _to_partition_representation(iceberg_type, raw_partition_field_value.value) |
| 392 | + transformed_value = partition_field.transform.transform(iceberg_type)(iceberg_typed_value) |
| 393 | + iceberg_typed_key_values[partition_field.name] = transformed_value |
| 394 | + return Record(**iceberg_typed_key_values) |
| 395 | + |
| 396 | + def to_path(self) -> str: |
| 397 | + return self.partition_spec.partition_to_path(self.partition, self.schema) |
| 398 | + |
| 399 | + |
| 400 | +@singledispatch |
| 401 | +def _to_partition_representation(type: IcebergType, value: Any) -> Any: |
| 402 | + return TypeError(f"Unsupported partition field type: {type}") |
| 403 | + |
| 404 | + |
| 405 | +@_to_partition_representation.register(TimestampType) |
| 406 | +@_to_partition_representation.register(TimestamptzType) |
| 407 | +def _(type: IcebergType, value: Optional[datetime]) -> Optional[int]: |
| 408 | + return datetime_to_micros(value) if value is not None else None |
| 409 | + |
| 410 | + |
| 411 | +@_to_partition_representation.register(DateType) |
| 412 | +def _(type: IcebergType, value: Optional[date]) -> Optional[int]: |
| 413 | + return date_to_days(value) if value is not None else None |
| 414 | + |
| 415 | + |
| 416 | +@_to_partition_representation.register(UUIDType) |
| 417 | +def _(type: IcebergType, value: Optional[uuid.UUID]) -> Optional[str]: |
| 418 | + return str(value) if value is not None else None |
| 419 | + |
| 420 | + |
| 421 | +@_to_partition_representation.register(PrimitiveType) |
| 422 | +def _(type: IcebergType, value: Optional[Any]) -> Optional[Any]: |
| 423 | + return value |
0 commit comments