forked from apache/iceberg-python
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathtest_partitioned_writes.py
More file actions
568 lines (500 loc) · 21.3 KB
/
test_partitioned_writes.py
File metadata and controls
568 lines (500 loc) · 21.3 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
# pylint:disable=redefined-outer-name
from datetime import date
from typing import Any, Set
import pyarrow as pa
import pytest
from pyspark.sql import SparkSession
from pyiceberg.catalog import Catalog
from pyiceberg.exceptions import NoSuchTableError
from pyiceberg.partitioning import PartitionField, PartitionSpec
from pyiceberg.schema import Schema
from pyiceberg.transforms import (
BucketTransform,
DayTransform,
HourTransform,
IdentityTransform,
MonthTransform,
Transform,
TruncateTransform,
YearTransform,
)
from utils import TABLE_SCHEMA, _create_table
@pytest.mark.integration
@pytest.mark.parametrize(
"part_col", ["int", "bool", "string", "string_long", "long", "float", "double", "date", "timestamp", "timestamptz", "binary"]
)
@pytest.mark.parametrize("format_version", [1, 2])
def test_query_filter_null_partitioned(
session_catalog: Catalog, spark: SparkSession, arrow_table_with_null: pa.Table, part_col: str, format_version: int
) -> None:
# Given
identifier = f"default.arrow_table_v{format_version}_with_null_partitioned_on_col_{part_col}"
nested_field = TABLE_SCHEMA.find_field(part_col)
partition_spec = PartitionSpec(
PartitionField(source_id=nested_field.field_id, field_id=1001, transform=IdentityTransform(), name=part_col)
)
# When
tbl = _create_table(
session_catalog=session_catalog,
identifier=identifier,
properties={"format-version": str(format_version)},
data=[arrow_table_with_null],
partition_spec=partition_spec,
)
# Then
assert tbl.format_version == format_version, f"Expected v{format_version}, got: v{tbl.format_version}"
df = spark.table(identifier)
assert df.count() == 3, f"Expected 3 total rows for {identifier}"
for col in arrow_table_with_null.column_names:
assert df.where(f"{col} is not null").count() == 2, f"Expected 2 non-null rows for {col}"
assert df.where(f"{col} is null").count() == 1, f"Expected 1 null row for {col} is null"
@pytest.mark.integration
@pytest.mark.parametrize(
"part_col", ["int", "bool", "string", "string_long", "long", "float", "double", "date", "timestamp", "timestamptz", "binary"]
)
@pytest.mark.parametrize("format_version", [1, 2])
def test_query_filter_without_data_partitioned(
session_catalog: Catalog,
spark: SparkSession,
arrow_table_without_data: pa.Table,
part_col: str,
arrow_table_with_null: pa.Table,
format_version: int,
) -> None:
# Given
identifier = f"default.arrow_table_v{format_version}_without_data_partitioned_on_col_{part_col}"
nested_field = TABLE_SCHEMA.find_field(part_col)
partition_spec = PartitionSpec(
PartitionField(source_id=nested_field.field_id, field_id=1001, transform=IdentityTransform(), name=part_col)
)
# When
tbl = _create_table(
session_catalog=session_catalog,
identifier=identifier,
properties={"format-version": str(format_version)},
data=[arrow_table_without_data],
partition_spec=partition_spec,
)
# Then
assert tbl.format_version == format_version, f"Expected v{format_version}, got: v{tbl.format_version}"
df = spark.table(identifier)
for col in arrow_table_with_null.column_names:
assert df.where(f"{col} is null").count() == 0, f"Expected 0 row for {col}"
assert df.where(f"{col} is not null").count() == 0, f"Expected 0 row for {col}"
@pytest.mark.integration
@pytest.mark.parametrize(
"part_col", ["int", "bool", "string", "string_long", "long", "float", "double", "date", "timestamp", "timestamptz", "binary"]
)
@pytest.mark.parametrize("format_version", [1, 2])
def test_query_filter_only_nulls_partitioned(
session_catalog: Catalog, spark: SparkSession, arrow_table_with_only_nulls: pa.Table, part_col: str, format_version: int
) -> None:
# Given
identifier = f"default.arrow_table_v{format_version}_with_only_nulls_partitioned_on_col_{part_col}"
nested_field = TABLE_SCHEMA.find_field(part_col)
partition_spec = PartitionSpec(
PartitionField(source_id=nested_field.field_id, field_id=1001, transform=IdentityTransform(), name=part_col)
)
# When
tbl = _create_table(
session_catalog=session_catalog,
identifier=identifier,
properties={"format-version": str(format_version)},
data=[arrow_table_with_only_nulls],
partition_spec=partition_spec,
)
# Then
assert tbl.format_version == format_version, f"Expected v{format_version}, got: v{tbl.format_version}"
df = spark.table(identifier)
for col in arrow_table_with_only_nulls.column_names:
assert df.where(f"{col} is null").count() == 2, f"Expected 2 row for {col}"
assert df.where(f"{col} is not null").count() == 0, f"Expected 0 rows for {col}"
@pytest.mark.integration
@pytest.mark.parametrize(
"part_col", ["int", "bool", "string", "string_long", "long", "float", "double", "date", "timestamptz", "timestamp", "binary"]
)
@pytest.mark.parametrize("format_version", [1, 2])
def test_query_filter_appended_null_partitioned(
session_catalog: Catalog, spark: SparkSession, arrow_table_with_null: pa.Table, part_col: str, format_version: int
) -> None:
# Given
identifier = f"default.arrow_table_v{format_version}_appended_with_null_partitioned_on_col_{part_col}"
nested_field = TABLE_SCHEMA.find_field(part_col)
partition_spec = PartitionSpec(
PartitionField(source_id=nested_field.field_id, field_id=1001, transform=IdentityTransform(), name=part_col)
)
# When
tbl = _create_table(
session_catalog=session_catalog,
identifier=identifier,
properties={"format-version": str(format_version)},
data=[],
partition_spec=partition_spec,
)
# Append with arrow_table_1 with lines [A,B,C] and then arrow_table_2 with lines[A,B,C,A,B,C]
tbl.append(arrow_table_with_null)
tbl.append(pa.concat_tables([arrow_table_with_null, arrow_table_with_null]))
# Then
assert tbl.format_version == format_version, f"Expected v{format_version}, got: v{tbl.format_version}"
df = spark.table(identifier)
for col in arrow_table_with_null.column_names:
assert df.where(f"{col} is not null").count() == 6, f"Expected 6 non-null rows for {col}"
assert df.where(f"{col} is null").count() == 3, f"Expected 3 null rows for {col}"
# expecting 6 files: first append with [A], [B], [C], second append with [A, A], [B, B], [C, C]
rows = spark.sql(f"select partition from {identifier}.files").collect()
assert len(rows) == 6
@pytest.mark.integration
@pytest.mark.parametrize(
"part_col", ["int", "bool", "string", "string_long", "long", "float", "double", "date", "timestamptz", "timestamp", "binary"]
)
def test_query_filter_v1_v2_append_null(
session_catalog: Catalog, spark: SparkSession, arrow_table_with_null: pa.Table, part_col: str
) -> None:
# Given
identifier = f"default.arrow_table_v1_v2_appended_with_null_partitioned_on_col_{part_col}"
nested_field = TABLE_SCHEMA.find_field(part_col)
partition_spec = PartitionSpec(
PartitionField(source_id=nested_field.field_id, field_id=1001, transform=IdentityTransform(), name=part_col)
)
# When
tbl = _create_table(
session_catalog=session_catalog,
identifier=identifier,
properties={"format-version": "1"},
data=[],
partition_spec=partition_spec,
)
tbl.append(arrow_table_with_null)
# Then
assert tbl.format_version == 1, f"Expected v1, got: v{tbl.format_version}"
# When
with tbl.transaction() as tx:
tx.upgrade_table_version(format_version=2)
tbl.append(arrow_table_with_null)
# Then
assert tbl.format_version == 2, f"Expected v2, got: v{tbl.format_version}"
for col in arrow_table_with_null.column_names: # type: ignore
df = spark.table(identifier)
assert df.where(f"{col} is not null").count() == 4, f"Expected 4 non-null rows for {col}"
assert df.where(f"{col} is null").count() == 2, f"Expected 2 null rows for {col}"
@pytest.mark.integration
def test_summaries_with_null(spark: SparkSession, session_catalog: Catalog, arrow_table_with_null: pa.Table) -> None:
identifier = "default.arrow_table_summaries"
try:
session_catalog.drop_table(identifier=identifier)
except NoSuchTableError:
pass
tbl = session_catalog.create_table(
identifier=identifier,
schema=TABLE_SCHEMA,
partition_spec=PartitionSpec(PartitionField(source_id=4, field_id=1001, transform=IdentityTransform(), name="int")),
properties={"format-version": "2"},
)
tbl.append(arrow_table_with_null)
tbl.append(arrow_table_with_null)
rows = spark.sql(
f"""
SELECT operation, summary
FROM {identifier}.snapshots
ORDER BY committed_at ASC
"""
).collect()
operations = [row.operation for row in rows]
assert operations == ["append", "append"]
summaries = [row.summary for row in rows]
file_size = int(summaries[0]["added-files-size"])
assert file_size > 0
assert summaries[0] == {
"changed-partition-count": "3",
"added-data-files": "3",
"added-files-size": str(file_size),
"added-records": "3",
"total-data-files": "3",
"total-delete-files": "0",
"total-equality-deletes": "0",
"total-files-size": str(file_size),
"total-position-deletes": "0",
"total-records": "3",
}
assert summaries[1] == {
"changed-partition-count": "3",
"added-data-files": "3",
"added-files-size": str(file_size),
"added-records": "3",
"total-data-files": "6",
"total-delete-files": "0",
"total-equality-deletes": "0",
"total-files-size": str(file_size * 2),
"total-position-deletes": "0",
"total-records": "6",
}
@pytest.mark.integration
def test_data_files_with_table_partitioned_with_null(
spark: SparkSession, session_catalog: Catalog, arrow_table_with_null: pa.Table
) -> None:
identifier = "default.arrow_data_files"
try:
session_catalog.drop_table(identifier=identifier)
except NoSuchTableError:
pass
tbl = session_catalog.create_table(
identifier=identifier,
schema=TABLE_SCHEMA,
partition_spec=PartitionSpec(PartitionField(source_id=4, field_id=1001, transform=IdentityTransform(), name="int")),
properties={"format-version": "1"},
)
tbl.append(arrow_table_with_null)
tbl.append(arrow_table_with_null)
# added_data_files_count, existing_data_files_count, deleted_data_files_count
rows = spark.sql(
f"""
SELECT added_data_files_count, existing_data_files_count, deleted_data_files_count
FROM {identifier}.all_manifests
"""
).collect()
assert [row.added_data_files_count for row in rows] == [3, 3, 3]
assert [row.existing_data_files_count for row in rows] == [
0,
0,
0,
]
assert [row.deleted_data_files_count for row in rows] == [0, 0, 0]
@pytest.mark.integration
def test_invalid_arguments(spark: SparkSession, session_catalog: Catalog) -> None:
identifier = "default.arrow_data_files"
try:
session_catalog.drop_table(identifier=identifier)
except NoSuchTableError:
pass
tbl = session_catalog.create_table(
identifier=identifier,
schema=TABLE_SCHEMA,
partition_spec=PartitionSpec(PartitionField(source_id=4, field_id=1001, transform=IdentityTransform(), name="int")),
properties={"format-version": "1"},
)
with pytest.raises(ValueError, match="Expected PyArrow table, got: not a df"):
tbl.append("not a df")
@pytest.mark.integration
@pytest.mark.parametrize(
"spec",
[
# mixed with non-identity is not supported
(
PartitionSpec(
PartitionField(source_id=4, field_id=1001, transform=BucketTransform(2), name="int_bucket"),
PartitionField(source_id=1, field_id=1002, transform=IdentityTransform(), name="bool"),
)
),
# none of non-identity is supported
(PartitionSpec(PartitionField(source_id=4, field_id=1001, transform=BucketTransform(2), name="int_bucket"))),
(PartitionSpec(PartitionField(source_id=5, field_id=1001, transform=BucketTransform(2), name="long_bucket"))),
(PartitionSpec(PartitionField(source_id=10, field_id=1001, transform=BucketTransform(2), name="date_bucket"))),
(PartitionSpec(PartitionField(source_id=8, field_id=1001, transform=BucketTransform(2), name="timestamp_bucket"))),
(PartitionSpec(PartitionField(source_id=9, field_id=1001, transform=BucketTransform(2), name="timestamptz_bucket"))),
(PartitionSpec(PartitionField(source_id=2, field_id=1001, transform=BucketTransform(2), name="string_bucket"))),
(PartitionSpec(PartitionField(source_id=12, field_id=1001, transform=BucketTransform(2), name="fixed_bucket"))),
(PartitionSpec(PartitionField(source_id=11, field_id=1001, transform=BucketTransform(2), name="binary_bucket"))),
(PartitionSpec(PartitionField(source_id=4, field_id=1001, transform=TruncateTransform(2), name="int_trunc"))),
(PartitionSpec(PartitionField(source_id=5, field_id=1001, transform=TruncateTransform(2), name="long_trunc"))),
(PartitionSpec(PartitionField(source_id=2, field_id=1001, transform=TruncateTransform(2), name="string_trunc"))),
(PartitionSpec(PartitionField(source_id=11, field_id=1001, transform=TruncateTransform(2), name="binary_trunc"))),
],
)
def test_unsupported_transform(
spec: PartitionSpec, spark: SparkSession, session_catalog: Catalog, arrow_table_with_null: pa.Table
) -> None:
identifier = "default.unsupported_transform"
try:
session_catalog.drop_table(identifier=identifier)
except NoSuchTableError:
pass
tbl = session_catalog.create_table(
identifier=identifier,
schema=TABLE_SCHEMA,
partition_spec=spec,
properties={"format-version": "1"},
)
with pytest.raises(
ValueError,
match="Not all partition types are supported for writes. Following partitions cannot be written using pyarrow: *",
):
tbl.append(arrow_table_with_null)
@pytest.mark.integration
@pytest.mark.parametrize(
"transform,expected_rows",
[
pytest.param(YearTransform(), 2, id="year_transform"),
pytest.param(MonthTransform(), 3, id="month_transform"),
pytest.param(DayTransform(), 3, id="day_transform"),
],
)
@pytest.mark.parametrize("part_col", ["date", "timestamp", "timestamptz"])
@pytest.mark.parametrize("format_version", [1, 2])
def test_append_ymd_transform_partitioned(
session_catalog: Catalog,
spark: SparkSession,
arrow_table_with_null: pa.Table,
transform: Transform[Any, Any],
expected_rows: int,
part_col: str,
format_version: int,
) -> None:
# Given
identifier = f"default.arrow_table_v{format_version}_with_{str(transform)}_partition_on_col_{part_col}"
nested_field = TABLE_SCHEMA.find_field(part_col)
partition_spec = PartitionSpec(
PartitionField(source_id=nested_field.field_id, field_id=1001, transform=transform, name=part_col)
)
# When
tbl = _create_table(
session_catalog=session_catalog,
identifier=identifier,
properties={"format-version": str(format_version)},
data=[arrow_table_with_null],
partition_spec=partition_spec,
)
# Then
assert tbl.format_version == format_version, f"Expected v{format_version}, got: v{tbl.format_version}"
df = spark.table(identifier)
assert df.count() == 3, f"Expected 3 total rows for {identifier}"
for col in arrow_table_with_null.column_names:
assert df.where(f"{col} is not null").count() == 2, f"Expected 2 non-null rows for {col}"
assert df.where(f"{col} is null").count() == 1, f"Expected 1 null row for {col} is null"
assert tbl.inspect.partitions().num_rows == expected_rows
files_df = spark.sql(
f"""
SELECT *
FROM {identifier}.files
"""
)
assert files_df.count() == expected_rows
@pytest.mark.integration
@pytest.mark.parametrize(
"transform,expected_partitions",
[
pytest.param(YearTransform(), {53, 54, None}, id="year_transform"),
pytest.param(MonthTransform(), {647, 648, 649, None}, id="month_transform"),
pytest.param(
DayTransform(), {date(2023, 12, 31), date(2024, 1, 1), date(2024, 1, 31), date(2024, 2, 1), None}, id="day_transform"
),
pytest.param(HourTransform(), {473328, 473352, 474072, 474096, 474102, None}, id="hour_transform"),
],
)
@pytest.mark.parametrize("format_version", [1, 2])
def test_append_transform_partition_verify_partitions_count(
session_catalog: Catalog,
spark: SparkSession,
arrow_table_date_timestamps: pa.Table,
arrow_table_date_timestamps_schema: Schema,
transform: Transform[Any, Any],
expected_partitions: Set[Any],
format_version: int,
) -> None:
# Given
part_col = "timestamptz"
identifier = f"default.arrow_table_v{format_version}_with_{str(transform)}_transform_partitioned_on_col_{part_col}"
nested_field = arrow_table_date_timestamps_schema.find_field(part_col)
partition_spec = PartitionSpec(
PartitionField(source_id=nested_field.field_id, field_id=1001, transform=transform, name=part_col),
)
# When
tbl = _create_table(
session_catalog=session_catalog,
identifier=identifier,
properties={"format-version": str(format_version)},
data=[arrow_table_date_timestamps],
partition_spec=partition_spec,
schema=arrow_table_date_timestamps_schema,
)
# Then
assert tbl.format_version == format_version, f"Expected v{format_version}, got: v{tbl.format_version}"
df = spark.table(identifier)
assert df.count() == 6, f"Expected 6 total rows for {identifier}"
for col in arrow_table_date_timestamps.column_names:
assert df.where(f"{col} is not null").count() == 5, f"Expected 2 non-null rows for {col}"
assert df.where(f"{col} is null").count() == 1, f"Expected 1 null row for {col} is null"
partitions_table = tbl.inspect.partitions()
assert partitions_table.num_rows == len(expected_partitions)
assert {part[part_col] for part in partitions_table["partition"].to_pylist()} == expected_partitions
files_df = spark.sql(
f"""
SELECT *
FROM {identifier}.files
"""
)
assert files_df.count() == len(expected_partitions)
@pytest.mark.integration
@pytest.mark.parametrize("format_version", [1, 2])
def test_append_multiple_partitions(
session_catalog: Catalog,
spark: SparkSession,
arrow_table_date_timestamps: pa.Table,
arrow_table_date_timestamps_schema: Schema,
format_version: int,
) -> None:
# Given
identifier = f"default.arrow_table_v{format_version}_with_multiple_partitions"
partition_spec = PartitionSpec(
PartitionField(
source_id=arrow_table_date_timestamps_schema.find_field("date").field_id,
field_id=1001,
transform=YearTransform(),
name="date_year",
),
PartitionField(
source_id=arrow_table_date_timestamps_schema.find_field("timestamptz").field_id,
field_id=1000,
transform=HourTransform(),
name="timestamptz_hour",
),
)
# When
tbl = _create_table(
session_catalog=session_catalog,
identifier=identifier,
properties={"format-version": str(format_version)},
data=[arrow_table_date_timestamps],
partition_spec=partition_spec,
schema=arrow_table_date_timestamps_schema,
)
# Then
assert tbl.format_version == format_version, f"Expected v{format_version}, got: v{tbl.format_version}"
df = spark.table(identifier)
assert df.count() == 6, f"Expected 6 total rows for {identifier}"
for col in arrow_table_date_timestamps.column_names:
assert df.where(f"{col} is not null").count() == 5, f"Expected 2 non-null rows for {col}"
assert df.where(f"{col} is null").count() == 1, f"Expected 1 null row for {col} is null"
partitions_table = tbl.inspect.partitions()
assert partitions_table.num_rows == 6
partitions = partitions_table["partition"].to_pylist()
assert {(part["date_year"], part["timestamptz_hour"]) for part in partitions} == {
(53, 473328),
(54, 473352),
(54, 474072),
(54, 474096),
(54, 474102),
(None, None),
}
files_df = spark.sql(
f"""
SELECT *
FROM {identifier}.files
"""
)
assert files_df.count() == 6