forked from apache/iceberg-python
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathtest_reads.py
More file actions
1322 lines (1053 loc) · 54.4 KB
/
test_reads.py
File metadata and controls
1322 lines (1053 loc) · 54.4 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
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
# 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
import math
import time
import uuid
from datetime import datetime, timedelta
from pathlib import PosixPath
from typing import Any
from urllib.parse import urlparse
import pyarrow as pa
import pyarrow.parquet as pq
import pytest
from hive_metastore.ttypes import LockRequest, LockResponse, LockState, UnlockRequest
from pyarrow.fs import S3FileSystem
from pydantic_core import ValidationError
from pyspark.sql import SparkSession
from pytest_lazy_fixtures import lf
from pyiceberg.catalog import Catalog
from pyiceberg.catalog.hive import HiveCatalog, _HiveClient
from pyiceberg.exceptions import CommitFailedException, NoSuchTableError
from pyiceberg.expressions import (
And,
EqualTo,
GreaterThanOrEqual,
IsNaN,
LessThan,
NotEqualTo,
NotNaN,
NotNull,
)
from pyiceberg.io.pyarrow import (
pyarrow_to_schema,
)
from pyiceberg.schema import Schema
from pyiceberg.table import Table
from pyiceberg.types import (
BinaryType,
BooleanType,
IntegerType,
LongType,
NestedField,
StringType,
TimestampType,
)
from pyiceberg.utils.concurrent import ExecutorFactory
DEFAULT_PROPERTIES = {"write.parquet.compression-codec": "zstd"}
TABLE_NAME = ("default", "t1")
def create_table(catalog: Catalog) -> Table:
try:
catalog.drop_table(TABLE_NAME)
except NoSuchTableError:
pass # Just to make sure that the table doesn't exist
schema = Schema(
NestedField(field_id=1, name="str", field_type=StringType(), required=False),
NestedField(field_id=2, name="int", field_type=IntegerType(), required=True),
NestedField(field_id=3, name="bool", field_type=BooleanType(), required=False),
NestedField(field_id=4, name="datetime", field_type=TimestampType(), required=False),
)
return catalog.create_table(identifier=TABLE_NAME, schema=schema)
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
def test_table_properties(catalog: Catalog) -> None:
table = create_table(catalog)
assert table.properties == DEFAULT_PROPERTIES
with table.transaction() as transaction:
transaction.set_properties(abc="🤪")
assert table.properties == dict(abc="🤪", **DEFAULT_PROPERTIES)
with table.transaction() as transaction:
transaction.remove_properties("abc")
assert table.properties == DEFAULT_PROPERTIES
table = table.transaction().set_properties(abc="def").commit_transaction()
assert table.properties == dict(abc="def", **DEFAULT_PROPERTIES)
table = table.transaction().remove_properties("abc").commit_transaction()
assert table.properties == DEFAULT_PROPERTIES
table = table.transaction().set_properties(abc=123).commit_transaction()
# properties are stored as strings in the iceberg spec
assert table.properties == dict(abc="123", **DEFAULT_PROPERTIES)
with pytest.raises(ValidationError) as exc_info:
table.transaction().set_properties(property_name=None).commit_transaction()
assert "None type is not a supported value in properties: property_name" in str(exc_info.value)
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive")])
def test_hive_properties(catalog: Catalog) -> None:
table = create_table(catalog)
table.transaction().set_properties({"abc": "def", "p1": "123"}).commit_transaction()
hive_client: _HiveClient = _HiveClient(catalog.properties["uri"])
with hive_client as open_client:
hive_table = open_client.get_table(*TABLE_NAME)
assert hive_table.parameters.get("abc") == "def"
assert hive_table.parameters.get("p1") == "123"
assert hive_table.parameters.get("not_exist_parameter") is None
table.transaction().remove_properties("abc").commit_transaction()
with hive_client as open_client:
hive_table = open_client.get_table(*TABLE_NAME)
assert hive_table.parameters.get("abc") is None
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive")])
def test_hive_preserves_hms_specific_properties(catalog: Catalog) -> None:
"""Test that HMS-specific table properties are preserved during table commits.
This verifies that HMS-specific properties that are not managed by Iceberg
are preserved during commits, rather than being lost.
Regression test for: https://github.com/apache/iceberg-python/issues/2926
"""
table = create_table(catalog)
hive_client: _HiveClient = _HiveClient(catalog.properties["uri"])
with hive_client as open_client:
hive_table = open_client.get_table(*TABLE_NAME)
# Add HMS-specific properties that aren't managed by Iceberg
hive_table.parameters["table_category"] = "production"
hive_table.parameters["data_owner"] = "data_team"
open_client.alter_table(TABLE_NAME[0], TABLE_NAME[1], hive_table)
with hive_client as open_client:
hive_table = open_client.get_table(*TABLE_NAME)
assert hive_table.parameters.get("table_category") == "production"
assert hive_table.parameters.get("data_owner") == "data_team"
table.transaction().set_properties({"iceberg_property": "new_value"}).commit_transaction()
# Verify that HMS-specific properties are STILL present after commit
with hive_client as open_client:
hive_table = open_client.get_table(*TABLE_NAME)
# HMS-specific properties should be preserved
assert hive_table.parameters.get("table_category") == "production", (
"HMS property 'table_category' was lost during commit!"
)
assert hive_table.parameters.get("data_owner") == "data_team", "HMS property 'data_owner' was lost during commit!"
# Iceberg properties should also be present
assert hive_table.parameters.get("iceberg_property") == "new_value"
@pytest.mark.integration
def test_iceberg_property_deletion_not_restored_from_old_hms_state(session_catalog_hive: Catalog) -> None:
"""Test that deleted Iceberg properties are truly removed and not restored from old HMS state.
When a property is removed through Iceberg, it should be deleted from HMS and not
come back from the old HMS state during merge operations.
"""
table = create_table(session_catalog_hive)
hive_client: _HiveClient = _HiveClient(session_catalog_hive.properties["uri"])
# Set multiple Iceberg properties
table.transaction().set_properties({"prop_to_keep": "keep_value", "prop_to_delete": "delete_me"}).commit_transaction()
# Verify both properties exist
with hive_client as open_client:
hive_table = open_client.get_table(*TABLE_NAME)
assert hive_table.parameters.get("prop_to_keep") == "keep_value"
assert hive_table.parameters.get("prop_to_delete") == "delete_me"
# Delete one property through Iceberg
table.transaction().remove_properties("prop_to_delete").commit_transaction()
# Verify property is deleted from HMS
with hive_client as open_client:
hive_table = open_client.get_table(*TABLE_NAME)
assert hive_table.parameters.get("prop_to_keep") == "keep_value"
assert hive_table.parameters.get("prop_to_delete") is None, "Deleted property should not exist in HMS!"
# Perform another Iceberg commit
table.transaction().set_properties({"new_prop": "new_value"}).commit_transaction()
# Ensure deleted property doesn't come back from old state
with hive_client as open_client:
hive_table = open_client.get_table(*TABLE_NAME)
assert hive_table.parameters.get("prop_to_keep") == "keep_value"
assert hive_table.parameters.get("new_prop") == "new_value"
assert hive_table.parameters.get("prop_to_delete") is None, "Deleted property should NOT be restored from old HMS state!"
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive")])
def test_iceberg_metadata_is_source_of_truth(catalog: Catalog) -> None:
"""Test that Iceberg metadata is the source of truth for all Iceberg-managed properties.
If an external tool sets an HMS property with the same name as an Iceberg-managed
property, Iceberg's value should win during commits.
"""
table = create_table(catalog)
hive_client: _HiveClient = _HiveClient(catalog.properties["uri"])
# Set an Iceberg property
table.transaction().set_properties({"my_prop": "iceberg_value"}).commit_transaction()
# External tool modifies the same property in HMS
with hive_client as open_client:
hive_table = open_client.get_table(*TABLE_NAME)
hive_table.parameters["my_prop"] = "hms_value" # Conflicting value
open_client.alter_table(TABLE_NAME[0], TABLE_NAME[1], hive_table)
# Verify HMS has the external value
with hive_client as open_client:
hive_table = open_client.get_table(*TABLE_NAME)
assert hive_table.parameters.get("my_prop") == "hms_value"
# Perform another Iceberg commit
table.transaction().set_properties({"another_prop": "test"}).commit_transaction()
# Iceberg's value should take precedence
with hive_client as open_client:
hive_table = open_client.get_table(*TABLE_NAME)
assert hive_table.parameters.get("my_prop") == "iceberg_value", (
"Iceberg property value should take precedence over conflicting HMS value!"
)
assert hive_table.parameters.get("another_prop") == "test"
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive")])
def test_hive_critical_properties_always_from_iceberg(catalog: Catalog) -> None:
"""Test that critical properties (EXTERNAL, table_type, metadata_location) always come from Iceberg.
These properties should never be carried over from old HMS state.
"""
table = create_table(catalog)
hive_client: _HiveClient = _HiveClient(catalog.properties["uri"])
# Get original metadata_location
with hive_client as open_client:
hive_table = open_client.get_table(*TABLE_NAME)
original_metadata_location = hive_table.parameters.get("metadata_location")
assert original_metadata_location is not None
assert hive_table.parameters.get("EXTERNAL") == "TRUE"
assert hive_table.parameters.get("table_type") == "ICEBERG"
# Try to tamper with critical properties via HMS
with hive_client as open_client:
hive_table = open_client.get_table(*TABLE_NAME)
hive_table.parameters["EXTERNAL"] = "FALSE" # Try to change
open_client.alter_table(TABLE_NAME[0], TABLE_NAME[1], hive_table)
# Perform Iceberg commit
table.transaction().set_properties({"test_prop": "value"}).commit_transaction()
# Critical properties should be restored by Iceberg
with hive_client as open_client:
hive_table = open_client.get_table(*TABLE_NAME)
assert hive_table.parameters.get("EXTERNAL") == "TRUE", "EXTERNAL should always be TRUE from Iceberg!"
assert hive_table.parameters.get("table_type") == "ICEBERG", "table_type should always be ICEBERG!"
# metadata_location should be updated (new metadata file)
new_metadata_location = hive_table.parameters.get("metadata_location")
assert new_metadata_location != original_metadata_location, "metadata_location should be updated!"
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive")])
def test_hive_native_properties_cannot_be_deleted_via_iceberg(catalog: Catalog) -> None:
"""Test that HMS-native properties (set outside Iceberg) cannot be deleted via Iceberg.
HMS-native properties are not visible to Iceberg, so remove_properties fails with KeyError.
However, if you first SET an HMS property via Iceberg (making it tracked in Iceberg metadata),
it can then be deleted via Iceberg.
"""
table = create_table(catalog)
hive_client: _HiveClient = _HiveClient(catalog.properties["uri"])
# Set an HMS-native property directly (not through Iceberg)
with hive_client as open_client:
hive_table = open_client.get_table(*TABLE_NAME)
hive_table.parameters["hms_native_prop"] = "native_value"
open_client.alter_table(TABLE_NAME[0], TABLE_NAME[1], hive_table)
# Verify the HMS-native property exists in HMS
with hive_client as open_client:
hive_table = open_client.get_table(*TABLE_NAME)
assert hive_table.parameters.get("hms_native_prop") == "native_value"
# Refresh the Iceberg table to get the latest state
table.refresh()
# Verify the HMS-native property is NOT visible in Iceberg
assert "hms_native_prop" not in table.properties
# Attempt to remove the HMS-native property via Iceberg - this should fail
# because the property is not tracked in Iceberg metadata (not visible to Iceberg)
with pytest.raises(KeyError):
table.transaction().remove_properties("hms_native_prop").commit_transaction()
# HMS-native property should still exist (cannot be deleted via Iceberg)
with hive_client as open_client:
hive_table = open_client.get_table(*TABLE_NAME)
assert hive_table.parameters.get("hms_native_prop") == "native_value", (
"HMS-native property should still exist since Iceberg removal failed!"
)
# Now SET the same property via Iceberg (this makes it tracked in Iceberg metadata)
table.transaction().set_properties({"hms_native_prop": "iceberg_value"}).commit_transaction()
# Verify it's updated in both places
with hive_client as open_client:
hive_table = open_client.get_table(*TABLE_NAME)
assert hive_table.parameters.get("hms_native_prop") == "iceberg_value"
# Now we CAN delete it via Iceberg (because it's now tracked in Iceberg metadata)
table.transaction().remove_properties("hms_native_prop").commit_transaction()
# Property should be deleted from HMS
with hive_client as open_client:
hive_table = open_client.get_table(*TABLE_NAME)
assert hive_table.parameters.get("hms_native_prop") is None, (
"Property should be deletable after being SET via Iceberg (making it tracked)!"
)
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
def test_table_properties_dict(catalog: Catalog) -> None:
table = create_table(catalog)
assert table.properties == DEFAULT_PROPERTIES
with table.transaction() as transaction:
transaction.set_properties({"abc": "🤪"})
assert table.properties == dict({"abc": "🤪"}, **DEFAULT_PROPERTIES)
with table.transaction() as transaction:
transaction.remove_properties("abc")
assert table.properties == DEFAULT_PROPERTIES
table = table.transaction().set_properties({"abc": "def"}).commit_transaction()
assert table.properties == dict({"abc": "def"}, **DEFAULT_PROPERTIES)
table = table.transaction().remove_properties("abc").commit_transaction()
assert table.properties == DEFAULT_PROPERTIES
table = table.transaction().set_properties({"abc": 123}).commit_transaction()
# properties are stored as strings in the iceberg spec
assert table.properties == dict({"abc": "123"}, **DEFAULT_PROPERTIES)
with pytest.raises(ValidationError) as exc_info:
table.transaction().set_properties({"property_name": None}).commit_transaction()
assert "None type is not a supported value in properties: property_name" in str(exc_info.value)
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
def test_table_properties_error(catalog: Catalog) -> None:
table = create_table(catalog)
properties = {"abc": "def"}
with pytest.raises(ValueError) as e:
table.transaction().set_properties(properties, abc="def").commit_transaction()
assert "Cannot pass both properties and kwargs" in str(e.value)
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
def test_pyarrow_nan(catalog: Catalog) -> None:
table_test_null_nan = catalog.load_table("default.test_null_nan")
arrow_table = table_test_null_nan.scan(row_filter=IsNaN("col_numeric"), selected_fields=("idx", "col_numeric")).to_arrow()
assert len(arrow_table) == 1
assert arrow_table["idx"][0].as_py() == 1
assert math.isnan(arrow_table["col_numeric"][0].as_py())
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
def test_pyarrow_nan_rewritten(catalog: Catalog) -> None:
table_test_null_nan_rewritten = catalog.load_table("default.test_null_nan_rewritten")
arrow_table = table_test_null_nan_rewritten.scan(
row_filter=IsNaN("col_numeric"), selected_fields=("idx", "col_numeric")
).to_arrow()
assert len(arrow_table) == 1
assert arrow_table["idx"][0].as_py() == 1
assert math.isnan(arrow_table["col_numeric"][0].as_py())
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
@pytest.mark.skip(reason="Fixing issues with NaN's: https://github.com/apache/arrow/issues/34162")
def test_pyarrow_not_nan_count(catalog: Catalog) -> None:
table_test_null_nan = catalog.load_table("default.test_null_nan")
not_nan = table_test_null_nan.scan(row_filter=NotNaN("col_numeric"), selected_fields=("idx",)).to_arrow()
assert len(not_nan) == 2
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
def test_pyarrow_batches_nan(catalog: Catalog) -> None:
table_test_null_nan = catalog.load_table("default.test_null_nan")
arrow_batch_reader = table_test_null_nan.scan(
row_filter=IsNaN("col_numeric"), selected_fields=("idx", "col_numeric")
).to_arrow_batch_reader()
assert isinstance(arrow_batch_reader, pa.RecordBatchReader)
arrow_table = arrow_batch_reader.read_all()
assert len(arrow_table) == 1
assert arrow_table["idx"][0].as_py() == 1
assert math.isnan(arrow_table["col_numeric"][0].as_py())
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
def test_pyarrow_batches_nan_rewritten(catalog: Catalog) -> None:
table_test_null_nan_rewritten = catalog.load_table("default.test_null_nan_rewritten")
arrow_batch_reader = table_test_null_nan_rewritten.scan(
row_filter=IsNaN("col_numeric"), selected_fields=("idx", "col_numeric")
).to_arrow_batch_reader()
assert isinstance(arrow_batch_reader, pa.RecordBatchReader)
arrow_table = arrow_batch_reader.read_all()
assert len(arrow_table) == 1
assert arrow_table["idx"][0].as_py() == 1
assert math.isnan(arrow_table["col_numeric"][0].as_py())
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
@pytest.mark.skip(reason="Fixing issues with NaN's: https://github.com/apache/arrow/issues/34162")
def test_pyarrow_batches_not_nan_count(catalog: Catalog) -> None:
table_test_null_nan = catalog.load_table("default.test_null_nan")
arrow_batch_reader = table_test_null_nan.scan(
row_filter=NotNaN("col_numeric"), selected_fields=("idx",)
).to_arrow_batch_reader()
assert isinstance(arrow_batch_reader, pa.RecordBatchReader)
arrow_table = arrow_batch_reader.read_all()
assert len(arrow_table) == 2
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
def test_duckdb_nan(catalog: Catalog) -> None:
table_test_null_nan_rewritten = catalog.load_table("default.test_null_nan_rewritten")
con = table_test_null_nan_rewritten.scan().to_duckdb("table_test_null_nan")
result = con.query("SELECT idx, col_numeric FROM table_test_null_nan WHERE isnan(col_numeric)").fetchone()
assert result[0] == 1
assert math.isnan(result[1])
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
def test_pyarrow_limit(catalog: Catalog) -> None:
table_test_limit = catalog.load_table("default.test_limit")
limited_result = table_test_limit.scan(selected_fields=("idx",), limit=1).to_arrow()
assert len(limited_result) == 1
empty_result = table_test_limit.scan(selected_fields=("idx",), limit=0).to_arrow()
assert len(empty_result) == 0
full_result = table_test_limit.scan(selected_fields=("idx",), limit=999).to_arrow()
assert len(full_result) == 10
# test `to_arrow_batch_reader`
limited_result = table_test_limit.scan(selected_fields=("idx",), limit=1).to_arrow_batch_reader().read_all()
assert len(limited_result) == 1
empty_result = table_test_limit.scan(selected_fields=("idx",), limit=0).to_arrow_batch_reader().read_all()
assert len(empty_result) == 0
full_result = table_test_limit.scan(selected_fields=("idx",), limit=999).to_arrow_batch_reader().read_all()
assert len(full_result) == 10
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
def test_pyarrow_limit_with_multiple_files(catalog: Catalog) -> None:
table_name = "default.test_pyarrow_limit_with_multiple_files"
try:
catalog.drop_table(table_name)
except NoSuchTableError:
pass
reference_table = catalog.load_table("default.test_limit")
data = reference_table.scan().to_arrow()
table_test_limit = catalog.create_table(table_name, schema=reference_table.schema())
n_files = 2
for _ in range(n_files):
table_test_limit.append(data)
assert len(table_test_limit.inspect.files()) == n_files
# test with multiple files
limited_result = table_test_limit.scan(selected_fields=("idx",), limit=1).to_arrow()
assert len(limited_result) == 1
empty_result = table_test_limit.scan(selected_fields=("idx",), limit=0).to_arrow()
assert len(empty_result) == 0
full_result = table_test_limit.scan(selected_fields=("idx",), limit=999).to_arrow()
assert len(full_result) == 10 * n_files
# test `to_arrow_batch_reader`
limited_result = table_test_limit.scan(selected_fields=("idx",), limit=1).to_arrow_batch_reader().read_all()
assert len(limited_result) == 1
empty_result = table_test_limit.scan(selected_fields=("idx",), limit=0).to_arrow_batch_reader().read_all()
assert len(empty_result) == 0
full_result = table_test_limit.scan(selected_fields=("idx",), limit=999).to_arrow_batch_reader().read_all()
assert len(full_result) == 10 * n_files
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
def test_daft_nan(catalog: Catalog) -> None:
table_test_null_nan_rewritten = catalog.load_table("default.test_null_nan_rewritten")
df = table_test_null_nan_rewritten.to_daft()
assert df.count_rows() == 3
assert math.isnan(df.to_pydict()["col_numeric"][0])
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
def test_daft_nan_rewritten(catalog: Catalog) -> None:
table_test_null_nan_rewritten = catalog.load_table("default.test_null_nan_rewritten")
df = table_test_null_nan_rewritten.to_daft()
df = df.where(df["col_numeric"].is_nan())
df = df.select("idx", "col_numeric")
assert df.count_rows() == 1
assert df.to_pydict()["idx"][0] == 1
assert math.isnan(df.to_pydict()["col_numeric"][0])
@pytest.mark.skip(reason="Bodo should not monekeypatch PyArrowFileIO, https://github.com/apache/iceberg-python/issues/2400")
@pytest.mark.integration
@pytest.mark.filterwarnings("ignore")
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
def test_bodo_nan(catalog: Catalog, monkeypatch: pytest.MonkeyPatch) -> None:
# Avoid local Mac issues (see https://github.com/apache/iceberg-python/issues/2225)
monkeypatch.setenv("BODO_DATAFRAME_LIBRARY_RUN_PARALLEL", "0")
monkeypatch.setenv("FI_PROVIDER", "tcp")
table_test_null_nan_rewritten = catalog.load_table("default.test_null_nan_rewritten")
df = table_test_null_nan_rewritten.to_bodo()
assert len(df) == 3
assert math.isnan(df.col_numeric.iloc[0])
@pytest.mark.integration
@pytest.mark.filterwarnings("ignore")
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
def test_ray_nan(catalog: Catalog, ray_session: Any) -> None:
table_test_null_nan_rewritten = catalog.load_table("default.test_null_nan_rewritten")
ray_dataset = table_test_null_nan_rewritten.scan().to_ray()
assert ray_dataset.count() == 3
assert math.isnan(ray_dataset.take()[0]["col_numeric"])
@pytest.mark.integration
@pytest.mark.filterwarnings("ignore")
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
def test_ray_nan_rewritten(catalog: Catalog, ray_session: Any) -> None:
table_test_null_nan_rewritten = catalog.load_table("default.test_null_nan_rewritten")
ray_dataset = table_test_null_nan_rewritten.scan(
row_filter=IsNaN("col_numeric"), selected_fields=("idx", "col_numeric")
).to_ray()
assert ray_dataset.count() == 1
assert ray_dataset.take()[0]["idx"] == 1
assert math.isnan(ray_dataset.take()[0]["col_numeric"])
@pytest.mark.integration
@pytest.mark.filterwarnings("ignore")
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
@pytest.mark.skip(reason="Fixing issues with NaN's: https://github.com/apache/arrow/issues/34162")
def test_ray_not_nan_count(catalog: Catalog, ray_session: Any) -> None:
table_test_null_nan_rewritten = catalog.load_table("default.test_null_nan_rewritten")
ray_dataset = table_test_null_nan_rewritten.scan(row_filter=NotNaN("col_numeric"), selected_fields=("idx",)).to_ray()
assert ray_dataset.count() == 2
@pytest.mark.integration
@pytest.mark.filterwarnings("ignore")
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
def test_ray_all_types(catalog: Catalog, ray_session: Any) -> None:
table_test_all_types = catalog.load_table("default.test_all_types")
ray_dataset = table_test_all_types.scan().to_ray()
pandas_dataframe = table_test_all_types.scan().to_pandas()
assert ray_dataset.count() == pandas_dataframe.shape[0]
assert pandas_dataframe.equals(ray_dataset.to_pandas())
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
def test_pyarrow_to_iceberg_all_types(catalog: Catalog) -> None:
table_test_all_types = catalog.load_table("default.test_all_types")
fs = S3FileSystem(
endpoint_override=catalog.properties["s3.endpoint"],
access_key=catalog.properties["s3.access-key-id"],
secret_key=catalog.properties["s3.secret-access-key"],
)
data_file_paths = [task.file.file_path for task in table_test_all_types.scan().plan_files()]
for data_file_path in data_file_paths:
uri = urlparse(data_file_path)
with fs.open_input_file(f"{uri.netloc}{uri.path}") as fout:
parquet_schema = pq.read_schema(fout)
stored_iceberg_schema = Schema.model_validate_json(parquet_schema.metadata.get(b"iceberg.schema"))
converted_iceberg_schema = pyarrow_to_schema(parquet_schema)
assert converted_iceberg_schema == stored_iceberg_schema
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
@pytest.mark.parametrize("format_version", [2, 3])
def test_pyarrow_deletes(catalog: Catalog, format_version: int) -> None:
# number, letter
# (1, 'a'),
# (2, 'b'),
# (3, 'c'),
# (4, 'd'),
# (5, 'e'),
# (6, 'f'),
# (7, 'g'),
# (8, 'h'),
# (9, 'i'), <- deleted
# (10, 'j'),
# (11, 'k'),
# (12, 'l')
test_positional_mor_deletes = catalog.load_table(f"default.test_positional_mor_deletes_v{format_version}")
if format_version == 2:
assert len(test_positional_mor_deletes.inspect.delete_files()) > 0, "Table should produce position delete files"
arrow_table = test_positional_mor_deletes.scan().to_arrow()
assert arrow_table["number"].to_pylist() == [1, 2, 3, 4, 5, 6, 7, 8, 10, 11, 12]
# Checking the filter
arrow_table = test_positional_mor_deletes.scan(
row_filter=And(GreaterThanOrEqual("letter", "e"), LessThan("letter", "k"))
).to_arrow()
assert arrow_table["number"].to_pylist() == [5, 6, 7, 8, 10]
# Testing the combination of a filter and a limit
arrow_table = test_positional_mor_deletes.scan(
row_filter=And(GreaterThanOrEqual("letter", "e"), LessThan("letter", "k")), limit=1
).to_arrow()
assert arrow_table["number"].to_pylist() == [5]
# Testing the slicing of indices
arrow_table = test_positional_mor_deletes.scan(limit=3).to_arrow()
assert arrow_table["number"].to_pylist() == [1, 2, 3]
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
@pytest.mark.parametrize("format_version", [2, 3])
def test_pyarrow_deletes_double(catalog: Catalog, format_version: int) -> None:
# number, letter
# (1, 'a'),
# (2, 'b'),
# (3, 'c'),
# (4, 'd'),
# (5, 'e'),
# (6, 'f'), <- second delete
# (7, 'g'),
# (8, 'h'),
# (9, 'i'), <- first delete
# (10, 'j'),
# (11, 'k'),
# (12, 'l')
test_positional_mor_double_deletes = catalog.load_table(f"default.test_positional_mor_double_deletes_v{format_version}")
if format_version == 2:
assert len(test_positional_mor_double_deletes.inspect.delete_files()) > 0, "Table should produce position delete files"
arrow_table = test_positional_mor_double_deletes.scan().to_arrow()
assert arrow_table["number"].to_pylist() == [1, 2, 3, 4, 5, 7, 8, 10, 11, 12]
# Checking the filter
arrow_table = test_positional_mor_double_deletes.scan(
row_filter=And(GreaterThanOrEqual("letter", "e"), LessThan("letter", "k"))
).to_arrow()
assert arrow_table["number"].to_pylist() == [5, 7, 8, 10]
# Testing the combination of a filter and a limit
arrow_table = test_positional_mor_double_deletes.scan(
row_filter=And(GreaterThanOrEqual("letter", "e"), LessThan("letter", "k")), limit=1
).to_arrow()
assert arrow_table["number"].to_pylist() == [5]
# Testing the slicing of indices
arrow_table = test_positional_mor_double_deletes.scan(limit=8).to_arrow()
assert arrow_table["number"].to_pylist() == [1, 2, 3, 4, 5, 7, 8, 10]
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
@pytest.mark.parametrize("format_version", [2, 3])
def test_pyarrow_batches_deletes(catalog: Catalog, format_version: int) -> None:
# number, letter
# (1, 'a'),
# (2, 'b'),
# (3, 'c'),
# (4, 'd'),
# (5, 'e'),
# (6, 'f'),
# (7, 'g'),
# (8, 'h'),
# (9, 'i'), <- deleted
# (10, 'j'),
# (11, 'k'),
# (12, 'l')
test_positional_mor_deletes = catalog.load_table(f"default.test_positional_mor_deletes_v{format_version}")
if format_version == 2:
assert len(test_positional_mor_deletes.inspect.delete_files()) > 0, "Table should produce position delete files"
arrow_table = test_positional_mor_deletes.scan().to_arrow_batch_reader().read_all()
assert arrow_table["number"].to_pylist() == [1, 2, 3, 4, 5, 6, 7, 8, 10, 11, 12]
# Checking the filter
arrow_table = (
test_positional_mor_deletes.scan(row_filter=And(GreaterThanOrEqual("letter", "e"), LessThan("letter", "k")))
.to_arrow_batch_reader()
.read_all()
)
assert arrow_table["number"].to_pylist() == [5, 6, 7, 8, 10]
# Testing the combination of a filter and a limit
arrow_table = (
test_positional_mor_deletes.scan(row_filter=And(GreaterThanOrEqual("letter", "e"), LessThan("letter", "k")), limit=1)
.to_arrow_batch_reader()
.read_all()
)
assert arrow_table["number"].to_pylist() == [5]
# Testing the slicing of indices
arrow_table = test_positional_mor_deletes.scan(limit=3).to_arrow_batch_reader().read_all()
assert arrow_table["number"].to_pylist() == [1, 2, 3]
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
@pytest.mark.parametrize("format_version", [2, 3])
def test_pyarrow_batches_deletes_double(catalog: Catalog, format_version: int) -> None:
# number, letter
# (1, 'a'),
# (2, 'b'),
# (3, 'c'),
# (4, 'd'),
# (5, 'e'),
# (6, 'f'), <- second delete
# (7, 'g'),
# (8, 'h'),
# (9, 'i'), <- first delete
# (10, 'j'),
# (11, 'k'),
# (12, 'l')
test_positional_mor_double_deletes = catalog.load_table(f"default.test_positional_mor_double_deletes_v{format_version}")
if format_version == 2:
assert len(test_positional_mor_double_deletes.inspect.delete_files()) > 0, "Table should produce position delete files"
arrow_table = test_positional_mor_double_deletes.scan().to_arrow_batch_reader().read_all()
assert arrow_table["number"].to_pylist() == [1, 2, 3, 4, 5, 7, 8, 10, 11, 12]
# Checking the filter
arrow_table = (
test_positional_mor_double_deletes.scan(row_filter=And(GreaterThanOrEqual("letter", "e"), LessThan("letter", "k")))
.to_arrow_batch_reader()
.read_all()
)
assert arrow_table["number"].to_pylist() == [5, 7, 8, 10]
# Testing the combination of a filter and a limit
arrow_table = (
test_positional_mor_double_deletes.scan(
row_filter=And(GreaterThanOrEqual("letter", "e"), LessThan("letter", "k")), limit=1
)
.to_arrow_batch_reader()
.read_all()
)
assert arrow_table["number"].to_pylist() == [5]
# Testing the slicing of indices
arrow_table = test_positional_mor_double_deletes.scan(limit=8).to_arrow_batch_reader().read_all()
assert arrow_table["number"].to_pylist() == [1, 2, 3, 4, 5, 7, 8, 10]
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
def test_partitioned_tables(catalog: Catalog) -> None:
for table_name, predicate in [
("test_partitioned_by_identity", "ts >= '2023-03-05T00:00:00+00:00'"),
("test_partitioned_by_years", "dt >= '2023-03-05'"),
("test_partitioned_by_months", "dt >= '2023-03-05'"),
("test_partitioned_by_days", "ts >= '2023-03-05T00:00:00+00:00'"),
("test_partitioned_by_hours", "ts >= '2023-03-05T00:00:00+00:00'"),
("test_partitioned_by_truncate", "letter >= 'e'"),
("test_partitioned_by_bucket", "number >= '5'"),
]:
table = catalog.load_table(f"default.{table_name}")
arrow_table = table.scan(selected_fields=("number",), row_filter=predicate).to_arrow()
assert set(arrow_table["number"].to_pylist()) == {5, 6, 7, 8, 9, 10, 11, 12}, f"Table {table_name}, predicate {predicate}"
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
def test_unpartitioned_uuid_table(catalog: Catalog) -> None:
unpartitioned_uuid = catalog.load_table("default.test_uuid_and_fixed_unpartitioned")
arrow_table_eq = unpartitioned_uuid.scan(row_filter="uuid_col == '102cb62f-e6f8-4eb0-9973-d9b012ff0967'").to_arrow()
assert arrow_table_eq["uuid_col"].to_pylist() == [uuid.UUID("102cb62f-e6f8-4eb0-9973-d9b012ff0967")]
arrow_table_neq = unpartitioned_uuid.scan(
row_filter="uuid_col != '102cb62f-e6f8-4eb0-9973-d9b012ff0967' and uuid_col != '639cccce-c9d2-494a-a78c-278ab234f024'"
).to_arrow()
assert arrow_table_neq["uuid_col"].to_pylist() == [
uuid.UUID("ec33e4b2-a834-4cc3-8c4a-a1d3bfc2f226"),
uuid.UUID("c1b0d8e0-0b0e-4b1e-9b0a-0e0b0d0c0a0b"),
uuid.UUID("923dae77-83d6-47cd-b4b0-d383e64ee57e"),
]
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
def test_unpartitioned_fixed_table(catalog: Catalog) -> None:
fixed_table = catalog.load_table("default.test_uuid_and_fixed_unpartitioned")
arrow_table_eq = fixed_table.scan(row_filter=EqualTo("fixed_col", b"1234567890123456789012345")).to_arrow()
assert arrow_table_eq["fixed_col"].to_pylist() == [b"1234567890123456789012345"]
arrow_table_neq = fixed_table.scan(
row_filter=And(
NotEqualTo("fixed_col", b"1234567890123456789012345"), NotEqualTo("uuid_col", "c1b0d8e0-0b0e-4b1e-9b0a-0e0b0d0c0a0b")
)
).to_arrow()
assert arrow_table_neq["fixed_col"].to_pylist() == [
b"1231231231231231231231231",
b"12345678901234567ass12345",
b"qweeqwwqq1231231231231111",
]
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
@pytest.mark.parametrize("format_version", [2, 3])
def test_scan_tag(catalog: Catalog, format_version: int) -> None:
test_positional_mor_deletes = catalog.load_table(f"default.test_positional_mor_deletes_v{format_version}")
arrow_table = test_positional_mor_deletes.scan().use_ref("tag_12").to_arrow()
assert arrow_table["number"].to_pylist() == [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12]
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
@pytest.mark.parametrize("format_version", [2, 3])
def test_scan_branch(catalog: Catalog, format_version: int) -> None:
test_positional_mor_deletes = catalog.load_table(f"default.test_positional_mor_deletes_v{format_version}")
arrow_table = test_positional_mor_deletes.scan().use_ref("without_5").to_arrow()
assert arrow_table["number"].to_pylist() == [1, 2, 3, 4, 6, 7, 8, 9, 10, 11, 12]
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
def test_filter_on_new_column(catalog: Catalog) -> None:
test_table_add_column = catalog.load_table("default.test_table_add_column")
arrow_table = test_table_add_column.scan(row_filter="b == '2'").to_arrow()
assert arrow_table["b"].to_pylist() == ["2"]
arrow_table = test_table_add_column.scan(row_filter="b is not null").to_arrow()
assert arrow_table["b"].to_pylist() == ["2"]
arrow_table = test_table_add_column.scan(row_filter="b is null").to_arrow()
assert arrow_table["b"].to_pylist() == [None]
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
def test_filter_case_sensitive_by_default(catalog: Catalog) -> None:
test_table_add_column = catalog.load_table("default.test_table_add_column")
arrow_table = test_table_add_column.scan().to_arrow()
assert "2" in arrow_table["b"].to_pylist()
arrow_table = test_table_add_column.scan(row_filter="b == '2'").to_arrow()
assert arrow_table["b"].to_pylist() == ["2"]
with pytest.raises(ValueError) as e:
_ = test_table_add_column.scan(row_filter="B == '2'").to_arrow()
assert "Could not find field with name B" in str(e.value)
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
def test_filter_case_sensitive(catalog: Catalog) -> None:
test_table_add_column = catalog.load_table("default.test_table_add_column")
arrow_table = test_table_add_column.scan().to_arrow()
assert "2" in arrow_table["b"].to_pylist()
arrow_table = test_table_add_column.scan(row_filter="b == '2'", case_sensitive=True).to_arrow()
assert arrow_table["b"].to_pylist() == ["2"]
with pytest.raises(ValueError) as e:
_ = test_table_add_column.scan(row_filter="B == '2'", case_sensitive=True).to_arrow()
assert "Could not find field with name B" in str(e.value)
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
def test_filter_case_insensitive(catalog: Catalog) -> None:
test_table_add_column = catalog.load_table("default.test_table_add_column")
arrow_table = test_table_add_column.scan().to_arrow()
assert "2" in arrow_table["b"].to_pylist()
arrow_table = test_table_add_column.scan(row_filter="b == '2'", case_sensitive=False).to_arrow()
assert arrow_table["b"].to_pylist() == ["2"]
arrow_table = test_table_add_column.scan(row_filter="B == '2'", case_sensitive=False).to_arrow()
assert arrow_table["b"].to_pylist() == ["2"]
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
def test_filters_on_top_level_struct(catalog: Catalog) -> None:
test_empty_struct = catalog.load_table("default.test_table_empty_list_and_map")
arrow_table = test_empty_struct.scan().to_arrow()
assert None in arrow_table["col_struct"].to_pylist()
arrow_table = test_empty_struct.scan(row_filter=NotNull("col_struct")).to_arrow()
assert arrow_table["col_struct"].to_pylist() == [{"test": 1}]
arrow_table = test_empty_struct.scan(row_filter="col_struct is not null", case_sensitive=False).to_arrow()
assert arrow_table["col_struct"].to_pylist() == [{"test": 1}]
arrow_table = test_empty_struct.scan(row_filter="COL_STRUCT is null", case_sensitive=False).to_arrow()
assert arrow_table["col_struct"].to_pylist() == [None]
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
def test_upgrade_table_version(catalog: Catalog) -> None:
table_test_table_version = catalog.load_table("default.test_table_version")
assert table_test_table_version.format_version == 1
with table_test_table_version.transaction() as transaction:
transaction.upgrade_table_version(format_version=1)
assert table_test_table_version.format_version == 1
with table_test_table_version.transaction() as transaction:
transaction.upgrade_table_version(format_version=2)
assert table_test_table_version.format_version == 2
with pytest.raises(ValueError) as e: # type: ignore
with table_test_table_version.transaction() as transaction:
transaction.upgrade_table_version(format_version=1)
assert "Cannot downgrade v2 table to v1" in str(e.value)
with pytest.raises(ValueError) as e:
with table_test_table_version.transaction() as transaction:
transaction.upgrade_table_version(format_version=3)
assert "Unsupported table format version: 3" in str(e.value)
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
def test_sanitize_character(catalog: Catalog) -> None:
table_test_table_sanitized_character = catalog.load_table("default.test_table_sanitized_character")
arrow_table = table_test_table_sanitized_character.scan().to_arrow()
assert len(arrow_table.schema.names), 1
assert len(table_test_table_sanitized_character.schema().fields), 1
assert arrow_table.schema.names[0] == table_test_table_sanitized_character.schema().fields[0].name
@pytest.mark.integration
@pytest.mark.parametrize("catalog", [lf("session_catalog_hive"), lf("session_catalog")])
def test_null_list_and_map(catalog: Catalog) -> None:
table_test_empty_list_and_map = catalog.load_table("default.test_table_empty_list_and_map")
arrow_table = table_test_empty_list_and_map.scan().to_arrow()
assert arrow_table["col_list"].to_pylist() == [None, []]
assert arrow_table["col_map"].to_pylist() == [None, []]
# This should be:
# assert arrow_table["col_list_with_struct"].to_pylist() == [None, [{'test': 1}]]
# Once https://github.com/apache/arrow/issues/38809 has been fixed
assert arrow_table["col_list_with_struct"].to_pylist() == [[], [{"test": 1}]]
@pytest.mark.integration