Skip to content

Commit 19ad24e

Browse files
smaheshwar-pltrSreesh Maheshwar
andauthored
Nit fixes to URL-encoding of partition field names (#1499)
* Revert "Add `make_name_compatible` suggestion so test passes" This reverts commit 61cdd08. * Nit fixes to URL-encoding of partition field names * Fix tests * Collapse * Make lint --------- Co-authored-by: Sreesh Maheshwar <[email protected]>
1 parent a95f9ee commit 19ad24e

File tree

3 files changed

+7
-48
lines changed

3 files changed

+7
-48
lines changed

mkdocs/docs/api.md

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1077,6 +1077,7 @@ with table.update_schema() as update:
10771077
with table.update_schema() as update:
10781078
update.add_column(("details", "confirmed_by"), StringType(), "Name of the exchange")
10791079
```
1080+
10801081
A complex type must exist before columns can be added to it. Fields in complex types are added in a tuple.
10811082

10821083
### Rename column

pyiceberg/partitioning.py

Lines changed: 2 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -234,11 +234,8 @@ def partition_to_path(self, data: Record, schema: Schema) -> str:
234234
partition_field = self.fields[pos]
235235
value_str = partition_field.transform.to_human_string(field_types[pos].field_type, value=data[pos])
236236

237-
value_str = quote_plus(value_str, safe="")
238-
value_strs.append(value_str)
239-
240-
field_str = quote_plus(partition_field.name, safe="")
241-
field_strs.append(field_str)
237+
value_strs.append(quote_plus(value_str, safe=""))
238+
field_strs.append(quote_plus(partition_field.name, safe=""))
242239

243240
path = "/".join([field_str + "=" + value_str for field_str, value_str in zip(field_strs, value_strs)])
244241
return path

tests/integration/test_partitioning_key.py

Lines changed: 4 additions & 43 deletions
Original file line numberDiff line numberDiff line change
@@ -18,15 +18,15 @@
1818
import uuid
1919
from datetime import date, datetime, timedelta, timezone
2020
from decimal import Decimal
21-
from typing import Any, Callable, List, Optional
21+
from typing import Any, List
2222

2323
import pytest
2424
from pyspark.sql import SparkSession
2525
from pyspark.sql.utils import AnalysisException
2626

2727
from pyiceberg.catalog import Catalog
2828
from pyiceberg.partitioning import PartitionField, PartitionFieldValue, PartitionKey, PartitionSpec
29-
from pyiceberg.schema import Schema
29+
from pyiceberg.schema import Schema, make_compatible_name
3030
from pyiceberg.transforms import (
3131
BucketTransform,
3232
DayTransform,
@@ -78,7 +78,7 @@
7878

7979

8080
@pytest.mark.parametrize(
81-
"partition_fields, partition_values, expected_partition_record, expected_hive_partition_path_slice, spark_create_table_sql_for_justification, spark_data_insert_sql_for_justification, make_compatible_name",
81+
"partition_fields, partition_values, expected_partition_record, expected_hive_partition_path_slice, spark_create_table_sql_for_justification, spark_data_insert_sql_for_justification",
8282
[
8383
# # Identity Transform
8484
(
@@ -99,7 +99,6 @@
9999
VALUES
100100
(false, 'Boolean field set to false');
101101
""",
102-
None,
103102
),
104103
(
105104
[PartitionField(source_id=2, field_id=1001, transform=IdentityTransform(), name="string_field")],
@@ -119,7 +118,6 @@
119118
VALUES
120119
('sample_string', 'Another string value')
121120
""",
122-
None,
123121
),
124122
(
125123
[PartitionField(source_id=4, field_id=1001, transform=IdentityTransform(), name="int_field")],
@@ -139,7 +137,6 @@
139137
VALUES
140138
(42, 'Associated string value for int 42')
141139
""",
142-
None,
143140
),
144141
(
145142
[PartitionField(source_id=5, field_id=1001, transform=IdentityTransform(), name="long_field")],
@@ -159,7 +156,6 @@
159156
VALUES
160157
(1234567890123456789, 'Associated string value for long 1234567890123456789')
161158
""",
162-
None,
163159
),
164160
(
165161
[PartitionField(source_id=6, field_id=1001, transform=IdentityTransform(), name="float_field")],
@@ -183,7 +179,6 @@
183179
# VALUES
184180
# (3.14, 'Associated string value for float 3.14')
185181
# """
186-
None,
187182
),
188183
(
189184
[PartitionField(source_id=7, field_id=1001, transform=IdentityTransform(), name="double_field")],
@@ -207,7 +202,6 @@
207202
# VALUES
208203
# (6.282, 'Associated string value for double 6.282')
209204
# """
210-
None,
211205
),
212206
(
213207
[PartitionField(source_id=8, field_id=1001, transform=IdentityTransform(), name="timestamp_field")],
@@ -227,7 +221,6 @@
227221
VALUES
228222
(CAST('2023-01-01 12:00:01.000999' AS TIMESTAMP_NTZ), 'Associated string value for timestamp 2023-01-01T12:00:00')
229223
""",
230-
None,
231224
),
232225
(
233226
[PartitionField(source_id=8, field_id=1001, transform=IdentityTransform(), name="timestamp_field")],
@@ -247,7 +240,6 @@
247240
VALUES
248241
(CAST('2023-01-01 12:00:01' AS TIMESTAMP_NTZ), 'Associated string value for timestamp 2023-01-01T12:00:00')
249242
""",
250-
None,
251243
),
252244
(
253245
[PartitionField(source_id=8, field_id=1001, transform=IdentityTransform(), name="timestamp_field")],
@@ -272,7 +264,6 @@
272264
# VALUES
273265
# (CAST('2023-01-01 12:00:00' AS TIMESTAMP_NTZ), 'Associated string value for timestamp 2023-01-01T12:00:00')
274266
# """
275-
None,
276267
),
277268
(
278269
[PartitionField(source_id=9, field_id=1001, transform=IdentityTransform(), name="timestamptz_field")],
@@ -297,7 +288,6 @@
297288
# VALUES
298289
# (CAST('2023-01-01 12:00:01.000999+03:00' AS TIMESTAMP), 'Associated string value for timestamp 2023-01-01 12:00:01.000999+03:00')
299290
# """
300-
None,
301291
),
302292
(
303293
[PartitionField(source_id=10, field_id=1001, transform=IdentityTransform(), name="date_field")],
@@ -317,7 +307,6 @@
317307
VALUES
318308
(CAST('2023-01-01' AS DATE), 'Associated string value for date 2023-01-01')
319309
""",
320-
None,
321310
),
322311
(
323312
[PartitionField(source_id=14, field_id=1001, transform=IdentityTransform(), name="uuid_field")],
@@ -337,7 +326,6 @@
337326
VALUES
338327
('f47ac10b-58cc-4372-a567-0e02b2c3d479', 'Associated string value for UUID f47ac10b-58cc-4372-a567-0e02b2c3d479')
339328
""",
340-
None,
341329
),
342330
(
343331
[PartitionField(source_id=11, field_id=1001, transform=IdentityTransform(), name="binary_field")],
@@ -357,7 +345,6 @@
357345
VALUES
358346
(CAST('example' AS BINARY), 'Associated string value for binary `example`')
359347
""",
360-
None,
361348
),
362349
(
363350
[PartitionField(source_id=13, field_id=1001, transform=IdentityTransform(), name="decimal_field")],
@@ -377,7 +364,6 @@
377364
VALUES
378365
(123.45, 'Associated string value for decimal 123.45')
379366
""",
380-
None,
381367
),
382368
# # Year Month Day Hour Transform
383369
# Month Transform
@@ -399,7 +385,6 @@
399385
VALUES
400386
(CAST('2023-01-01 11:55:59.999999' AS TIMESTAMP_NTZ), 'Event at 2023-01-01 11:55:59.999999');
401387
""",
402-
None,
403388
),
404389
(
405390
[PartitionField(source_id=9, field_id=1001, transform=MonthTransform(), name="timestamptz_field_month")],
@@ -419,7 +404,6 @@
419404
VALUES
420405
(CAST('2023-01-01 12:00:01.000999+03:00' AS TIMESTAMP), 'Event at 2023-01-01 12:00:01.000999+03:00');
421406
""",
422-
None,
423407
),
424408
(
425409
[PartitionField(source_id=10, field_id=1001, transform=MonthTransform(), name="date_field_month")],
@@ -439,7 +423,6 @@
439423
VALUES
440424
(CAST('2023-01-01' AS DATE), 'Event on 2023-01-01');
441425
""",
442-
None,
443426
),
444427
# Year Transform
445428
(
@@ -460,7 +443,6 @@
460443
VALUES
461444
(CAST('2023-01-01 11:55:59.999999' AS TIMESTAMP), 'Event at 2023-01-01 11:55:59.999999');
462445
""",
463-
None,
464446
),
465447
(
466448
[PartitionField(source_id=9, field_id=1001, transform=YearTransform(), name="timestamptz_field_year")],
@@ -480,7 +462,6 @@
480462
VALUES
481463
(CAST('2023-01-01 12:00:01.000999+03:00' AS TIMESTAMP), 'Event at 2023-01-01 12:00:01.000999+03:00');
482464
""",
483-
None,
484465
),
485466
(
486467
[PartitionField(source_id=10, field_id=1001, transform=YearTransform(), name="date_field_year")],
@@ -500,7 +481,6 @@
500481
VALUES
501482
(CAST('2023-01-01' AS DATE), 'Event on 2023-01-01');
502483
""",
503-
None,
504484
),
505485
# # Day Transform
506486
(
@@ -521,7 +501,6 @@
521501
VALUES
522502
(CAST('2023-01-01' AS DATE), 'Event on 2023-01-01');
523503
""",
524-
None,
525504
),
526505
(
527506
[PartitionField(source_id=9, field_id=1001, transform=DayTransform(), name="timestamptz_field_day")],
@@ -541,7 +520,6 @@
541520
VALUES
542521
(CAST('2023-01-01 12:00:01.000999+03:00' AS TIMESTAMP), 'Event at 2023-01-01 12:00:01.000999+03:00');
543522
""",
544-
None,
545523
),
546524
(
547525
[PartitionField(source_id=10, field_id=1001, transform=DayTransform(), name="date_field_day")],
@@ -561,7 +539,6 @@
561539
VALUES
562540
(CAST('2023-01-01' AS DATE), 'Event on 2023-01-01');
563541
""",
564-
None,
565542
),
566543
# Hour Transform
567544
(
@@ -582,7 +559,6 @@
582559
VALUES
583560
(CAST('2023-01-01 11:55:59.999999' AS TIMESTAMP), 'Event within the 11th hour of 2023-01-01');
584561
""",
585-
None,
586562
),
587563
(
588564
[PartitionField(source_id=9, field_id=1001, transform=HourTransform(), name="timestamptz_field_hour")],
@@ -602,7 +578,6 @@
602578
VALUES
603579
(CAST('2023-01-01 12:00:01.000999+03:00' AS TIMESTAMP), 'Event at 2023-01-01 12:00:01.000999+03:00');
604580
""",
605-
None,
606581
),
607582
# Truncate Transform
608583
(
@@ -623,7 +598,6 @@
623598
VALUES
624599
(12345, 'Sample data for int');
625600
""",
626-
None,
627601
),
628602
(
629603
[PartitionField(source_id=5, field_id=1001, transform=TruncateTransform(2), name="bigint_field_trunc")],
@@ -643,7 +617,6 @@
643617
VALUES
644618
(4294967297, 'Sample data for long');
645619
""",
646-
None,
647620
),
648621
(
649622
[PartitionField(source_id=2, field_id=1001, transform=TruncateTransform(3), name="string_field_trunc")],
@@ -663,7 +636,6 @@
663636
VALUES
664637
('abcdefg', 'Another sample for string');
665638
""",
666-
None,
667639
),
668640
(
669641
[PartitionField(source_id=13, field_id=1001, transform=TruncateTransform(width=5), name="decimal_field_trunc")],
@@ -683,7 +655,6 @@
683655
VALUES
684656
(678.90, 'Associated string value for decimal 678.90')
685657
""",
686-
None,
687658
),
688659
(
689660
[PartitionField(source_id=11, field_id=1001, transform=TruncateTransform(10), name="binary_field_trunc")],
@@ -703,7 +674,6 @@
703674
VALUES
704675
(binary('HELLOICEBERG'), 'Sample data for binary');
705676
""",
706-
None,
707677
),
708678
# Bucket Transform
709679
(
@@ -724,7 +694,6 @@
724694
VALUES
725695
(10, 'Integer with value 10');
726696
""",
727-
None,
728697
),
729698
# Test multiple field combinations could generate the Partition record and hive partition path correctly
730699
(
@@ -753,7 +722,6 @@
753722
VALUES
754723
(CAST('2023-01-01 11:55:59.999999' AS TIMESTAMP), CAST('2023-01-01' AS DATE), 'some data');
755724
""",
756-
None,
757725
),
758726
# Test that special characters are URL-encoded
759727
(
@@ -773,7 +741,6 @@
773741
VALUES
774742
('special string')
775743
""",
776-
lambda name: name.replace("#", "_x23").replace("+", "_x2B"),
777744
),
778745
],
779746
)
@@ -787,7 +754,6 @@ def test_partition_key(
787754
expected_hive_partition_path_slice: str,
788755
spark_create_table_sql_for_justification: str,
789756
spark_data_insert_sql_for_justification: str,
790-
make_compatible_name: Optional[Callable[[str], str]],
791757
) -> None:
792758
partition_field_values = [PartitionFieldValue(field, value) for field, value in zip(partition_fields, partition_values)]
793759
spec = PartitionSpec(*partition_fields)
@@ -823,11 +789,6 @@ def test_partition_key(
823789
snapshot.manifests(iceberg_table.io)[0].fetch_manifest_entry(iceberg_table.io)[0].data_file.file_path
824790
)
825791
# Special characters in partition value are sanitized when written to the data file's partition field
826-
# Use `make_compatible_name` to match the sanitize behavior
827-
sanitized_record = (
828-
Record(**{make_compatible_name(k): v for k, v in vars(expected_partition_record).items()})
829-
if make_compatible_name
830-
else expected_partition_record
831-
)
792+
sanitized_record = Record(**{make_compatible_name(k): v for k, v in vars(expected_partition_record).items()})
832793
assert spark_partition_for_justification == sanitized_record
833794
assert expected_hive_partition_path_slice in spark_path_for_justification

0 commit comments

Comments
 (0)