Skip to content
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

URL-encode partition field names in file locations #1457

Merged
merged 13 commits into from
Jan 2, 2025
8 changes: 5 additions & 3 deletions pyiceberg/partitioning.py
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@
Tuple,
TypeVar,
)
from urllib.parse import quote
from urllib.parse import quote_plus

from pydantic import (
BeforeValidator,
Expand Down Expand Up @@ -234,9 +234,11 @@ def partition_to_path(self, data: Record, schema: Schema) -> str:
partition_field = self.fields[pos]
value_str = partition_field.transform.to_human_string(field_types[pos].field_type, value=data[pos])

value_str = quote(value_str, safe="")
value_str = quote_plus(value_str, safe="")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It defaults to utf-8, so that's good 👍

value_strs.append(value_str)
field_strs.append(partition_field.name)

field_str = quote_plus(partition_field.name, safe="")
field_strs.append(field_str)
Comment on lines +240 to +241
Copy link
Contributor

@Fokko Fokko Dec 29, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit, I would just collapse these:

Suggested change
field_str = quote_plus(partition_field.name, safe="")
field_strs.append(field_str)
field_strs.append(quote_plus(partition_field.name, safe=""))


path = "/".join([field_str + "=" + value_str for field_str, value_str in zip(field_strs, value_strs)])
return path
Expand Down
25 changes: 25 additions & 0 deletions tests/integration/test_partitioning_key.py
Original file line number Diff line number Diff line change
Expand Up @@ -70,6 +70,7 @@
NestedField(field_id=12, name="fixed_field", field_type=FixedType(16), required=False),
NestedField(field_id=13, name="decimal_field", field_type=DecimalType(5, 2), required=False),
NestedField(field_id=14, name="uuid_field", field_type=UUIDType(), required=False),
NestedField(field_id=15, name="special#string#field", field_type=StringType(), required=False),
)


Expand Down Expand Up @@ -722,6 +723,30 @@
(CAST('2023-01-01 11:55:59.999999' AS TIMESTAMP), CAST('2023-01-01' AS DATE), 'some data');
""",
),
# Test that special characters are URL-encoded
(
[PartitionField(source_id=15, field_id=1001, transform=IdentityTransform(), name="special#string#field")],
["special string"],
Record(**{"special#string#field": "special string"}), # type: ignore
"special%23string%23field=special+string",
# Spark currently writes differently to PyIceberg w.r.t special column name sanitization so justification
# (comparing expected value with Spark behavior) would fail: PyIceberg produces
# Record[special_x23string_x23field='special string'], not Record[special#string#field='special string'].
# None,
# None,
f"""CREATE TABLE {identifier} (
`special#string#field` string
)
USING iceberg
PARTITIONED BY (
identity(`special#string#field`)
)
""",
f"""INSERT INTO {identifier}
VALUES
('special string')
""",
),
],
)
@pytest.mark.integration
Expand Down
24 changes: 23 additions & 1 deletion tests/table/test_partitioning.py
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,8 @@
# under the License.
from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionField, PartitionSpec
from pyiceberg.schema import Schema
from pyiceberg.transforms import BucketTransform, TruncateTransform
from pyiceberg.transforms import BucketTransform, IdentityTransform, TruncateTransform
from pyiceberg.typedef import Record
from pyiceberg.types import (
IntegerType,
NestedField,
Expand Down Expand Up @@ -118,6 +119,27 @@ def test_deserialize_partition_spec() -> None:
)


def test_partition_spec_to_path() -> None:
schema = Schema(
NestedField(field_id=1, name="str", field_type=StringType(), required=False),
NestedField(field_id=2, name="other_str", field_type=StringType(), required=False),
NestedField(field_id=3, name="int", field_type=IntegerType(), required=True),
)

spec = PartitionSpec(
PartitionField(source_id=1, field_id=1000, transform=TruncateTransform(width=19), name="my#str%bucket"),
PartitionField(source_id=2, field_id=1001, transform=IdentityTransform(), name="other str+bucket"),
PartitionField(source_id=3, field_id=1002, transform=BucketTransform(num_buckets=25), name="my!int:bucket"),
spec_id=3,
)

record = Record(**{"my#str%bucket": "my+str", "other str+bucket": "( )", "my!int:bucket": 10}) # type: ignore
Copy link
Contributor Author

@smaheshwar-pltr smaheshwar-pltr Dec 23, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

mypy complains here and elsewhere but I think it's fine


# Both partition names fields and values should be URL encoded, with spaces mapping to plus signs, to match the Java
# behaviour: https://github.com/apache/iceberg/blob/ca3db931b0f024f0412084751ac85dd4ef2da7e7/api/src/main/java/org/apache/iceberg/PartitionSpec.java#L198-L204
assert spec.partition_to_path(record, schema) == "my%23str%25bucket=my%2Bstr/other+str%2Bbucket=%28+%29/my%21int%3Abucket=10"
Copy link
Contributor Author

@smaheshwar-pltr smaheshwar-pltr Dec 23, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Cross-checked with Java implementation (integration tests will do this eventually), in particular WRT to ' ' and '+' encoding. It is consistent.



def test_partition_type(table_schema_simple: Schema) -> None:
spec = PartitionSpec(
PartitionField(source_id=1, field_id=1000, transform=TruncateTransform(width=19), name="str_truncate"),
Expand Down
Loading