-
Notifications
You must be signed in to change notification settings - Fork 4.5k
Expand file tree
/
Copy pathframes_test.py
More file actions
3203 lines (2757 loc) · 118 KB
/
frames_test.py
File metadata and controls
3203 lines (2757 loc) · 118 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.
import re
import sys
import unittest
import warnings
import numpy as np
import pandas as pd
import pytest
from parameterized import parameterized
import apache_beam as beam
from apache_beam.dataframe import expressions
from apache_beam.dataframe import frame_base
from apache_beam.dataframe import frames
from apache_beam.dataframe.convert import to_dataframe
from apache_beam.dataframe.doctests import teststring
from apache_beam.runners.interactive import interactive_beam as ib
from apache_beam.runners.interactive import interactive_environment as ie
from apache_beam.runners.interactive.interactive_runner import InteractiveRunner
from apache_beam.runners.interactive.testing.mock_env import isolated_env
# Get major, minor version
PD_VERSION = tuple(map(int, pd.__version__.split('.')[0:2]))
GROUPBY_DF = pd.DataFrame({
'group': ['a' if i % 5 == 0 or i % 3 == 0 else 'b' for i in range(100)],
'foo': [None if i % 11 == 0 else i for i in range(100)],
'bar': [None if i % 7 == 0 else 99 - i for i in range(100)],
'baz': [None if i % 13 == 0 else i * 2 for i in range(100)],
'bool': [i % 17 == 0 for i in range(100)],
'str': [str(i) for i in range(100)],
})
if PD_VERSION < (2, 0):
# All these are things that are fixed in the Pandas 2 transition.
pytestmark = pytest.mark.filterwarnings("ignore::FutureWarning")
def _get_deferred_args(*args):
return [
frame_base.DeferredFrame.wrap(
expressions.ConstantExpression(arg, arg[0:0])) for arg in args
]
class _AbstractFrameTest(unittest.TestCase):
"""Test sub-class with utilities for verifying DataFrame operations."""
def _run_error_test(
self, func, *args, construction_time=True, distributed=True):
"""Verify that func(*args) raises the same exception in pandas and in Beam.
Note that by default this only checks for exceptions that the Beam DataFrame
API raises during expression generation (i.e. construction time).
Exceptions raised while the pipeline is executing are less helpful, but
are sometimes unavoidable (e.g. data validation exceptions), to check for
these exceptions use construction_time=False."""
deferred_args = _get_deferred_args(*args)
# Get expected error
try:
expected = func(*args)
except Exception as e:
expected_error = e
else:
raise AssertionError(
"Expected an error, but executing with pandas successfully "
f"returned:\n{expected}")
# Get actual error
if construction_time:
try:
_ = func(*deferred_args)._expr
except Exception as e:
actual = e
else:
raise AssertionError(
f"Expected an error:\n{expected_error}\nbut Beam successfully "
f"generated an expression.")
else: # not construction_time
# Check for an error raised during pipeline execution
expr = func(*deferred_args)._expr
session_type = (
expressions.PartitioningSession
if distributed else expressions.Session)
try:
result = session_type({}).evaluate(expr)
except Exception as e:
actual = e
else:
raise AssertionError(
f"Expected an error:\n{expected_error}\nbut Beam successfully "
f"Computed the result:\n{result}.")
# Verify
if (not isinstance(actual, type(expected_error)) or
str(expected_error) not in str(actual)):
raise AssertionError(
f'Expected {expected_error!r} to be raised, but got {actual!r}'
) from actual
def _run_inplace_test(self, func, arg, **kwargs):
"""Verify an inplace operation performed by func.
Checks that func performs the same inplace operation on arg, in pandas and
in Beam."""
def wrapper(df):
df = df.copy()
func(df)
return df
self._run_test(wrapper, arg, **kwargs)
def _run_test(
self,
func,
*args,
distributed=True,
nonparallel=False,
check_proxy=True,
lenient_dtype_check=False):
"""Verify that func(*args) produces the same result in pandas and in Beam.
Args:
distributed (bool): Whether or not to use PartitioningSession to
simulate parallel execution.
nonparallel (bool): Whether or not this function contains a
non-parallelizable operation. If True, the expression will be
generated twice, once outside of an allow_non_parallel_operations
block (to verify NonParallelOperation is raised), and again inside
of an allow_non_parallel_operations block to actually generate an
expression to verify.
check_proxy (bool): Whether or not to check that the proxy of the
generated expression matches the actual result, defaults to True.
This option should NOT be set to False in tests added for new
operations if at all possible. Instead make sure the new operation
produces the correct proxy. This flag only exists as an escape hatch
until existing failures can be addressed
(https://github.com/apache/beam/issues/20926).
lenient_dtype_check (bool): Whether or not to check that numeric columns
are still numeric between actual and proxy. i.e. verify that they
are at least int64 or float64, and not necessarily have the exact
same dtype. This may need to be set to True for some non-deferred
operations, where the dtype of the values in the proxy are not known
ahead of time, causing int64 to float64 coercion issues.
"""
# Compute expected value
expected = func(*args)
# Compute actual value
deferred_args = _get_deferred_args(*args)
if nonparallel:
# First run outside a nonparallel block to confirm this raises as expected
with self.assertRaises(expressions.NonParallelOperation) as raised:
func(*deferred_args)
if raised.exception.msg.startswith(
"Encountered non-parallelizable form of"):
raise AssertionError(
"Default NonParallelOperation raised, please specify a reason in "
"the Singleton() partitioning requirement for this operation."
) from raised.exception
# Re-run in an allow non parallel block to get an expression to verify
with beam.dataframe.allow_non_parallel_operations():
expr = func(*deferred_args)._expr
else:
expr = func(*deferred_args)._expr
# Compute the result of the generated expression
session_type = (
expressions.PartitioningSession if distributed else expressions.Session)
actual = session_type({}).evaluate(expr)
# Verify
if isinstance(expected, pd.core.generic.NDFrame):
if distributed:
if expected.index.is_unique:
expected = expected.sort_index()
actual = actual.sort_index()
elif isinstance(expected, pd.Series):
expected = expected.sort_values()
actual = actual.sort_values()
else:
expected = expected.sort_values(list(expected.columns))
actual = actual.sort_values(list(actual.columns))
if isinstance(expected, pd.Series):
if lenient_dtype_check:
pd.testing.assert_series_equal(
expected.astype('Float64'), actual.astype('Float64'))
else:
pd.testing.assert_series_equal(expected, actual)
elif isinstance(expected, pd.DataFrame):
if lenient_dtype_check:
pd.testing.assert_frame_equal(
expected.astype('Float64'), actual.astype('Float64'))
else:
pd.testing.assert_frame_equal(expected, actual)
else:
raise ValueError(
f"Expected value is a {type(expected)},"
"not a Series or DataFrame.")
else:
# Expectation is not a pandas object
if isinstance(expected, float):
if np.isnan(expected):
cmp = np.isnan
else:
cmp = lambda x: np.isclose(expected, x)
else:
cmp = lambda x: x == expected
self.assertTrue(
cmp(actual), 'Expected:\n\n%r\n\nActual:\n\n%r' % (expected, actual))
if check_proxy:
# Verify that the actual result agrees with the proxy
proxy = expr.proxy()
if type(actual) in (np.float32, np.float64):
self.assertTrue(type(actual) == type(proxy) or np.isnan(proxy))
else:
self.assertEqual(type(actual), type(proxy))
if isinstance(expected, pd.core.generic.NDFrame):
if isinstance(expected, pd.Series):
if lenient_dtype_check:
self.assertEqual(
actual.astype('Float64').dtype, proxy.astype('Float64').dtype)
else:
self.assertEqual(actual.dtype, proxy.dtype)
self.assertEqual(actual.name, proxy.name)
elif isinstance(expected, pd.DataFrame):
if lenient_dtype_check:
pd.testing.assert_series_equal(
actual.astype('Float64').dtypes, proxy.astype('Float64').dtypes)
else:
pd.testing.assert_series_equal(actual.dtypes, proxy.dtypes)
else:
raise ValueError(
f"Expected value is a {type(expected)},"
"not a Series or DataFrame.")
self.assertEqual(actual.index.names, proxy.index.names)
for i in range(actual.index.nlevels):
if lenient_dtype_check:
self.assertEqual(
actual.astype('Float64').index.get_level_values(i).dtype,
proxy.astype('Float64').index.get_level_values(i).dtype)
else:
self.assertEqual(
actual.index.get_level_values(i).dtype,
proxy.index.get_level_values(i).dtype)
class DeferredFrameTest(_AbstractFrameTest):
"""Miscellaneous tessts for DataFrame operations."""
def test_series_arithmetic(self):
a = pd.Series([1, 2, 3])
b = pd.Series([100, 200, 300])
self._run_test(lambda a, b: a - 2 * b, a, b)
self._run_test(lambda a, b: a.subtract(2).multiply(b).divide(a), a, b)
def test_dataframe_arithmetic(self):
df = pd.DataFrame({'a': [1, 2, 3], 'b': [100, 200, 300]})
df2 = pd.DataFrame({'a': [3000, 1000, 2000], 'b': [7, 11, 13]})
self._run_test(lambda df, df2: df - 2 * df2, df, df2)
self._run_test(
lambda df, df2: df.subtract(2).multiply(df2).divide(df), df, df2)
@unittest.skipIf(PD_VERSION < (1, 3), "dropna=False is new in pandas 1.3")
def test_value_counts_dropna_false(self):
df = pd.DataFrame({
'first_name': ['John', 'Anne', 'John', 'Beth'],
'middle_name': ['Smith', pd.NA, pd.NA, 'Louise']
})
# TODO(https://github.com/apache/beam/issues/21014): Remove the
# assertRaises this when the underlying bug in
# https://github.com/pandas-dev/pandas/issues/36470 is fixed.
with self.assertRaises(NotImplementedError):
self._run_test(lambda df: df.value_counts(dropna=False), df)
def test_get_column(self):
df = pd.DataFrame({
'Animal': ['Falcon', 'Falcon', 'Parrot', 'Parrot'],
'Speed': [380., 370., 24., 26.]
})
self._run_test(lambda df: df['Animal'], df)
self._run_test(lambda df: df.Speed, df)
self._run_test(lambda df: df.get('Animal'), df)
self._run_test(lambda df: df.get('FOO', df.Animal), df)
def test_series_xs(self):
# pandas doctests only verify DataFrame.xs, here we verify Series.xs as well
d = {
'num_legs': [4, 4, 2, 2],
'num_wings': [0, 0, 2, 2],
'class': ['mammal', 'mammal', 'mammal', 'bird'],
'animal': ['cat', 'dog', 'bat', 'penguin'],
'locomotion': ['walks', 'walks', 'flies', 'walks']
}
df = pd.DataFrame(data=d)
df = df.set_index(['class', 'animal', 'locomotion'])
self._run_test(lambda df: df.num_legs.xs('mammal'), df)
self._run_test(lambda df: df.num_legs.xs(('mammal', 'dog')), df)
self._run_test(lambda df: df.num_legs.xs('cat', level=1), df)
self._run_test(
lambda df: df.num_legs.xs(('bird', 'walks'), level=[0, 'locomotion']),
df)
def test_dataframe_xs(self):
# Test cases reported in BEAM-13421
df = pd.DataFrame(
np.array([
['state', 'day1', 12],
['state', 'day1', 1],
['state', 'day2', 14],
['county', 'day1', 9],
]),
columns=['provider', 'time', 'value'])
self._run_test(lambda df: df.xs('state'), df.set_index(['provider']))
self._run_test(
lambda df: df.xs('state'), df.set_index(['provider', 'time']))
def test_set_column(self):
def new_column(df):
df['NewCol'] = df['Speed']
df = pd.DataFrame({
'Animal': ['Falcon', 'Falcon', 'Parrot', 'Parrot'],
'Speed': [380., 370., 24., 26.]
})
self._run_inplace_test(new_column, df)
def test_set_column_from_index(self):
def new_column(df):
df['NewCol'] = df.index
df = pd.DataFrame({
'Animal': ['Falcon', 'Falcon', 'Parrot', 'Parrot'],
'Speed': [380., 370., 24., 26.]
})
self._run_inplace_test(new_column, df)
def test_tz_with_utc_zone_set_explicitly(self):
test = """
>>> s = pd.Series(["1/1/2020 10:00:00+00:00", "2/1/2020 11:00:00+03:00"])
>>> s = pd.to_datetime(s, utc=True)
>>> s
0 2020-01-01 10:00:00+00:00
1 2020-02-01 08:00:00+00:00
dtype: datetime64[ns, UTC]
>>> s.dt.tz
datetime.timezone.utc
"""
teststring(test)
def test_tz_localize_ambiguous_series(self):
# This replicates a tz_localize doctest:
# s.tz_localize('CET', ambiguous=np.array([True, True, False]))
# But using a DeferredSeries instead of a np array
s = pd.Series(
range(3),
index=pd.DatetimeIndex([
'2018-10-28 01:20:00', '2018-10-28 02:36:00', '2018-10-28 03:46:00'
]))
ambiguous = pd.Series([True, True, False], index=s.index)
self._run_test(
lambda s, ambiguous: s.tz_localize('CET', ambiguous=ambiguous),
s,
ambiguous)
def test_tz_convert(self):
# This replicates a tz_localize doctest:
# s.tz_localize('CET', ambiguous=np.array([True, True, False]))
# But using a DeferredSeries instead of a np array
s = pd.Series(
range(3),
index=pd.DatetimeIndex([
'2018-10-27 01:20:00', '2018-10-27 02:36:00', '2018-10-27 03:46:00'
],
tz='Europe/Berlin'))
self._run_test(lambda s: s.tz_convert('America/Los_Angeles'), s)
def test_sort_index_columns(self):
df = pd.DataFrame({
'c': range(10),
'a': range(10),
'b': range(10),
np.nan: range(10),
})
self._run_test(lambda df: df.sort_index(axis=1), df)
self._run_test(lambda df: df.sort_index(axis=1, ascending=False), df)
self._run_test(lambda df: df.sort_index(axis=1, na_position='first'), df)
def test_where_callable_args(self):
df = pd.DataFrame(
np.arange(10, dtype=np.int64).reshape(-1, 2), columns=['A', 'B'])
self._run_test(
lambda df: df.where(lambda df: df % 2 == 0, lambda df: df * 10), df)
def test_where_concrete_args(self):
df = pd.DataFrame(
np.arange(10, dtype=np.int64).reshape(-1, 2), columns=['A', 'B'])
self._run_test(
lambda df: df.where(
df % 2 == 0, pd.Series({
'A': 123, 'B': 456
}), axis=1),
df)
def test_combine_dataframe(self):
df = pd.DataFrame({'A': [0, 0], 'B': [4, 4]})
df2 = pd.DataFrame({'A': [1, 1], 'B': [3, 3]})
take_smaller = lambda s1, s2: s1 if s1.sum() < s2.sum() else s2
self._run_test(
lambda df, df2: df.combine(df2, take_smaller),
df,
df2,
nonparallel=True)
def test_combine_dataframe_fill(self):
df1 = pd.DataFrame({'A': [0, 0], 'B': [None, 4]})
df2 = pd.DataFrame({'A': [1, 1], 'B': [3, 3]})
take_smaller = lambda s1, s2: s1 if s1.sum() < s2.sum() else s2
self._run_test(
lambda df1, df2: df1.combine(df2, take_smaller, fill_value=-5),
df1,
df2,
nonparallel=True)
def test_combine_Series(self):
s1 = pd.Series({'falcon': 330.0, 'eagle': 160.0})
s2 = pd.Series({'falcon': 345.0, 'eagle': 200.0, 'duck': 30.0})
self._run_test(
lambda s1, s2: s1.combine(s2, max),
s1,
s2,
nonparallel=True,
check_proxy=False)
def test_combine_first_dataframe(self):
df1 = pd.DataFrame({'A': [None, 0], 'B': [None, 4]})
df2 = pd.DataFrame({'A': [1, 1], 'B': [3, 3]})
self._run_test(lambda df1, df2: df1.combine_first(df2), df1, df2)
def test_combine_first_series(self):
s1 = pd.Series([1, np.nan])
s2 = pd.Series([3, 4])
self._run_test(lambda s1, s2: s1.combine_first(s2), s1, s2)
def test_add_prefix(self):
df = pd.DataFrame({'A': [1, 2, 3, 4], 'B': [3, 4, 5, 6]})
s = pd.Series([1, 2, 3, 4])
self._run_test(lambda df: df.add_prefix('col_'), df)
self._run_test(lambda s: s.add_prefix('col_'), s)
def test_add_suffix(self):
df = pd.DataFrame({'A': [1, 2, 3, 4], 'B': [3, 4, 5, 6]})
s = pd.Series([1, 2, 3, 4])
self._run_test(lambda df: df.add_suffix('_col'), df)
self._run_test(lambda s: s.add_prefix('_col'), s)
def test_set_index(self):
df = pd.DataFrame({
# [19, 18, ..]
'index1': reversed(range(20)), # [15, 16, .., 0, 1, .., 13, 14]
'index2': np.roll(range(20), 5), # ['', 'a', 'bb', ...]
'values': [chr(ord('a') + i) * i for i in range(20)],
})
self._run_test(lambda df: df.set_index(['index1', 'index2']), df)
self._run_test(lambda df: df.set_index(['index1', 'index2'], drop=True), df)
self._run_test(lambda df: df.set_index('values'), df)
self._run_error_test(lambda df: df.set_index('bad'), df)
self._run_error_test(
lambda df: df.set_index(['index2', 'bad', 'really_bad']), df)
def test_set_axis(self):
df = pd.DataFrame({"A": [1, 2, 3], "B": [4, 5, 6]}, index=['X', 'Y', 'Z'])
self._run_test(lambda df: df.set_axis(['I', 'II'], axis='columns'), df)
self._run_test(lambda df: df.set_axis([0, 1], axis=1), df)
self._run_inplace_test(
lambda df: df.set_axis(['i', 'ii'], axis='columns'), df)
with self.assertRaises(NotImplementedError):
self._run_test(lambda df: df.set_axis(['a', 'b', 'c'], axis='index'), df)
self._run_test(lambda df: df.set_axis([0, 1, 2], axis=0), df)
def test_series_set_axis(self):
s = pd.Series(list(range(3)), index=['X', 'Y', 'Z'])
with self.assertRaises(NotImplementedError):
self._run_test(lambda s: s.set_axis(['a', 'b', 'c']), s)
self._run_test(lambda s: s.set_axis([1, 2, 3]), s)
def test_series_drop_ignore_errors(self):
midx = pd.MultiIndex(
levels=[['lama', 'cow', 'falcon'], ['speed', 'weight', 'length']],
codes=[[0, 0, 0, 1, 1, 1, 2, 2, 2], [0, 1, 2, 0, 1, 2, 0, 1, 2]])
s = pd.Series([45, 200, 1.2, 30, 250, 1.5, 320, 1, 0.3], index=midx)
# drop() requires singleton partitioning unless errors are ignored
# Add some additional tests here to make sure the implementation works in
# non-singleton partitioning.
self._run_test(lambda s: s.drop('lama', level=0, errors='ignore'), s)
self._run_test(lambda s: s.drop(('cow', 'speed'), errors='ignore'), s)
self._run_test(lambda s: s.drop('falcon', level=0, errors='ignore'), s)
def test_dataframe_drop_ignore_errors(self):
midx = pd.MultiIndex(
levels=[['lama', 'cow', 'falcon'], ['speed', 'weight', 'length']],
codes=[[0, 0, 0, 1, 1, 1, 2, 2, 2], [0, 1, 2, 0, 1, 2, 0, 1, 2]])
df = pd.DataFrame(
index=midx,
columns=['big', 'small'],
data=[[45, 30], [200, 100], [1.5, 1], [30, 20], [250, 150], [1.5, 0.8],
[320, 250], [1, 0.8], [0.3, 0.2]])
# drop() requires singleton partitioning unless errors are ignored
# Add some additional tests here to make sure the implementation works in
# non-singleton partitioning.
self._run_test(
lambda df: df.drop(index='lama', level=0, errors='ignore'), df)
self._run_test(
lambda df: df.drop(index=('cow', 'speed'), errors='ignore'), df)
self._run_test(
lambda df: df.drop(index='falcon', level=0, errors='ignore'), df)
self._run_test(
lambda df: df.drop(index='cow', columns='small', errors='ignore'), df)
def test_merge(self):
# This is from the pandas doctests, but fails due to re-indexing being
# order-sensitive.
df1 = pd.DataFrame({
'lkey': ['foo', 'bar', 'baz', 'foo'], 'value': [1, 2, 3, 5]
})
df2 = pd.DataFrame({
'rkey': ['foo', 'bar', 'baz', 'foo'], 'value': [5, 6, 7, 8]
})
self._run_test(
lambda df1, df2: df1.merge(df2, left_on='lkey', right_on='rkey').rename(
index=lambda x: '*'),
df1,
df2,
nonparallel=True,
check_proxy=False)
self._run_test(
lambda df1, df2: df1.merge(
df2, left_on='lkey', right_on='rkey', suffixes=('_left', '_right')).
rename(index=lambda x: '*'),
df1,
df2,
nonparallel=True,
check_proxy=False)
def test_merge_left_join(self):
# This is from the pandas doctests, but fails due to re-indexing being
# order-sensitive.
df1 = pd.DataFrame({'a': ['foo', 'bar'], 'b': [1, 2]})
df2 = pd.DataFrame({'a': ['foo', 'baz'], 'c': [3, 4]})
self._run_test(
lambda df1, df2: df1.merge(df2, how='left', on='a').rename(
index=lambda x: '*'),
df1,
df2,
nonparallel=True,
check_proxy=False)
def test_merge_on_index(self):
# This is from the pandas doctests, but fails due to re-indexing being
# order-sensitive.
df1 = pd.DataFrame({
'lkey': ['foo', 'bar', 'baz', 'foo'], 'value': [1, 2, 3, 5]
}).set_index('lkey')
df2 = pd.DataFrame({
'rkey': ['foo', 'bar', 'baz', 'foo'], 'value': [5, 6, 7, 8]
}).set_index('rkey')
self._run_test(
lambda df1, df2: df1.merge(df2, left_index=True, right_index=True),
df1,
df2,
check_proxy=False)
def test_merge_same_key(self):
df1 = pd.DataFrame({
'key': ['foo', 'bar', 'baz', 'foo'], 'value': [1, 2, 3, 5]
})
df2 = pd.DataFrame({
'key': ['foo', 'bar', 'baz', 'foo'], 'value': [5, 6, 7, 8]
})
self._run_test(
lambda df1, df2: df1.merge(df2, on='key').rename(index=lambda x: '*'),
df1,
df2,
nonparallel=True,
check_proxy=False)
self._run_test(
lambda df1, df2: df1.merge(df2, on='key', suffixes=('_left', '_right')).
rename(index=lambda x: '*'),
df1,
df2,
nonparallel=True,
check_proxy=False)
def test_merge_same_key_doctest(self):
df1 = pd.DataFrame({'a': ['foo', 'bar'], 'b': [1, 2]})
df2 = pd.DataFrame({'a': ['foo', 'baz'], 'c': [3, 4]})
self._run_test(
lambda df1, df2: df1.merge(df2, how='left', on='a').rename(
index=lambda x: '*'),
df1,
df2,
nonparallel=True,
check_proxy=False)
# Test without specifying 'on'
self._run_test(
lambda df1, df2: df1.merge(df2, how='left').rename(index=lambda x: '*'),
df1,
df2,
nonparallel=True,
check_proxy=False)
def test_merge_same_key_suffix_collision(self):
df1 = pd.DataFrame({'a': ['foo', 'bar'], 'b': [1, 2], 'a_lsuffix': [5, 6]})
df2 = pd.DataFrame({'a': ['foo', 'baz'], 'c': [3, 4], 'a_rsuffix': [7, 8]})
self._run_test(
lambda df1, df2: df1.merge(
df2, how='left', on='a', suffixes=('_lsuffix', '_rsuffix')).rename(
index=lambda x: '*'),
df1,
df2,
nonparallel=True,
check_proxy=False)
# Test without specifying 'on'
self._run_test(
lambda df1, df2: df1.merge(
df2, how='left', suffixes=('_lsuffix', '_rsuffix')).rename(
index=lambda x: '*'),
df1,
df2,
nonparallel=True,
check_proxy=False)
def test_swaplevel(self):
df = pd.DataFrame(
{"Grade": ["A", "B", "A", "C"]},
index=[
["Final exam", "Final exam", "Coursework", "Coursework"],
["History", "Geography", "History", "Geography"],
["January", "February", "March", "April"],
])
self._run_test(lambda df: df.swaplevel(), df)
def test_value_counts_with_nans(self):
# similar to doctests that verify value_counts, but include nan values to
# make sure we handle them correctly.
df = pd.DataFrame({
'num_legs': [2, 4, 4, 6, np.nan, np.nan],
'num_wings': [2, 0, 0, 0, np.nan, 2]
},
index=['falcon', 'dog', 'cat', 'ant', 'car', 'plane'])
self._run_test(lambda df: df.value_counts(), df)
self._run_test(lambda df: df.value_counts(normalize=True), df)
# Ensure we don't drop rows due to nan values in unused columns.
self._run_test(lambda df: df.value_counts('num_wings'), df)
if PD_VERSION >= (1, 3):
# dropna=False is new in pandas 1.3
# TODO(https://github.com/apache/beam/issues/21014): Remove the
# assertRaises this when the underlying bug in
# https://github.com/pandas-dev/pandas/issues/36470 is fixed.
with self.assertRaises(NotImplementedError):
self._run_test(lambda df: df.value_counts(dropna=False), df)
# Test the defaults.
self._run_test(lambda df: df.num_wings.value_counts(), df)
self._run_test(lambda df: df.num_wings.value_counts(normalize=True), df)
self._run_test(lambda df: df.num_wings.value_counts(dropna=False), df)
# Test the combination interactions.
for normalize in (True, False):
for dropna in (True, False):
self._run_test(
lambda df, dropna=dropna, normalize=normalize: df.num_wings.
value_counts(dropna=dropna, normalize=normalize),
df)
def test_value_counts_does_not_support_sort(self):
df = pd.DataFrame({
'num_legs': [2, 4, 4, 6, np.nan, np.nan],
'num_wings': [2, 0, 0, 0, np.nan, 2]
},
index=['falcon', 'dog', 'cat', 'ant', 'car', 'plane'])
with self.assertRaisesRegex(frame_base.WontImplementError,
r"value_counts\(sort\=True\)"):
self._run_test(lambda df: df.value_counts(sort=True), df)
with self.assertRaisesRegex(frame_base.WontImplementError,
r"value_counts\(sort\=True\)"):
self._run_test(lambda df: df.num_wings.value_counts(sort=True), df)
def test_series_getitem(self):
s = pd.Series([x**2 for x in range(10)])
self._run_test(lambda s: s[...], s)
self._run_test(lambda s: s[:], s)
self._run_test(lambda s: s[s < 10], s)
self._run_test(lambda s: s[lambda s: s < 10], s)
s.index = s.index.map(float)
self._run_test(lambda s: s[1.5:6], s)
def test_series_truncate(self):
s = pd.Series(['a', 'b', 'c', 'd', 'e', 'f'])
self._run_test(lambda s: s.truncate(before=1, after=3), s)
def test_dataframe_truncate(self):
df = pd.DataFrame({
'C': list('abcde'), 'B': list('fghij'), 'A': list('klmno')
},
index=[1, 2, 3, 4, 5])
self._run_test(lambda df: df.truncate(before=1, after=3), df)
self._run_test(lambda df: df.truncate(before='A', after='B', axis=1), df)
self._run_test(lambda df: df['A'].truncate(before=2, after=4), df)
@parameterized.expand([
(pd.Series(range(10)), ), # unique
(pd.Series(list(range(100)) + [0]), ), # non-unique int
(pd.Series(list(range(100)) + [0]) / 100, ), # non-unique flt
(pd.Series(['a', 'b', 'c', 'd']), ), # unique str
(pd.Series(['a', 'b', 'a', 'c', 'd']), ), # non-unique str
])
def test_series_is_unique(self, series):
self._run_test(lambda s: s.is_unique, series)
@parameterized.expand([
(pd.Series(range(10)), ), # False
(pd.Series([1, 2, np.nan, 3, np.nan]), ), # True
(pd.Series(['a', 'b', 'c', 'd', 'e']), ), # False
(pd.Series(['a', 'b', None, 'c', None]), ), # True
])
def test_series_hasnans(self, series):
self._run_test(lambda s: s.hasnans, series)
def test_dataframe_getitem(self):
df = pd.DataFrame({'A': [x**2 for x in range(6)], 'B': list('abcdef')})
self._run_test(lambda df: df['A'], df)
self._run_test(lambda df: df[['A', 'B']], df)
self._run_test(lambda df: df[:], df)
self._run_test(lambda df: df[df.A < 10], df)
df.index = df.index.map(float)
self._run_test(lambda df: df[1.5:4], df)
def test_loc(self):
dates = pd.date_range('1/1/2000', periods=8)
# TODO(https://github.com/apache/beam/issues/20765):
# We do not preserve the freq attribute on a DateTime index
dates.freq = None
df = pd.DataFrame(
np.arange(32).reshape((8, 4)),
index=dates,
columns=['A', 'B', 'C', 'D'])
self._run_test(lambda df: df.loc[:], df)
self._run_test(lambda df: df.loc[:, 'A'], df)
self._run_test(lambda df: df.loc[:dates[3]], df)
self._run_test(lambda df: df.loc[df.A > 10], df)
self._run_test(lambda df: df.loc[lambda df: df.A > 10], df)
self._run_test(lambda df: df.C.loc[df.A > 10], df)
self._run_test(lambda df, s: df.loc[s.loc[1:3]], df, pd.Series(dates))
@unittest.skipIf(PD_VERSION >= (2, 0), 'append removed in Pandas 2.0')
def test_append_sort(self):
# yapf: disable
df1 = pd.DataFrame({'int': [1, 2, 3], 'str': ['a', 'b', 'c']},
columns=['int', 'str'],
index=[1, 3, 5])
df2 = pd.DataFrame({'int': [4, 5, 6], 'str': ['d', 'e', 'f']},
columns=['str', 'int'],
index=[2, 4, 6])
# yapf: enable
self._run_test(lambda df1, df2: df1.append(df2, sort=True), df1, df2)
self._run_test(lambda df1, df2: df1.append(df2, sort=False), df1, df2)
self._run_test(lambda df1, df2: df2.append(df1, sort=True), df1, df2)
self._run_test(lambda df1, df2: df2.append(df1, sort=False), df1, df2)
def test_smallest_largest(self):
df = pd.DataFrame({'A': [1, 1, 2, 2], 'B': [2, 3, 5, 7]})
self._run_test(lambda df: df.nlargest(1, 'A', keep='all'), df)
self._run_test(lambda df: df.nsmallest(3, 'A', keep='all'), df)
self._run_test(lambda df: df.nlargest(3, ['A', 'B'], keep='all'), df)
def test_series_cov_corr(self):
for s in [pd.Series([1, 2, 3]),
pd.Series(range(100)),
pd.Series([x**3 for x in range(-50, 50)])]:
self._run_test(lambda s: s.std(), s)
self._run_test(lambda s: s.var(), s)
self._run_test(lambda s: s.corr(s), s)
self._run_test(lambda s: s.corr(s + 1), s)
self._run_test(lambda s: s.corr(s * s), s)
self._run_test(lambda s: s.cov(s * s), s)
self._run_test(lambda s: s.skew(), s)
self._run_test(lambda s: s.kurtosis(), s)
self._run_test(lambda s: s.kurt(), s)
def test_dataframe_cov_corr(self):
df = pd.DataFrame(np.random.randn(20, 3), columns=['a', 'b', 'c'])
df.loc[df.index[:5], 'a'] = np.nan
df.loc[df.index[5:10], 'b'] = np.nan
self._run_test(lambda df: df.corr(), df)
self._run_test(lambda df: df.cov(), df)
self._run_test(lambda df: df.corr(min_periods=12), df)
self._run_test(lambda df: df.cov(min_periods=12), df)
self._run_test(lambda df: df.corrwith(df.a), df)
self._run_test(lambda df: df[['a', 'b']].corrwith(df[['b', 'c']]), df)
df2 = pd.DataFrame(np.random.randn(20, 3), columns=['a', 'b', 'c'])
self._run_test(
lambda df, df2: df.corrwith(df2, axis=1), df, df2, check_proxy=False)
def test_corrwith_bad_axis(self):
df = pd.DataFrame({'a': range(3), 'b': range(3, 6), 'c': range(6, 9)})
self._run_error_test(lambda df: df.corrwith(df.a, axis=2), df)
self._run_error_test(lambda df: df.corrwith(df, axis=5), df)
@unittest.skipIf(PD_VERSION < (1, 2), "na_action added in pandas 1.2.0")
@pytest.mark.filterwarnings(
"ignore:The default of observed=False is deprecated:FutureWarning")
def test_applymap_na_action(self):
# Replicates a doctest for na_action which is incompatible with
# doctest framework
df = pd.DataFrame([[pd.NA, 2.12], [3.356, 4.567]])
self._run_test(
lambda df: df.applymap(lambda x: len(str(x)), na_action='ignore'),
df,
# TODO: generate proxy using naive type inference on fn
check_proxy=False)
@unittest.skipIf(PD_VERSION < (2, 1), "map added in 2.1.0")
def test_map_na_action(self):
# Replicates a doctest for na_action which is incompatible with
# doctest framework
df = pd.DataFrame([[pd.NA, 2.12], [3.356, 4.567]])
self._run_test(
lambda df: df.map(lambda x: len(str(x)), na_action='ignore'),
df,
# TODO: generate proxy using naive type inference on fn
check_proxy=False)
def test_dataframe_eval_query(self):
df = pd.DataFrame(np.random.randn(20, 3), columns=['a', 'b', 'c'])
self._run_test(lambda df: df.eval('foo = a + b - c'), df)
self._run_test(lambda df: df.query('a > b + c'), df)
self._run_inplace_test(lambda df: df.eval('foo = a + b - c'), df)
# Verify that attempting to access locals raises a useful error
deferred_df = frame_base.DeferredFrame.wrap(
expressions.ConstantExpression(df, df[0:0]))
self.assertRaises(
NotImplementedError, lambda: deferred_df.eval('foo = a + @b - c'))
self.assertRaises(
NotImplementedError, lambda: deferred_df.query('a > @b + c'))
def test_index_name_assignment(self):
df = pd.DataFrame({'a': ['foo', 'bar'], 'b': [1, 2]})
df = df.set_index(['a', 'b'], drop=False)
def change_index_names(df):
df.index.names = ['A', None]
self._run_inplace_test(change_index_names, df)
def test_quantile(self):
df = pd.DataFrame(
np.array([[1, 1], [2, 10], [3, 100], [4, 100]]), columns=['a', 'b'])
self._run_test(
lambda df: df.quantile(0.1, axis='columns'), df, check_proxy=False)
self._run_test(
lambda df: df.quantile(0.1, axis='columns'), df, check_proxy=False)
with self.assertRaisesRegex(frame_base.WontImplementError,
r"df\.quantile\(q=0\.1, axis='columns'\)"):
self._run_test(lambda df: df.quantile([0.1, 0.5], axis='columns'), df)
def test_dataframe_melt(self):
df = pd.DataFrame({
'A': {
0: 'a', 1: 'b', 2: 'c'
},
'B': {
0: 1, 1: 3, 2: 5
},
'C': {
0: 2, 1: 4, 2: 6
}
})
self._run_test(
lambda df: df.melt(id_vars=['A'], value_vars=['B'], ignore_index=False),
df)
self._run_test(
lambda df: df.melt(
id_vars=['A'], value_vars=['B', 'C'], ignore_index=False),
df)
self._run_test(
lambda df: df.melt(
id_vars=['A'], value_vars=['B'], var_name='myVarname', value_name=
'myValname', ignore_index=False),
df)
self._run_test(
lambda df: df.melt(
id_vars=['A'], value_vars=['B', 'C'], ignore_index=False),
df)
df.columns = [list('ABC'), list('DEF')]
self._run_test(
lambda df: df.melt(
col_level=0, id_vars=['A'], value_vars=['B'], ignore_index=False),
df)
self._run_test(
lambda df: df.melt(
id_vars=[('A', 'D')], value_vars=[('B', 'E')], ignore_index=False),
df)
def test_fillna_columns(self):
df = pd.DataFrame(
[[np.nan, 2, np.nan, 0], [3, 4, np.nan, 1], [np.nan, np.nan, np.nan, 5],
[np.nan, 3, np.nan, 4], [3, np.nan, np.nan, 4]],
columns=list('ABCD'))
self._run_test(lambda df: df.fillna(method='ffill', axis='columns'), df)
self._run_test(
lambda df: df.fillna(method='ffill', axis='columns', limit=1), df)
self._run_test(
lambda df: df.fillna(method='bfill', axis='columns', limit=1), df)
# Intended behavior is unclear here. See
# https://github.com/pandas-dev/pandas/issues/40989
# self._run_test(lambda df: df.fillna(axis='columns', value=100,
# limit=2), df)
def test_dataframe_fillna_dataframe_as_value(self):
df = pd.DataFrame([[np.nan, 2, np.nan, 0], [3, 4, np.nan, 1],
[np.nan, np.nan, np.nan, 5], [np.nan, 3, np.nan, 4]],
columns=list("ABCD"))
df2 = pd.DataFrame(np.zeros((4, 4)), columns=list("ABCE"))
self._run_test(lambda df, df2: df.fillna(df2), df, df2)
def test_dataframe_fillna_series_as_value(self):
df = pd.DataFrame([[np.nan, 2, np.nan, 0], [3, 4, np.nan, 1],
[np.nan, np.nan, np.nan, 5], [np.nan, 3, np.nan, 4]],
columns=list("ABCD"))
s = pd.Series(range(4), index=list("ABCE"))