/
internal.py
1700 lines (1494 loc) · 66.2 KB
/
internal.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.
#
"""
An internal immutable DataFrame with some metadata to manage indexes.
"""
import re
from typing import Any, Dict, List, Optional, Sequence, Tuple, Union, TYPE_CHECKING, cast
from itertools import accumulate
import py4j
import numpy as np
import pandas as pd
from pandas.api.types import CategoricalDtype # noqa: F401
from pyspark._globals import _NoValue, _NoValueType
from pyspark.sql import functions as F, Column, DataFrame as SparkDataFrame, Window
from pyspark.sql.functions import pandas_udf
from pyspark.sql.types import ( # noqa: F401
BooleanType,
DataType,
IntegralType,
LongType,
StructField,
StructType,
StringType,
)
# For running doctests and reference resolution in PyCharm.
from pyspark import pandas as ps
from pyspark.pandas._typing import Label
if TYPE_CHECKING:
# This is required in old Python 3.5 to prevent circular reference.
from pyspark.pandas.series import Series # noqa: F401 (SPARK-34943)
from pyspark.pandas.spark.utils import as_nullable_spark_type, force_decimal_precision_scale
from pyspark.pandas.data_type_ops.base import DataTypeOps
from pyspark.pandas.typedef import (
Dtype,
as_spark_type,
extension_dtypes,
infer_pd_series_spark_type,
spark_type_to_pandas_dtype,
)
from pyspark.pandas.utils import (
column_labels_level,
default_session,
is_name_like_tuple,
is_testing,
lazy_property,
name_like_string,
scol_for,
spark_column_equals,
verify_temp_column_name,
)
# A function to turn given numbers to Spark columns that represent pandas-on-Spark index.
SPARK_INDEX_NAME_FORMAT = "__index_level_{}__".format
SPARK_DEFAULT_INDEX_NAME = SPARK_INDEX_NAME_FORMAT(0)
# A pattern to check if the name of a Spark column is a pandas-on-Spark index name or not.
SPARK_INDEX_NAME_PATTERN = re.compile(r"__index_level_[0-9]+__")
NATURAL_ORDER_COLUMN_NAME = "__natural_order__"
HIDDEN_COLUMNS = {NATURAL_ORDER_COLUMN_NAME}
DEFAULT_SERIES_NAME = 0
SPARK_DEFAULT_SERIES_NAME = str(DEFAULT_SERIES_NAME)
class InternalField:
"""
The internal field to store the dtype as well as the Spark's StructField optionally.
Parameters
----------
dtype : numpy.dtype or pandas' ExtensionDtype
The dtype for the field
struct_field : StructField, optional
The `StructField` for the field. If None, InternalFrame will properly set.
"""
def __init__(self, dtype: Dtype, struct_field: Optional[StructField] = None):
self._dtype = dtype
self._struct_field = struct_field
@staticmethod
def from_struct_field(
struct_field: StructField, *, use_extension_dtypes: bool = False
) -> "InternalField":
"""
Returns a new InternalField object created from the given StructField.
The dtype will be inferred from the data type of the given StructField.
Parameters
----------
struct_field : StructField
The StructField used to create a new InternalField object.
use_extension_dtypes : bool
If True, try to use the extension dtypes.
Returns
-------
InternalField
"""
return InternalField(
dtype=spark_type_to_pandas_dtype(
struct_field.dataType, use_extension_dtypes=use_extension_dtypes
),
struct_field=struct_field,
)
@property
def dtype(self) -> Dtype:
"""Return the dtype for the field."""
return self._dtype
@property
def struct_field(self) -> Optional[StructField]:
"""Return the StructField for the field."""
return self._struct_field
@property
def name(self) -> str:
"""Return the field name if the StructField exists."""
assert self.struct_field is not None
return self.struct_field.name
@property
def spark_type(self) -> DataType:
"""Return the spark data type for the field if the StructField exists."""
assert self.struct_field is not None
return self.struct_field.dataType
@property
def nullable(self) -> bool:
"""Return the nullability for the field if the StructField exists."""
assert self.struct_field is not None
return self.struct_field.nullable
@property
def metadata(self) -> Dict[str, Any]:
"""Return the metadata for the field if the StructField exists."""
assert self.struct_field is not None
return self.struct_field.metadata
@property
def is_extension_dtype(self) -> bool:
"""Return whether the dtype for the field is an extension type or not."""
return isinstance(self.dtype, extension_dtypes)
def normalize_spark_type(self) -> "InternalField":
"""Return a new InternalField object with normalized Spark data type."""
assert self.struct_field is not None
return self.copy(
spark_type=force_decimal_precision_scale(as_nullable_spark_type(self.spark_type)),
nullable=True,
)
def copy(
self,
*,
name: Union[str, _NoValueType] = _NoValue,
dtype: Union[Dtype, _NoValueType] = _NoValue,
spark_type: Union[DataType, _NoValueType] = _NoValue,
nullable: Union[bool, _NoValueType] = _NoValue,
metadata: Union[Optional[Dict[str, Any]], _NoValueType] = _NoValue,
) -> "InternalField":
"""Copy the InternalField object."""
if name is _NoValue:
name = self.name
if dtype is _NoValue:
dtype = self.dtype
if spark_type is _NoValue:
spark_type = self.spark_type
if nullable is _NoValue:
nullable = self.nullable
if metadata is _NoValue:
metadata = self.metadata
return InternalField(
dtype=cast(Dtype, dtype),
struct_field=StructField(
name=cast(str, name),
dataType=cast(DataType, spark_type),
nullable=cast(bool, nullable),
metadata=cast(Optional[Dict[str, Any]], metadata),
),
)
def __eq__(self, other: Any) -> bool:
return (
isinstance(other, InternalField)
and self.dtype == other.dtype
and self.struct_field == other.struct_field
)
def __repr__(self) -> str:
return "InternalField(dtype={dtype},struct_field={struct_field})".format(
dtype=self.dtype, struct_field=self.struct_field
)
class InternalFrame(object):
"""
The internal immutable DataFrame which manages Spark DataFrame and column names and index
information.
.. note:: this is an internal class. It is not supposed to be exposed to users and users
should not directly access to it.
The internal immutable DataFrame represents the index information for a DataFrame it belongs to.
For instance, if we have a pandas-on-Spark DataFrame as below, pandas DataFrame does not
store the index as columns.
>>> psdf = ps.DataFrame({
... 'A': [1, 2, 3, 4],
... 'B': [5, 6, 7, 8],
... 'C': [9, 10, 11, 12],
... 'D': [13, 14, 15, 16],
... 'E': [17, 18, 19, 20]}, columns = ['A', 'B', 'C', 'D', 'E'])
>>> psdf # doctest: +NORMALIZE_WHITESPACE
A B C D E
0 1 5 9 13 17
1 2 6 10 14 18
2 3 7 11 15 19
3 4 8 12 16 20
However, all columns including index column are also stored in Spark DataFrame internally
as below.
>>> psdf._internal.to_internal_spark_frame.show() # doctest: +NORMALIZE_WHITESPACE
+-----------------+---+---+---+---+---+
|__index_level_0__| A| B| C| D| E|
+-----------------+---+---+---+---+---+
| 0| 1| 5| 9| 13| 17|
| 1| 2| 6| 10| 14| 18|
| 2| 3| 7| 11| 15| 19|
| 3| 4| 8| 12| 16| 20|
+-----------------+---+---+---+---+---+
In order to fill this gap, the current metadata is used by mapping Spark's internal column
to pandas-on-Spark's index. See the method below:
* `spark_frame` represents the internal Spark DataFrame
* `data_spark_column_names` represents non-indexing Spark column names
* `data_spark_columns` represents non-indexing Spark columns
* `data_fields` represents non-indexing InternalFields
* `index_spark_column_names` represents internal index Spark column names
* `index_spark_columns` represents internal index Spark columns
* `index_fields` represents index InternalFields
* `spark_column_names` represents all columns
* `index_names` represents the external index name as a label
* `to_internal_spark_frame` represents Spark DataFrame derived by the metadata. Includes index.
* `to_pandas_frame` represents pandas DataFrame derived by the metadata
>>> internal = psdf._internal
>>> internal.spark_frame.show() # doctest: +NORMALIZE_WHITESPACE +ELLIPSIS
+-----------------+---+---+---+---+---+-----------------+
|__index_level_0__| A| B| C| D| E|__natural_order__|
+-----------------+---+---+---+---+---+-----------------+
| 0| 1| 5| 9| 13| 17| ...|
| 1| 2| 6| 10| 14| 18| ...|
| 2| 3| 7| 11| 15| 19| ...|
| 3| 4| 8| 12| 16| 20| ...|
+-----------------+---+---+---+---+---+-----------------+
>>> internal.data_spark_column_names
['A', 'B', 'C', 'D', 'E']
>>> internal.index_spark_column_names
['__index_level_0__']
>>> internal.spark_column_names
['__index_level_0__', 'A', 'B', 'C', 'D', 'E']
>>> internal.index_names
[None]
>>> internal.data_fields # doctest: +NORMALIZE_WHITESPACE
[InternalField(dtype=int64,struct_field=StructField(A,LongType,false)),
InternalField(dtype=int64,struct_field=StructField(B,LongType,false)),
InternalField(dtype=int64,struct_field=StructField(C,LongType,false)),
InternalField(dtype=int64,struct_field=StructField(D,LongType,false)),
InternalField(dtype=int64,struct_field=StructField(E,LongType,false))]
>>> internal.index_fields
[InternalField(dtype=int64,struct_field=StructField(__index_level_0__,LongType,false))]
>>> internal.to_internal_spark_frame.show() # doctest: +NORMALIZE_WHITESPACE
+-----------------+---+---+---+---+---+
|__index_level_0__| A| B| C| D| E|
+-----------------+---+---+---+---+---+
| 0| 1| 5| 9| 13| 17|
| 1| 2| 6| 10| 14| 18|
| 2| 3| 7| 11| 15| 19|
| 3| 4| 8| 12| 16| 20|
+-----------------+---+---+---+---+---+
>>> internal.to_pandas_frame
A B C D E
0 1 5 9 13 17
1 2 6 10 14 18
2 3 7 11 15 19
3 4 8 12 16 20
In case that index is set to one of the existing column as below:
>>> psdf1 = psdf.set_index("A")
>>> psdf1 # doctest: +NORMALIZE_WHITESPACE
B C D E
A
1 5 9 13 17
2 6 10 14 18
3 7 11 15 19
4 8 12 16 20
>>> psdf1._internal.to_internal_spark_frame.show() # doctest: +NORMALIZE_WHITESPACE
+---+---+---+---+---+
| A| B| C| D| E|
+---+---+---+---+---+
| 1| 5| 9| 13| 17|
| 2| 6| 10| 14| 18|
| 3| 7| 11| 15| 19|
| 4| 8| 12| 16| 20|
+---+---+---+---+---+
>>> internal = psdf1._internal
>>> internal.spark_frame.show() # doctest: +NORMALIZE_WHITESPACE +ELLIPSIS
+-----------------+---+---+---+---+---+-----------------+
|__index_level_0__| A| B| C| D| E|__natural_order__|
+-----------------+---+---+---+---+---+-----------------+
| 0| 1| 5| 9| 13| 17| ...|
| 1| 2| 6| 10| 14| 18| ...|
| 2| 3| 7| 11| 15| 19| ...|
| 3| 4| 8| 12| 16| 20| ...|
+-----------------+---+---+---+---+---+-----------------+
>>> internal.data_spark_column_names
['B', 'C', 'D', 'E']
>>> internal.index_spark_column_names
['A']
>>> internal.spark_column_names
['A', 'B', 'C', 'D', 'E']
>>> internal.index_names
[('A',)]
>>> internal.data_fields
[InternalField(dtype=int64,struct_field=StructField(B,LongType,false)),
InternalField(dtype=int64,struct_field=StructField(C,LongType,false)),
InternalField(dtype=int64,struct_field=StructField(D,LongType,false)),
InternalField(dtype=int64,struct_field=StructField(E,LongType,false))]
>>> internal.index_fields
[InternalField(dtype=int64,struct_field=StructField(A,LongType,false))]
>>> internal.to_internal_spark_frame.show() # doctest: +NORMALIZE_WHITESPACE
+---+---+---+---+---+
| A| B| C| D| E|
+---+---+---+---+---+
| 1| 5| 9| 13| 17|
| 2| 6| 10| 14| 18|
| 3| 7| 11| 15| 19|
| 4| 8| 12| 16| 20|
+---+---+---+---+---+
>>> internal.to_pandas_frame # doctest: +NORMALIZE_WHITESPACE
B C D E
A
1 5 9 13 17
2 6 10 14 18
3 7 11 15 19
4 8 12 16 20
In case that index becomes a multi index as below:
>>> psdf2 = psdf.set_index("A", append=True)
>>> psdf2 # doctest: +NORMALIZE_WHITESPACE
B C D E
A
0 1 5 9 13 17
1 2 6 10 14 18
2 3 7 11 15 19
3 4 8 12 16 20
>>> psdf2._internal.to_internal_spark_frame.show() # doctest: +NORMALIZE_WHITESPACE
+-----------------+---+---+---+---+---+
|__index_level_0__| A| B| C| D| E|
+-----------------+---+---+---+---+---+
| 0| 1| 5| 9| 13| 17|
| 1| 2| 6| 10| 14| 18|
| 2| 3| 7| 11| 15| 19|
| 3| 4| 8| 12| 16| 20|
+-----------------+---+---+---+---+---+
>>> internal = psdf2._internal
>>> internal.spark_frame.show() # doctest: +NORMALIZE_WHITESPACE +ELLIPSIS
+-----------------+---+---+---+---+---+-----------------+
|__index_level_0__| A| B| C| D| E|__natural_order__|
+-----------------+---+---+---+---+---+-----------------+
| 0| 1| 5| 9| 13| 17| ...|
| 1| 2| 6| 10| 14| 18| ...|
| 2| 3| 7| 11| 15| 19| ...|
| 3| 4| 8| 12| 16| 20| ...|
+-----------------+---+---+---+---+---+-----------------+
>>> internal.data_spark_column_names
['B', 'C', 'D', 'E']
>>> internal.index_spark_column_names
['__index_level_0__', 'A']
>>> internal.spark_column_names
['__index_level_0__', 'A', 'B', 'C', 'D', 'E']
>>> internal.index_names
[None, ('A',)]
>>> internal.data_fields # doctest: +NORMALIZE_WHITESPACE
[InternalField(dtype=int64,struct_field=StructField(B,LongType,false)),
InternalField(dtype=int64,struct_field=StructField(C,LongType,false)),
InternalField(dtype=int64,struct_field=StructField(D,LongType,false)),
InternalField(dtype=int64,struct_field=StructField(E,LongType,false))]
>>> internal.index_fields # doctest: +NORMALIZE_WHITESPACE
[InternalField(dtype=int64,struct_field=StructField(__index_level_0__,LongType,false)),
InternalField(dtype=int64,struct_field=StructField(A,LongType,false))]
>>> internal.to_internal_spark_frame.show() # doctest: +NORMALIZE_WHITESPACE
+-----------------+---+---+---+---+---+
|__index_level_0__| A| B| C| D| E|
+-----------------+---+---+---+---+---+
| 0| 1| 5| 9| 13| 17|
| 1| 2| 6| 10| 14| 18|
| 2| 3| 7| 11| 15| 19|
| 3| 4| 8| 12| 16| 20|
+-----------------+---+---+---+---+---+
>>> internal.to_pandas_frame # doctest: +NORMALIZE_WHITESPACE
B C D E
A
0 1 5 9 13 17
1 2 6 10 14 18
2 3 7 11 15 19
3 4 8 12 16 20
For multi-level columns, it also holds column_labels
>>> columns = pd.MultiIndex.from_tuples([('X', 'A'), ('X', 'B'),
... ('Y', 'C'), ('Y', 'D')])
>>> psdf3 = ps.DataFrame([
... [1, 2, 3, 4],
... [5, 6, 7, 8],
... [9, 10, 11, 12],
... [13, 14, 15, 16],
... [17, 18, 19, 20]], columns = columns)
>>> psdf3 # doctest: +NORMALIZE_WHITESPACE
X Y
A B C D
0 1 2 3 4
1 5 6 7 8
2 9 10 11 12
3 13 14 15 16
4 17 18 19 20
>>> internal = psdf3._internal
>>> internal.spark_frame.show() # doctest: +NORMALIZE_WHITESPACE +ELLIPSIS
+-----------------+------+------+------+------+-----------------+
|__index_level_0__|(X, A)|(X, B)|(Y, C)|(Y, D)|__natural_order__|
+-----------------+------+------+------+------+-----------------+
| 0| 1| 2| 3| 4| ...|
| 1| 5| 6| 7| 8| ...|
| 2| 9| 10| 11| 12| ...|
| 3| 13| 14| 15| 16| ...|
| 4| 17| 18| 19| 20| ...|
+-----------------+------+------+------+------+-----------------+
>>> internal.data_spark_column_names
['(X, A)', '(X, B)', '(Y, C)', '(Y, D)']
>>> internal.column_labels
[('X', 'A'), ('X', 'B'), ('Y', 'C'), ('Y', 'D')]
For Series, it also holds scol to represent the column.
>>> psseries = psdf1.B
>>> psseries
A
1 5
2 6
3 7
4 8
Name: B, dtype: int64
>>> internal = psseries._internal
>>> internal.spark_frame.show() # doctest: +NORMALIZE_WHITESPACE +ELLIPSIS
+-----------------+---+---+---+---+---+-----------------+
|__index_level_0__| A| B| C| D| E|__natural_order__|
+-----------------+---+---+---+---+---+-----------------+
| 0| 1| 5| 9| 13| 17| ...|
| 1| 2| 6| 10| 14| 18| ...|
| 2| 3| 7| 11| 15| 19| ...|
| 3| 4| 8| 12| 16| 20| ...|
+-----------------+---+---+---+---+---+-----------------+
>>> internal.data_spark_column_names
['B']
>>> internal.index_spark_column_names
['A']
>>> internal.spark_column_names
['A', 'B']
>>> internal.index_names
[('A',)]
>>> internal.data_fields
[InternalField(dtype=int64,struct_field=StructField(B,LongType,false))]
>>> internal.index_fields
[InternalField(dtype=int64,struct_field=StructField(A,LongType,false))]
>>> internal.to_internal_spark_frame.show() # doctest: +NORMALIZE_WHITESPACE
+---+---+
| A| B|
+---+---+
| 1| 5|
| 2| 6|
| 3| 7|
| 4| 8|
+---+---+
>>> internal.to_pandas_frame # doctest: +NORMALIZE_WHITESPACE
B
A
1 5
2 6
3 7
4 8
"""
def __init__(
self,
spark_frame: SparkDataFrame,
index_spark_columns: Optional[List[Column]],
index_names: Optional[List[Optional[Label]]] = None,
index_fields: Optional[List[InternalField]] = None,
column_labels: Optional[List[Label]] = None,
data_spark_columns: Optional[List[Column]] = None,
data_fields: Optional[List[InternalField]] = None,
column_label_names: Optional[List[Optional[Label]]] = None,
):
"""
Create a new internal immutable DataFrame to manage Spark DataFrame, column fields and
index fields and names.
:param spark_frame: Spark DataFrame to be managed.
:param index_spark_columns: list of Spark Column
Spark Columns for the index.
:param index_names: list of tuples
the index names.
:param index_fields: list of InternalField
the InternalFields for the index columns
:param column_labels: list of tuples with the same length
The multi-level values in the tuples.
:param data_spark_columns: list of Spark Column
Spark Columns to appear as columns. If this is None, calculated
from spark_frame.
:param data_fields: list of InternalField
the InternalFields for the data columns
:param column_label_names: Names for each of the column index levels.
See the examples below to refer what each parameter means.
>>> column_labels = pd.MultiIndex.from_tuples(
... [('a', 'x'), ('a', 'y'), ('b', 'z')], names=["column_labels_a", "column_labels_b"])
>>> row_index = pd.MultiIndex.from_tuples(
... [('foo', 'bar'), ('foo', 'bar'), ('zoo', 'bar')],
... names=["row_index_a", "row_index_b"])
>>> psdf = ps.DataFrame(
... [[1, 2, 3], [4, 5, 6], [7, 8, 9]], index=row_index, columns=column_labels)
>>> psdf.set_index(('a', 'x'), append=True, inplace=True)
>>> psdf # doctest: +NORMALIZE_WHITESPACE
column_labels_a a b
column_labels_b y z
row_index_a row_index_b (a, x)
foo bar 1 2 3
4 5 6
zoo bar 7 8 9
>>> internal = psdf._internal
>>> internal.spark_frame.show() # doctest: +NORMALIZE_WHITESPACE +ELLIPSIS
+-----------------+-----------------+------+------+------+...
|__index_level_0__|__index_level_1__|(a, x)|(a, y)|(b, z)|...
+-----------------+-----------------+------+------+------+...
| foo| bar| 1| 2| 3|...
| foo| bar| 4| 5| 6|...
| zoo| bar| 7| 8| 9|...
+-----------------+-----------------+------+------+------+...
>>> internal.index_spark_columns # doctest: +SKIP
[Column<'__index_level_0__'>, Column<'__index_level_1__'>, Column<'(a, x)'>]
>>> internal.index_names
[('row_index_a',), ('row_index_b',), ('a', 'x')]
>>> internal.index_fields # doctest: +NORMALIZE_WHITESPACE
[InternalField(dtype=object,struct_field=StructField(__index_level_0__,StringType,false)),
InternalField(dtype=object,struct_field=StructField(__index_level_1__,StringType,false)),
InternalField(dtype=int64,struct_field=StructField((a, x),LongType,false))]
>>> internal.column_labels
[('a', 'y'), ('b', 'z')]
>>> internal.data_spark_columns # doctest: +SKIP
[Column<'(a, y)'>, Column<'(b, z)'>]
>>> internal.data_fields # doctest: +NORMALIZE_WHITESPACE
[InternalField(dtype=int64,struct_field=StructField((a, y),LongType,false)),
InternalField(dtype=int64,struct_field=StructField((b, z),LongType,false))]
>>> internal.column_label_names
[('column_labels_a',), ('column_labels_b',)]
"""
assert isinstance(spark_frame, SparkDataFrame)
assert not spark_frame.isStreaming, "pandas-on-Spark does not support Structured Streaming."
if not index_spark_columns:
if data_spark_columns is not None:
if column_labels is not None:
data_spark_columns = [
scol.alias(name_like_string(label))
for scol, label in zip(data_spark_columns, column_labels)
]
spark_frame = spark_frame.select(data_spark_columns)
assert not any(SPARK_INDEX_NAME_PATTERN.match(name) for name in spark_frame.columns), (
"Index columns should not appear in columns of the Spark DataFrame. Avoid "
"index column names [%s]." % SPARK_INDEX_NAME_PATTERN
)
# Create default index.
spark_frame, force_nullable = InternalFrame.attach_default_index(spark_frame)
index_spark_columns = [scol_for(spark_frame, SPARK_DEFAULT_INDEX_NAME)]
index_fields = [
InternalField.from_struct_field(
StructField(SPARK_DEFAULT_INDEX_NAME, LongType(), nullable=False)
)
]
if data_spark_columns is not None:
data_struct_fields = [
field
for field in spark_frame.schema.fields
if field.name != SPARK_DEFAULT_INDEX_NAME
]
data_spark_columns = [
scol_for(spark_frame, field.name) for field in data_struct_fields
]
if data_fields is not None:
data_fields = [
field.copy(
name=name_like_string(struct_field.name),
nullable=(force_nullable or field.nullable),
)
for field, struct_field in zip(data_fields, data_struct_fields)
]
if NATURAL_ORDER_COLUMN_NAME not in spark_frame.columns:
spark_frame = spark_frame.withColumn(
NATURAL_ORDER_COLUMN_NAME, F.monotonically_increasing_id()
)
self._sdf = spark_frame # type: SparkDataFrame
# index_spark_columns
assert all(
isinstance(index_scol, Column) for index_scol in index_spark_columns
), index_spark_columns
self._index_spark_columns = index_spark_columns # type: List[Column]
# data_spark_columns
if data_spark_columns is None:
data_spark_columns = [
scol_for(spark_frame, col)
for col in spark_frame.columns
if all(
not spark_column_equals(scol_for(spark_frame, col), index_scol)
for index_scol in index_spark_columns
)
and col not in HIDDEN_COLUMNS
]
self._data_spark_columns = data_spark_columns # type: List[Column]
else:
assert all(isinstance(scol, Column) for scol in data_spark_columns)
self._data_spark_columns = data_spark_columns
# fields
if index_fields is None:
index_fields = [None] * len(index_spark_columns)
if data_fields is None:
data_fields = [None] * len(data_spark_columns)
assert len(index_spark_columns) == len(index_fields), (
len(index_spark_columns),
len(index_fields),
)
assert len(data_spark_columns) == len(data_fields), (
len(data_spark_columns),
len(data_fields),
)
if any(field is None or field.struct_field is None for field in index_fields) and any(
field is None or field.struct_field is None for field in data_fields
):
schema = spark_frame.select(index_spark_columns + data_spark_columns).schema
fields = [
InternalField.from_struct_field(struct_field)
if field is None
else InternalField(field.dtype, struct_field)
if field.struct_field is None
else field
for field, struct_field in zip(index_fields + data_fields, schema.fields)
]
index_fields = fields[: len(index_spark_columns)]
data_fields = fields[len(index_spark_columns) :]
elif any(field is None or field.struct_field is None for field in index_fields):
schema = spark_frame.select(index_spark_columns).schema
index_fields = [
InternalField.from_struct_field(struct_field)
if field is None
else InternalField(field.dtype, struct_field)
if field.struct_field is None
else field
for field, struct_field in zip(index_fields, schema.fields)
]
elif any(field is None or field.struct_field is None for field in data_fields):
schema = spark_frame.select(data_spark_columns).schema
data_fields = [
InternalField.from_struct_field(struct_field)
if field is None
else InternalField(field.dtype, struct_field)
if field.struct_field is None
else field
for field, struct_field in zip(data_fields, schema.fields)
]
assert all(
isinstance(ops.dtype, Dtype.__args__) # type: ignore
and (
ops.dtype == np.dtype("object")
or as_spark_type(ops.dtype, raise_error=False) is not None
)
for ops in index_fields
), index_fields
if is_testing():
struct_fields = spark_frame.select(index_spark_columns).schema.fields
assert all(
index_field.struct_field == struct_field
for index_field, struct_field in zip(index_fields, struct_fields)
), (index_fields, struct_fields)
self._index_fields = index_fields # type: List[InternalField]
assert all(
isinstance(ops.dtype, Dtype.__args__) # type: ignore
and (
ops.dtype == np.dtype("object")
or as_spark_type(ops.dtype, raise_error=False) is not None
)
for ops in data_fields
), data_fields
if is_testing():
struct_fields = spark_frame.select(data_spark_columns).schema.fields
assert all(
data_field.struct_field == struct_field
for data_field, struct_field in zip(data_fields, struct_fields)
), (data_fields, struct_fields)
self._data_fields = data_fields # type: List[InternalField]
# index_names
if not index_names:
index_names = [None] * len(index_spark_columns)
assert len(index_spark_columns) == len(index_names), (
len(index_spark_columns),
len(index_names),
)
assert all(
is_name_like_tuple(index_name, check_type=True) for index_name in index_names
), index_names
self._index_names = index_names # type: List[Optional[Label]]
# column_labels
if column_labels is None:
self._column_labels = [
(col,) for col in spark_frame.select(self._data_spark_columns).columns
] # type: List[Label]
else:
assert len(column_labels) == len(self._data_spark_columns), (
len(column_labels),
len(self._data_spark_columns),
)
if len(column_labels) == 1:
column_label = column_labels[0]
assert is_name_like_tuple(column_label, check_type=True), column_label
else:
assert all(
is_name_like_tuple(column_label, check_type=True)
for column_label in column_labels
), column_labels
assert len(set(len(label) for label in column_labels)) <= 1, column_labels
self._column_labels = column_labels
# column_label_names
if column_label_names is None:
self._column_label_names = [None] * column_labels_level(
self._column_labels
) # type: List[Optional[Label]]
else:
if len(self._column_labels) > 0:
assert len(column_label_names) == column_labels_level(self._column_labels), (
len(column_label_names),
column_labels_level(self._column_labels),
)
else:
assert len(column_label_names) > 0, len(column_label_names)
assert all(
is_name_like_tuple(column_label_name, check_type=True)
for column_label_name in column_label_names
), column_label_names
self._column_label_names = column_label_names
@staticmethod
def attach_default_index(
sdf: SparkDataFrame, default_index_type: Optional[str] = None
) -> Tuple[SparkDataFrame, bool]:
"""
This method attaches a default index to Spark DataFrame. Spark does not have the index
notion so corresponding column should be generated.
There are several types of default index can be configured by `compute.default_index_type`.
>>> spark_frame = ps.range(10).to_spark()
>>> spark_frame
DataFrame[id: bigint]
It adds the default index column '__index_level_0__'.
>>> spark_frame = InternalFrame.attach_default_index(spark_frame)[0]
>>> spark_frame
DataFrame[__index_level_0__: bigint, id: bigint]
It throws an exception if the given column name already exists.
>>> InternalFrame.attach_default_index(spark_frame)[0]
... # doctest: +ELLIPSIS
Traceback (most recent call last):
...
AssertionError: '__index_level_0__' already exists...
"""
index_column = SPARK_DEFAULT_INDEX_NAME
assert (
index_column not in sdf.columns
), "'%s' already exists in the Spark column names '%s'" % (index_column, sdf.columns)
if default_index_type is None:
default_index_type = ps.get_option("compute.default_index_type")
if default_index_type == "sequence":
return InternalFrame.attach_sequence_column(sdf, column_name=index_column)
elif default_index_type == "distributed-sequence":
return InternalFrame.attach_distributed_sequence_column(sdf, column_name=index_column)
elif default_index_type == "distributed":
return InternalFrame.attach_distributed_column(sdf, column_name=index_column)
else:
raise ValueError(
"'compute.default_index_type' should be one of 'sequence',"
" 'distributed-sequence' and 'distributed'"
)
@staticmethod
def attach_sequence_column(
sdf: SparkDataFrame, column_name: str
) -> Tuple[SparkDataFrame, bool]:
scols = [scol_for(sdf, column) for column in sdf.columns]
sequential_index = (
F.row_number().over(Window.orderBy(F.monotonically_increasing_id())).cast("long") - 1
)
return sdf.select(sequential_index.alias(column_name), *scols), False
@staticmethod
def attach_distributed_column(
sdf: SparkDataFrame, column_name: str
) -> Tuple[SparkDataFrame, bool]:
scols = [scol_for(sdf, column) for column in sdf.columns]
return sdf.select(F.monotonically_increasing_id().alias(column_name), *scols), False
@staticmethod
def attach_distributed_sequence_column(
sdf: SparkDataFrame, column_name: str
) -> Tuple[SparkDataFrame, bool]:
"""
This method attaches a Spark column that has a sequence in a distributed manner.
This is equivalent to the column assigned when default index type 'distributed-sequence'.
>>> sdf = ps.DataFrame(['a', 'b', 'c']).to_spark()
>>> sdf, force_nullable = (
... InternalFrame.attach_distributed_sequence_column(sdf, column_name="sequence")
... )
>>> sdf.show() # doctest: +NORMALIZE_WHITESPACE
+--------+---+
|sequence| 0|
+--------+---+
| 0| a|
| 1| b|
| 2| c|
+--------+---+
>>> force_nullable
True
"""
if len(sdf.columns) > 0:
try:
jdf = sdf._jdf.toDF() # type: ignore
sql_ctx = sdf.sql_ctx
encoders = sql_ctx._jvm.org.apache.spark.sql.Encoders # type: ignore
encoder = encoders.tuple(jdf.exprEnc(), encoders.scalaLong())
jrdd = jdf.localCheckpoint(False).rdd().zipWithIndex()
df = SparkDataFrame(
sql_ctx.sparkSession._jsparkSession.createDataset( # type: ignore
jrdd, encoder
).toDF(),
sql_ctx,
)
columns = df.columns
return (
df.selectExpr(
"`{}` as `{}`".format(columns[1], column_name), "`{}`.*".format(columns[0])
),
True,
)
except py4j.protocol.Py4JError:
if is_testing():
raise
return InternalFrame._attach_distributed_sequence_column(sdf, column_name)
else:
cnt = sdf.count()
if cnt > 0:
return default_session().range(cnt).toDF(column_name), False
else:
return (
default_session().createDataFrame(
[],
schema=StructType().add(column_name, data_type=LongType(), nullable=False),
),
False,
)
@staticmethod
def _attach_distributed_sequence_column(
sdf: SparkDataFrame, column_name: str
) -> Tuple[SparkDataFrame, bool]:
"""
>>> sdf = ps.DataFrame(['a', 'b', 'c']).to_spark()
>>> sdf, force_nullable = (
... InternalFrame._attach_distributed_sequence_column(sdf, column_name="sequence")
... )
>>> sdf.sort("sequence").show() # doctest: +NORMALIZE_WHITESPACE
+--------+---+
|sequence| 0|
+--------+---+
| 0| a|
| 1| b|
| 2| c|
+--------+---+
>>> force_nullable
False
"""
scols = [scol_for(sdf, column) for column in sdf.columns]
spark_partition_column = verify_temp_column_name(sdf, "__spark_partition_id__")
offset_column = verify_temp_column_name(sdf, "__offset__")
row_number_column = verify_temp_column_name(sdf, "__row_number__")
# 1. Calculates counts per each partition ID. `counts` here is, for instance,
# {
# 1: 83,
# 6: 83,
# 3: 83,
# ...
# }
sdf = sdf.withColumn(spark_partition_column, F.spark_partition_id())
# Checkpoint the DataFrame to fix the partition ID.
sdf = sdf.localCheckpoint(eager=False)