Skip to content

Commit 1034823

Browse files
committed
clean up; rename ambiguous function name
1 parent d6e9e73 commit 1034823

File tree

1 file changed

+8
-8
lines changed

1 file changed

+8
-8
lines changed

pyiceberg/partitioning.py

Lines changed: 8 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -228,7 +228,7 @@ def partition_to_path(self, data: Record, schema: Schema) -> str:
228228
field_strs = []
229229
value_strs = []
230230
for pos, value in enumerate(data.record_fields()):
231-
partition_field = self.fields[pos] # partition field
231+
partition_field = self.fields[pos]
232232
value_str = partition_field.transform.to_human_string(field_types[pos].field_type, value=value)
233233

234234
value_str = quote(value_str, safe='')
@@ -388,7 +388,7 @@ def partition(self) -> Record: # partition key transformed with iceberg interna
388388
raise ValueError("partition_fields must contain exactly one field.")
389389
partition_field = partition_fields[0]
390390
iceberg_type = self.schema.find_field(name_or_id=raw_partition_field_value.field.source_id).field_type
391-
iceberg_typed_value = _to_iceberg_internal_representation(iceberg_type, raw_partition_field_value.value)
391+
iceberg_typed_value = _to_partition_representation(iceberg_type, raw_partition_field_value.value)
392392
transformed_value = partition_field.transform.transform(iceberg_type)(iceberg_typed_value)
393393
iceberg_typed_key_values[partition_field.name] = transformed_value
394394
return Record(**iceberg_typed_key_values)
@@ -398,26 +398,26 @@ def to_path(self) -> str:
398398

399399

400400
@singledispatch
401-
def _to_iceberg_internal_representation(type: IcebergType, value: Any) -> Any:
401+
def _to_partition_representation(type: IcebergType, value: Any) -> Any:
402402
return TypeError(f"Unsupported partition field type: {type}")
403403

404404

405-
@_to_iceberg_internal_representation.register(TimestampType)
406-
@_to_iceberg_internal_representation.register(TimestamptzType)
405+
@_to_partition_representation.register(TimestampType)
406+
@_to_partition_representation.register(TimestamptzType)
407407
def _(type: IcebergType, value: Optional[datetime]) -> Optional[int]:
408408
return datetime_to_micros(value) if value is not None else None
409409

410410

411-
@_to_iceberg_internal_representation.register(DateType)
411+
@_to_partition_representation.register(DateType)
412412
def _(type: IcebergType, value: Optional[date]) -> Optional[int]:
413413
return date_to_days(value) if value is not None else None
414414

415415

416-
@_to_iceberg_internal_representation.register(UUIDType)
416+
@_to_partition_representation.register(UUIDType)
417417
def _(type: IcebergType, value: Optional[uuid.UUID]) -> Optional[str]:
418418
return str(value) if value is not None else None
419419

420420

421-
@_to_iceberg_internal_representation.register(PrimitiveType)
421+
@_to_partition_representation.register(PrimitiveType)
422422
def _(type: IcebergType, value: Optional[Any]) -> Optional[Any]:
423423
return value

0 commit comments

Comments
 (0)