-
Notifications
You must be signed in to change notification settings - Fork 4.5k
Expand file tree
/
Copy pathparquetio_test.py
More file actions
1019 lines (925 loc) · 37.8 KB
/
parquetio_test.py
File metadata and controls
1019 lines (925 loc) · 37.8 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.
#
# pytype: skip-file
import glob
import json
import logging
import os
import re
import shutil
import tempfile
import unittest
from datetime import datetime
from tempfile import TemporaryDirectory
import hamcrest as hc
import pandas
import pytest
import pytz
from parameterized import param
from parameterized import parameterized
import apache_beam as beam
from apache_beam import Create
from apache_beam import Map
from apache_beam.io import filebasedsource
from apache_beam.io import source_test_utils
from apache_beam.io.iobase import RangeTracker
from apache_beam.io.parquetio import ReadAllFromParquet
from apache_beam.io.parquetio import ReadAllFromParquetBatched
from apache_beam.io.parquetio import ReadFromParquet
from apache_beam.io.parquetio import ReadFromParquetBatched
from apache_beam.io.parquetio import WriteToParquet
from apache_beam.io.parquetio import WriteToParquetBatched
from apache_beam.io.parquetio import _create_parquet_sink
from apache_beam.io.parquetio import _create_parquet_source
from apache_beam.testing.test_pipeline import TestPipeline
from apache_beam.testing.test_stream import TestStream
from apache_beam.testing.util import assert_that
from apache_beam.testing.util import equal_to
from apache_beam.transforms.display import DisplayData
from apache_beam.transforms.display_test import DisplayDataItemMatcher
from apache_beam.transforms.util import LogElements
try:
import pyarrow as pa
import pyarrow.parquet as pq
ARROW_MAJOR_VERSION, _, _ = map(int, pa.__version__.split('.'))
except ImportError:
pa = None
pq = None
ARROW_MAJOR_VERSION = 0
@unittest.skipIf(pa is None, "PyArrow is not installed.")
@pytest.mark.uses_pyarrow
class TestParquet(unittest.TestCase):
def setUp(self):
# Reducing the size of thread pools. Without this test execution may fail in
# environments with limited amount of resources.
filebasedsource.MAX_NUM_THREADS_FOR_SIZE_ESTIMATION = 2
self.temp_dir = tempfile.mkdtemp()
self.RECORDS = [{
'name': 'Thomas', 'favorite_number': 1, 'favorite_color': 'blue'
},
{
'name': 'Henry',
'favorite_number': 3,
'favorite_color': 'green'
},
{
'name': 'Toby',
'favorite_number': 7,
'favorite_color': 'brown'
},
{
'name': 'Gordon',
'favorite_number': 4,
'favorite_color': 'blue'
},
{
'name': 'Emily',
'favorite_number': -1,
'favorite_color': 'Red'
},
{
'name': 'Percy',
'favorite_number': 6,
'favorite_color': 'Green'
},
{
'name': 'Peter',
'favorite_number': 3,
'favorite_color': None
}]
self.SCHEMA = pa.schema([('name', pa.string(), False),
('favorite_number', pa.int64(), False),
('favorite_color', pa.string())])
self.SCHEMA96 = pa.schema([('name', pa.string(), False),
('favorite_number', pa.timestamp('ns'), False),
('favorite_color', pa.string())])
self.RECORDS_NESTED = [{
'items': [
{
'name': 'Thomas',
'favorite_number': 1,
'favorite_color': 'blue'
},
{
'name': 'Henry',
'favorite_number': 3,
'favorite_color': 'green'
},
]
},
{
'items': [
{
'name': 'Toby',
'favorite_number': 7,
'favorite_color': 'brown'
},
]
}]
self.SCHEMA_NESTED = pa.schema([(
'items',
pa.list_(
pa.struct([('name', pa.string(), False),
('favorite_number', pa.int64(), False),
('favorite_color', pa.string())])))])
def tearDown(self):
shutil.rmtree(self.temp_dir)
def _record_to_columns(self, records, schema):
col_list = []
for n in schema.names:
column = []
for r in records:
column.append(r[n])
col_list.append(column)
return col_list
def _records_as_arrow(self, schema=None, count=None):
if schema is None:
schema = self.SCHEMA
if count is None:
count = len(self.RECORDS)
len_records = len(self.RECORDS)
data = []
for i in range(count):
data.append(self.RECORDS[i % len_records])
col_data = self._record_to_columns(data, schema)
col_array = [pa.array(c, schema.types[cn]) for cn, c in enumerate(col_data)]
return pa.Table.from_arrays(col_array, schema=schema)
def _write_data(
self,
directory=None,
schema=None,
prefix=tempfile.template,
row_group_size=1000,
codec='none',
count=None):
if directory is None:
directory = self.temp_dir
with tempfile.NamedTemporaryFile(delete=False, dir=directory,
prefix=prefix) as f:
table = self._records_as_arrow(schema, count)
pq.write_table(
table,
f,
row_group_size=row_group_size,
compression=codec,
use_deprecated_int96_timestamps=True)
return f.name
def _write_pattern(self, num_files, with_filename=False):
assert num_files > 0
temp_dir = tempfile.mkdtemp(dir=self.temp_dir)
file_list = []
for _ in range(num_files):
file_list.append(self._write_data(directory=temp_dir, prefix='mytemp'))
if with_filename:
return (temp_dir + os.path.sep + 'mytemp*', file_list)
return temp_dir + os.path.sep + 'mytemp*'
def _run_parquet_test(
self,
pattern,
columns,
desired_bundle_size,
perform_splitting,
expected_result):
source = _create_parquet_source(pattern, columns=columns)
if perform_splitting:
assert desired_bundle_size
sources_info = [
(split.source, split.start_position, split.stop_position)
for split in source.split(desired_bundle_size=desired_bundle_size)
]
if len(sources_info) < 2:
raise ValueError(
'Test is trivial. Please adjust it so that at least '
'two splits get generated')
source_test_utils.assert_sources_equal_reference_source(
(source, None, None), sources_info)
else:
read_records = source_test_utils.read_from_source(source, None, None)
self.assertCountEqual(expected_result, read_records)
def test_read_without_splitting(self):
file_name = self._write_data()
expected_result = [self._records_as_arrow()]
self._run_parquet_test(file_name, None, None, False, expected_result)
def test_read_with_splitting(self):
file_name = self._write_data()
expected_result = [self._records_as_arrow()]
self._run_parquet_test(file_name, None, 100, True, expected_result)
def test_source_display_data(self):
file_name = 'some_parquet_source'
source = \
_create_parquet_source(
file_name,
validate=False
)
dd = DisplayData.create_from(source)
expected_items = [
DisplayDataItemMatcher('compression', 'auto'),
DisplayDataItemMatcher('file_pattern', file_name)
]
hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items))
def test_read_display_data(self):
file_name = 'some_parquet_source'
read = \
ReadFromParquet(
file_name,
validate=False)
read_batched = \
ReadFromParquetBatched(
file_name,
validate=False)
expected_items = [
DisplayDataItemMatcher('compression', 'auto'),
DisplayDataItemMatcher('file_pattern', file_name)
]
hc.assert_that(
DisplayData.create_from(read).items,
hc.contains_inanyorder(*expected_items))
hc.assert_that(
DisplayData.create_from(read_batched).items,
hc.contains_inanyorder(*expected_items))
def test_sink_display_data(self):
file_name = 'some_parquet_sink'
sink = _create_parquet_sink(
file_name,
self.SCHEMA,
'none',
False,
False,
'.end',
0,
None,
'application/x-parquet')
dd = DisplayData.create_from(sink)
expected_items = [
DisplayDataItemMatcher('schema', str(self.SCHEMA)),
DisplayDataItemMatcher(
'file_pattern',
'some_parquet_sink-%(shard_num)05d-of-%(num_shards)05d.end'),
DisplayDataItemMatcher('codec', 'none'),
DisplayDataItemMatcher('compression', 'uncompressed')
]
hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items))
def test_write_display_data(self):
file_name = 'some_parquet_sink'
write = WriteToParquet(file_name, self.SCHEMA)
dd = DisplayData.create_from(write)
expected_items = [
DisplayDataItemMatcher('codec', 'none'),
DisplayDataItemMatcher('schema', str(self.SCHEMA)),
DisplayDataItemMatcher('row_group_buffer_size', str(64 * 1024 * 1024)),
DisplayDataItemMatcher(
'file_pattern',
'some_parquet_sink-%(shard_num)05d-of-%(num_shards)05d'),
DisplayDataItemMatcher('compression', 'uncompressed')
]
hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items))
def test_write_batched_display_data(self):
file_name = 'some_parquet_sink'
write = WriteToParquetBatched(file_name, self.SCHEMA)
dd = DisplayData.create_from(write)
expected_items = [
DisplayDataItemMatcher('codec', 'none'),
DisplayDataItemMatcher('schema', str(self.SCHEMA)),
DisplayDataItemMatcher(
'file_pattern',
'some_parquet_sink-%(shard_num)05d-of-%(num_shards)05d'),
DisplayDataItemMatcher('compression', 'uncompressed')
]
hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items))
@unittest.skipIf(
ARROW_MAJOR_VERSION >= 13,
'pyarrow 13.x and above does not throw ArrowInvalid error')
def test_sink_transform_int96(self):
with self.assertRaisesRegex(Exception, 'would lose data'):
# Should throw an error "ArrowInvalid: Casting from timestamp[ns] to
# timestamp[us] would lose data"
dst = tempfile.NamedTemporaryFile()
path = dst.name
with TestPipeline() as p:
_ = p \
| Create(self.RECORDS) \
| WriteToParquet(
path, self.SCHEMA96, num_shards=1, shard_name_template='')
def test_sink_transform(self):
with TemporaryDirectory() as tmp_dirname:
path = os.path.join(tmp_dirname + "tmp_filename")
with TestPipeline() as p:
_ = p \
| Create(self.RECORDS) \
| WriteToParquet(
path, self.SCHEMA, num_shards=1, shard_name_template='')
with TestPipeline() as p:
# json used for stable sortability
readback = \
p \
| ReadFromParquet(path) \
| Map(json.dumps)
assert_that(readback, equal_to([json.dumps(r) for r in self.RECORDS]))
def test_sink_transform_batched(self):
with TemporaryDirectory() as tmp_dirname:
path = os.path.join(tmp_dirname + "tmp_filename")
with TestPipeline() as p:
_ = p \
| Create([self._records_as_arrow()]) \
| WriteToParquetBatched(
path, self.SCHEMA, num_shards=1, shard_name_template='')
with TestPipeline() as p:
# json used for stable sortability
readback = \
p \
| ReadFromParquet(path) \
| Map(json.dumps)
assert_that(readback, equal_to([json.dumps(r) for r in self.RECORDS]))
def test_sink_transform_compliant_nested_type(self):
if ARROW_MAJOR_VERSION < 4:
return unittest.skip(
'Writing with compliant nested type is only '
'supported in pyarrow 4.x and above')
with TemporaryDirectory() as tmp_dirname:
path = os.path.join(tmp_dirname + 'tmp_filename')
with TestPipeline() as p:
_ = p \
| Create(self.RECORDS_NESTED) \
| WriteToParquet(
path, self.SCHEMA_NESTED, num_shards=1,
shard_name_template='', use_compliant_nested_type=True)
with TestPipeline() as p:
# json used for stable sortability
readback = \
p \
| ReadFromParquet(path) \
| Map(json.dumps)
assert_that(
readback, equal_to([json.dumps(r) for r in self.RECORDS_NESTED]))
def test_schema_read_write(self):
with TemporaryDirectory() as tmp_dirname:
path = os.path.join(tmp_dirname, 'tmp_filename')
rows = [beam.Row(a=1, b='x'), beam.Row(a=2, b='y')]
stable_repr = lambda row: json.dumps(row._asdict())
with TestPipeline() as p:
_ = p | Create(rows) | WriteToParquet(path)
with TestPipeline() as p:
readback = (
p
| ReadFromParquet(path + '*', as_rows=True)
| Map(stable_repr))
assert_that(readback, equal_to([stable_repr(r) for r in rows]))
def test_write_with_nullable_fields_missing_data(self):
"""Test WriteToParquet with nullable fields where some fields are missing.
This test addresses the bug reported in:
https://github.com/apache/beam/issues/35791
where WriteToParquet fails with a KeyError if any nullable
field is missing in the data.
"""
# Define PyArrow schema with all fields nullable
schema = pa.schema([
pa.field("id", pa.int64(), nullable=True),
pa.field("name", pa.string(), nullable=True),
pa.field("age", pa.int64(), nullable=True),
pa.field("email", pa.string(), nullable=True),
])
# Sample data with missing nullable fields
data = [
{
'id': 1, 'name': 'Alice', 'age': 30
}, # missing 'email'
{
'id': 2, 'name': 'Bob', 'age': 25, 'email': '[email protected]'
}, # all fields present
{
'id': 3, 'name': 'Charlie', 'age': None, 'email': None
}, # explicit None values
{
'id': 4, 'name': 'David'
}, # missing 'age' and 'email'
]
with TemporaryDirectory() as tmp_dirname:
path = os.path.join(tmp_dirname, 'nullable_test')
# Write data with missing nullable fields - this should not raise KeyError
with TestPipeline() as p:
_ = (
p
| Create(data)
| WriteToParquet(
path, schema, num_shards=1, shard_name_template=''))
# Read back and verify the data
with TestPipeline() as p:
readback = (
p
| ReadFromParquet(path + '*')
| Map(json.dumps, sort_keys=True))
# Expected data should have None for missing nullable fields
expected_data = [
{
'id': 1, 'name': 'Alice', 'age': 30, 'email': None
},
{
'id': 2, 'name': 'Bob', 'age': 25, 'email': '[email protected]'
},
{
'id': 3, 'name': 'Charlie', 'age': None, 'email': None
},
{
'id': 4, 'name': 'David', 'age': None, 'email': None
},
]
assert_that(
readback,
equal_to([json.dumps(r, sort_keys=True) for r in expected_data]))
def test_batched_read(self):
with TemporaryDirectory() as tmp_dirname:
path = os.path.join(tmp_dirname + "tmp_filename")
with TestPipeline() as p:
_ = p \
| Create(self.RECORDS, reshuffle=False) \
| WriteToParquet(
path, self.SCHEMA, num_shards=1, shard_name_template='')
with TestPipeline() as p:
# json used for stable sortability
readback = \
p \
| ReadFromParquetBatched(path)
assert_that(readback, equal_to([self._records_as_arrow()]))
@parameterized.expand([
param(compression_type='snappy'),
param(compression_type='gzip'),
param(compression_type='brotli'),
param(compression_type='lz4'),
param(compression_type='zstd')
])
def test_sink_transform_compressed(self, compression_type):
if compression_type == 'lz4' and ARROW_MAJOR_VERSION == 1:
return unittest.skip(
"Writing with LZ4 compression is not supported in "
"pyarrow 1.x")
with TemporaryDirectory() as tmp_dirname:
path = os.path.join(tmp_dirname + "tmp_filename")
with TestPipeline() as p:
_ = p \
| Create(self.RECORDS) \
| WriteToParquet(
path, self.SCHEMA, codec=compression_type,
num_shards=1, shard_name_template='')
with TestPipeline() as p:
# json used for stable sortability
readback = \
p \
| ReadFromParquet(path + '*') \
| Map(json.dumps)
assert_that(readback, equal_to([json.dumps(r) for r in self.RECORDS]))
def test_read_reentrant(self):
file_name = self._write_data(count=6, row_group_size=3)
source = _create_parquet_source(file_name)
source_test_utils.assert_reentrant_reads_succeed((source, None, None))
def test_read_without_splitting_multiple_row_group(self):
file_name = self._write_data(count=12000, row_group_size=1000)
# We expect 12000 elements, split into batches of 1000 elements. Create
# a list of pa.Table instances to model this expecation
expected_result = [
pa.Table.from_batches([batch]) for batch in self._records_as_arrow(
count=12000).to_batches(max_chunksize=1000)
]
self._run_parquet_test(file_name, None, None, False, expected_result)
def test_read_with_splitting_multiple_row_group(self):
file_name = self._write_data(count=12000, row_group_size=1000)
# We expect 12000 elements, split into batches of 1000 elements. Create
# a list of pa.Table instances to model this expecation
expected_result = [
pa.Table.from_batches([batch]) for batch in self._records_as_arrow(
count=12000).to_batches(max_chunksize=1000)
]
self._run_parquet_test(file_name, None, 10000, True, expected_result)
def test_dynamic_work_rebalancing(self):
# This test depends on count being sufficiently large + the ratio of
# count to row_group_size also being sufficiently large (but the required
# ratio to pass varies for values of row_group_size and, somehow, the
# version of pyarrow being tested against.)
file_name = self._write_data(count=320, row_group_size=20)
source = _create_parquet_source(file_name)
splits = [split for split in source.split(desired_bundle_size=float('inf'))]
assert len(splits) == 1
source_test_utils.assert_split_at_fraction_exhaustive(
splits[0].source, splits[0].start_position, splits[0].stop_position)
def test_min_bundle_size(self):
file_name = self._write_data(count=120, row_group_size=20)
source = _create_parquet_source(
file_name, min_bundle_size=100 * 1024 * 1024)
splits = [split for split in source.split(desired_bundle_size=1)]
self.assertEqual(len(splits), 1)
source = _create_parquet_source(file_name, min_bundle_size=0)
splits = [split for split in source.split(desired_bundle_size=1)]
self.assertNotEqual(len(splits), 1)
def _convert_to_timestamped_record(self, record):
timestamped_record = record.copy()
timestamped_record['favorite_number'] =\
pandas.Timestamp(timestamped_record['favorite_number'])
return timestamped_record
def test_int96_type_conversion(self):
file_name = self._write_data(
count=120, row_group_size=20, schema=self.SCHEMA96)
orig = self._records_as_arrow(count=120, schema=self.SCHEMA96)
expected_result = [
pa.Table.from_batches([batch], schema=self.SCHEMA96)
for batch in orig.to_batches(max_chunksize=20)
]
self._run_parquet_test(file_name, None, None, False, expected_result)
def test_split_points(self):
file_name = self._write_data(count=12000, row_group_size=3000)
source = _create_parquet_source(file_name)
splits = [split for split in source.split(desired_bundle_size=float('inf'))]
assert len(splits) == 1
range_tracker = splits[0].source.get_range_tracker(
splits[0].start_position, splits[0].stop_position)
split_points_report = []
for _ in splits[0].source.read(range_tracker):
split_points_report.append(range_tracker.split_points())
# There are a total of four row groups. Each row group has 3000 records.
# When reading records of the first group, range_tracker.split_points()
# should return (0, iobase.RangeTracker.SPLIT_POINTS_UNKNOWN)
self.assertEqual(
split_points_report,
[
(0, RangeTracker.SPLIT_POINTS_UNKNOWN),
(1, RangeTracker.SPLIT_POINTS_UNKNOWN),
(2, RangeTracker.SPLIT_POINTS_UNKNOWN),
(3, 1),
])
def test_selective_columns(self):
file_name = self._write_data()
orig = self._records_as_arrow()
name_column = self.SCHEMA.field('name')
expected_result = [
pa.Table.from_arrays(
[orig.column('name')],
schema=pa.schema([('name', name_column.type, name_column.nullable)
]))
]
self._run_parquet_test(file_name, ['name'], None, False, expected_result)
def test_sink_transform_multiple_row_group(self):
with TemporaryDirectory() as tmp_dirname:
path = os.path.join(tmp_dirname + "tmp_filename")
# Pin to FnApiRunner since test assumes fixed bundle size
with TestPipeline('FnApiRunner') as p:
# writing 623200 bytes of data
_ = p \
| Create(self.RECORDS * 4000) \
| WriteToParquet(
path, self.SCHEMA, num_shards=1, codec='none',
shard_name_template='', row_group_buffer_size=250000)
self.assertEqual(pq.read_metadata(path).num_row_groups, 3)
def test_read_all_from_parquet_single_file(self):
path = self._write_data()
with TestPipeline() as p:
assert_that(
p \
| Create([path]) \
| ReadAllFromParquet(),
equal_to(self.RECORDS))
with TestPipeline() as p:
assert_that(
p \
| Create([path]) \
| ReadAllFromParquetBatched(),
equal_to([self._records_as_arrow()]))
def test_read_all_from_parquet_many_single_files(self):
path1 = self._write_data()
path2 = self._write_data()
path3 = self._write_data()
with TestPipeline() as p:
assert_that(
p \
| Create([path1, path2, path3]) \
| ReadAllFromParquet(),
equal_to(self.RECORDS * 3))
with TestPipeline() as p:
assert_that(
p \
| Create([path1, path2, path3]) \
| ReadAllFromParquetBatched(),
equal_to([self._records_as_arrow()] * 3))
def test_read_all_from_parquet_file_pattern(self):
file_pattern = self._write_pattern(5)
with TestPipeline() as p:
assert_that(
p \
| Create([file_pattern]) \
| ReadAllFromParquet(),
equal_to(self.RECORDS * 5))
with TestPipeline() as p:
assert_that(
p \
| Create([file_pattern]) \
| ReadAllFromParquetBatched(),
equal_to([self._records_as_arrow()] * 5))
def test_read_all_from_parquet_many_file_patterns(self):
file_pattern1 = self._write_pattern(5)
file_pattern2 = self._write_pattern(2)
file_pattern3 = self._write_pattern(3)
with TestPipeline() as p:
assert_that(
p \
| Create([file_pattern1, file_pattern2, file_pattern3]) \
| ReadAllFromParquet(),
equal_to(self.RECORDS * 10))
with TestPipeline() as p:
assert_that(
p \
| Create([file_pattern1, file_pattern2, file_pattern3]) \
| ReadAllFromParquetBatched(),
equal_to([self._records_as_arrow()] * 10))
def test_read_all_from_parquet_with_filename(self):
file_pattern, file_paths = self._write_pattern(3, with_filename=True)
result = [(path, record) for path in file_paths for record in self.RECORDS]
with TestPipeline() as p:
assert_that(
p \
| Create([file_pattern]) \
| ReadAllFromParquet(with_filename=True),
equal_to(result))
class GenerateEvent(beam.PTransform):
@staticmethod
def sample_data():
return GenerateEvent()
def expand(self, input):
elemlist = [{'age': 10}, {'age': 20}, {'age': 30}]
elem = elemlist
return (
input
| TestStream().add_elements(
elements=elem,
event_timestamp=datetime(
2021, 3, 1, 0, 0, 1, 0,
tzinfo=pytz.UTC).timestamp()).add_elements(
elements=elem,
event_timestamp=datetime(
2021, 3, 1, 0, 0, 2, 0,
tzinfo=pytz.UTC).timestamp()).add_elements(
elements=elem,
event_timestamp=datetime(
2021, 3, 1, 0, 0, 3, 0,
tzinfo=pytz.UTC).timestamp()).add_elements(
elements=elem,
event_timestamp=datetime(
2021, 3, 1, 0, 0, 4, 0,
tzinfo=pytz.UTC).timestamp()).
advance_watermark_to(
datetime(2021, 3, 1, 0, 0, 5, 0,
tzinfo=pytz.UTC).timestamp()).add_elements(
elements=elem,
event_timestamp=datetime(
2021, 3, 1, 0, 0, 5, 0,
tzinfo=pytz.UTC).timestamp()).
add_elements(
elements=elem,
event_timestamp=datetime(
2021, 3, 1, 0, 0, 6,
0, tzinfo=pytz.UTC).timestamp()).add_elements(
elements=elem,
event_timestamp=datetime(
2021, 3, 1, 0, 0, 7, 0,
tzinfo=pytz.UTC).timestamp()).add_elements(
elements=elem,
event_timestamp=datetime(
2021, 3, 1, 0, 0, 8, 0,
tzinfo=pytz.UTC).timestamp()).add_elements(
elements=elem,
event_timestamp=datetime(
2021, 3, 1, 0, 0, 9, 0,
tzinfo=pytz.UTC).timestamp()).
advance_watermark_to(
datetime(2021, 3, 1, 0, 0, 10, 0,
tzinfo=pytz.UTC).timestamp()).add_elements(
elements=elem,
event_timestamp=datetime(
2021, 3, 1, 0, 0, 10, 0,
tzinfo=pytz.UTC).timestamp()).add_elements(
elements=elem,
event_timestamp=datetime(
2021, 3, 1, 0, 0, 11, 0,
tzinfo=pytz.UTC).timestamp()).
add_elements(
elements=elem,
event_timestamp=datetime(
2021, 3, 1, 0, 0, 12, 0,
tzinfo=pytz.UTC).timestamp()).add_elements(
elements=elem,
event_timestamp=datetime(
2021, 3, 1, 0, 0, 13, 0,
tzinfo=pytz.UTC).timestamp()).add_elements(
elements=elem,
event_timestamp=datetime(
2021, 3, 1, 0, 0, 14, 0,
tzinfo=pytz.UTC).timestamp()).
advance_watermark_to(
datetime(2021, 3, 1, 0, 0, 15, 0,
tzinfo=pytz.UTC).timestamp()).add_elements(
elements=elem,
event_timestamp=datetime(
2021, 3, 1, 0, 0, 15, 0,
tzinfo=pytz.UTC).timestamp()).add_elements(
elements=elem,
event_timestamp=datetime(
2021, 3, 1, 0, 0, 16, 0,
tzinfo=pytz.UTC).timestamp()).
add_elements(
elements=elem,
event_timestamp=datetime(
2021, 3, 1, 0, 0, 17, 0,
tzinfo=pytz.UTC).timestamp()).add_elements(
elements=elem,
event_timestamp=datetime(
2021, 3, 1, 0, 0, 18, 0,
tzinfo=pytz.UTC).timestamp()).add_elements(
elements=elem,
event_timestamp=datetime(
2021, 3, 1, 0, 0, 19, 0,
tzinfo=pytz.UTC).timestamp()).
advance_watermark_to(
datetime(2021, 3, 1, 0, 0, 20, 0,
tzinfo=pytz.UTC).timestamp()).add_elements(
elements=elem,
event_timestamp=datetime(
2021, 3, 1, 0, 0, 20, 0,
tzinfo=pytz.UTC).timestamp()).advance_watermark_to(
datetime(
2021, 3, 1, 0, 0, 25, 0, tzinfo=pytz.UTC).
timestamp()).advance_watermark_to_infinity())
class WriteStreamingTest(unittest.TestCase):
def setUp(self):
super().setUp()
self.tempdir = tempfile.mkdtemp()
def tearDown(self):
if os.path.exists(self.tempdir):
shutil.rmtree(self.tempdir)
def test_write_streaming_2_shards_default_shard_name_template(
self, num_shards=2):
with TestPipeline() as p:
output = (p | GenerateEvent.sample_data())
#ParquetIO
pyschema = pa.schema([('age', pa.int64())])
output2 = output | 'WriteToParquet' >> beam.io.WriteToParquet(
file_path_prefix=self.tempdir + "/ouput_WriteToParquet",
file_name_suffix=".parquet",
num_shards=num_shards,
triggering_frequency=60,
schema=pyschema)
_ = output2 | 'LogElements after WriteToParquet' >> LogElements(
prefix='after WriteToParquet ', with_window=True, level=logging.INFO)
# Regex to match the expected windowed file pattern
# Example:
# ouput_WriteToParquet-[1614556800.0, 1614556805.0)-00000-of-00002.parquet
# It captures: window_interval, shard_num, total_shards
pattern_string = (
r'.*-\[(?P<window_start>[\d\.]+), '
r'(?P<window_end>[\d\.]+|Infinity)\)-'
r'(?P<shard_num>\d{5})-of-(?P<total_shards>\d{5})\.parquet$')
pattern = re.compile(pattern_string)
file_names = []
for file_name in glob.glob(self.tempdir + '/ouput_WriteToParquet*'):
match = pattern.match(file_name)
self.assertIsNotNone(
match, f"File name {file_name} did not match expected pattern.")
if match:
file_names.append(file_name)
print("Found files matching expected pattern:", file_names)
self.assertEqual(
len(file_names),
num_shards,
"expected %d files, but got: %d" % (num_shards, len(file_names)))
def test_write_streaming_2_shards_custom_shard_name_template(
self, num_shards=2, shard_name_template='-V-SSSSS-of-NNNNN'):
with TestPipeline() as p:
output = (p | GenerateEvent.sample_data())
#ParquetIO
pyschema = pa.schema([('age', pa.int64())])
output2 = output | 'WriteToParquet' >> beam.io.WriteToParquet(
file_path_prefix=self.tempdir + "/ouput_WriteToParquet",
file_name_suffix=".parquet",
shard_name_template=shard_name_template,
num_shards=num_shards,
triggering_frequency=60,
schema=pyschema)
_ = output2 | 'LogElements after WriteToParquet' >> LogElements(
prefix='after WriteToParquet ', with_window=True, level=logging.INFO)
# Regex to match the expected windowed file pattern
# Example:
# ouput_WriteToParquet-[2021-03-01T00-00-00, 2021-03-01T00-01-00)-
# 00000-of-00002.parquet
# It captures: window_interval, shard_num, total_shards
pattern_string = (
r'.*-\[(?P<window_start>\d{4}-\d{2}-\d{2}T\d{2}-\d{2}-\d{2}), '
r'(?P<window_end>\d{4}-\d{2}-\d{2}T\d{2}-\d{2}-\d{2}|Infinity)\)-'
r'(?P<shard_num>\d{5})-of-(?P<total_shards>\d{5})\.parquet$')
pattern = re.compile(pattern_string)
file_names = []
for file_name in glob.glob(self.tempdir + '/ouput_WriteToParquet*'):
match = pattern.match(file_name)
self.assertIsNotNone(
match, f"File name {file_name} did not match expected pattern.")
if match:
file_names.append(file_name)
print("Found files matching expected pattern:", file_names)
self.assertEqual(
len(file_names),
num_shards,
"expected %d files, but got: %d" % (num_shards, len(file_names)))
def test_write_streaming_2_shards_custom_shard_name_template_5s_window(
self,
num_shards=2,
shard_name_template='-V-SSSSS-of-NNNNN',
triggering_frequency=5):
with TestPipeline() as p:
output = (p | GenerateEvent.sample_data())
#ParquetIO
pyschema = pa.schema([('age', pa.int64())])
output2 = output | 'WriteToParquet' >> beam.io.WriteToParquet(
file_path_prefix=self.tempdir + "/ouput_WriteToParquet",
file_name_suffix=".parquet",
shard_name_template=shard_name_template,
num_shards=num_shards,
triggering_frequency=triggering_frequency,
schema=pyschema)
_ = output2 | 'LogElements after WriteToParquet' >> LogElements(
prefix='after WriteToParquet ', with_window=True, level=logging.INFO)
# Regex to match the expected windowed file pattern
# Example:
# ouput_WriteToParquet-[2021-03-01T00-00-00, 2021-03-01T00-01-00)-
# 00000-of-00002.parquet
# It captures: window_interval, shard_num, total_shards
pattern_string = (
r'.*-\[(?P<window_start>\d{4}-\d{2}-\d{2}T\d{2}-\d{2}-\d{2}), '
r'(?P<window_end>\d{4}-\d{2}-\d{2}T\d{2}-\d{2}-\d{2}|Infinity)\)-'
r'(?P<shard_num>\d{5})-of-(?P<total_shards>\d{5})\.parquet$')
pattern = re.compile(pattern_string)
file_names = []
for file_name in glob.glob(self.tempdir + '/ouput_WriteToParquet*'):
match = pattern.match(file_name)
self.assertIsNotNone(
match, f"File name {file_name} did not match expected pattern.")
if match:
file_names.append(file_name)
print("Found files matching expected pattern:", file_names)
# for 5s window size, the input should be processed by 5 windows with
# 2 shards per window
self.assertEqual(
len(file_names),
10,
"expected %d files, but got: %d" % (num_shards, len(file_names)))
def test_write_streaming_undef_shards_default_shard_name_template_windowed_pcoll( # pylint: disable=line-too-long
self):
with TestPipeline() as p:
output = (
p | GenerateEvent.sample_data()
| 'User windowing' >> beam.transforms.core.WindowInto(
beam.transforms.window.FixedWindows(10),
trigger=beam.transforms.trigger.AfterWatermark(),
accumulation_mode=beam.transforms.trigger.AccumulationMode.
DISCARDING,
allowed_lateness=beam.utils.timestamp.Duration(seconds=0)))
#ParquetIO
pyschema = pa.schema([('age', pa.int64())])
output2 = output | 'WriteToParquet' >> beam.io.WriteToParquet(
file_path_prefix=self.tempdir + "/ouput_WriteToParquet",
file_name_suffix=".parquet",
num_shards=0,
schema=pyschema)
_ = output2 | 'LogElements after WriteToParquet' >> LogElements(
prefix='after WriteToParquet ', with_window=True, level=logging.INFO)
# Regex to match the expected windowed file pattern
# Example:
# ouput_WriteToParquet-[1614556800.0, 1614556805.0)-00000-of-00002.parquet
# It captures: window_interval, shard_num, total_shards
pattern_string = (
r'.*-\[(?P<window_start>[\d\.]+), '
r'(?P<window_end>[\d\.]+|Infinity)\)-'