-
Notifications
You must be signed in to change notification settings - Fork 393
Construction of filenames for partitioned writes #453
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
e380cfa
da14e69
0a062cb
f9be89f
d6e9e73
1034823
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -16,9 +16,21 @@ | |
| # under the License. | ||
| from __future__ import annotations | ||
|
|
||
| import uuid | ||
| from abc import ABC, abstractmethod | ||
| from dataclasses import dataclass | ||
| from datetime import date, datetime | ||
| from functools import cached_property, singledispatch | ||
| from typing import Any, Dict, Generic, List, Optional, Tuple, TypeVar | ||
| from typing import ( | ||
| Any, | ||
| Dict, | ||
| Generic, | ||
| List, | ||
| Optional, | ||
| Tuple, | ||
| TypeVar, | ||
| ) | ||
| from urllib.parse import quote | ||
|
|
||
| from pydantic import ( | ||
| BeforeValidator, | ||
|
|
@@ -41,8 +53,18 @@ | |
| YearTransform, | ||
| parse_transform, | ||
| ) | ||
| from pyiceberg.typedef import IcebergBaseModel | ||
| from pyiceberg.types import NestedField, StructType | ||
| from pyiceberg.typedef import IcebergBaseModel, Record | ||
| from pyiceberg.types import ( | ||
| DateType, | ||
| IcebergType, | ||
| NestedField, | ||
| PrimitiveType, | ||
| StructType, | ||
| TimestampType, | ||
| TimestamptzType, | ||
| UUIDType, | ||
| ) | ||
| from pyiceberg.utils.datetime import date_to_days, datetime_to_micros | ||
|
|
||
| INITIAL_PARTITION_SPEC_ID = 0 | ||
| PARTITION_FIELD_ID_START: int = 1000 | ||
|
|
@@ -199,6 +221,23 @@ def partition_type(self, schema: Schema) -> StructType: | |
| nested_fields.append(NestedField(field.field_id, field.name, result_type, required=False)) | ||
| return StructType(*nested_fields) | ||
|
|
||
| def partition_to_path(self, data: Record, schema: Schema) -> str: | ||
| partition_type = self.partition_type(schema) | ||
| field_types = partition_type.fields | ||
|
|
||
| field_strs = [] | ||
| value_strs = [] | ||
| for pos, value in enumerate(data.record_fields()): | ||
| partition_field = self.fields[pos] | ||
| value_str = partition_field.transform.to_human_string(field_types[pos].field_type, value=value) | ||
|
|
||
| value_str = quote(value_str, safe='') | ||
| value_strs.append(value_str) | ||
| field_strs.append(partition_field.name) | ||
|
|
||
| path = "/".join([field_str + "=" + value_str for field_str, value_str in zip(field_strs, value_strs)]) | ||
| return path | ||
|
|
||
|
|
||
| UNPARTITIONED_PARTITION_SPEC = PartitionSpec(spec_id=0) | ||
|
|
||
|
|
@@ -326,3 +365,59 @@ def _visit_partition_field(schema: Schema, field: PartitionField, visitor: Parti | |
| return visitor.unknown(field.field_id, source_name, field.source_id, repr(transform)) | ||
| else: | ||
| raise ValueError(f"Unknown transform {transform}") | ||
|
|
||
|
|
||
| @dataclass(frozen=True) | ||
| class PartitionFieldValue: | ||
| field: PartitionField | ||
| value: Any | ||
|
|
||
|
|
||
| @dataclass(frozen=True) | ||
| class PartitionKey: | ||
| raw_partition_field_values: List[PartitionFieldValue] | ||
| partition_spec: PartitionSpec | ||
| schema: Schema | ||
|
|
||
| @cached_property | ||
| def partition(self) -> Record: # partition key transformed with iceberg internal representation as input | ||
| iceberg_typed_key_values = {} | ||
| for raw_partition_field_value in self.raw_partition_field_values: | ||
| partition_fields = self.partition_spec.source_id_to_fields_map[raw_partition_field_value.field.source_id] | ||
| if len(partition_fields) != 1: | ||
| raise ValueError("partition_fields must contain exactly one field.") | ||
| partition_field = partition_fields[0] | ||
| iceberg_type = self.schema.find_field(name_or_id=raw_partition_field_value.field.source_id).field_type | ||
| iceberg_typed_value = _to_partition_representation(iceberg_type, raw_partition_field_value.value) | ||
| transformed_value = partition_field.transform.transform(iceberg_type)(iceberg_typed_value) | ||
| iceberg_typed_key_values[partition_field.name] = transformed_value | ||
| return Record(**iceberg_typed_key_values) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We're now getting into the realm of premature optimization, but ideally you don't need to set the names of the keys. The concept of a Record is that is only contains the data. Just below:
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hi @Fokko, thanks for the guidance! My intention of adding the keys is because this PartitionKey.partition is not only used for generating the file path but also used to initiate the Datafile.partition in the io.pyarrow.write_file(). As the integration test shows, prints So I assume this data_file.partition is Record with keys. |
||
|
|
||
| def to_path(self) -> str: | ||
| return self.partition_spec.partition_to_path(self.partition, self.schema) | ||
|
|
||
|
|
||
| @singledispatch | ||
| def _to_partition_representation(type: IcebergType, value: Any) -> Any: | ||
| return TypeError(f"Unsupported partition field type: {type}") | ||
|
|
||
|
|
||
| @_to_partition_representation.register(TimestampType) | ||
| @_to_partition_representation.register(TimestamptzType) | ||
| def _(type: IcebergType, value: Optional[datetime]) -> Optional[int]: | ||
| return datetime_to_micros(value) if value is not None else None | ||
|
|
||
|
|
||
| @_to_partition_representation.register(DateType) | ||
| def _(type: IcebergType, value: Optional[date]) -> Optional[int]: | ||
| return date_to_days(value) if value is not None else None | ||
|
|
||
|
|
||
| @_to_partition_representation.register(UUIDType) | ||
| def _(type: IcebergType, value: Optional[uuid.UUID]) -> Optional[str]: | ||
| return str(value) if value is not None else None | ||
|
|
||
|
|
||
| @_to_partition_representation.register(PrimitiveType) | ||
| def _(type: IcebergType, value: Optional[Any]) -> Optional[Any]: | ||
| return value | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Spark builds a row accessor that takes in an arrow table row and converts it to key values. The accessor seems a little unnecessary since the partition field could not be nested or a map/list, so here the class just uses a naive list of field-value pairs. Willing to change it if this is inappropriate.