-
Notifications
You must be signed in to change notification settings - Fork 28.2k
/
indexing.py
1739 lines (1474 loc) · 63.5 KB
/
indexing.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 loc indexer for pandas-on-Spark DataFrame/Series.
"""
from abc import ABCMeta, abstractmethod
from collections.abc import Iterable
from functools import reduce
from typing import Any, Optional, List, Tuple, TYPE_CHECKING, Union, cast, Sized
import pandas as pd
from pandas.api.types import is_list_like
from pyspark import sql as spark
from pyspark.sql import functions as F
from pyspark.sql.types import BooleanType, LongType
from pyspark.sql.utils import AnalysisException
import numpy as np
from pyspark import pandas as ps # noqa: F401
from pyspark.pandas.internal import (
InternalFrame,
NATURAL_ORDER_COLUMN_NAME,
SPARK_DEFAULT_SERIES_NAME,
)
from pyspark.pandas.exceptions import SparkPandasIndexingError, SparkPandasNotImplementedError
from pyspark.pandas.typedef.typehints import (
Dtype,
Scalar,
extension_dtypes,
spark_type_to_pandas_dtype,
)
from pyspark.pandas.utils import (
is_name_like_tuple,
is_name_like_value,
lazy_property,
name_like_string,
same_anchor,
scol_for,
verify_temp_column_name,
)
if TYPE_CHECKING:
from pyspark.pandas.frame import DataFrame # noqa: F401 (SPARK-34943)
from pyspark.pandas.series import Series # noqa: F401 (SPARK-34943)
class IndexerLike(object):
def __init__(self, kdf_or_kser):
from pyspark.pandas.frame import DataFrame
from pyspark.pandas.series import Series
assert isinstance(kdf_or_kser, (DataFrame, Series)), "unexpected argument type: {}".format(
type(kdf_or_kser)
)
self._kdf_or_kser = kdf_or_kser
@property
def _is_df(self):
from pyspark.pandas.frame import DataFrame
return isinstance(self._kdf_or_kser, DataFrame)
@property
def _is_series(self):
from pyspark.pandas.series import Series
return isinstance(self._kdf_or_kser, Series)
@property
def _kdf(self):
if self._is_df:
return self._kdf_or_kser
else:
assert self._is_series
return self._kdf_or_kser._kdf
@property
def _internal(self):
return self._kdf._internal
class AtIndexer(IndexerLike):
"""
Access a single value for a row/column label pair.
If the index is not unique, all matching pairs are returned as an array.
Similar to ``loc``, in that both provide label-based lookups. Use ``at`` if you only need to
get a single value in a DataFrame or Series.
.. note:: Unlike pandas, pandas-on-Spark only allows using ``at`` to get values but not to
set them.
.. note:: Warning: If ``row_index`` matches a lot of rows, large amounts of data will be
fetched, potentially causing your machine to run out of memory.
Raises
------
KeyError
When label does not exist in DataFrame
Examples
--------
>>> kdf = ps.DataFrame([[0, 2, 3], [0, 4, 1], [10, 20, 30]],
... index=[4, 5, 5], columns=['A', 'B', 'C'])
>>> kdf
A B C
4 0 2 3
5 0 4 1
5 10 20 30
Get value at specified row/column pair
>>> kdf.at[4, 'B']
2
Get array if an index occurs multiple times
>>> kdf.at[5, 'B']
array([ 4, 20])
"""
def __getitem__(self, key) -> Union["Series", "DataFrame", Scalar]:
if self._is_df:
if not isinstance(key, tuple) or len(key) != 2:
raise TypeError("Use DataFrame.at like .at[row_index, column_name]")
row_sel, col_sel = key
else:
assert self._is_series, type(self._kdf_or_kser)
if isinstance(key, tuple) and len(key) != 1:
raise TypeError("Use Series.at like .at[row_index]")
row_sel = key
col_sel = self._kdf_or_kser._column_label
if self._internal.index_level == 1:
if not is_name_like_value(row_sel, allow_none=False, allow_tuple=False):
raise ValueError("At based indexing on a single index can only have a single value")
row_sel = (row_sel,)
else:
if not is_name_like_tuple(row_sel, allow_none=False):
raise ValueError("At based indexing on multi-index can only have tuple values")
if col_sel is not None:
if not is_name_like_value(col_sel, allow_none=False):
raise ValueError("At based indexing on multi-index can only have tuple values")
if not is_name_like_tuple(col_sel):
col_sel = (col_sel,)
cond = reduce(
lambda x, y: x & y,
[scol == row for scol, row in zip(self._internal.index_spark_columns, row_sel)],
)
pdf = (
self._internal.spark_frame.drop(NATURAL_ORDER_COLUMN_NAME)
.filter(cond)
.select(self._internal.spark_column_for(col_sel))
.toPandas()
)
if len(pdf) < 1:
raise KeyError(name_like_string(row_sel))
values = pdf.iloc[:, 0].values
return (
values if (len(row_sel) < self._internal.index_level or len(values) > 1) else values[0]
)
class iAtIndexer(IndexerLike):
"""
Access a single value for a row/column pair by integer position.
Similar to ``iloc``, in that both provide integer-based lookups. Use
``iat`` if you only need to get or set a single value in a DataFrame
or Series.
Raises
------
KeyError
When label does not exist in DataFrame
Examples
--------
>>> df = ps.DataFrame([[0, 2, 3], [0, 4, 1], [10, 20, 30]],
... columns=['A', 'B', 'C'])
>>> df
A B C
0 0 2 3
1 0 4 1
2 10 20 30
Get value at specified row/column pair
>>> df.iat[1, 2]
1
Get value within a series
>>> kser = ps.Series([1, 2, 3], index=[10, 20, 30])
>>> kser
10 1
20 2
30 3
dtype: int64
>>> kser.iat[1]
2
"""
def __getitem__(self, key) -> Union["Series", "DataFrame", Scalar]:
if self._is_df:
if not isinstance(key, tuple) or len(key) != 2:
raise TypeError(
"Use DataFrame.iat like .iat[row_integer_position, column_integer_position]"
)
row_sel, col_sel = key
if not isinstance(row_sel, int) or not isinstance(col_sel, int):
raise ValueError("iAt based indexing can only have integer indexers")
return self._kdf_or_kser.iloc[row_sel, col_sel]
else:
assert self._is_series, type(self._kdf_or_kser)
if not isinstance(key, int) and len(key) != 1:
raise TypeError("Use Series.iat like .iat[row_integer_position]")
if not isinstance(key, int):
raise ValueError("iAt based indexing can only have integer indexers")
return self._kdf_or_kser.iloc[key]
class LocIndexerLike(IndexerLike, metaclass=ABCMeta):
def _select_rows(
self, rows_sel: Any
) -> Tuple[Optional[spark.Column], Optional[int], Optional[int]]:
"""
Dispatch the logic for select rows to more specific methods by `rows_sel` argument types.
Parameters
----------
rows_sel : the key specified to select rows.
Returns
-------
Tuple of Spark column, int, int:
* The Spark column for the condition to filter the rows.
* The number of rows when the selection can be simplified by limit.
* The remaining index rows if the result index size is shrunk.
"""
from pyspark.pandas.series import Series
if rows_sel is None:
return None, None, None
elif isinstance(rows_sel, Series):
return self._select_rows_by_series(rows_sel)
elif isinstance(rows_sel, spark.Column):
return self._select_rows_by_spark_column(rows_sel)
elif isinstance(rows_sel, slice):
if rows_sel == slice(None):
# If slice is None - select everything, so nothing to do
return None, None, None
return self._select_rows_by_slice(rows_sel)
elif isinstance(rows_sel, tuple):
return self._select_rows_else(rows_sel)
elif is_list_like(rows_sel):
return self._select_rows_by_iterable(rows_sel)
else:
return self._select_rows_else(rows_sel)
def _select_cols(
self, cols_sel: Any, missing_keys: Optional[List[Tuple]] = None
) -> Tuple[
List[Tuple], Optional[List[spark.Column]], Optional[List[Dtype]], bool, Optional[Tuple]
]:
"""
Dispatch the logic for select columns to more specific methods by `cols_sel` argument types.
Parameters
----------
cols_sel : the key specified to select columns.
Returns
-------
Tuple of list of column label, list of Spark columns, list of dtypes, bool:
* The column labels selected.
* The Spark columns selected.
* The dtypes selected.
* The boolean value whether Series should be returned or not.
* The Series name if needed.
"""
from pyspark.pandas.series import Series
if cols_sel is None:
column_labels = self._internal.column_labels
data_spark_columns = self._internal.data_spark_columns
data_dtypes = self._internal.data_dtypes
return column_labels, data_spark_columns, data_dtypes, False, None
elif isinstance(cols_sel, Series):
return self._select_cols_by_series(cols_sel, missing_keys)
elif isinstance(cols_sel, spark.Column):
return self._select_cols_by_spark_column(cols_sel, missing_keys)
elif isinstance(cols_sel, slice):
if cols_sel == slice(None):
# If slice is None - select everything, so nothing to do
column_labels = self._internal.column_labels
data_spark_columns = self._internal.data_spark_columns
data_dtypes = self._internal.data_dtypes
return column_labels, data_spark_columns, data_dtypes, False, None
return self._select_cols_by_slice(cols_sel, missing_keys)
elif isinstance(cols_sel, tuple):
return self._select_cols_else(cols_sel, missing_keys)
elif is_list_like(cols_sel):
return self._select_cols_by_iterable(cols_sel, missing_keys)
else:
return self._select_cols_else(cols_sel, missing_keys)
# Methods for row selection
@abstractmethod
def _select_rows_by_series(
self, rows_sel: "Series"
) -> Tuple[Optional[spark.Column], Optional[int], Optional[int]]:
""" Select rows by `Series` type key. """
pass
@abstractmethod
def _select_rows_by_spark_column(
self, rows_sel: spark.column
) -> Tuple[Optional[spark.Column], Optional[int], Optional[int]]:
""" Select rows by Spark `Column` type key. """
pass
@abstractmethod
def _select_rows_by_slice(
self, rows_sel: slice
) -> Tuple[Optional[spark.Column], Optional[int], Optional[int]]:
""" Select rows by `slice` type key. """
pass
@abstractmethod
def _select_rows_by_iterable(
self, rows_sel: Iterable
) -> Tuple[Optional[spark.Column], Optional[int], Optional[int]]:
""" Select rows by `Iterable` type key. """
pass
@abstractmethod
def _select_rows_else(
self, rows_sel: Any
) -> Tuple[Optional[spark.Column], Optional[int], Optional[int]]:
""" Select rows by other type key. """
pass
# Methods for col selection
@abstractmethod
def _select_cols_by_series(
self, cols_sel: "Series", missing_keys: Optional[List[Tuple]]
) -> Tuple[
List[Tuple], Optional[List[spark.Column]], Optional[List[Dtype]], bool, Optional[Tuple]
]:
""" Select columns by `Series` type key. """
pass
@abstractmethod
def _select_cols_by_spark_column(
self, cols_sel: spark.Column, missing_keys: Optional[List[Tuple]]
) -> Tuple[
List[Tuple], Optional[List[spark.Column]], Optional[List[Dtype]], bool, Optional[Tuple]
]:
""" Select columns by Spark `Column` type key. """
pass
@abstractmethod
def _select_cols_by_slice(
self, cols_sel: slice, missing_keys: Optional[List[Tuple]]
) -> Tuple[
List[Tuple], Optional[List[spark.Column]], Optional[List[Dtype]], bool, Optional[Tuple]
]:
""" Select columns by `slice` type key. """
pass
@abstractmethod
def _select_cols_by_iterable(
self, cols_sel: Iterable, missing_keys: Optional[List[Tuple]]
) -> Tuple[
List[Tuple], Optional[List[spark.Column]], Optional[List[Dtype]], bool, Optional[Tuple]
]:
""" Select columns by `Iterable` type key. """
pass
@abstractmethod
def _select_cols_else(
self, cols_sel: Any, missing_keys: Optional[List[Tuple]]
) -> Tuple[
List[Tuple], Optional[List[spark.Column]], Optional[List[Dtype]], bool, Optional[Tuple]
]:
""" Select columns by other type key. """
pass
def __getitem__(self, key) -> Union["Series", "DataFrame"]:
from pyspark.pandas.frame import DataFrame
from pyspark.pandas.series import Series, first_series
if self._is_series:
if isinstance(key, Series) and not same_anchor(key, self._kdf_or_kser):
kdf = self._kdf_or_kser.to_frame()
temp_col = verify_temp_column_name(kdf, "__temp_col__")
kdf[temp_col] = key
return type(self)(kdf[self._kdf_or_kser.name])[kdf[temp_col]]
cond, limit, remaining_index = self._select_rows(key)
if cond is None and limit is None:
return self._kdf_or_kser
column_label = self._kdf_or_kser._column_label
column_labels = [column_label]
data_spark_columns = [self._internal.spark_column_for(column_label)]
data_dtypes = [self._internal.dtype_for(column_label)]
returns_series = True
series_name = self._kdf_or_kser.name
else:
assert self._is_df
if isinstance(key, tuple):
if len(key) != 2:
raise SparkPandasIndexingError("Only accepts pairs of candidates")
rows_sel, cols_sel = key
else:
rows_sel = key
cols_sel = None
if isinstance(rows_sel, Series) and not same_anchor(rows_sel, self._kdf_or_kser):
kdf = self._kdf_or_kser.copy()
temp_col = verify_temp_column_name(kdf, "__temp_col__")
kdf[temp_col] = rows_sel
return type(self)(kdf)[kdf[temp_col], cols_sel][list(self._kdf_or_kser.columns)]
cond, limit, remaining_index = self._select_rows(rows_sel)
(
column_labels,
data_spark_columns,
data_dtypes,
returns_series,
series_name,
) = self._select_cols(cols_sel)
if cond is None and limit is None and returns_series:
kser = self._kdf_or_kser._kser_for(column_labels[0])
if series_name is not None and series_name != kser.name:
kser = kser.rename(series_name)
return kser
if remaining_index is not None:
index_spark_columns = self._internal.index_spark_columns[-remaining_index:]
index_names = self._internal.index_names[-remaining_index:]
index_dtypes = self._internal.index_dtypes[-remaining_index:]
else:
index_spark_columns = self._internal.index_spark_columns
index_names = self._internal.index_names
index_dtypes = self._internal.index_dtypes
if len(column_labels) > 0:
column_labels = column_labels.copy()
column_labels_level = max(
len(label) if label is not None else 1 for label in column_labels
)
none_column = 0
for i, label in enumerate(column_labels):
if label is None:
label = (none_column,)
none_column += 1
if len(label) < column_labels_level:
label = tuple(list(label) + ([""]) * (column_labels_level - len(label)))
column_labels[i] = label
if i == 0 and none_column == 1:
column_labels = [None]
column_label_names = self._internal.column_label_names[-column_labels_level:]
else:
column_label_names = self._internal.column_label_names
try:
sdf = self._internal.spark_frame
if cond is not None:
index_columns = sdf.select(index_spark_columns).columns
data_columns = sdf.select(data_spark_columns).columns
sdf = sdf.filter(cond).select(index_spark_columns + data_spark_columns)
index_spark_columns = [scol_for(sdf, col) for col in index_columns]
data_spark_columns = [scol_for(sdf, col) for col in data_columns]
if limit is not None:
if limit >= 0:
sdf = sdf.limit(limit)
else:
sdf = sdf.limit(sdf.count() + limit)
sdf = sdf.drop(NATURAL_ORDER_COLUMN_NAME)
except AnalysisException:
raise KeyError(
"[{}] don't exist in columns".format(
[col._jc.toString() for col in data_spark_columns]
)
)
internal = InternalFrame(
spark_frame=sdf,
index_spark_columns=index_spark_columns,
index_names=index_names,
index_dtypes=index_dtypes,
column_labels=column_labels,
data_spark_columns=data_spark_columns,
data_dtypes=data_dtypes,
column_label_names=column_label_names,
)
kdf = DataFrame(internal)
if returns_series:
kdf_or_kser = first_series(kdf)
if series_name is not None and series_name != kdf_or_kser.name:
kdf_or_kser = kdf_or_kser.rename(series_name)
else:
kdf_or_kser = kdf
if remaining_index is not None and remaining_index == 0:
pdf_or_pser = kdf_or_kser.head(2).to_pandas()
length = len(pdf_or_pser)
if length == 0:
raise KeyError(name_like_string(key))
elif length == 1:
return pdf_or_pser.iloc[0]
else:
return kdf_or_kser
else:
return kdf_or_kser
def __setitem__(self, key, value):
from pyspark.pandas.frame import DataFrame
from pyspark.pandas.series import Series, first_series
if self._is_series:
if (
isinstance(key, Series)
and (isinstance(self, iLocIndexer) or not same_anchor(key, self._kdf_or_kser))
) or (
isinstance(value, Series)
and (isinstance(self, iLocIndexer) or not same_anchor(value, self._kdf_or_kser))
):
if self._kdf_or_kser.name is None:
kdf = self._kdf_or_kser.to_frame()
column_label = kdf._internal.column_labels[0]
else:
kdf = self._kdf_or_kser._kdf.copy()
column_label = self._kdf_or_kser._column_label
temp_natural_order = verify_temp_column_name(kdf, "__temp_natural_order__")
temp_key_col = verify_temp_column_name(kdf, "__temp_key_col__")
temp_value_col = verify_temp_column_name(kdf, "__temp_value_col__")
kdf[temp_natural_order] = F.monotonically_increasing_id()
if isinstance(key, Series):
kdf[temp_key_col] = key
if isinstance(value, Series):
kdf[temp_value_col] = value
kdf = kdf.sort_values(temp_natural_order).drop(temp_natural_order)
kser = kdf._kser_for(column_label)
if isinstance(key, Series):
key = F.col(
"`{}`".format(kdf[temp_key_col]._internal.data_spark_column_names[0])
)
if isinstance(value, Series):
value = F.col(
"`{}`".format(kdf[temp_value_col]._internal.data_spark_column_names[0])
)
type(self)(kser)[key] = value
if self._kdf_or_kser.name is None:
kser = kser.rename()
self._kdf_or_kser._kdf._update_internal_frame(
kser._kdf[
self._kdf_or_kser._kdf._internal.column_labels
]._internal.resolved_copy,
requires_same_anchor=False,
)
return
if isinstance(value, DataFrame):
raise ValueError("Incompatible indexer with DataFrame")
cond, limit, remaining_index = self._select_rows(key)
if cond is None:
cond = F.lit(True)
if limit is not None:
cond = cond & (self._internal.spark_frame[self._sequence_col] < F.lit(limit))
if isinstance(value, (Series, spark.Column)):
if remaining_index is not None and remaining_index == 0:
raise ValueError(
"No axis named {} for object type {}".format(key, type(value).__name__)
)
if isinstance(value, Series):
value = value.spark.column
else:
value = F.lit(value)
scol = (
F.when(cond, value)
.otherwise(self._internal.spark_column_for(self._kdf_or_kser._column_label))
.alias(name_like_string(self._kdf_or_kser.name or SPARK_DEFAULT_SERIES_NAME))
)
internal = self._internal.with_new_spark_column(
self._kdf_or_kser._column_label, scol # TODO: dtype?
)
self._kdf_or_kser._kdf._update_internal_frame(internal, requires_same_anchor=False)
else:
assert self._is_df
if isinstance(key, tuple):
if len(key) != 2:
raise SparkPandasIndexingError("Only accepts pairs of candidates")
rows_sel, cols_sel = key
else:
rows_sel = key
cols_sel = None
if isinstance(value, DataFrame):
if len(value.columns) == 1:
value = first_series(value)
else:
raise ValueError("Only a dataframe with one column can be assigned")
if (
isinstance(rows_sel, Series)
and (isinstance(self, iLocIndexer) or not same_anchor(rows_sel, self._kdf_or_kser))
) or (
isinstance(value, Series)
and (isinstance(self, iLocIndexer) or not same_anchor(value, self._kdf_or_kser))
):
kdf = self._kdf_or_kser.copy()
temp_natural_order = verify_temp_column_name(kdf, "__temp_natural_order__")
temp_key_col = verify_temp_column_name(kdf, "__temp_key_col__")
temp_value_col = verify_temp_column_name(kdf, "__temp_value_col__")
kdf[temp_natural_order] = F.monotonically_increasing_id()
if isinstance(rows_sel, Series):
kdf[temp_key_col] = rows_sel
if isinstance(value, Series):
kdf[temp_value_col] = value
kdf = kdf.sort_values(temp_natural_order).drop(temp_natural_order)
if isinstance(rows_sel, Series):
rows_sel = F.col(
"`{}`".format(kdf[temp_key_col]._internal.data_spark_column_names[0])
)
if isinstance(value, Series):
value = F.col(
"`{}`".format(kdf[temp_value_col]._internal.data_spark_column_names[0])
)
type(self)(kdf)[rows_sel, cols_sel] = value
self._kdf_or_kser._update_internal_frame(
kdf[list(self._kdf_or_kser.columns)]._internal.resolved_copy,
requires_same_anchor=False,
)
return
cond, limit, remaining_index = self._select_rows(rows_sel)
missing_keys = []
_, data_spark_columns, _, _, _ = self._select_cols(cols_sel, missing_keys=missing_keys)
if cond is None:
cond = F.lit(True)
if limit is not None:
cond = cond & (self._internal.spark_frame[self._sequence_col] < F.lit(limit))
if isinstance(value, (Series, spark.Column)):
if remaining_index is not None and remaining_index == 0:
raise ValueError("Incompatible indexer with Series")
if len(data_spark_columns) > 1:
raise ValueError("shape mismatch")
if isinstance(value, Series):
value = value.spark.column
else:
value = F.lit(value)
new_data_spark_columns = []
new_dtypes = []
for new_scol, spark_column_name, new_dtype in zip(
self._internal.data_spark_columns,
self._internal.data_spark_column_names,
self._internal.data_dtypes,
):
for scol in data_spark_columns:
if new_scol._jc.equals(scol._jc):
new_scol = F.when(cond, value).otherwise(scol).alias(spark_column_name)
new_dtype = spark_type_to_pandas_dtype(
self._internal.spark_frame.select(new_scol).schema[0].dataType,
use_extension_dtypes=isinstance(new_dtype, extension_dtypes),
)
break
new_data_spark_columns.append(new_scol)
new_dtypes.append(new_dtype)
column_labels = self._internal.column_labels.copy()
for label in missing_keys:
if not is_name_like_tuple(label):
label = (label,)
if len(label) < self._internal.column_labels_level:
label = tuple(
list(label) + ([""] * (self._internal.column_labels_level - len(label)))
)
elif len(label) > self._internal.column_labels_level:
raise KeyError(
"Key length ({}) exceeds index depth ({})".format(
len(label), self._internal.column_labels_level
)
)
column_labels.append(label)
new_data_spark_columns.append(F.when(cond, value).alias(name_like_string(label)))
new_dtypes.append(None)
internal = self._internal.with_new_columns(
new_data_spark_columns, column_labels=column_labels, data_dtypes=new_dtypes
)
self._kdf_or_kser._update_internal_frame(internal, requires_same_anchor=False)
class LocIndexer(LocIndexerLike):
"""
Access a group of rows and columns by label(s) or a boolean Series.
``.loc[]`` is primarily label based, but may also be used with a
conditional boolean Series derived from the DataFrame or Series.
Allowed inputs are:
- A single label, e.g. ``5`` or ``'a'``, (note that ``5`` is
interpreted as a *label* of the index, and **never** as an
integer position along the index) for column selection.
- A list or array of labels, e.g. ``['a', 'b', 'c']``.
- A slice object with labels, e.g. ``'a':'f'``.
- A conditional boolean Series derived from the DataFrame or Series
- A boolean array of the same length as the column axis being sliced,
e.g. ``[True, False, True]``.
- An alignable boolean pandas Series to the column axis being sliced.
The index of the key will be aligned before masking.
Not allowed inputs which pandas allows are:
- A boolean array of the same length as the row axis being sliced,
e.g. ``[True, False, True]``.
- A ``callable`` function with one argument (the calling Series, DataFrame
or Panel) and that returns valid output for indexing (one of the above)
.. note:: MultiIndex is not supported yet.
.. note:: Note that contrary to usual python slices, **both** the
start and the stop are included, and the step of the slice is not allowed.
.. note:: With a list or array of labels for row selection,
pandas-on-Spark behaves as a filter without reordering by the labels.
See Also
--------
Series.loc : Access group of values using labels.
Examples
--------
**Getting values**
>>> df = ps.DataFrame([[1, 2], [4, 5], [7, 8]],
... index=['cobra', 'viper', 'sidewinder'],
... columns=['max_speed', 'shield'])
>>> df
max_speed shield
cobra 1 2
viper 4 5
sidewinder 7 8
Single label. Note this returns the row as a Series.
>>> df.loc['viper']
max_speed 4
shield 5
Name: viper, dtype: int64
List of labels. Note using ``[[]]`` returns a DataFrame.
Also note that pandas-on-Spark behaves just a filter without reordering by the labels.
>>> df.loc[['viper', 'sidewinder']]
max_speed shield
viper 4 5
sidewinder 7 8
>>> df.loc[['sidewinder', 'viper']]
max_speed shield
viper 4 5
sidewinder 7 8
Single label for column.
>>> df.loc['cobra', 'shield']
2
List of labels for row.
>>> df.loc[['cobra'], 'shield']
cobra 2
Name: shield, dtype: int64
List of labels for column.
>>> df.loc['cobra', ['shield']]
shield 2
Name: cobra, dtype: int64
List of labels for both row and column.
>>> df.loc[['cobra'], ['shield']]
shield
cobra 2
Slice with labels for row and single label for column. As mentioned
above, note that both the start and stop of the slice are included.
>>> df.loc['cobra':'viper', 'max_speed']
cobra 1
viper 4
Name: max_speed, dtype: int64
Conditional that returns a boolean Series
>>> df.loc[df['shield'] > 6]
max_speed shield
sidewinder 7 8
Conditional that returns a boolean Series with column labels specified
>>> df.loc[df['shield'] > 6, ['max_speed']]
max_speed
sidewinder 7
A boolean array of the same length as the column axis being sliced.
>>> df.loc[:, [False, True]]
shield
cobra 2
viper 5
sidewinder 8
An alignable boolean Series to the column axis being sliced.
>>> df.loc[:, pd.Series([False, True], index=['max_speed', 'shield'])]
shield
cobra 2
viper 5
sidewinder 8
**Setting values**
Setting value for all items matching the list of labels.
>>> df.loc[['viper', 'sidewinder'], ['shield']] = 50
>>> df
max_speed shield
cobra 1 2
viper 4 50
sidewinder 7 50
Setting value for an entire row
>>> df.loc['cobra'] = 10
>>> df
max_speed shield
cobra 10 10
viper 4 50
sidewinder 7 50
Set value for an entire column
>>> df.loc[:, 'max_speed'] = 30
>>> df
max_speed shield
cobra 30 10
viper 30 50
sidewinder 30 50
Set value for an entire list of columns
>>> df.loc[:, ['max_speed', 'shield']] = 100
>>> df
max_speed shield
cobra 100 100
viper 100 100
sidewinder 100 100
Set value with Series
>>> df.loc[:, 'shield'] = df['shield'] * 2
>>> df
max_speed shield
cobra 100 200
viper 100 200
sidewinder 100 200
**Getting values on a DataFrame with an index that has integer labels**
Another example using integers for the index
>>> df = ps.DataFrame([[1, 2], [4, 5], [7, 8]],
... index=[7, 8, 9],
... columns=['max_speed', 'shield'])
>>> df
max_speed shield
7 1 2
8 4 5
9 7 8
Slice with integer labels for rows. As mentioned above, note that both
the start and stop of the slice are included.
>>> df.loc[7:9]
max_speed shield
7 1 2
8 4 5
9 7 8
"""
@staticmethod
def _NotImplemented(description):
return SparkPandasNotImplementedError(
description=description,
pandas_function=".loc[..., ...]",
spark_target_function="select, where",
)
def _select_rows_by_series(
self, rows_sel: "Series"
) -> Tuple[Optional[spark.Column], Optional[int], Optional[int]]:
assert isinstance(rows_sel.spark.data_type, BooleanType), rows_sel.spark.data_type
return rows_sel.spark.column, None, None
def _select_rows_by_spark_column(
self, rows_sel: spark.Column
) -> Tuple[Optional[spark.Column], Optional[int], Optional[int]]:
spark_type = self._internal.spark_frame.select(rows_sel).schema[0].dataType
assert isinstance(spark_type, BooleanType), spark_type
return rows_sel, None, None
def _select_rows_by_slice(
self, rows_sel: slice
) -> Tuple[Optional[spark.Column], Optional[int], Optional[int]]:
from pyspark.pandas.indexes import MultiIndex
if rows_sel.step is not None:
raise LocIndexer._NotImplemented("Cannot use step with Spark.")
elif self._internal.index_level == 1:
sdf = self._internal.spark_frame
index = self._kdf_or_kser.index
index_column = index.to_series()
index_data_type = index_column.spark.data_type
start = rows_sel.start
stop = rows_sel.stop
# get natural order from '__natural_order__' from start to stop
# to keep natural order.
start_and_stop = (
sdf.select(index_column.spark.column, NATURAL_ORDER_COLUMN_NAME)
.where(
(index_column.spark.column == F.lit(start).cast(index_data_type))
| (index_column.spark.column == F.lit(stop).cast(index_data_type))
)
.collect()
)
start = [row[1] for row in start_and_stop if row[0] == start]
start = start[0] if len(start) > 0 else None