forked from Kyligence/spark
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathframe.py
13736 lines (11756 loc) · 488 KB
/
frame.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
#
# 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.
#
"""
A wrapper class for Spark DataFrame to behave like pandas DataFrame.
"""
from collections import defaultdict, namedtuple
from collections.abc import Mapping
import re
import warnings
import inspect
import json
import types
from functools import partial, reduce
import sys
from itertools import zip_longest, chain
from types import TracebackType
from typing import (
Any,
Callable,
Dict,
Generic,
IO,
Iterable,
Iterator,
List,
Optional,
Sequence,
Tuple,
Type,
Union,
cast,
no_type_check,
TYPE_CHECKING,
)
import datetime
import numpy as np
import pandas as pd
from pandas.api.types import ( # type: ignore[attr-defined]
is_bool_dtype,
is_list_like,
is_dict_like,
is_scalar,
)
from pandas.tseries.frequencies import DateOffset, to_offset
if TYPE_CHECKING:
from pandas.io.formats.style import Styler
from pandas.core.dtypes.common import infer_dtype_from_object
from pandas.core.accessor import CachedAccessor
from pandas.core.dtypes.inference import is_sequence
from pyspark import StorageLevel
from pyspark.sql import Column as PySparkColumn, DataFrame as PySparkDataFrame, functions as F
from pyspark.sql.functions import pandas_udf
from pyspark.sql.types import (
ArrayType,
BooleanType,
DataType,
DoubleType,
NumericType,
Row,
StringType,
StructField,
StructType,
DecimalType,
TimestampType,
TimestampNTZType,
)
from pyspark.sql.window import Window
from pyspark import pandas as ps # For running doctests and reference resolution in PyCharm.
from pyspark.pandas._typing import (
Axis,
DataFrameOrSeries,
Dtype,
Label,
Name,
Scalar,
T,
GenericColumn,
)
from pyspark.pandas.accessors import PandasOnSparkFrameMethods
from pyspark.pandas.config import option_context, get_option
from pyspark.pandas.correlation import (
compute,
CORRELATION_VALUE_1_COLUMN,
CORRELATION_VALUE_2_COLUMN,
CORRELATION_CORR_OUTPUT_COLUMN,
CORRELATION_COUNT_OUTPUT_COLUMN,
)
from pyspark.pandas.spark import functions as SF
from pyspark.pandas.spark.accessors import SparkFrameMethods, CachedSparkFrameMethods
from pyspark.pandas.utils import (
align_diff_frames,
column_labels_level,
combine_frames,
default_session,
is_name_like_tuple,
is_name_like_value,
is_testing,
name_like_string,
same_anchor,
scol_for,
validate_arguments_and_invoke_function,
validate_axis,
validate_bool_kwarg,
validate_how,
validate_mode,
verify_temp_column_name,
log_advice,
)
from pyspark.pandas.generic import Frame
from pyspark.pandas.internal import (
InternalField,
InternalFrame,
HIDDEN_COLUMNS,
NATURAL_ORDER_COLUMN_NAME,
SPARK_INDEX_NAME_FORMAT,
SPARK_DEFAULT_INDEX_NAME,
SPARK_DEFAULT_SERIES_NAME,
SPARK_INDEX_NAME_PATTERN,
)
from pyspark.pandas.missing.frame import MissingPandasLikeDataFrame
from pyspark.pandas.typedef.typehints import (
as_spark_type,
infer_return_type,
pandas_on_spark_type,
spark_type_to_pandas_dtype,
DataFrameType,
SeriesType,
ScalarType,
create_tuple_for_frame_type,
)
from pyspark.pandas.plot import PandasOnSparkPlotAccessor
# For supporting Spark Connect
from pyspark.sql.connect.dataframe import DataFrame as ConnectDataFrame
from pyspark.sql.connect.column import Column as ConnectColumn
from pyspark.sql.utils import is_remote
if TYPE_CHECKING:
from pyspark.sql._typing import OptionalPrimitiveType
from pyspark.pandas.groupby import DataFrameGroupBy
from pyspark.pandas.resample import DataFrameResampler
from pyspark.pandas.indexes import Index
from pyspark.pandas.series import Series
# These regular expression patterns are compiled and defined here to avoid compiling the same
# pattern every time it is used in _repr_ and _repr_html_ in DataFrame.
# Two patterns basically seek the footer string from Pandas'
REPR_PATTERN = re.compile(r"\n\n\[(?P<rows>[0-9]+) rows x (?P<columns>[0-9]+) columns\]$")
REPR_HTML_PATTERN = re.compile(
r"\n\<p\>(?P<rows>[0-9]+) rows × (?P<columns>[0-9]+) columns\<\/p\>\n\<\/div\>$"
)
_flex_doc_FRAME = """
Get {desc} of dataframe and other, element-wise (binary operator `{op_name}`).
Equivalent to ``{equiv}``. With the reverse version, `{reverse}`.
Among flexible wrappers (`add`, `sub`, `mul`, `div`) to
arithmetic operators: `+`, `-`, `*`, `/`, `//`.
Parameters
----------
other : scalar
Any single data
Returns
-------
DataFrame
Result of the arithmetic operation.
Examples
--------
>>> df = ps.DataFrame({{'angles': [0, 3, 4],
... 'degrees': [360, 180, 360]}},
... index=['circle', 'triangle', 'rectangle'],
... columns=['angles', 'degrees'])
>>> df
angles degrees
circle 0 360
triangle 3 180
rectangle 4 360
Add a scalar with operator version which returns the same
results. Also, the reverse version.
>>> df + 1
angles degrees
circle 1 361
triangle 4 181
rectangle 5 361
>>> df.add(1)
angles degrees
circle 1 361
triangle 4 181
rectangle 5 361
>>> df.add(df)
angles degrees
circle 0 720
triangle 6 360
rectangle 8 720
>>> df + df + df
angles degrees
circle 0 1080
triangle 9 540
rectangle 12 1080
>>> df.radd(1)
angles degrees
circle 1 361
triangle 4 181
rectangle 5 361
Divide and true divide by constant with reverse version.
>>> df / 10
angles degrees
circle 0.0 36.0
triangle 0.3 18.0
rectangle 0.4 36.0
>>> df.div(10)
angles degrees
circle 0.0 36.0
triangle 0.3 18.0
rectangle 0.4 36.0
>>> df.rdiv(10)
angles degrees
circle inf 0.027778
triangle 3.333333 0.055556
rectangle 2.500000 0.027778
>>> df.truediv(10)
angles degrees
circle 0.0 36.0
triangle 0.3 18.0
rectangle 0.4 36.0
>>> df.rtruediv(10)
angles degrees
circle inf 0.027778
triangle 3.333333 0.055556
rectangle 2.500000 0.027778
Subtract by constant with reverse version.
>>> df - 1
angles degrees
circle -1 359
triangle 2 179
rectangle 3 359
>>> df.sub(1)
angles degrees
circle -1 359
triangle 2 179
rectangle 3 359
>>> df.rsub(1)
angles degrees
circle 1 -359
triangle -2 -179
rectangle -3 -359
Multiply by constant with the reverse version.
>>> df * 1
angles degrees
circle 0 360
triangle 3 180
rectangle 4 360
>>> df.mul(1)
angles degrees
circle 0 360
triangle 3 180
rectangle 4 360
>>> df.rmul(1)
angles degrees
circle 0 360
triangle 3 180
rectangle 4 360
Floor Divide by constant with reverse version.
>>> df // 10
angles degrees
circle 0.0 36.0
triangle 0.0 18.0
rectangle 0.0 36.0
>>> df.floordiv(10)
angles degrees
circle 0.0 36.0
triangle 0.0 18.0
rectangle 0.0 36.0
>>> df.rfloordiv(10) # doctest: +SKIP
angles degrees
circle inf 0.0
triangle 3.0 0.0
rectangle 2.0 0.0
Mod by constant with reverse version.
>>> df % 2
angles degrees
circle 0 0
triangle 1 0
rectangle 0 0
>>> df.mod(2)
angles degrees
circle 0 0
triangle 1 0
rectangle 0 0
>>> df.rmod(2)
angles degrees
circle NaN 2
triangle 2.0 2
rectangle 2.0 2
Power by constant with reverse version.
>>> df ** 2
angles degrees
circle 0.0 129600.0
triangle 9.0 32400.0
rectangle 16.0 129600.0
>>> df.pow(2)
angles degrees
circle 0.0 129600.0
triangle 9.0 32400.0
rectangle 16.0 129600.0
>>> df.rpow(2)
angles degrees
circle 1.0 2.348543e+108
triangle 8.0 1.532496e+54
rectangle 16.0 2.348543e+108
"""
class DataFrame(Frame, Generic[T]):
"""
pandas-on-Spark DataFrame that corresponds to pandas DataFrame logically. This holds Spark
DataFrame internally.
:ivar _internal: an internal immutable Frame to manage metadata.
:type _internal: InternalFrame
Parameters
----------
data : numpy ndarray (structured or homogeneous), dict, pandas DataFrame,
Spark DataFrame, pandas-on-Spark DataFrame or pandas-on-Spark Series.
Dict can contain Series, arrays, constants, or list-like objects
index : Index or array-like
Index to use for the resulting frame. Will default to RangeIndex if
no indexing information part of input data and no index provided
columns : Index or array-like
Column labels to use for the resulting frame. Will default to
RangeIndex (0, 1, 2, ..., n) if no column labels are provided
dtype : dtype, default None
Data type to force. Only a single dtype is allowed. If None, infer
copy : boolean, default False
Copy data from inputs. Only affects DataFrame / 2d ndarray input
.. versionchanged:: 3.4.0
Since 3.4.0, it deals with `data` and `index` in this approach:
1, when `data` is a distributed dataset (Internal DataFrame/Spark DataFrame/
pandas-on-Spark DataFrame/pandas-on-Spark Series), it will first parallelize
the `index` if necessary, and then try to combine the `data` and `index`;
Note that if `data` and `index` doesn't have the same anchor, then
`compute.ops_on_diff_frames` should be turned on;
2, when `data` is a local dataset (Pandas DataFrame/numpy ndarray/list/etc),
it will first collect the `index` to driver if necessary, and then apply
the `Pandas.DataFrame(...)` creation internally;
Examples
--------
Constructing DataFrame from a dictionary.
>>> d = {'col1': [1, 2], 'col2': [3, 4]}
>>> df = ps.DataFrame(data=d, columns=['col1', 'col2'])
>>> df
col1 col2
0 1 3
1 2 4
Constructing DataFrame from pandas DataFrame
>>> df = ps.DataFrame(pd.DataFrame(data=d, columns=['col1', 'col2']))
>>> df
col1 col2
0 1 3
1 2 4
Notice that the inferred dtype is int64.
>>> df.dtypes
col1 int64
col2 int64
dtype: object
To enforce a single dtype:
>>> df = ps.DataFrame(data=d, dtype=np.int8)
>>> df.dtypes
col1 int8
col2 int8
dtype: object
Constructing DataFrame from numpy ndarray:
>>> import numpy as np
>>> ps.DataFrame(data=np.array([[1, 2, 3, 4, 5], [6, 7, 8, 9, 0]]),
... columns=['a', 'b', 'c', 'd', 'e'])
a b c d e
0 1 2 3 4 5
1 6 7 8 9 0
Constructing DataFrame from numpy ndarray with Pandas index:
>>> import numpy as np
>>> import pandas as pd
>>> ps.DataFrame(data=np.array([[1, 2, 3, 4, 5], [6, 7, 8, 9, 0]]),
... index=pd.Index([1, 4]), columns=['a', 'b', 'c', 'd', 'e'])
a b c d e
1 1 2 3 4 5
4 6 7 8 9 0
Constructing DataFrame from numpy ndarray with pandas-on-Spark index:
>>> import numpy as np
>>> import pandas as pd
>>> ps.DataFrame(data=np.array([[1, 2, 3, 4, 5], [6, 7, 8, 9, 0]]),
... index=ps.Index([1, 4]), columns=['a', 'b', 'c', 'd', 'e'])
a b c d e
1 1 2 3 4 5
4 6 7 8 9 0
Constructing DataFrame from Pandas DataFrame with Pandas index:
>>> import numpy as np
>>> import pandas as pd
>>> pdf = pd.DataFrame(data=np.array([[1, 2, 3, 4, 5], [6, 7, 8, 9, 0]]),
... columns=['a', 'b', 'c', 'd', 'e'])
>>> ps.DataFrame(data=pdf, index=pd.Index([1, 4]))
a b c d e
1 6.0 7.0 8.0 9.0 0.0
4 NaN NaN NaN NaN NaN
Constructing DataFrame from Pandas DataFrame with pandas-on-Spark index:
>>> import numpy as np
>>> import pandas as pd
>>> pdf = pd.DataFrame(data=np.array([[1, 2, 3, 4, 5], [6, 7, 8, 9, 0]]),
... columns=['a', 'b', 'c', 'd', 'e'])
>>> ps.DataFrame(data=pdf, index=ps.Index([1, 4]))
a b c d e
1 6.0 7.0 8.0 9.0 0.0
4 NaN NaN NaN NaN NaN
Constructing DataFrame from Spark DataFrame with Pandas index:
>>> import pandas as pd
>>> sdf = spark.createDataFrame([("Data", 1), ("Bricks", 2)], ["x", "y"])
>>> ps.DataFrame(data=sdf, index=pd.Index([0, 1, 2]))
Traceback (most recent call last):
...
ValueError: Cannot combine the series or dataframe...'compute.ops_on_diff_frames' option.
Enable 'compute.ops_on_diff_frames' to combine SparkDataFrame and Pandas index
>>> with ps.option_context("compute.ops_on_diff_frames", True):
... ps.DataFrame(data=sdf, index=pd.Index([0, 1, 2]))
x y
0 Data 1.0
1 Bricks 2.0
2 None NaN
Constructing DataFrame from Spark DataFrame with pandas-on-Spark index:
>>> import pandas as pd
>>> sdf = spark.createDataFrame([("Data", 1), ("Bricks", 2)], ["x", "y"])
>>> ps.DataFrame(data=sdf, index=ps.Index([0, 1, 2]))
Traceback (most recent call last):
...
ValueError: Cannot combine the series or dataframe...'compute.ops_on_diff_frames' option.
Enable 'compute.ops_on_diff_frames' to combine Spark DataFrame and pandas-on-Spark index
>>> with ps.option_context("compute.ops_on_diff_frames", True):
... ps.DataFrame(data=sdf, index=ps.Index([0, 1, 2]))
x y
0 Data 1.0
1 Bricks 2.0
2 None NaN
"""
def __init__( # type: ignore[no-untyped-def]
self, data=None, index=None, columns=None, dtype=None, copy=False
):
index_assigned = False
if isinstance(data, InternalFrame):
assert columns is None
assert dtype is None
assert not copy
if index is None:
internal = data
elif isinstance(data, (PySparkDataFrame, ConnectDataFrame)):
assert columns is None
assert dtype is None
assert not copy
if index is None:
internal = InternalFrame(spark_frame=data, index_spark_columns=None)
elif isinstance(data, ps.DataFrame):
assert columns is None
assert dtype is None
assert not copy
if index is None:
internal = data._internal
elif isinstance(data, ps.Series):
assert dtype is None
assert not copy
# For pandas compatibility when `columns` contains only one valid column.
if columns is not None:
assert isinstance(columns, (dict, list, tuple))
assert len(columns) == 1
columns = list(columns.keys()) if isinstance(columns, dict) else columns
assert columns[0] == data._internal.data_spark_column_names[0]
if index is None:
internal = data.to_frame()._internal
else:
from pyspark.pandas.indexes.base import Index
if index is not None and isinstance(index, Index):
# with local data, collect ps.Index to driver
# to avoid mismatched results between
# ps.DataFrame([1, 2], index=ps.Index([1, 2]))
# and
# pd.DataFrame([1, 2], index=pd.Index([1, 2]))
index = index._to_pandas()
pdf = pd.DataFrame(data=data, index=index, columns=columns, dtype=dtype, copy=copy)
internal = InternalFrame.from_pandas(pdf)
index_assigned = True
if index is not None and not index_assigned:
# TODO(SPARK-40226): Support MultiIndex
if isinstance(index, (ps.MultiIndex, pd.MultiIndex)):
raise ValueError("Cannot combine a Distributed Dataset with a MultiIndex")
data_df = ps.DataFrame(data=data, index=None, columns=columns, dtype=dtype, copy=copy)
index_ps = ps.Index(index)
index_df = index_ps.to_frame()
if same_anchor(data_df, index_df):
data_labels = data_df._internal.column_labels
data_pssers = [data_df._psser_for(label) for label in data_labels]
index_labels = index_df._internal.column_labels
index_pssers = [index_df._psser_for(label) for label in index_labels]
internal = data_df._internal.with_new_columns(data_pssers + index_pssers)
combined = ps.DataFrame(internal).set_index(index_labels)
combined.index.name = index_ps.name
else:
# drop un-matched rows in `data`
# note that `combine_frames` cannot work with a MultiIndex for now
combined = combine_frames(data_df, index_df, how="right")
combined_labels = combined._internal.column_labels
index_labels = [label for label in combined_labels if label[0] == "that"]
combined = combined.set_index(index_labels)
combined._internal._column_labels = data_df._internal.column_labels
combined._internal._column_label_names = data_df._internal._column_label_names
combined._internal._index_names = index_df._internal.column_labels
combined.index.name = index_ps.name
internal = combined._internal
object.__setattr__(self, "_internal_frame", internal)
@property
def _pssers(self) -> Dict[Label, "Series"]:
"""Return a dict of column label -> Series which anchors `self`."""
from pyspark.pandas.series import Series
if not hasattr(self, "_psseries"):
object.__setattr__(
self,
"_psseries",
{label: Series(data=self, index=label) for label in self._internal.column_labels},
)
else:
psseries = cast(Dict[Label, Series], self._psseries) # type: ignore[has-type]
assert len(self._internal.column_labels) == len(psseries), (
len(self._internal.column_labels),
len(psseries),
)
if any(self is not psser._psdf for psser in psseries.values()):
# Refresh the dict to contain only Series anchoring `self`.
self._psseries = {
label: (
psseries[label]
if self is psseries[label]._psdf
else Series(data=self, index=label)
)
for label in self._internal.column_labels
}
return self._psseries
@property
def _internal(self) -> InternalFrame:
return cast(InternalFrame, self._internal_frame) # type: ignore[has-type]
def _update_internal_frame(
self,
internal: InternalFrame,
check_same_anchor: bool = True,
anchor_force_disconnect: bool = False,
) -> None:
"""
Update InternalFrame with the given one.
If the column_label is changed or the new InternalFrame is not the same `anchor` or the
`anchor_force_disconnect` flag is set to True, disconnect the original anchor and create
a new one.
If `check_same_anchor` is `False`, checking whether the same anchor is ignored
and force to update the InternalFrame, e.g., replacing the internal with the resolved_copy,
updating the underlying Spark DataFrame which need to combine a different Spark DataFrame.
Parameters
----------
internal : InternalFrame
The new InternalFrame
check_same_anchor : bool
Whether checking the same anchor
anchor_force_disconnect : bool
Force to disconnect the original anchor and create a new one
"""
from pyspark.pandas.series import Series
if hasattr(self, "_psseries"):
psseries = {}
for old_label, new_label in zip_longest(
self._internal.column_labels, internal.column_labels
):
if old_label is not None:
psser = self._pssers[old_label]
renamed = old_label != new_label
not_same_anchor = check_same_anchor and not same_anchor(internal, psser)
if renamed or not_same_anchor or anchor_force_disconnect:
psdf: DataFrame = DataFrame(self._internal.select_column(old_label))
psser._update_anchor(psdf)
psser = None
else:
psser = None
if new_label is not None:
if psser is None:
psser = Series(data=self, index=new_label)
psseries[new_label] = psser
self._psseries = psseries
self._internal_frame = internal
if hasattr(self, "_repr_pandas_cache"):
del self._repr_pandas_cache
@property
def ndim(self) -> int:
"""
Return an int representing the number of array dimensions.
return 2 for DataFrame.
Examples
--------
>>> df = ps.DataFrame([[1, 2], [4, 5], [7, 8]],
... index=['cobra', 'viper', None],
... columns=['max_speed', 'shield'])
>>> df # doctest: +SKIP
max_speed shield
cobra 1 2
viper 4 5
None 7 8
>>> df.ndim
2
"""
return 2
@property
def axes(self) -> List:
"""
Return a list representing the axes of the DataFrame.
It has the row axis labels and column axis labels as the only members.
They are returned in that order.
Examples
--------
>>> df = ps.DataFrame({'col1': [1, 2], 'col2': [3, 4]})
>>> df.axes
[Int64Index([0, 1], dtype='int64'), Index(['col1', 'col2'], dtype='object')]
"""
return [self.index, self.columns]
def _reduce_for_stat_function(
self,
sfun: Callable[["Series"], GenericColumn],
name: str,
axis: Optional[Axis] = None,
numeric_only: bool = True,
skipna: bool = True,
**kwargs: Any,
) -> "Series":
"""
Applies sfun to each column and returns a pd.Series where the number of rows equals the
number of columns.
Parameters
----------
sfun : either an 1-arg function that takes a Column and returns a Column, or
a 2-arg function that takes a Column and its DataType and returns a Column.
axis: used only for sanity check because the series only supports index axis.
name : original pandas API name.
axis : axis to apply. 0 or 1, or 'index' or 'columns.
numeric_only : bool, default True
Include only float, int, boolean columns. False is not supported. This parameter
is mainly for pandas compatibility. Only 'DataFrame.count' uses this parameter
currently.
skipna : bool, default True
Exclude NA/null values when computing the result.
"""
from pyspark.pandas.series import Series, first_series
axis = validate_axis(axis)
if axis == 0:
min_count = kwargs.get("min_count", 0)
exprs = [
cast(GenericColumn, F.lit(None).cast(StringType()).alias(SPARK_DEFAULT_INDEX_NAME))
]
new_column_labels = []
for label in self._internal.column_labels:
psser = self._psser_for(label)
is_numeric_or_boolean = isinstance(
psser.spark.data_type, (NumericType, BooleanType)
)
keep_column = not numeric_only or is_numeric_or_boolean
if keep_column:
if not skipna and get_option("compute.eager_check") and psser.hasnans:
scol: GenericColumn = F.first(F.lit(np.nan))
else:
scol = sfun(psser)
if min_count > 0:
scol = F.when(Frame._count_expr(psser) >= min_count, scol)
exprs.append(scol.alias(name_like_string(label)))
new_column_labels.append(label)
if len(exprs) == 1:
return Series([])
sdf = self._internal.spark_frame.select(*exprs) # type: ignore[arg-type]
# The data is expected to be small so it's fine to transpose/use the default index.
with ps.option_context("compute.max_rows", 1):
internal = InternalFrame(
spark_frame=sdf,
index_spark_columns=[scol_for(sdf, SPARK_DEFAULT_INDEX_NAME)],
column_labels=new_column_labels,
column_label_names=self._internal.column_label_names,
)
return first_series(DataFrame(internal).transpose())
else:
# Here we execute with the first 1000 to get the return type.
# If the records were less than 1000, it uses pandas API directly for a shortcut.
limit = get_option("compute.shortcut_limit")
pdf = self.head(limit + 1)._to_internal_pandas()
pser = getattr(pdf, name)(axis=axis, numeric_only=numeric_only, **kwargs)
if len(pdf) <= limit:
return Series(pser)
@pandas_udf(returnType=as_spark_type(pser.dtype.type)) # type: ignore[call-overload]
def calculate_columns_axis(*cols: pd.Series) -> pd.Series:
return getattr(pd.concat(cols, axis=1), name)(
axis=axis, numeric_only=numeric_only, **kwargs
)
column_name = verify_temp_column_name(
self._internal.spark_frame.select(self._internal.index_spark_columns),
"__calculate_columns_axis__",
)
sdf = self._internal.spark_frame.select(
self._internal.index_spark_columns
+ [calculate_columns_axis(*self._internal.data_spark_columns).alias(column_name)]
)
internal = InternalFrame(
spark_frame=sdf,
index_spark_columns=[
scol_for(sdf, col) for col in self._internal.index_spark_column_names
],
index_names=self._internal.index_names,
index_fields=self._internal.index_fields,
)
return first_series(DataFrame(internal)).rename(pser.name)
def _psser_for(self, label: Label) -> "Series":
"""
Create Series with a proper column label.
The given label must be verified to exist in `InternalFrame.column_labels`.
For example, in some method, self is like:
>>> self = ps.range(3)
`self._psser_for(label)` can be used with `InternalFrame.column_labels`:
>>> self._psser_for(self._internal.column_labels[0])
0 0
1 1
2 2
Name: id, dtype: int64
`self._psser_for(label)` must not be used directly with user inputs.
In that case, `self[label]` should be used instead, which checks the label exists or not:
>>> self['id']
0 0
1 1
2 2
Name: id, dtype: int64
"""
return self._pssers[label]
def _apply_series_op(
self,
op: Callable[["Series"], Union["Series", GenericColumn]],
should_resolve: bool = False,
) -> "DataFrame":
applied = []
for label in self._internal.column_labels:
applied.append(op(self._psser_for(label)))
internal = self._internal.with_new_columns(applied)
if should_resolve:
internal = internal.resolved_copy
return DataFrame(internal)
# Arithmetic Operators
def _map_series_op(self, op: str, other: Any) -> "DataFrame":
from pyspark.pandas.base import IndexOpsMixin
if not isinstance(other, DataFrame) and (
isinstance(other, IndexOpsMixin) or is_sequence(other)
):
raise TypeError(
"%s with a sequence is currently not supported; "
"however, got %s." % (op, type(other).__name__)
)
if isinstance(other, DataFrame):
if self._internal.column_labels_level != other._internal.column_labels_level:
raise ValueError("cannot join with no overlapping index names")
if not same_anchor(self, other):
# Different DataFrames
def apply_op(
psdf: DataFrame,
this_column_labels: List[Label],
that_column_labels: List[Label],
) -> Iterator[Tuple["Series", Label]]:
for this_label, that_label in zip(this_column_labels, that_column_labels):
yield (
getattr(psdf._psser_for(this_label), op)(
psdf._psser_for(that_label)
).rename(this_label),
this_label,
)
return align_diff_frames(apply_op, self, other, fillna=True, how="full")
else:
applied = []
column_labels = []
for label in self._internal.column_labels:
if label in other._internal.column_labels:
applied.append(getattr(self._psser_for(label), op)(other._psser_for(label)))
else:
applied.append(
F.lit(None)
.cast(self._internal.spark_type_for(label))
.alias(name_like_string(label))
)
column_labels.append(label)
for label in other._internal.column_labels:
if label not in column_labels:
applied.append(
F.lit(None)
.cast(other._internal.spark_type_for(label))
.alias(name_like_string(label))
)
column_labels.append(label)
internal = self._internal.with_new_columns(applied, column_labels=column_labels)
return DataFrame(internal)
else:
return self._apply_series_op(lambda psser: getattr(psser, op)(other))
def __add__(self, other: Any) -> "DataFrame":
return self._map_series_op("add", other)
def __radd__(self, other: Any) -> "DataFrame":
return self._map_series_op("radd", other)
def __truediv__(self, other: Any) -> "DataFrame":
return self._map_series_op("truediv", other)
def __rtruediv__(self, other: Any) -> "DataFrame":
return self._map_series_op("rtruediv", other)
def __mul__(self, other: Any) -> "DataFrame":
return self._map_series_op("mul", other)
def __rmul__(self, other: Any) -> "DataFrame":
return self._map_series_op("rmul", other)
def __sub__(self, other: Any) -> "DataFrame":
return self._map_series_op("sub", other)
def __rsub__(self, other: Any) -> "DataFrame":
return self._map_series_op("rsub", other)
def __pow__(self, other: Any) -> "DataFrame":
return self._map_series_op("pow", other)
def __rpow__(self, other: Any) -> "DataFrame":
return self._map_series_op("rpow", other)
def __mod__(self, other: Any) -> "DataFrame":
return self._map_series_op("mod", other)
def __rmod__(self, other: Any) -> "DataFrame":
return self._map_series_op("rmod", other)
def __floordiv__(self, other: Any) -> "DataFrame":
return self._map_series_op("floordiv", other)
def __rfloordiv__(self, other: Any) -> "DataFrame":
return self._map_series_op("rfloordiv", other)
def __abs__(self) -> "DataFrame":
return self._apply_series_op(lambda psser: abs(psser))
def __neg__(self) -> "DataFrame":
return self._apply_series_op(lambda psser: -psser)
def add(self, other: Any) -> "DataFrame":
return self + other
# create accessor for plot