forked from Kyligence/spark
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathdataframe.py
5614 lines (4632 loc) · 182 KB
/
dataframe.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.
#
import json
import os
import sys
import random
import warnings
from collections.abc import Iterable
from functools import reduce
from typing import (
Any,
Callable,
Dict,
Iterator,
List,
Optional,
Sequence,
Tuple,
Type,
Union,
cast,
overload,
TYPE_CHECKING,
)
from py4j.java_gateway import JavaObject, JVMView
from pyspark import copy_func, _NoValue
from pyspark._globals import _NoValueType
from pyspark.context import SparkContext
from pyspark.errors import PySparkTypeError, PySparkValueError
from pyspark.rdd import (
RDD,
_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, _to_seq, _to_list, _to_java_column
from pyspark.sql.readwriter import DataFrameWriter, DataFrameWriterV2
from pyspark.sql.streaming import DataStreamWriter
from pyspark.sql.types import (
StructType,
Row,
_parse_datatype_json_string,
)
from pyspark.sql.utils import get_active_spark_context
from pyspark.sql.pandas.conversion import PandasConversionMixin
from pyspark.sql.pandas.map_ops import PandasMapOpsMixin
if TYPE_CHECKING:
from pyspark._typing import PrimitiveType
from pyspark.pandas.frame import DataFrame as PandasOnSparkDataFrame
from pyspark.sql._typing import ColumnOrName, LiteralType, OptionalPrimitiveType
from pyspark.sql.context import SQLContext
from pyspark.sql.session import SparkSession
from pyspark.sql.group import GroupedData
from pyspark.sql.observation import Observation
__all__ = ["DataFrame", "DataFrameNaFunctions", "DataFrameStatFunctions"]
class DataFrame(PandasMapOpsMixin, PandasConversionMixin):
"""A distributed collection of data grouped into named columns.
.. versionadded:: 1.3.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Examples
--------
A :class:`DataFrame` is equivalent to a relational table in Spark SQL,
and can be created using various functions in :class:`SparkSession`:
>>> people = spark.createDataFrame([
... {"deptId": 1, "age": 40, "name": "Hyukjin Kwon", "gender": "M", "salary": 50},
... {"deptId": 1, "age": 50, "name": "Takuya Ueshin", "gender": "M", "salary": 100},
... {"deptId": 2, "age": 60, "name": "Xinrong Meng", "gender": "F", "salary": 150},
... {"deptId": 3, "age": 20, "name": "Haejoon Lee", "gender": "M", "salary": 200}
... ])
Once created, it can be manipulated using the various domain-specific-language
(DSL) functions defined in: :class:`DataFrame`, :class:`Column`.
To select a column from the :class:`DataFrame`, use the apply method:
>>> age_col = people.age
A more concrete example:
>>> # To create DataFrame using SparkSession
... department = spark.createDataFrame([
... {"id": 1, "name": "PySpark"},
... {"id": 2, "name": "ML"},
... {"id": 3, "name": "Spark SQL"}
... ])
>>> people.filter(people.age > 30).join(
... department, people.deptId == department.id).groupBy(
... department.name, "gender").agg({"salary": "avg", "age": "max"}).show()
+-------+------+-----------+--------+
| name|gender|avg(salary)|max(age)|
+-------+------+-----------+--------+
| ML| F| 150.0| 60|
|PySpark| M| 75.0| 50|
+-------+------+-----------+--------+
Notes
-----
A DataFrame should only be created as described above. It should not be directly
created via using the constructor.
"""
def __init__(
self,
jdf: JavaObject,
sql_ctx: Union["SQLContext", "SparkSession"],
):
from pyspark.sql.context import SQLContext
self._sql_ctx: Optional["SQLContext"] = None
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
# initialized lazily
self._schema: Optional[StructType] = None
self._lazy_rdd: Optional[RDD[Row]] = None
# 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
@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."
)
if self._sql_ctx is None:
self._sql_ctx = SQLContext._get_or_create(self._sc)
return self._sql_ctx
@property
def sparkSession(self) -> "SparkSession":
"""Returns Spark session that created this :class:`DataFrame`.
.. versionadded:: 3.3.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Returns
-------
:class:`SparkSession`
Examples
--------
>>> df = spark.range(1)
>>> type(df.sparkSession)
<class '...session.SparkSession'>
"""
return self._session
@property
def rdd(self) -> "RDD[Row]":
"""Returns the content as an :class:`pyspark.RDD` of :class:`Row`.
.. versionadded:: 1.3.0
Returns
-------
:class:`RDD`
Examples
--------
>>> df = spark.range(1)
>>> type(df.rdd)
<class 'pyspark.rdd.RDD'>
"""
if self._lazy_rdd is None:
jrdd = self._jdf.javaToPython()
self._lazy_rdd = RDD(
jrdd, self.sparkSession._sc, BatchedSerializer(CPickleSerializer())
)
return self._lazy_rdd
@property
def na(self) -> "DataFrameNaFunctions":
"""Returns a :class:`DataFrameNaFunctions` for handling missing values.
.. versionadded:: 1.3.1
.. versionchanged:: 3.4.0
Supports Spark Connect.
Returns
-------
:class:`DataFrameNaFunctions`
Examples
--------
>>> df = spark.sql("SELECT 1 AS c1, int(NULL) AS c2")
>>> type(df.na)
<class '...dataframe.DataFrameNaFunctions'>
Replace the missing values as 2.
>>> df.na.fill(2).show()
+---+---+
| c1| c2|
+---+---+
| 1| 2|
+---+---+
"""
return DataFrameNaFunctions(self)
@property
def stat(self) -> "DataFrameStatFunctions":
"""Returns a :class:`DataFrameStatFunctions` for statistic functions.
.. versionadded:: 1.4.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Returns
-------
:class:`DataFrameStatFunctions`
Examples
--------
>>> import pyspark.sql.functions as f
>>> df = spark.range(3).withColumn("c", f.expr("id + 1"))
>>> type(df.stat)
<class '...dataframe.DataFrameStatFunctions'>
>>> df.stat.corr("id", "c")
1.0
"""
return DataFrameStatFunctions(self)
def toJSON(self, use_unicode: bool = True) -> RDD[str]:
"""Converts a :class:`DataFrame` into a :class:`RDD` of string.
Each row is turned into a JSON document as one element in the returned RDD.
.. versionadded:: 1.3.0
Parameters
----------
use_unicode : bool, optional, default True
Whether to convert to unicode or not.
Returns
-------
:class:`RDD`
Examples
--------
>>> df = spark.createDataFrame([(2, "Alice"), (5, "Bob")], schema=["age", "name"])
>>> df.toJSON().first()
'{"age":2,"name":"Alice"}'
"""
rdd = self._jdf.toJSON()
return RDD(rdd.toJavaRDD(), self._sc, UTF8Deserializer(use_unicode))
def registerTempTable(self, name: str) -> None:
"""Registers this :class:`DataFrame` as a temporary table using the given name.
The lifetime of this temporary table is tied to the :class:`SparkSession`
that was used to create this :class:`DataFrame`.
.. versionadded:: 1.3.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
.. deprecated:: 2.0.0
Use :meth:`DataFrame.createOrReplaceTempView` instead.
Parameters
----------
name : str
Name of the temporary table to register.
Examples
--------
>>> df = spark.createDataFrame([(2, "Alice"), (5, "Bob")], schema=["age", "name"])
>>> df.registerTempTable("people")
>>> df2 = spark.sql("SELECT * FROM people")
>>> sorted(df.collect()) == sorted(df2.collect())
True
>>> spark.catalog.dropTempView("people")
True
"""
warnings.warn("Deprecated in 2.0, use createOrReplaceTempView instead.", FutureWarning)
self._jdf.createOrReplaceTempView(name)
def createTempView(self, name: str) -> None:
"""Creates a local temporary view with this :class:`DataFrame`.
The lifetime of this temporary table is tied to the :class:`SparkSession`
that was used to create this :class:`DataFrame`.
throws :class:`TempTableAlreadyExistsException`, if the view name already exists in the
catalog.
.. versionadded:: 2.0.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
name : str
Name of the view.
Examples
--------
Create a local temporary view.
>>> df = spark.createDataFrame([(2, "Alice"), (5, "Bob")], schema=["age", "name"])
>>> df.createTempView("people")
>>> df2 = spark.sql("SELECT * FROM people")
>>> sorted(df.collect()) == sorted(df2.collect())
True
Throw an exception if the table already exists.
>>> df.createTempView("people") # doctest: +IGNORE_EXCEPTION_DETAIL
Traceback (most recent call last):
...
AnalysisException: "Temporary table 'people' already exists;"
>>> spark.catalog.dropTempView("people")
True
"""
self._jdf.createTempView(name)
def createOrReplaceTempView(self, name: str) -> None:
"""Creates or replaces a local temporary view with this :class:`DataFrame`.
The lifetime of this temporary table is tied to the :class:`SparkSession`
that was used to create this :class:`DataFrame`.
.. versionadded:: 2.0.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
name : str
Name of the view.
Examples
--------
Create a local temporary view named 'people'.
>>> df = spark.createDataFrame([(2, "Alice"), (5, "Bob")], schema=["age", "name"])
>>> df.createOrReplaceTempView("people")
Replace the local temporary view.
>>> df2 = df.filter(df.age > 3)
>>> df2.createOrReplaceTempView("people")
>>> df3 = spark.sql("SELECT * FROM people")
>>> sorted(df3.collect()) == sorted(df2.collect())
True
>>> spark.catalog.dropTempView("people")
True
"""
self._jdf.createOrReplaceTempView(name)
def createGlobalTempView(self, name: str) -> None:
"""Creates a global temporary view with this :class:`DataFrame`.
The lifetime of this temporary view is tied to this Spark application.
throws :class:`TempTableAlreadyExistsException`, if the view name already exists in the
catalog.
.. versionadded:: 2.1.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
name : str
Name of the view.
Examples
--------
Create a global temporary view.
>>> df = spark.createDataFrame([(2, "Alice"), (5, "Bob")], schema=["age", "name"])
>>> df.createGlobalTempView("people")
>>> df2 = spark.sql("SELECT * FROM global_temp.people")
>>> sorted(df.collect()) == sorted(df2.collect())
True
Throws an exception if the global temporary view already exists.
>>> df.createGlobalTempView("people") # doctest: +IGNORE_EXCEPTION_DETAIL
Traceback (most recent call last):
...
AnalysisException: "Temporary table 'people' already exists;"
>>> spark.catalog.dropGlobalTempView("people")
True
"""
self._jdf.createGlobalTempView(name)
def createOrReplaceGlobalTempView(self, name: str) -> None:
"""Creates or replaces a global temporary view using the given name.
The lifetime of this temporary view is tied to this Spark application.
.. versionadded:: 2.2.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
name : str
Name of the view.
Examples
--------
Create a global temporary view.
>>> df = spark.createDataFrame([(2, "Alice"), (5, "Bob")], schema=["age", "name"])
>>> df.createOrReplaceGlobalTempView("people")
Replace the global temporary view.
>>> df2 = df.filter(df.age > 3)
>>> df2.createOrReplaceGlobalTempView("people")
>>> df3 = spark.sql("SELECT * FROM global_temp.people")
>>> sorted(df3.collect()) == sorted(df2.collect())
True
>>> spark.catalog.dropGlobalTempView("people")
True
"""
self._jdf.createOrReplaceGlobalTempView(name)
@property
def write(self) -> DataFrameWriter:
"""
Interface for saving the content of the non-streaming :class:`DataFrame` out into external
storage.
.. versionadded:: 1.4.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Returns
-------
:class:`DataFrameWriter`
Examples
--------
>>> df = spark.createDataFrame([(2, "Alice"), (5, "Bob")], schema=["age", "name"])
>>> type(df.write)
<class '...readwriter.DataFrameWriter'>
Write the DataFrame as a table.
>>> _ = spark.sql("DROP TABLE IF EXISTS tab2")
>>> df.write.saveAsTable("tab2")
>>> _ = spark.sql("DROP TABLE tab2")
"""
return DataFrameWriter(self)
@property
def writeStream(self) -> DataStreamWriter:
"""
Interface for saving the content of the streaming :class:`DataFrame` out into external
storage.
.. versionadded:: 2.0.0
Notes
-----
This API is evolving.
Returns
-------
:class:`DataStreamWriter`
Examples
--------
>>> import tempfile
>>> df = spark.readStream.format("rate").load()
>>> type(df.writeStream)
<class '...streaming.readwriter.DataStreamWriter'>
>>> with tempfile.TemporaryDirectory() as d:
... # Create a table with Rate source.
... df.writeStream.toTable(
... "my_table", checkpointLocation=d)
<...streaming.query.StreamingQuery object at 0x...>
"""
return DataStreamWriter(self)
@property
def schema(self) -> StructType:
"""Returns the schema of this :class:`DataFrame` as a :class:`pyspark.sql.types.StructType`.
.. versionadded:: 1.3.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Returns
-------
:class:`StructType`
Examples
--------
>>> df = spark.createDataFrame(
... [(14, "Tom"), (23, "Alice"), (16, "Bob")], ["age", "name"])
Retrieve the schema of the current DataFrame.
>>> df.schema
StructType([StructField('age', LongType(), True),
StructField('name', StringType(), True)])
"""
if self._schema is None:
try:
self._schema = cast(
StructType, _parse_datatype_json_string(self._jdf.schema().json())
)
except Exception as e:
raise PySparkValueError(
error_class="CANNOT_PARSE_DATATYPE",
message_parameters={"error": str(e)},
)
return self._schema
def printSchema(self) -> None:
"""Prints out the schema in the tree format.
.. versionadded:: 1.3.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Examples
--------
>>> df = spark.createDataFrame(
... [(14, "Tom"), (23, "Alice"), (16, "Bob")], ["age", "name"])
>>> df.printSchema()
root
|-- age: long (nullable = true)
|-- name: string (nullable = true)
"""
print(self._jdf.schema().treeString())
def explain(
self, extended: Optional[Union[bool, str]] = None, mode: Optional[str] = None
) -> None:
"""Prints the (logical and physical) plans to the console for debugging purposes.
.. versionadded:: 1.3.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
extended : bool, optional
default ``False``. If ``False``, prints only the physical plan.
When this is a string without specifying the ``mode``, it works as the mode is
specified.
mode : str, optional
specifies the expected output format of plans.
* ``simple``: Print only a physical plan.
* ``extended``: Print both logical and physical plans.
* ``codegen``: Print a physical plan and generated codes if they are available.
* ``cost``: Print a logical plan and statistics if they are available.
* ``formatted``: Split explain output into two sections: a physical plan outline \
and node details.
.. versionchanged:: 3.0.0
Added optional argument `mode` to specify the expected output format of plans.
Examples
--------
>>> df = spark.createDataFrame(
... [(14, "Tom"), (23, "Alice"), (16, "Bob")], ["age", "name"])
Print out the physical plan only (default).
>>> df.explain() # doctest: +SKIP
== Physical Plan ==
*(1) Scan ExistingRDD[age...,name...]
Print out all of the parsed, analyzed, optimized and physical plans.
>>> df.explain(True)
== Parsed Logical Plan ==
...
== Analyzed Logical Plan ==
...
== Optimized Logical Plan ==
...
== Physical Plan ==
...
Print out the plans with two sections: a physical plan outline and node details
>>> df.explain(mode="formatted") # doctest: +SKIP
== Physical Plan ==
* Scan ExistingRDD (...)
(1) Scan ExistingRDD [codegen id : ...]
Output [2]: [age..., name...]
...
Print a logical plan and statistics if they are available.
>>> df.explain("cost")
== Optimized Logical Plan ==
...Statistics...
...
"""
if extended is not None and mode is not None:
raise PySparkValueError(
error_class="CANNOT_SET_TOGETHER",
message_parameters={"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(
error_class="NOT_BOOL_OR_STR",
message_parameters={
"arg_name": "extended",
"arg_type": type(extended).__name__,
},
)
if (mode is not None) and (not isinstance(mode, str)):
raise PySparkTypeError(
error_class="NOT_STR",
message_parameters={"arg_name": "mode", "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: "DataFrame") -> "DataFrame":
"""Return a new :class:`DataFrame` containing rows in this :class:`DataFrame` but
not in another :class:`DataFrame` while preserving duplicates.
This is equivalent to `EXCEPT ALL` in SQL.
As standard in SQL, this function resolves columns by position (not by name).
.. versionadded:: 2.4.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
other : :class:`DataFrame`
The other :class:`DataFrame` to compare to.
Returns
-------
:class:`DataFrame`
Examples
--------
>>> df1 = spark.createDataFrame(
... [("a", 1), ("a", 1), ("a", 1), ("a", 2), ("b", 3), ("c", 4)], ["C1", "C2"])
>>> df2 = spark.createDataFrame([("a", 1), ("b", 3)], ["C1", "C2"])
>>> df1.exceptAll(df2).show()
+---+---+
| C1| C2|
+---+---+
| a| 1|
| a| 1|
| a| 2|
| c| 4|
+---+---+
"""
return DataFrame(self._jdf.exceptAll(other._jdf), self.sparkSession)
def isLocal(self) -> bool:
"""Returns ``True`` if the :func:`collect` and :func:`take` methods can be run locally
(without any Spark executors).
.. versionadded:: 1.3.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Returns
-------
bool
Examples
--------
>>> df = spark.sql("SHOW TABLES")
>>> df.isLocal()
True
"""
return self._jdf.isLocal()
@property
def isStreaming(self) -> bool:
"""Returns ``True`` if this :class:`DataFrame` contains one or more sources that
continuously return data as it arrives. A :class:`DataFrame` that reads data from a
streaming source must be executed as a :class:`StreamingQuery` using the :func:`start`
method in :class:`DataStreamWriter`. Methods that return a single answer, (e.g.,
:func:`count` or :func:`collect`) will throw an :class:`AnalysisException` when there
is a streaming source present.
.. versionadded:: 2.0.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Notes
-----
This API is evolving.
Returns
-------
bool
Whether it's streaming DataFrame or not.
Examples
--------
>>> df = spark.readStream.format("rate").load()
>>> df.isStreaming
True
"""
return self._jdf.isStreaming()
def isEmpty(self) -> bool:
"""Returns ``True`` if this :class:`DataFrame` is empty.
.. versionadded:: 3.3.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Returns
-------
bool
Whether it's empty DataFrame or not.
Examples
--------
>>> df_empty = spark.createDataFrame([], 'a STRING')
>>> df_non_empty = spark.createDataFrame(["a"], 'STRING')
>>> df_empty.isEmpty()
True
>>> df_non_empty.isEmpty()
False
"""
return self._jdf.isEmpty()
def show(self, n: int = 20, truncate: Union[bool, int] = True, vertical: bool = False) -> None:
"""Prints the first ``n`` rows to the console.
.. versionadded:: 1.3.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
n : int, optional
Number of rows to show.
truncate : bool or int, optional
If set to ``True``, truncate strings longer than 20 chars by default.
If set to a number greater than one, truncates long strings to length ``truncate``
and align cells right.
vertical : bool, optional
If set to ``True``, print output rows vertically (one line
per column value).
Examples
--------
>>> df = spark.createDataFrame([
... (14, "Tom"), (23, "Alice"), (16, "Bob")], ["age", "name"])
Show only top 2 rows.
>>> df.show(2)
+---+-----+
|age| name|
+---+-----+
| 14| Tom|
| 23|Alice|
+---+-----+
only showing top 2 rows
Show :class:`DataFrame` where the maximum number of characters is 3.
>>> df.show(truncate=3)
+---+----+
|age|name|
+---+----+
| 14| Tom|
| 23| Ali|
| 16| Bob|
+---+----+
Show :class:`DataFrame` vertically.
>>> df.show(vertical=True)
-RECORD 0-----
age | 14
name | Tom
-RECORD 1-----
age | 23
name | Alice
-RECORD 2-----
age | 16
name | Bob
"""
if not isinstance(n, int) or isinstance(n, bool):
raise PySparkTypeError(
error_class="NOT_INT",
message_parameters={"arg_name": "n", "arg_type": type(n).__name__},
)
if not isinstance(vertical, bool):
raise PySparkTypeError(
error_class="NOT_BOOL",
message_parameters={"arg_name": "vertical", "arg_type": type(vertical).__name__},
)
if isinstance(truncate, bool) and truncate:
print(self._jdf.showString(n, 20, vertical))
else:
try:
int_truncate = int(truncate)
except ValueError:
raise PySparkTypeError(
error_class="NOT_BOOL",
message_parameters={
"arg_name": "truncate",
"arg_type": type(truncate).__name__,
},
)
print(self._jdf.showString(n, int_truncate, vertical))
def __repr__(self) -> str:
if not self._support_repr_html and self.sparkSession._jconf.isReplEagerEvalEnabled():
vertical = False
return self._jdf.showString(
self.sparkSession._jconf.replEagerEvalMaxNumRows(),
self.sparkSession._jconf.replEagerEvalTruncate(),
vertical,
)
else:
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
if self.sparkSession._jconf.isReplEagerEvalEnabled():
return self._jdf.htmlString(
self.sparkSession._jconf.replEagerEvalMaxNumRows(),
self.sparkSession._jconf.replEagerEvalTruncate(),
)
else:
return None
def checkpoint(self, eager: bool = True) -> "DataFrame":
"""Returns a checkpointed version of this :class:`DataFrame`. Checkpointing can be used to
truncate the logical plan of this :class:`DataFrame`, which is especially useful in
iterative algorithms where the plan may grow exponentially. It will be saved to files
inside the checkpoint directory set with :meth:`SparkContext.setCheckpointDir`.
.. versionadded:: 2.1.0
Parameters
----------
eager : bool, optional, default True
Whether to checkpoint this :class:`DataFrame` immediately.
Returns
-------
:class:`DataFrame`
Checkpointed DataFrame.
Notes
-----
This API is experimental.
Examples
--------
>>> import tempfile
>>> df = spark.createDataFrame([
... (14, "Tom"), (23, "Alice"), (16, "Bob")], ["age", "name"])
>>> with tempfile.TemporaryDirectory() as d:
... spark.sparkContext.setCheckpointDir("/tmp/bb")
... df.checkpoint(False)
DataFrame[age: bigint, name: string]
"""
jdf = self._jdf.checkpoint(eager)
return DataFrame(jdf, self.sparkSession)
def localCheckpoint(self, eager: bool = True) -> "DataFrame":
"""Returns a locally checkpointed version of this :class:`DataFrame`. Checkpointing can be
used to truncate the logical plan of this :class:`DataFrame`, which is especially useful in
iterative algorithms where the plan may grow exponentially. Local checkpoints are
stored in the executors using the caching subsystem and therefore they are not reliable.
.. versionadded:: 2.3.0
Parameters
----------
eager : bool, optional, default True
Whether to checkpoint this :class:`DataFrame` immediately.
Returns
-------
:class:`DataFrame`
Checkpointed DataFrame.
Notes
-----
This API is experimental.