-
Notifications
You must be signed in to change notification settings - Fork 29.2k
Expand file tree
/
Copy pathdataframe.py
More file actions
2137 lines (1857 loc) · 77.3 KB
/
dataframe.py
File metadata and controls
2137 lines (1857 loc) · 77.3 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
#
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
#
import os
import json
import sys
import random
import warnings
from collections.abc import Iterable
from functools import reduce, cached_property
from typing import (
Any,
Callable,
Dict,
Iterator,
List,
Optional,
Sequence,
Tuple,
Type,
Union,
cast,
overload,
TYPE_CHECKING,
)
from pyspark import _NoValue
from pyspark.resource import ResourceProfile
from pyspark._globals import _NoValueType
from pyspark.errors import (
AnalysisException,
PySparkTypeError,
PySparkValueError,
PySparkIndexError,
PySparkAttributeError,
)
from pyspark.util import (
_load_from_socket,
_local_iterator_from_socket,
)
from pyspark.serializers import BatchedSerializer, CPickleSerializer, UTF8Deserializer
from pyspark.storagelevel import StorageLevel
from pyspark.traceback_utils import SCCallSiteSync
from pyspark.sql.column import Column
from pyspark.sql.functions import builtin as F
from pyspark.sql.classic.column import _to_seq, _to_list, _to_java_column
from pyspark.sql.readwriter import DataFrameWriter, DataFrameWriterV2
from pyspark.sql.merge import MergeIntoWriter
from pyspark.sql.streaming import DataStreamWriter
from pyspark.sql.types import (
StructType,
Row,
_parse_datatype_json_string,
)
from pyspark.sql.dataframe import (
DataFrame as ParentDataFrame,
DataFrameNaFunctions as ParentDataFrameNaFunctions,
DataFrameStatFunctions as ParentDataFrameStatFunctions,
)
from pyspark.sql.utils import get_active_spark_context, to_java_array, to_scala_map
from pyspark.sql.pandas.conversion import PandasConversionMixin
from pyspark.sql.pandas.map_ops import PandasMapOpsMixin
from pyspark.sql.table_arg import TableArg
if TYPE_CHECKING:
from py4j.java_gateway import JavaObject
import pyarrow as pa
from pyspark.core.rdd import RDD
from pyspark.core.context import SparkContext
from pyspark._typing import PrimitiveType
from pyspark.pandas.frame import DataFrame as PandasOnSparkDataFrame
from pyspark.sql._typing import (
ColumnOrName,
ColumnOrNameOrOrdinal,
LiteralType,
OptionalPrimitiveType,
)
from pyspark.sql.pandas._typing import (
PandasMapIterFunction,
ArrowMapIterFunction,
DataFrameLike as PandasDataFrameLike,
)
from pyspark.sql.context import SQLContext
from pyspark.sql.session import SparkSession
from pyspark.sql.group import GroupedData
from pyspark.sql.observation import Observation
from pyspark.sql.metrics import ExecutionInfo
class DataFrame(ParentDataFrame, PandasMapOpsMixin, PandasConversionMixin):
def __new__(
cls,
jdf: "JavaObject",
sql_ctx: Union["SQLContext", "SparkSession"],
) -> "DataFrame":
self = object.__new__(cls)
self.__init__(jdf, sql_ctx) # type: ignore[misc]
return self
def __init__(
self,
jdf: "JavaObject",
sql_ctx: Union["SQLContext", "SparkSession"],
):
from pyspark.sql.context import SQLContext
if isinstance(sql_ctx, SQLContext):
assert not os.environ.get("SPARK_TESTING") # Sanity check for our internal usage.
assert isinstance(sql_ctx, SQLContext)
# We should remove this if-else branch in the future release, and rename
# sql_ctx to session in the constructor. This is an internal code path but
# was kept with a warning because it's used intensively by third-party libraries.
warnings.warn("DataFrame constructor is internal. Do not directly use it.")
self._sql_ctx = sql_ctx
session = sql_ctx.sparkSession
else:
session = sql_ctx
self._session: "SparkSession" = session
self._sc: "SparkContext" = sql_ctx._sc
self._jdf: "JavaObject" = jdf
self.is_cached = False
# Check whether _repr_html is supported or not, we use it to avoid calling _jdf twice
# by __repr__ and _repr_html_ while eager evaluation opens.
self._support_repr_html = False
@cached_property
def sql_ctx(self) -> "SQLContext":
from pyspark.sql.context import SQLContext
warnings.warn(
"DataFrame.sql_ctx is an internal property, and will be removed "
"in future releases. Use DataFrame.sparkSession instead."
)
return SQLContext._get_or_create(self._sc)
@property
def sparkSession(self) -> "SparkSession":
return self._session
@cached_property
def rdd(self) -> "RDD[Row]":
from pyspark.core.rdd import RDD
jrdd = self._jdf.javaToPython()
return RDD(jrdd, self.sparkSession._sc, BatchedSerializer(CPickleSerializer()))
@property
def na(self) -> ParentDataFrameNaFunctions:
return DataFrameNaFunctions(self)
@property
def stat(self) -> ParentDataFrameStatFunctions:
return DataFrameStatFunctions(self)
def toJSON(self, use_unicode: bool = True) -> Union["RDD[str]", "DataFrame"]:
from pyspark.core.rdd import RDD
returnDataFrame = self.sparkSession._jconf.pysparkToJSONReturnDataFrame()
if returnDataFrame:
jdf = self._jdf.toJSON()
return DataFrame(jdf, self.sparkSession)
else:
warnings.warn(
"The return type of DataFrame.toJSON will be changed from RDD to DataFrame "
"in future releases."
)
rdd = self._jdf.toJSON()
return RDD(rdd.toJavaRDD(), self._sc, UTF8Deserializer(use_unicode))
def registerTempTable(self, name: str) -> None:
warnings.warn("Deprecated in 2.0, use createOrReplaceTempView instead.", FutureWarning)
self._jdf.createOrReplaceTempView(name)
def createTempView(self, name: str) -> None:
self._jdf.createTempView(name)
def createOrReplaceTempView(self, name: str) -> None:
self._jdf.createOrReplaceTempView(name)
def createGlobalTempView(self, name: str) -> None:
self._jdf.createGlobalTempView(name)
def createOrReplaceGlobalTempView(self, name: str) -> None:
self._jdf.createOrReplaceGlobalTempView(name)
@property
def write(self) -> DataFrameWriter:
return DataFrameWriter(self)
@property
def writeStream(self) -> DataStreamWriter:
return DataStreamWriter(self)
@cached_property
def schema(self) -> StructType:
try:
return cast(StructType, _parse_datatype_json_string(self._jdf.schema().json()))
except AnalysisException as e:
raise e
except Exception as e:
raise PySparkValueError(
errorClass="CANNOT_PARSE_DATATYPE",
messageParameters={"msg": str(e)},
)
def printSchema(self, level: Optional[int] = None) -> None:
if level:
print(self._jdf.schema().treeString(level))
else:
print(self._jdf.schema().treeString())
def explain(
self, extended: Optional[Union[bool, str]] = None, mode: Optional[str] = None
) -> None:
if extended is not None and mode is not None:
raise PySparkValueError(
errorClass="CANNOT_SET_TOGETHER",
messageParameters={"arg_list": "extended and mode"},
)
# For the no argument case: df.explain()
is_no_argument = extended is None and mode is None
# For the cases below:
# explain(True)
# explain(extended=False)
is_extended_case = isinstance(extended, bool) and mode is None
# For the case when extended is mode:
# df.explain("formatted")
is_extended_as_mode = isinstance(extended, str) and mode is None
# For the mode specified:
# df.explain(mode="formatted")
is_mode_case = extended is None and isinstance(mode, str)
if not (is_no_argument or is_extended_case or is_extended_as_mode or is_mode_case):
if (extended is not None) and (not isinstance(extended, (bool, str))):
raise PySparkTypeError(
errorClass="NOT_EXPECTED_TYPE",
messageParameters={
"expected_type": "bool or str",
"arg_name": "extended",
"arg_type": type(extended).__name__,
},
)
if (mode is not None) and (not isinstance(mode, str)):
raise PySparkTypeError(
errorClass="NOT_EXPECTED_TYPE",
messageParameters={
"arg_name": "mode",
"expected_type": "str",
"arg_type": type(mode).__name__,
},
)
# Sets an explain mode depending on a given argument
if is_no_argument:
explain_mode = "simple"
elif is_extended_case:
explain_mode = "extended" if extended else "simple"
elif is_mode_case:
explain_mode = cast(str, mode)
elif is_extended_as_mode:
explain_mode = cast(str, extended)
assert self._sc._jvm is not None
print(self._sc._jvm.PythonSQLUtils.explainString(self._jdf.queryExecution(), explain_mode))
def exceptAll(self, other: ParentDataFrame) -> ParentDataFrame:
return DataFrame(self._jdf.exceptAll(other._jdf), self.sparkSession)
def zipWithIndex(self, indexColName: str = "index") -> ParentDataFrame:
return DataFrame(self._jdf.zipWithIndex(indexColName), self.sparkSession)
def isLocal(self) -> bool:
return self._jdf.isLocal()
@property
def isStreaming(self) -> bool:
return self._jdf.isStreaming()
def isEmpty(self) -> bool:
return self._jdf.isEmpty()
def show(self, n: int = 20, truncate: Union[bool, int] = True, vertical: bool = False) -> None:
print(self._show_string(n, truncate, vertical))
def _show_string(
self, n: int = 20, truncate: Union[bool, int] = True, vertical: bool = False
) -> str:
if not isinstance(n, int) or isinstance(n, bool):
raise PySparkTypeError(
errorClass="NOT_EXPECTED_TYPE",
messageParameters={
"expected_type": "int",
"arg_name": "n",
"arg_type": type(n).__name__,
},
)
if not isinstance(vertical, bool):
raise PySparkTypeError(
errorClass="NOT_EXPECTED_TYPE",
messageParameters={
"expected_type": "bool",
"arg_name": "vertical",
"arg_type": type(vertical).__name__,
},
)
if isinstance(truncate, bool) and truncate:
return self._jdf.showString(n, 20, vertical)
else:
try:
int_truncate = int(truncate)
except ValueError:
raise PySparkTypeError(
errorClass="NOT_EXPECTED_TYPE",
messageParameters={
"expected_type": "bool",
"arg_name": "truncate",
"arg_type": type(truncate).__name__,
},
)
return self._jdf.showString(n, int_truncate, vertical)
def __repr__(self) -> str:
if not self._support_repr_html:
(
isReplEagerEvalEnabled,
replEagerEvalMaxNumRows,
replEagerEvalTruncate,
) = self.sparkSession._jconf.getConfs(
[
"spark.sql.repl.eagerEval.enabled",
"spark.sql.repl.eagerEval.maxNumRows",
"spark.sql.repl.eagerEval.truncate",
]
)
if isReplEagerEvalEnabled == "true":
vertical = False
return self._jdf.showString(
int(replEagerEvalMaxNumRows),
int(replEagerEvalTruncate),
vertical,
)
return "DataFrame[%s]" % (", ".join("%s: %s" % c for c in self.dtypes))
def _repr_html_(self) -> Optional[str]:
"""Returns a :class:`DataFrame` with html code when you enabled eager evaluation
by 'spark.sql.repl.eagerEval.enabled', this only called by REPL you are
using support eager evaluation with HTML.
"""
if not self._support_repr_html:
self._support_repr_html = True
(
isReplEagerEvalEnabled,
replEagerEvalMaxNumRows,
replEagerEvalTruncate,
) = self.sparkSession._jconf.getConfs(
[
"spark.sql.repl.eagerEval.enabled",
"spark.sql.repl.eagerEval.maxNumRows",
"spark.sql.repl.eagerEval.truncate",
]
)
if isReplEagerEvalEnabled == "true":
return self._jdf.htmlString(
int(replEagerEvalMaxNumRows),
int(replEagerEvalTruncate),
)
else:
return None
def checkpoint(self, eager: bool = True) -> ParentDataFrame:
jdf = self._jdf.checkpoint(eager)
return DataFrame(jdf, self.sparkSession)
def localCheckpoint(
self, eager: bool = True, storageLevel: Optional[StorageLevel] = None
) -> ParentDataFrame:
if storageLevel is None:
jdf = self._jdf.localCheckpoint(eager)
else:
jdf = self._jdf.localCheckpoint(eager, self._sc._getJavaStorageLevel(storageLevel))
return DataFrame(jdf, self.sparkSession)
def withWatermark(self, eventTime: str, delayThreshold: str) -> ParentDataFrame:
if not eventTime or type(eventTime) is not str:
raise PySparkTypeError(
errorClass="NOT_EXPECTED_TYPE",
messageParameters={
"arg_name": "eventTime",
"expected_type": "str",
"arg_type": type(eventTime).__name__,
},
)
if not delayThreshold or type(delayThreshold) is not str:
raise PySparkTypeError(
errorClass="NOT_EXPECTED_TYPE",
messageParameters={
"arg_name": "delayThreshold",
"expected_type": "str",
"arg_type": type(delayThreshold).__name__,
},
)
jdf = self._jdf.withWatermark(eventTime, delayThreshold)
return DataFrame(jdf, self.sparkSession)
def hint(
self, name: str, *parameters: Union["PrimitiveType", "Column", List["PrimitiveType"]]
) -> ParentDataFrame:
if len(parameters) == 1 and isinstance(parameters[0], list):
parameters = parameters[0] # type: ignore[assignment]
if not isinstance(name, str):
raise PySparkTypeError(
errorClass="NOT_EXPECTED_TYPE",
messageParameters={
"arg_name": "name",
"expected_type": "str",
"arg_type": type(name).__name__,
},
)
allowed_types = (str, float, int, Column, list)
allowed_primitive_types = (str, float, int)
allowed_types_repr = ", ".join(
[t.__name__ for t in allowed_types[:-1]]
+ ["list[" + t.__name__ + "]" for t in allowed_primitive_types]
)
for p in parameters:
if not isinstance(p, allowed_types):
raise PySparkTypeError(
errorClass="DISALLOWED_TYPE_FOR_CONTAINER",
messageParameters={
"arg_name": "parameters",
"arg_type": type(parameters).__name__,
"allowed_types": allowed_types_repr,
"item_type": type(p).__name__,
},
)
if isinstance(p, list):
if not all(isinstance(e, allowed_primitive_types) for e in p):
raise PySparkTypeError(
errorClass="DISALLOWED_TYPE_FOR_CONTAINER",
messageParameters={
"arg_name": "parameters",
"arg_type": type(parameters).__name__,
"allowed_types": allowed_types_repr,
"item_type": type(p).__name__ + "[" + type(p[0]).__name__ + "]",
},
)
def _converter(parameter: Union[str, list, float, int, Column]) -> Any:
if isinstance(parameter, Column):
return _to_java_column(parameter)
elif isinstance(parameter, list):
# for list input, we are assuming only one element type exist in the list.
# for empty list, we are converting it into an empty long[] in the JVM side.
gateway = self._sc._gateway
assert gateway is not None
jclass = gateway.jvm.long
if len(parameter) >= 1:
mapping = {
str: gateway.jvm.java.lang.String,
float: gateway.jvm.double,
int: gateway.jvm.long,
}
jclass = mapping[type(parameter[0])]
return to_java_array(gateway, jclass, parameter)
else:
return parameter
jdf = self._jdf.hint(name, self._jseq(parameters, _converter))
return DataFrame(jdf, self.sparkSession)
def count(self) -> int:
return int(self._jdf.count())
def collect(self) -> List[Row]:
with SCCallSiteSync(self._sc):
sock_info = self._jdf.collectToPython()
with _load_from_socket(sock_info, BatchedSerializer(CPickleSerializer())) as stream:
return list(stream)
def toLocalIterator(self, prefetchPartitions: bool = False) -> Iterator[Row]:
with SCCallSiteSync(self._sc):
sock_info = self._jdf.toPythonIterator(prefetchPartitions)
return _local_iterator_from_socket(sock_info, BatchedSerializer(CPickleSerializer()))
def limit(self, num: int) -> ParentDataFrame:
jdf = self._jdf.limit(num)
return DataFrame(jdf, self.sparkSession)
def offset(self, num: int) -> ParentDataFrame:
jdf = self._jdf.offset(num)
return DataFrame(jdf, self.sparkSession)
def take(self, num: int) -> List[Row]:
return self.limit(num).collect()
def tail(self, num: int) -> List[Row]:
with SCCallSiteSync(self._sc):
sock_info = self._jdf.tailToPython(num)
with _load_from_socket(sock_info, BatchedSerializer(CPickleSerializer())) as stream:
return list(stream)
def foreach(self, f: Callable[[Row], None]) -> None:
self.rdd.foreach(f)
def foreachPartition(self, f: Callable[[Iterator[Row]], None]) -> None:
self.rdd.foreachPartition(f) # type: ignore[arg-type]
def cache(self) -> ParentDataFrame:
self.is_cached = True
self._jdf.cache()
return self
def persist(
self,
storageLevel: StorageLevel = (StorageLevel.MEMORY_AND_DISK_DESER),
) -> ParentDataFrame:
self.is_cached = True
javaStorageLevel = self._sc._getJavaStorageLevel(storageLevel)
self._jdf.persist(javaStorageLevel)
return self
@property
def storageLevel(self) -> StorageLevel:
java_storage_level = self._jdf.storageLevel()
storage_level = StorageLevel(
java_storage_level.useDisk(),
java_storage_level.useMemory(),
java_storage_level.useOffHeap(),
java_storage_level.deserialized(),
java_storage_level.replication(),
)
return storage_level
def unpersist(self, blocking: bool = False) -> ParentDataFrame:
self.is_cached = False
self._jdf.unpersist(blocking)
return self
def coalesce(self, numPartitions: int) -> ParentDataFrame:
return DataFrame(self._jdf.coalesce(numPartitions), self.sparkSession)
def repartition(
self, numPartitions: Union[int, "ColumnOrName"], *cols: "ColumnOrName"
) -> ParentDataFrame:
if isinstance(numPartitions, int):
if len(cols) == 0:
return DataFrame(self._jdf.repartition(numPartitions), self.sparkSession)
else:
return DataFrame(
self._jdf.repartition(numPartitions, self._jcols(*cols)),
self.sparkSession,
)
elif isinstance(numPartitions, (str, Column)):
cols = (numPartitions,) + cols
return DataFrame(self._jdf.repartition(self._jcols(*cols)), self.sparkSession)
else:
raise PySparkTypeError(
errorClass="NOT_EXPECTED_TYPE",
messageParameters={
"expected_type": "Column or str",
"arg_name": "numPartitions",
"arg_type": type(numPartitions).__name__,
},
)
def repartitionByRange(
self, numPartitions: Union[int, "ColumnOrName"], *cols: "ColumnOrName"
) -> ParentDataFrame:
if isinstance(numPartitions, int):
if len(cols) == 0:
raise PySparkValueError(
errorClass="CANNOT_BE_EMPTY",
messageParameters={"item": "partition-by expression"},
)
else:
return DataFrame(
self._jdf.repartitionByRange(numPartitions, self._jcols(*cols)),
self.sparkSession,
)
elif isinstance(numPartitions, (str, Column)):
cols = (numPartitions,) + cols
return DataFrame(self._jdf.repartitionByRange(self._jcols(*cols)), self.sparkSession)
else:
raise PySparkTypeError(
errorClass="NOT_EXPECTED_TYPE",
messageParameters={
"expected_type": "Column, int or str",
"arg_name": "numPartitions",
"arg_type": type(numPartitions).__name__,
},
)
def repartitionById(
self, numPartitions: int, partitionIdCol: "ColumnOrName"
) -> ParentDataFrame:
if not isinstance(numPartitions, (int, bool)):
raise PySparkTypeError(
errorClass="NOT_EXPECTED_TYPE",
messageParameters={
"expected_type": "int",
"arg_name": "numPartitions",
"arg_type": type(numPartitions).__name__,
},
)
if numPartitions <= 0:
raise PySparkValueError(
errorClass="VALUE_NOT_POSITIVE",
messageParameters={
"arg_name": "numPartitions",
"arg_value": str(numPartitions),
},
)
return DataFrame(
self._jdf.repartitionById(numPartitions, _to_java_column(partitionIdCol)),
self.sparkSession,
)
def distinct(self) -> ParentDataFrame:
return DataFrame(self._jdf.distinct(), self.sparkSession)
@overload
def sample(self, fraction: float, seed: Optional[int] = ...) -> ParentDataFrame: ...
@overload
def sample(
self,
withReplacement: Optional[bool],
fraction: float,
seed: Optional[int] = ...,
) -> ParentDataFrame: ...
def sample( # type: ignore[misc]
self,
withReplacement: Optional[Union[float, bool]] = None,
fraction: Optional[Union[int, float]] = None,
seed: Optional[int] = None,
) -> ParentDataFrame:
_w, _f, _s = self._preapare_args_for_sample(withReplacement, fraction, seed)
jdf = self._jdf.sample(*[_w, _f, _s])
return DataFrame(jdf, self.sparkSession)
def sampleBy(
self, col: "ColumnOrName", fractions: Dict[Any, float], seed: Optional[int] = None
) -> ParentDataFrame:
if isinstance(col, str):
col = Column(col)
elif not isinstance(col, Column):
raise PySparkTypeError(
errorClass="NOT_EXPECTED_TYPE",
messageParameters={
"expected_type": "Column or str",
"arg_name": "col",
"arg_type": type(col).__name__,
},
)
if not isinstance(fractions, dict):
raise PySparkTypeError(
errorClass="NOT_EXPECTED_TYPE",
messageParameters={
"expected_type": "dict",
"arg_name": "fractions",
"arg_type": type(fractions).__name__,
},
)
for k, v in fractions.items():
if not isinstance(k, (float, int, str)):
raise PySparkTypeError(
errorClass="DISALLOWED_TYPE_FOR_CONTAINER",
messageParameters={
"arg_name": "fractions",
"arg_type": type(fractions).__name__,
"allowed_types": "float, int, str",
"item_type": type(k).__name__,
},
)
fractions[k] = float(v)
col = col._jc
seed = seed if seed is not None else random.randint(0, sys.maxsize)
return DataFrame(
self._jdf.stat().sampleBy(col, self._jmap(fractions), seed), self.sparkSession
)
def randomSplit(
self, weights: List[float], seed: Optional[int] = None
) -> List[ParentDataFrame]:
for w in weights:
if w < 0.0:
raise PySparkValueError(
errorClass="VALUE_NOT_POSITIVE",
messageParameters={"arg_name": "weights", "arg_value": str(w)},
)
seed = seed if seed is not None else random.randint(0, sys.maxsize)
df_array = self._jdf.randomSplit(
_to_list(self.sparkSession._sc, cast(List["ColumnOrName"], weights)), int(seed)
)
return [DataFrame(df, self.sparkSession) for df in df_array]
@property
def dtypes(self) -> List[Tuple[str, str]]:
return [(str(f.name), f.dataType.simpleString()) for f in self.schema.fields]
@property
def columns(self) -> List[str]:
return [f.name for f in self.schema.fields]
def metadataColumn(self, colName: str) -> Column:
if not isinstance(colName, str):
raise PySparkTypeError(
errorClass="NOT_EXPECTED_TYPE",
messageParameters={
"arg_name": "colName",
"expected_type": "str",
"arg_type": type(colName).__name__,
},
)
jc = self._jdf.metadataColumn(colName)
return Column(jc)
def colRegex(self, colName: str) -> Column:
if not isinstance(colName, str):
raise PySparkTypeError(
errorClass="NOT_EXPECTED_TYPE",
messageParameters={
"arg_name": "colName",
"expected_type": "str",
"arg_type": type(colName).__name__,
},
)
jc = self._jdf.colRegex(colName)
return Column(jc)
def to(self, schema: StructType) -> ParentDataFrame:
assert schema is not None
jschema = self._jdf.sparkSession().parseDataType(schema.json())
return DataFrame(self._jdf.to(jschema), self.sparkSession)
def alias(self, alias: str) -> ParentDataFrame:
assert isinstance(alias, str), "alias should be a string"
return DataFrame(getattr(self._jdf, "as")(alias), self.sparkSession)
def crossJoin(self, other: ParentDataFrame) -> ParentDataFrame:
jdf = self._jdf.crossJoin(other._jdf)
return DataFrame(jdf, self.sparkSession)
def join(
self,
other: ParentDataFrame,
on: Optional[Union[str, List[str], Column, List[Column]]] = None,
how: Optional[str] = None,
) -> ParentDataFrame:
if on is not None and not isinstance(on, list):
on = [on] # type: ignore[assignment]
if on is not None:
if isinstance(on[0], str):
on = self._jseq(cast(List[str], on))
else:
assert isinstance(on[0], Column), "on should be Column or list of Column"
on = reduce(lambda x, y: x.__and__(y), cast(List[Column], on))
on = on._jc
if on is None and how is None:
jdf = self._jdf.join(other._jdf)
else:
if how is None:
how = "inner"
if on is None:
on = self._jseq([])
assert isinstance(how, str), "how should be a string"
jdf = self._jdf.join(other._jdf, on, how)
return DataFrame(jdf, self.sparkSession)
def lateralJoin(
self,
other: ParentDataFrame,
on: Optional[Column] = None,
how: Optional[str] = None,
) -> ParentDataFrame:
if on is None and how is None:
jdf = self._jdf.lateralJoin(other._jdf)
elif on is None:
jdf = self._jdf.lateralJoin(other._jdf, how)
elif how is None:
jdf = self._jdf.lateralJoin(other._jdf, on._jc)
else:
jdf = self._jdf.lateralJoin(other._jdf, on._jc, how)
return DataFrame(jdf, self.sparkSession)
# TODO(SPARK-22947): Fix the DataFrame API.
def _joinAsOf(
self,
other: ParentDataFrame,
leftAsOfColumn: Union[str, Column],
rightAsOfColumn: Union[str, Column],
on: Optional[Union[str, List[str], Column, List[Column]]] = None,
how: Optional[str] = None,
*,
tolerance: Optional[Column] = None,
allowExactMatches: bool = True,
direction: str = "backward",
) -> ParentDataFrame:
"""
Perform an as-of join.
This is similar to a left-join except that we match on the nearest
key rather than equal keys.
.. versionchanged:: 4.0.0
Supports Spark Connect.
Parameters
----------
other : :class:`DataFrame`
Right side of the join
leftAsOfColumn : str or :class:`Column`
a string for the as-of join column name, or a Column
rightAsOfColumn : str or :class:`Column`
a string for the as-of join column name, or a Column
on : str, list or :class:`Column`, optional
a string for the join column name, a list of column names,
a join expression (Column), or a list of Columns.
If `on` is a string or a list of strings indicating the name of the join column(s),
the column(s) must exist on both sides, and this performs an equi-join.
how : str, optional
default ``inner``. Must be one of: ``inner`` and ``left``.
tolerance : :class:`Column`, optional
an asof tolerance within this range; must be compatible
with the merge index.
allowExactMatches : bool, optional
default ``True``.
direction : str, optional
default ``backward``. Must be one of: ``backward``, ``forward``, and ``nearest``.
Examples
--------
The following performs an as-of join between ``left`` and ``right``.
>>> left = spark.createDataFrame([(1, "a"), (5, "b"), (10, "c")], ["a", "left_val"])
>>> right = spark.createDataFrame([(1, 1), (2, 2), (3, 3), (6, 6), (7, 7)],
... ["a", "right_val"])
>>> left._joinAsOf(
... right, leftAsOfColumn="a", rightAsOfColumn="a"
... ).select(left.a, 'left_val', 'right_val').sort("a").collect()
[Row(a=1, left_val='a', right_val=1),
Row(a=5, left_val='b', right_val=3),
Row(a=10, left_val='c', right_val=7)]
>>> from pyspark.sql import functions as sf
>>> left._joinAsOf(
... right, leftAsOfColumn="a", rightAsOfColumn="a", tolerance=sf.lit(1)
... ).select(left.a, 'left_val', 'right_val').sort("a").collect()
[Row(a=1, left_val='a', right_val=1)]
>>> left._joinAsOf(
... right, leftAsOfColumn="a", rightAsOfColumn="a", how="left", tolerance=sf.lit(1)
... ).select(left.a, 'left_val', 'right_val').sort("a").collect()
[Row(a=1, left_val='a', right_val=1),
Row(a=5, left_val='b', right_val=None),
Row(a=10, left_val='c', right_val=None)]
>>> left._joinAsOf(
... right, leftAsOfColumn="a", rightAsOfColumn="a", allowExactMatches=False
... ).select(left.a, 'left_val', 'right_val').sort("a").collect()
[Row(a=5, left_val='b', right_val=3),
Row(a=10, left_val='c', right_val=7)]
>>> left._joinAsOf(
... right, leftAsOfColumn="a", rightAsOfColumn="a", direction="forward"
... ).select(left.a, 'left_val', 'right_val').sort("a").collect()
[Row(a=1, left_val='a', right_val=1),
Row(a=5, left_val='b', right_val=6)]
"""
if isinstance(leftAsOfColumn, str):
leftAsOfColumn = self[leftAsOfColumn]
left_as_of_jcol = leftAsOfColumn._jc
if isinstance(rightAsOfColumn, str):
rightAsOfColumn = other[rightAsOfColumn]
right_as_of_jcol = rightAsOfColumn._jc
if on is not None and not isinstance(on, list):
on = [on] # type: ignore[assignment]
if on is not None:
if isinstance(on[0], str):
on = self._jseq(cast(List[str], on))
else:
assert isinstance(on[0], Column), "on should be Column or list of Column"
on = reduce(lambda x, y: x.__and__(y), cast(List[Column], on))
on = on._jc
if how is None:
how = "inner"
assert isinstance(how, str), "how should be a string"
if tolerance is not None:
assert isinstance(tolerance, Column), "tolerance should be Column"
tolerance = tolerance._jc
jdf = self._jdf.joinAsOf(
other._jdf,
left_as_of_jcol,
right_as_of_jcol,
on,
how,
tolerance,
allowExactMatches,
direction,
)
return DataFrame(jdf, self.sparkSession)
def sortWithinPartitions(
self,
*cols: Union[Sequence["ColumnOrNameOrOrdinal"], "ColumnOrNameOrOrdinal"],
**kwargs: Any,
) -> ParentDataFrame:
_cols = self._preapare_cols_for_sort(F.col, cols, kwargs)
jdf = self._jdf.sortWithinPartitions(self._jseq(_cols, _to_java_column))
return DataFrame(jdf, self.sparkSession)
def sort(
self,
*cols: Union[Sequence["ColumnOrNameOrOrdinal"], "ColumnOrNameOrOrdinal"],
**kwargs: Any,
) -> ParentDataFrame:
_cols = self._preapare_cols_for_sort(F.col, cols, kwargs)
jdf = self._jdf.sort(self._jseq(_cols, _to_java_column))
return DataFrame(jdf, self.sparkSession)
orderBy = sort
def _jseq(
self,
cols: Sequence,
converter: Optional[Callable[..., Union["PrimitiveType", "JavaObject"]]] = None,
) -> "JavaObject":
"""Return a JVM Seq of Columns from a list of Column or names"""
return _to_seq(self.sparkSession._sc, cols, converter)
def _jmap(self, jm: Dict) -> "JavaObject":
"""Return a JVM Scala Map from a dict"""
return to_scala_map(self.sparkSession._sc._jvm, jm)
def _jcols(self, *cols: Union[Sequence["ColumnOrName"], "ColumnOrName"]) -> "JavaObject":
"""Return a JVM Seq of Columns from a list of Column or column names
If `cols` has only one list in it, cols[0] will be used as the list.
"""
if (
len(cols) == 1
and not isinstance(cols[0], (str, Column))
and isinstance(cols[0], Sequence)
):
cols = tuple(cols[0])
return self._jseq(cols, _to_java_column)
def _jcols_ordinal(
self, *cols: Union[Sequence["ColumnOrNameOrOrdinal"], "ColumnOrNameOrOrdinal"]
) -> "JavaObject":
"""Return a JVM Seq of Columns from a list of Column or column names or column ordinals.
If `cols` has only one list in it, cols[0] will be used as the list.
"""
if (
len(cols) == 1
and not isinstance(cols[0], (int, str, Column))
and isinstance(cols[0], Sequence)
):
cols = tuple(cols[0])
_cols = []
for c in cols:
if isinstance(c, int) and not isinstance(c, bool):
if c < 1: