/
test_parquet.py
1415 lines (1144 loc) · 50.3 KB
/
test_parquet.py
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
from __future__ import (absolute_import, division, print_function,
unicode_literals)
import os
from distutils.version import LooseVersion
import numpy as np
import pandas as pd
import pandas.util.testing as tm
import pytest
import dask
import dask.multiprocessing
import dask.dataframe as dd
from dask.dataframe.utils import assert_eq
from dask.dataframe.io.parquet import _parse_pandas_metadata
from dask.utils import natural_sort_key
try:
import fastparquet
except ImportError:
fastparquet = False
try:
import pyarrow as pa
pa_version = LooseVersion(pa.__version__)
check_pa_divs = pa_version >= '0.9.0'
except ImportError:
pa_version = None
check_pa_divs = False
try:
import pyarrow.parquet as pq
except ImportError:
pq = False
SKIP_FASTPARQUET = not fastparquet
SKIP_FASTPARQUET_REASON = 'fastparquet not found'
FASTPARQUET_MARK = pytest.mark.skipif(SKIP_FASTPARQUET, reason=SKIP_FASTPARQUET_REASON)
if pa_version == '0.13.0':
SKIP_PYARROW = True
SKIP_PYARROW_REASON = 'pyarrow 0.13.0 not supported'
else:
SKIP_PYARROW = not pq
SKIP_PYARROW_REASON = 'pyarrow not found'
PYARROW_MARK = pytest.mark.skipif(SKIP_PYARROW, reason=SKIP_PYARROW_REASON)
def check_fastparquet():
if SKIP_FASTPARQUET:
pytest.skip(SKIP_FASTPARQUET_REASON)
def check_pyarrow():
if SKIP_PYARROW:
pytest.skip(SKIP_PYARROW_REASON)
def should_check_divs(engine):
if engine == 'fastparquet':
return True
elif engine == 'pyarrow' and check_pa_divs:
return True
return False
nrows = 40
npartitions = 15
df = pd.DataFrame({'x': [i * 7 % 5 for i in range(nrows)], # Not sorted
'y': [i * 2.5 for i in range(nrows)] # Sorted
},
index=pd.Index([10 * i for i in range(nrows)], name='myindex'))
ddf = dd.from_pandas(df, npartitions=npartitions)
@pytest.fixture(params=[
pytest.param('fastparquet', marks=FASTPARQUET_MARK),
pytest.param('pyarrow', marks=PYARROW_MARK)])
def engine(request):
return request.param
def write_read_engines(**kwargs):
"""Product of both engines for write/read:
To add custom marks, pass keyword of the form: `mark_writer_reader=reason`,
or `mark_engine=reason` to apply to all parameters with that engine."""
backends = {'pyarrow', 'fastparquet'}
marks = {(w, r): [] for w in backends for r in backends}
# Skip if uninstalled
for name, skip, reason in [('fastparquet', SKIP_FASTPARQUET, SKIP_FASTPARQUET_REASON),
('pyarrow', SKIP_PYARROW, SKIP_PYARROW_REASON)]:
if skip:
val = pytest.mark.skip(reason=reason)
for k in marks:
if name in k:
marks[k].append(val)
# Custom marks
for kw, val in kwargs.items():
kind, rest = kw.split('_', 1)
key = tuple(rest.split('_'))
if (kind not in ('xfail', 'skip') or len(key) > 2 or
set(key).difference(backends)):
raise ValueError("unknown keyword %r" % kw)
val = getattr(pytest.mark, kind)(reason=val)
if len(key) == 2:
marks[key].append(val)
else:
for k in marks:
if key in k:
marks[k].append(val)
return pytest.mark.parametrize(('write_engine', 'read_engine'),
[pytest.param(*k, marks=tuple(v))
for (k, v) in sorted(marks.items())])
pyarrow_fastparquet_msg = "fastparquet fails reading pyarrow written directories"
write_read_engines_xfail = write_read_engines(xfail_pyarrow_fastparquet=pyarrow_fastparquet_msg)
@write_read_engines_xfail
def test_local(tmpdir, write_engine, read_engine):
tmp = str(tmpdir)
data = pd.DataFrame({'i32': np.arange(1000, dtype=np.int32),
'i64': np.arange(1000, dtype=np.int64),
'f': np.arange(1000, dtype=np.float64),
'bhello': np.random.choice(['hello', 'yo', 'people'], size=1000).astype("O")})
df = dd.from_pandas(data, chunksize=500)
df.to_parquet(tmp, write_index=False, engine=write_engine)
files = os.listdir(tmp)
assert '_common_metadata' in files
assert 'part.0.parquet' in files
df2 = dd.read_parquet(tmp, index=False, engine=read_engine)
assert len(df2.divisions) > 1
out = df2.compute(scheduler='sync').reset_index()
for column in df.columns:
assert (data[column] == out[column]).all()
@write_read_engines_xfail
def test_index(tmpdir, write_engine, read_engine):
fn = str(tmpdir)
ddf.to_parquet(fn, engine=write_engine)
# Infer divisions for engines/versions that support it
ddf2 = dd.read_parquet(fn, engine=read_engine, infer_divisions=should_check_divs(read_engine))
assert_eq(ddf, ddf2, check_divisions=should_check_divs(read_engine))
# infer_divisions False
ddf2_no_divs = dd.read_parquet(fn, engine=read_engine, infer_divisions=False)
assert_eq(ddf.clear_divisions(), ddf2_no_divs, check_divisions=True)
# infer_divisions unspecified
ddf2_default = dd.read_parquet(fn, engine=read_engine)
if read_engine == 'fastparquet':
# The fastparquet engine infers divisions by default because it only supports reading datasets that have a
# global _metadata file
assert_eq(ddf, ddf2_default, check_divisions=True)
else:
# pyarrow does not infer divisions by default because doing so requires reading metadata from each file in
# the dataset, which could be expensive
assert_eq(ddf.clear_divisions(), ddf2_default, check_divisions=True)
@pytest.mark.parametrize('index', [False, True])
@write_read_engines_xfail
def test_empty(tmpdir, write_engine, read_engine, index):
fn = str(tmpdir)
df = pd.DataFrame({'a': ['a', 'b', 'b'], 'b': [4, 5, 6]})[:0]
if index:
df.set_index('a', inplace=True, drop=True)
ddf = dd.from_pandas(df, npartitions=2)
ddf.to_parquet(fn, write_index=index, engine=write_engine)
read_df = dd.read_parquet(fn, engine=read_engine)
assert_eq(ddf, read_df)
@write_read_engines()
def test_read_glob(tmpdir, write_engine, read_engine):
if write_engine == read_engine == 'fastparquet' and os.name == 'nt':
# fastparquet or dask is not normalizing filepaths correctly on
# windows.
pytest.skip("filepath bug.")
fn = str(tmpdir)
ddf.to_parquet(fn, engine=write_engine)
if os.path.exists(os.path.join(fn, '_metadata')):
os.unlink(os.path.join(fn, '_metadata'))
files = os.listdir(fn)
assert '_metadata' not in files
# Infer divisions for engines/versions that support it
ddf2 = dd.read_parquet(os.path.join(fn, '*.parquet'), engine=read_engine,
infer_divisions=should_check_divs(write_engine) and should_check_divs(read_engine))
assert_eq(ddf, ddf2, check_divisions=should_check_divs(write_engine) and should_check_divs(read_engine))
# No divisions
ddf2_no_divs = dd.read_parquet(os.path.join(fn, '*.parquet'),
engine=read_engine, infer_divisions=False)
assert_eq(ddf.clear_divisions(), ddf2_no_divs, check_divisions=True)
@write_read_engines()
def test_read_list(tmpdir, write_engine, read_engine):
if write_engine == read_engine == 'fastparquet' and os.name == 'nt':
# fastparquet or dask is not normalizing filepaths correctly on
# windows.
pytest.skip("filepath bug.")
tmpdir = str(tmpdir)
ddf.to_parquet(tmpdir, engine=write_engine)
files = sorted([os.path.join(tmpdir, f)
for f in os.listdir(tmpdir)
if not f.endswith('_metadata')],
key=natural_sort_key)
# Infer divisions for engines/versions that support it
ddf2 = dd.read_parquet(files, engine=read_engine,
infer_divisions=should_check_divs(write_engine) and should_check_divs(read_engine))
assert_eq(ddf, ddf2, check_divisions=should_check_divs(write_engine) and should_check_divs(read_engine))
# No divisions
ddf2_no_divs = dd.read_parquet(files, engine=read_engine, infer_divisions=False)
assert_eq(ddf.clear_divisions(), ddf2_no_divs, check_divisions=True)
@write_read_engines_xfail
def test_columns_index(tmpdir, write_engine, read_engine):
fn = str(tmpdir)
ddf.to_parquet(fn, engine=write_engine)
# With Index
# ----------
# ### Emtpy columns ###
# With divisions if supported
assert_eq(dd.read_parquet(fn, columns=[], engine=read_engine, infer_divisions=should_check_divs(read_engine)),
ddf[[]], check_divisions=should_check_divs(read_engine))
# No divisions
assert_eq(dd.read_parquet(fn, columns=[], engine=read_engine, infer_divisions=False),
ddf[[]].clear_divisions(), check_divisions=True)
# ### Single column, auto select index ###
# With divisions if supported
assert_eq(dd.read_parquet(fn, columns=['x'], engine=read_engine, infer_divisions=should_check_divs(read_engine)),
ddf[['x']], check_divisions=should_check_divs(read_engine))
# No divisions
assert_eq(dd.read_parquet(fn, columns=['x'], engine=read_engine, infer_divisions=False),
ddf[['x']].clear_divisions(), check_divisions=True)
# ### Single column, specify index ###
# With divisions if supported
assert_eq(dd.read_parquet(fn, index='myindex', columns=['x'], engine=read_engine,
infer_divisions=should_check_divs(read_engine)),
ddf[['x']], check_divisions=should_check_divs(read_engine))
# No divisions
assert_eq(dd.read_parquet(fn, index='myindex', columns=['x'], engine=read_engine,
infer_divisions=False),
ddf[['x']].clear_divisions(), check_divisions=True)
# ### Two columns, specify index ###
# With divisions if supported
assert_eq(dd.read_parquet(fn, index='myindex', columns=['x', 'y'], engine=read_engine,
infer_divisions=should_check_divs(read_engine)),
ddf, check_divisions=should_check_divs(read_engine))
# No divisions
assert_eq(dd.read_parquet(fn, index='myindex', columns=['x', 'y'], engine=read_engine,
infer_divisions=False),
ddf.clear_divisions(), check_divisions=True)
@write_read_engines_xfail
def test_nonsense_column(tmpdir, write_engine, read_engine):
fn = str(tmpdir)
ddf.to_parquet(fn, engine=write_engine)
with pytest.raises((ValueError, KeyError)):
# fastparquet fails early, pyarrow only on compute
dd.read_parquet(fn, columns=['nonesense'], engine=read_engine
).compute()
with pytest.raises((Exception, KeyError)):
# fastparquet fails early, pyarrow only on compute
dd.read_parquet(fn, columns=['nonesense'] + list(ddf.columns),
engine=read_engine).compute()
@write_read_engines_xfail
def test_columns_no_index(tmpdir, write_engine, read_engine):
fn = str(tmpdir)
ddf.to_parquet(fn, engine=write_engine)
ddf2 = ddf.reset_index()
# No Index
# --------
# All columns, none as index
assert_eq(dd.read_parquet(fn, index=False, engine=read_engine,
infer_divisions=False),
ddf2, check_index=False, check_divisions=True)
# Two columns, none as index
assert_eq(dd.read_parquet(fn, index=False, columns=['x', 'y'], engine=read_engine,
infer_divisions=False),
ddf2[['x', 'y']], check_index=False, check_divisions=True)
# One column and one index, all as columns
assert_eq(dd.read_parquet(fn, index=False, columns=['myindex', 'x'], engine=read_engine,
infer_divisions=False),
ddf2[['myindex', 'x']], check_index=False, check_divisions=True)
@write_read_engines_xfail
def test_infer_divisions_not_sorted(tmpdir, write_engine, read_engine):
fn = str(tmpdir)
ddf.to_parquet(fn, engine=write_engine)
if read_engine == 'pyarrow' and not check_pa_divs:
match = 'requires pyarrow >=0.9.0'
ex = NotImplementedError
else:
match = 'not known to be sorted across partitions'
ex = ValueError
with pytest.raises(ex, match=match):
dd.read_parquet(fn, index='x', engine=read_engine, infer_divisions=True)
@write_read_engines_xfail
def test_infer_divisions_no_index(tmpdir, write_engine, read_engine):
if read_engine == 'pyarrow' and pa.__version__ >= LooseVersion('0.13.0'):
pytest.skip("No longer an error from pyarrow 0.13.0")
fn = str(tmpdir)
ddf.to_parquet(fn, engine=write_engine, write_index=False)
if read_engine == 'pyarrow' and not check_pa_divs:
match = 'requires pyarrow >=0.9.0'
ex = NotImplementedError
else:
match = 'no index column was discovered'
ex = ValueError
with pytest.raises(ex, match=match):
dd.read_parquet(fn, engine=read_engine, infer_divisions=True)
def test_columns_index_with_multi_index(tmpdir, engine):
fn = os.path.join(str(tmpdir), 'test.parquet')
index = pd.MultiIndex.from_arrays([np.arange(10), np.arange(10) + 1],
names=['x0', 'x1'])
df = pd.DataFrame(np.random.randn(10, 2), columns=['a', 'b'], index=index)
df2 = df.reset_index(drop=False)
if engine == 'fastparquet':
fastparquet.write(fn, df, write_index=True)
# fastparquet doesn't support multi-index
with pytest.raises(ValueError):
ddf = dd.read_parquet(fn, engine=engine)
else:
import pyarrow as pa
pq.write_table(pa.Table.from_pandas(df), fn)
# Pyarrow supports multi-index reads
ddf = dd.read_parquet(fn, engine=engine)
assert_eq(ddf, df)
d = dd.read_parquet(fn, columns='a', engine=engine)
assert_eq(d, df['a'])
d = dd.read_parquet(fn, index=['a', 'b'], columns=['x0', 'x1'], engine=engine)
assert_eq(d, df2.set_index(['a', 'b'])[['x0', 'x1']])
# Just index
d = dd.read_parquet(fn, index=False, engine=engine)
assert_eq(d, df2)
d = dd.read_parquet(fn, index=['a'], engine=engine)
assert_eq(d, df2.set_index('a')[['b']])
d = dd.read_parquet(fn, index=['x0'], engine=engine)
assert_eq(d, df2.set_index('x0')[['a', 'b']])
# Just columns
d = dd.read_parquet(fn, columns=['x0', 'a'], engine=engine)
assert_eq(d, df2.set_index('x1')[['x0', 'a']])
# Both index and columns
d = dd.read_parquet(fn, index=False, columns=['x0', 'b'], engine=engine)
assert_eq(d, df2[['x0', 'b']])
for index in ['x1', 'b']:
d = dd.read_parquet(fn, index=index, columns=['x0', 'a'], engine=engine)
assert_eq(d, df2.set_index(index)[['x0', 'a']])
# Columns and index intersect
for index in ['a', 'x0']:
with pytest.raises(ValueError):
d = dd.read_parquet(fn, index=index, columns=['x0', 'a'], engine=engine)
# Series output
for ind, col, sol_df in [(None, 'x0', df2.set_index('x1')),
(False, 'b', df2),
(False, 'x0', df2),
('a', 'x0', df2.set_index('a')),
('a', 'b', df2.set_index('a'))]:
d = dd.read_parquet(fn, index=ind, columns=col, engine=engine)
assert_eq(d, sol_df[col])
@write_read_engines_xfail
def test_no_index(tmpdir, write_engine, read_engine):
fn = str(tmpdir)
df = pd.DataFrame({'a': [1, 2, 3], 'b': [4, 5, 6]})
ddf = dd.from_pandas(df, npartitions=2)
ddf.to_parquet(fn, write_index=False, engine=write_engine)
ddf2 = dd.read_parquet(fn, engine=read_engine)
assert_eq(df, ddf2, check_index=False)
def test_read_series(tmpdir, engine):
fn = str(tmpdir)
ddf.to_parquet(fn, engine=engine)
ddf2 = dd.read_parquet(fn, columns=['x'], engine=engine, infer_divisions=should_check_divs(engine))
assert_eq(ddf[['x']], ddf2, check_divisions=should_check_divs(engine))
ddf2 = dd.read_parquet(fn, columns='x', index='myindex', engine=engine, infer_divisions=should_check_divs(engine))
assert_eq(ddf.x, ddf2, check_divisions=should_check_divs(engine))
def test_names(tmpdir, engine):
fn = str(tmpdir)
ddf.to_parquet(fn, engine=engine)
def read(fn, **kwargs):
return dd.read_parquet(fn, engine=engine, **kwargs)
assert (set(read(fn).dask) == set(read(fn).dask))
assert (set(read(fn).dask) !=
set(read(fn, columns=['x']).dask))
assert (set(read(fn, columns=('x',)).dask) ==
set(read(fn, columns=['x']).dask))
@pytest.mark.xfail(reason="parquet column fusion is special cased today"
" we'll need to find a more general solution near-term")
@pytest.mark.parametrize('c', [['x'], 'x', ['x', 'y'], []])
def test_optimize(tmpdir, c):
check_fastparquet()
fn = str(tmpdir)
ddf.to_parquet(fn)
ddf2 = dd.read_parquet(fn)
assert_eq(df[c], ddf2[c])
x = ddf2[c]
with dask.config.set(fuse_rename_keys=False):
dsk = x.__dask_optimize__(x.dask, x.__dask_keys__())
assert len(dsk) == x.npartitions
assert all(v[4] == c for v in dsk.values())
@pytest.mark.skipif(not hasattr(pd.DataFrame, 'to_parquet'),
reason="no to_parquet method")
@write_read_engines()
def test_roundtrip_from_pandas(tmpdir, write_engine, read_engine):
fn = str(tmpdir.join('test.parquet'))
df = pd.DataFrame({'x': [1, 2, 3]})
df.to_parquet(fn, engine=write_engine)
ddf = dd.read_parquet(fn, engine=read_engine)
assert_eq(df, ddf)
@write_read_engines_xfail
def test_categorical(tmpdir, write_engine, read_engine):
tmp = str(tmpdir)
df = pd.DataFrame({'x': ['a', 'b', 'c'] * 100}, dtype='category')
ddf = dd.from_pandas(df, npartitions=3)
dd.to_parquet(ddf, tmp, engine=write_engine)
ddf2 = dd.read_parquet(tmp, categories='x', engine=read_engine)
assert ddf2.compute().x.cat.categories.tolist() == ['a', 'b', 'c']
ddf2 = dd.read_parquet(tmp, categories=['x'], engine=read_engine)
assert ddf2.compute().x.cat.categories.tolist() == ['a', 'b', 'c']
# autocat
if read_engine != 'pyarrow':
ddf2 = dd.read_parquet(tmp, engine=read_engine)
assert ddf2.compute().x.cat.categories.tolist() == ['a', 'b', 'c']
ddf2.loc[:1000].compute()
df.index.name = 'index' # defaults to 'index' in this case
assert assert_eq(df, ddf2)
# dereference cats
ddf2 = dd.read_parquet(tmp, categories=[], engine=read_engine)
ddf2.loc[:1000].compute()
assert (df.x == ddf2.x).all()
def test_append(tmpdir, engine):
"""Test that appended parquet equal to the original one."""
check_fastparquet()
tmp = str(tmpdir)
df = pd.DataFrame({'i32': np.arange(1000, dtype=np.int32),
'i64': np.arange(1000, dtype=np.int64),
'f': np.arange(1000, dtype=np.float64),
'bhello': np.random.choice(['hello', 'yo', 'people'],
size=1000).astype("O")})
df.index.name = 'index'
half = len(df) // 2
ddf1 = dd.from_pandas(df.iloc[:half], chunksize=100)
ddf2 = dd.from_pandas(df.iloc[half:], chunksize=100)
ddf1.to_parquet(tmp)
ddf2.to_parquet(tmp, append=True)
ddf3 = dd.read_parquet(tmp, engine=engine)
assert_eq(df, ddf3)
def test_append_create(tmpdir):
"""Test that appended parquet equal to the original one."""
check_fastparquet()
tmp = str(tmpdir)
df = pd.DataFrame({'i32': np.arange(1000, dtype=np.int32),
'i64': np.arange(1000, dtype=np.int64),
'f': np.arange(1000, dtype=np.float64),
'bhello': np.random.choice(['hello', 'yo', 'people'],
size=1000).astype("O")})
df.index.name = 'index'
half = len(df) // 2
ddf1 = dd.from_pandas(df.iloc[:half], chunksize=100)
ddf2 = dd.from_pandas(df.iloc[half:], chunksize=100)
ddf1.to_parquet(tmp, append=True)
ddf2.to_parquet(tmp, append=True)
ddf3 = dd.read_parquet(tmp, engine='fastparquet')
assert_eq(df, ddf3)
def test_append_with_partition(tmpdir):
check_fastparquet()
tmp = str(tmpdir)
df0 = pd.DataFrame({'lat': np.arange(0, 10), 'lon': np.arange(10, 20),
'value': np.arange(100, 110)})
df0.index.name = 'index'
df1 = pd.DataFrame({'lat': np.arange(10, 20), 'lon': np.arange(10, 20),
'value': np.arange(120, 130)})
df1.index.name = 'index'
dd_df0 = dd.from_pandas(df0, npartitions=1)
dd_df1 = dd.from_pandas(df1, npartitions=1)
dd.to_parquet(dd_df0, tmp, partition_on=['lon'])
dd.to_parquet(dd_df1, tmp, partition_on=['lon'], append=True,
ignore_divisions=True)
out = dd.read_parquet(tmp).compute()
out['lon'] = out.lon.astype('int') # just to pass assert
# sort required since partitioning breaks index order
assert_eq(out.sort_values('value'), pd.concat([df0, df1])[out.columns],
check_index=False)
def test_partition_on_cats(tmpdir):
check_fastparquet()
tmp = str(tmpdir)
d = pd.DataFrame({'a': np.random.rand(50),
'b': np.random.choice(['x', 'y', 'z'], size=50),
'c': np.random.choice(['x', 'y', 'z'], size=50)})
d = dd.from_pandas(d, 2)
d.to_parquet(tmp, partition_on=['b'], engine='fastparquet')
df = dd.read_parquet(tmp, engine='fastparquet')
assert set(df.b.cat.categories) == {'x', 'y', 'z'}
d.to_parquet(tmp, partition_on=['b', 'c'], engine='fastparquet')
df = dd.read_parquet(tmp, engine='fastparquet')
assert set(df.b.cat.categories) == {'x', 'y', 'z'}
assert set(df.c.cat.categories) == {'x', 'y', 'z'}
df = dd.read_parquet(tmp, columns=['a', 'c'], engine='fastparquet')
assert set(df.c.cat.categories) == {'x', 'y', 'z'}
assert 'b' not in df.columns
df = dd.read_parquet(tmp, index='c', engine='fastparquet')
assert set(df.index.categories) == {'x', 'y', 'z'}
assert 'c' not in df.columns
# series
df = dd.read_parquet(tmp, columns='b', engine='fastparquet')
assert set(df.cat.categories) == {'x', 'y', 'z'}
def test_append_wo_index(tmpdir):
"""Test append with write_index=False."""
check_fastparquet()
tmp = str(tmpdir.join('tmp1.parquet'))
df = pd.DataFrame({'i32': np.arange(1000, dtype=np.int32),
'i64': np.arange(1000, dtype=np.int64),
'f': np.arange(1000, dtype=np.float64),
'bhello': np.random.choice(['hello', 'yo', 'people'],
size=1000).astype("O")})
half = len(df) // 2
ddf1 = dd.from_pandas(df.iloc[:half], chunksize=100)
ddf2 = dd.from_pandas(df.iloc[half:], chunksize=100)
ddf1.to_parquet(tmp)
with pytest.raises(ValueError) as excinfo:
ddf2.to_parquet(tmp, write_index=False, append=True)
assert 'Appended columns' in str(excinfo.value)
tmp = str(tmpdir.join('tmp2.parquet'))
ddf1.to_parquet(tmp, write_index=False)
ddf2.to_parquet(tmp, write_index=False, append=True)
ddf3 = dd.read_parquet(tmp, index='f')
assert_eq(df.set_index('f'), ddf3)
def test_append_overlapping_divisions(tmpdir):
"""Test raising of error when divisions overlapping."""
check_fastparquet()
tmp = str(tmpdir)
df = pd.DataFrame({'i32': np.arange(1000, dtype=np.int32),
'i64': np.arange(1000, dtype=np.int64),
'f': np.arange(1000, dtype=np.float64),
'bhello': np.random.choice(['hello', 'yo', 'people'],
size=1000).astype("O")})
half = len(df) // 2
ddf1 = dd.from_pandas(df.iloc[:half], chunksize=100)
ddf2 = dd.from_pandas(df.iloc[half - 10:], chunksize=100)
ddf1.to_parquet(tmp)
with pytest.raises(ValueError) as excinfo:
ddf2.to_parquet(tmp, append=True)
assert 'Appended divisions' in str(excinfo.value)
ddf2.to_parquet(tmp, append=True, ignore_divisions=True)
def test_append_different_columns(tmpdir):
"""Test raising of error when non equal columns."""
check_fastparquet()
tmp = str(tmpdir)
df1 = pd.DataFrame({'i32': np.arange(100, dtype=np.int32)})
df2 = pd.DataFrame({'i64': np.arange(100, dtype=np.int64)})
df3 = pd.DataFrame({'i32': np.arange(100, dtype=np.int64)})
ddf1 = dd.from_pandas(df1, chunksize=2)
ddf2 = dd.from_pandas(df2, chunksize=2)
ddf3 = dd.from_pandas(df3, chunksize=2)
ddf1.to_parquet(tmp)
with pytest.raises(ValueError) as excinfo:
ddf2.to_parquet(tmp, append=True)
assert 'Appended columns' in str(excinfo.value)
with pytest.raises(ValueError) as excinfo:
ddf3.to_parquet(tmp, append=True)
assert 'Appended dtypes' in str(excinfo.value)
@write_read_engines_xfail
def test_ordering(tmpdir, write_engine, read_engine):
tmp = str(tmpdir)
df = pd.DataFrame({'a': [1, 2, 3],
'b': [10, 20, 30],
'c': [100, 200, 300]},
index=pd.Index([-1, -2, -3], name='myindex'),
columns=['c', 'a', 'b'])
ddf = dd.from_pandas(df, npartitions=2)
dd.to_parquet(ddf, tmp, engine=write_engine)
if read_engine == 'fastparquet':
pf = fastparquet.ParquetFile(tmp)
assert pf.columns == ['myindex', 'c', 'a', 'b']
ddf2 = dd.read_parquet(tmp, index='myindex', engine=read_engine)
assert_eq(ddf, ddf2, check_divisions=False)
def test_read_parquet_custom_columns(tmpdir, engine):
import glob
tmp = str(tmpdir)
data = pd.DataFrame({'i32': np.arange(1000, dtype=np.int32),
'f': np.arange(1000, dtype=np.float64)})
df = dd.from_pandas(data, chunksize=50)
df.to_parquet(tmp)
df2 = dd.read_parquet(tmp,
columns=['i32', 'f'],
engine=engine,
infer_divisions=should_check_divs(engine))
assert_eq(df[['i32', 'f']], df2,
check_index=False, check_divisions=should_check_divs(engine))
import glob
fns = glob.glob(os.path.join(tmp, '*.parquet'))
df2 = dd.read_parquet(fns,
columns=['i32'],
engine=engine).compute()
df2.sort_values('i32', inplace=True)
assert_eq(df[['i32']], df2,
check_index=False, check_divisions=False)
df3 = dd.read_parquet(tmp,
columns=['f', 'i32'],
engine=engine,
infer_divisions=should_check_divs(engine))
assert_eq(df[['f', 'i32']], df3,
check_index=False, check_divisions=should_check_divs(engine))
@pytest.mark.parametrize('df,write_kwargs,read_kwargs', [
(pd.DataFrame({'x': [3, 2, 1]}), {}, {}),
(pd.DataFrame({'x': ['c', 'a', 'b']}), {'object_encoding': 'utf8'}, {}),
(pd.DataFrame({'x': ['cc', 'a', 'bbb']}), {'object_encoding': 'utf8'}, {}),
(pd.DataFrame({'x': [b'a', b'b', b'c']}), {'object_encoding': 'bytes'}, {}),
(pd.DataFrame({'x': pd.Categorical(['a', 'b', 'a'])}),
{'object_encoding': 'utf8'}, {'categories': ['x']}),
(pd.DataFrame({'x': pd.Categorical([1, 2, 1])}), {}, {'categories': ['x']}),
(pd.DataFrame({'x': list(map(pd.Timestamp, [3000, 2000, 1000]))}), {}, {}),
(pd.DataFrame({'x': [3000, 2000, 1000]}).astype('M8[ns]'), {}, {}),
pytest.param(pd.DataFrame({'x': [3, 2, 1]}).astype('M8[ns]'), {}, {},
marks=pytest.mark.xfail(
reason="Parquet doesn't support nanosecond precision")),
(pd.DataFrame({'x': [3, 2, 1]}).astype('M8[us]'), {}, {}),
(pd.DataFrame({'x': [3, 2, 1]}).astype('M8[ms]'), {}, {}),
(pd.DataFrame({'x': [3, 2, 1]}).astype('uint16'), {}, {}),
(pd.DataFrame({'x': [3, 2, 1]}).astype('float32'), {}, {}),
(pd.DataFrame({'x': [3, 1, 2]}, index=[3, 2, 1]), {}, {}),
(pd.DataFrame({'x': [3, 1, 5]}, index=pd.Index([1, 2, 3], name='foo')), {}, {}),
(pd.DataFrame({'x': [1, 2, 3],
'y': [3, 2, 1]}), {}, {}),
(pd.DataFrame({'x': [1, 2, 3],
'y': [3, 2, 1]}, columns=['y', 'x']), {}, {}),
(pd.DataFrame({'0': [3, 2, 1]}), {}, {}),
(pd.DataFrame({'x': [3, 2, None]}), {}, {}),
(pd.DataFrame({'-': [3., 2., None]}), {}, {}),
(pd.DataFrame({'.': [3., 2., None]}), {}, {}),
(pd.DataFrame({' ': [3., 2., None]}), {}, {}),
])
def test_roundtrip(tmpdir, df, write_kwargs, read_kwargs):
check_fastparquet()
tmp = str(tmpdir)
if df.index.name is None:
df.index.name = 'index'
ddf = dd.from_pandas(df, npartitions=2)
dd.to_parquet(ddf, tmp, **write_kwargs)
ddf2 = dd.read_parquet(tmp, index=df.index.name, **read_kwargs)
assert_eq(ddf, ddf2)
def test_categories(tmpdir):
check_fastparquet()
fn = str(tmpdir)
df = pd.DataFrame({'x': [1, 2, 3, 4, 5],
'y': list('caaab')})
ddf = dd.from_pandas(df, npartitions=2)
ddf['y'] = ddf.y.astype('category')
ddf.to_parquet(fn)
ddf2 = dd.read_parquet(fn, categories=['y'])
with pytest.raises(NotImplementedError):
ddf2.y.cat.categories
assert set(ddf2.y.compute().cat.categories) == {'a', 'b', 'c'}
cats_set = ddf2.map_partitions(lambda x: x.y.cat.categories).compute()
assert cats_set.tolist() == ['a', 'c', 'a', 'b']
assert_eq(ddf.y, ddf2.y, check_names=False)
with pytest.raises(TypeError):
# attempt to load as category that which is not so encoded
ddf2 = dd.read_parquet(fn, categories=['x']).compute()
with pytest.raises(ValueError):
# attempt to load as category unknown column
ddf2 = dd.read_parquet(fn, categories=['foo'])
def test_empty_partition(tmpdir, engine):
fn = str(tmpdir)
df = pd.DataFrame({"a": range(10), "b": range(10)})
ddf = dd.from_pandas(df, npartitions=5)
ddf2 = ddf[ddf.a <= 5]
ddf2.to_parquet(fn, engine=engine)
ddf3 = dd.read_parquet(fn, engine=engine)
sol = ddf2.compute()
assert_eq(sol, ddf3, check_names=False, check_index=False)
def test_timestamp_index(tmpdir, engine):
fn = str(tmpdir)
df = tm.makeTimeDataFrame()
df.index.name = 'foo'
ddf = dd.from_pandas(df, npartitions=5)
ddf.to_parquet(fn, engine=engine)
ddf2 = dd.read_parquet(fn, engine=engine, infer_divisions=should_check_divs(engine))
assert_eq(ddf, ddf2, check_divisions=should_check_divs(engine))
def test_to_parquet_default_writes_nulls(tmpdir):
check_fastparquet()
check_pyarrow()
fn = str(tmpdir.join('test.parquet'))
df = pd.DataFrame({'c1': [1., np.nan, 2, np.nan, 3]})
ddf = dd.from_pandas(df, npartitions=1)
ddf.to_parquet(fn)
table = pq.read_table(fn)
assert table[1].null_count == 2
@write_read_engines(
xfail_pyarrow_fastparquet=pyarrow_fastparquet_msg,
xfail_pyarrow_pyarrow=("Race condition writing using pyarrow with partition_on. "
"Fixed on master, but not on pyarrow 0.8.0")
)
def test_partition_on(tmpdir, write_engine, read_engine):
tmpdir = str(tmpdir)
df = pd.DataFrame({'a': np.random.choice(['A', 'B', 'C'], size=100),
'b': np.random.random(size=100),
'c': np.random.randint(1, 5, size=100)})
d = dd.from_pandas(df, npartitions=2)
d.to_parquet(tmpdir, partition_on=['a'], engine=write_engine)
out = dd.read_parquet(tmpdir, engine=read_engine).compute()
for val in df.a.unique():
assert set(df.b[df.a == val]) == set(out.b[out.a == val])
@write_read_engines_xfail
def test_filters(tmpdir, write_engine, read_engine):
fn = str(tmpdir)
df = pd.DataFrame({'at': ['ab', 'aa', 'ba', 'da', 'bb']})
ddf = dd.from_pandas(df, npartitions=1)
# Ok with 1 partition and filters
ddf.repartition(npartitions=1, force=True).to_parquet(fn, write_index=False,
engine=write_engine)
ddf2 = dd.read_parquet(fn, index=False, engine=read_engine,
filters=[('at', '==', 'aa')]).compute()
assert_eq(ddf2, ddf)
# with >1 partition and no filters
ddf.repartition(npartitions=2, force=True).to_parquet(fn, engine=write_engine)
dd.read_parquet(fn, engine=read_engine).compute()
assert_eq(ddf2, ddf)
# with >1 partition and filters using base fastparquet
if read_engine == 'fastparquet':
ddf.repartition(npartitions=2, force=True).to_parquet(fn, engine=write_engine)
df2 = fastparquet.ParquetFile(fn).to_pandas(filters=[('at', '==', 'aa')])
assert len(df2) > 0
# with >1 partition and filters
ddf.repartition(npartitions=2, force=True).to_parquet(fn, engine=write_engine)
dd.read_parquet(fn, engine=read_engine, filters=[('at', '==', 'aa')]).compute()
assert len(ddf2) > 0
def test_divisions_read_with_filters(tmpdir):
check_fastparquet()
tmpdir = str(tmpdir)
#generate dataframe
size = 100
categoricals = []
for value in ['a', 'b', 'c', 'd']:
categoricals += [value] * int(size / 4)
df = pd.DataFrame({'a': categoricals,
'b': np.random.random(size=size),
'c': np.random.randint(1, 5, size=size)})
d = dd.from_pandas(df, npartitions=4)
#save it
d.to_parquet(tmpdir, partition_on=['a'], engine='fastparquet')
#read it
out = dd.read_parquet(tmpdir,
engine='fastparquet',
filters=[('a', '==', 'b')])
#test it
expected_divisions = (25, 49)
assert out.divisions == expected_divisions
def test_divisions_are_known_read_with_filters(tmpdir):
check_fastparquet()
tmpdir = str(tmpdir)
#generate dataframe
df = pd.DataFrame({'unique': [0, 0, 1, 1, 2, 2, 3, 3],
'id': ['id1', 'id2',
'id1', 'id2',
'id1', 'id2',
'id1', 'id2']},
index=[0, 0, 1, 1, 2, 2, 3, 3])
d = dd.from_pandas(df, npartitions=2)
#save it
d.to_parquet(tmpdir, partition_on=['id'], engine='fastparquet')
#read it
out = dd.read_parquet(tmpdir,
engine='fastparquet',
filters=[('id', '==', 'id1')])
#test it
assert out.known_divisions
expected_divisions = (0, 2, 3)
assert out.divisions == expected_divisions
def test_read_from_fastparquet_parquetfile(tmpdir):
check_fastparquet()
fn = str(tmpdir)
df = pd.DataFrame({
'a': np.random.choice(['A', 'B', 'C'], size=100),
'b': np.random.random(size=100),
'c': np.random.randint(1, 5, size=100)
})
d = dd.from_pandas(df, npartitions=2)
d.to_parquet(fn, partition_on=['a'], engine='fastparquet')
pq_f = fastparquet.ParquetFile(fn)
# OK with no filters
out = dd.read_parquet(pq_f).compute()
for val in df.a.unique():
assert set(df.b[df.a == val]) == set(out.b[out.a == val])
# OK with filters
out = dd.read_parquet(pq_f, filters=[('a', '==', 'B')]).compute()
assert set(df.b[df.a == 'B']) == set(out.b)
# Engine should not be set to 'pyarrow'
with pytest.raises(AssertionError):
out = dd.read_parquet(pq_f, engine='pyarrow')
@pytest.mark.parametrize('scheduler', ['threads', 'processes'])
def test_to_parquet_lazy(tmpdir, scheduler, engine):
tmpdir = str(tmpdir)
df = pd.DataFrame({'a': [1, 2, 3, 4],
'b': [1., 2., 3., 4.]})
df.index.name = 'index'
ddf = dd.from_pandas(df, npartitions=2)
value = ddf.to_parquet(tmpdir, compute=False, engine=engine)
assert hasattr(value, 'dask')
value.compute(scheduler=scheduler)
assert os.path.exists(tmpdir)
ddf2 = dd.read_parquet(tmpdir, engine=engine, infer_divisions=should_check_divs(engine))
assert_eq(ddf, ddf2, check_divisions=should_check_divs(engine))
def test_timestamp96(tmpdir):
check_fastparquet()
fn = str(tmpdir)
df = pd.DataFrame({'a': ['now']}, dtype='M8[ns]')
ddf = dd.from_pandas(df, 1)
ddf.to_parquet(fn, write_index=False, times='int96')
pf = fastparquet.ParquetFile(fn)
assert pf._schema[1].type == fastparquet.parquet_thrift.Type.INT96
out = dd.read_parquet(fn).compute()
assert_eq(out, df)
def test_drill_scheme(tmpdir):
check_fastparquet()
fn = str(tmpdir)
N = 5
df1 = pd.DataFrame({c: np.random.random(N)
for i, c in enumerate(['a', 'b', 'c'])})
df2 = pd.DataFrame({c: np.random.random(N)
for i, c in enumerate(['a', 'b', 'c'])})
files = []
for d in ['test_data1', 'test_data2']:
dn = os.path.join(fn, d)
if not os.path.exists(dn):
os.mkdir(dn)
files.append(os.path.join(dn, 'data1.parq'))
fastparquet.write(files[0], df1)