-
Notifications
You must be signed in to change notification settings - Fork 28.5k
/
Copy pathsession.py
2454 lines (2036 loc) · 80.2 KB
/
session.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 os
import sys
import warnings
import filecmp
from collections.abc import Sized
from functools import reduce, cached_property
from threading import RLock
from types import TracebackType
from typing import (
Any,
ClassVar,
Dict,
Iterable,
List,
Optional,
Tuple,
Type,
Union,
Set,
cast,
no_type_check,
overload,
TYPE_CHECKING,
)
from pyspark.conf import SparkConf
from pyspark.util import default_api_mode, is_remote_only
from pyspark.sql.conf import RuntimeConfig
from pyspark.sql.dataframe import DataFrame
from pyspark.sql.functions import lit
from pyspark.sql.pandas.conversion import SparkConversionMixin
from pyspark.sql.profiler import AccumulatorProfilerCollector, Profile
from pyspark.sql.readwriter import DataFrameReader
from pyspark.sql.sql_formatter import SQLStringFormatter
from pyspark.sql.streaming import DataStreamReader
from pyspark.sql.types import (
AtomicType,
DataType,
StructField,
StructType,
_make_type_verifier,
_infer_schema,
_has_nulltype,
_merge_type,
_create_converter,
_from_numpy_type,
)
from pyspark.errors.exceptions.captured import install_exception_handler
from pyspark.sql.utils import (
is_timestamp_ntz_preferred,
to_str,
try_remote_session_classmethod,
remote_only,
)
from pyspark.errors import PySparkValueError, PySparkTypeError, PySparkRuntimeError
if TYPE_CHECKING:
from py4j.java_gateway import JavaClass, JavaObject, JVMView
import pyarrow as pa
from pyspark.core.context import SparkContext
from pyspark.core.rdd import RDD
from pyspark.sql._typing import AtomicValue, RowLike, OptionalPrimitiveType
from pyspark.sql.catalog import Catalog
from pyspark.sql.pandas._typing import ArrayLike, DataFrameLike as PandasDataFrameLike
from pyspark.sql.streaming import StreamingQueryManager
from pyspark.sql.tvf import TableValuedFunction
from pyspark.sql.udf import UDFRegistration
from pyspark.sql.udtf import UDTFRegistration
from pyspark.sql.datasource import DataSourceRegistration
from pyspark.sql.dataframe import DataFrame as ParentDataFrame
# Running MyPy type checks will always require pandas and
# other dependencies so importing here is fine.
from pyspark.sql.connect.client import SparkConnectClient
from pyspark.sql.connect.shell.progress import ProgressHandler
__all__ = ["SparkSession"]
def _monkey_patch_RDD(sparkSession: "SparkSession") -> None:
@no_type_check
def toDF(self, schema=None, sampleRatio=None):
"""
Converts current :class:`RDD` into a :class:`DataFrame`
This is a shorthand for ``spark.createDataFrame(rdd, schema, sampleRatio)``
Parameters
----------
schema : :class:`pyspark.sql.types.DataType`, str or list, optional
a :class:`pyspark.sql.types.DataType` or a datatype string or a list of
column names, default is None. The data type string format equals to
:class:`pyspark.sql.types.DataType.simpleString`, except that top level struct type can
omit the ``struct<>`` and atomic types use ``typeName()`` as their format, e.g. use
``byte`` instead of ``tinyint`` for :class:`pyspark.sql.types.ByteType`.
We can also use ``int`` as a short name for :class:`pyspark.sql.types.IntegerType`.
sampleRatio : float, optional
the sample ratio of rows used for inferring
Returns
-------
:class:`DataFrame`
Examples
--------
>>> rdd = spark.range(1).rdd.map(lambda x: tuple(x))
>>> rdd.collect()
[(0,)]
>>> rdd.toDF().show()
+---+
| _1|
+---+
| 0|
+---+
"""
return sparkSession.createDataFrame(self, schema, sampleRatio)
if not is_remote_only():
from pyspark import RDD
RDD.toDF = toDF # type: ignore[method-assign]
class classproperty(property):
"""Same as Python's @property decorator, but for class attributes.
Examples
--------
>>> class Builder:
... def build(self):
... return MyClass()
...
>>> class MyClass:
... @classproperty
... def builder(cls):
... print("instantiating new builder")
... return Builder()
...
>>> c1 = MyClass.builder
instantiating new builder
>>> c2 = MyClass.builder
instantiating new builder
>>> c1 == c2
False
>>> isinstance(c1.build(), MyClass)
True
"""
def __get__(self, instance: Any, owner: Any = None) -> "SparkSession.Builder":
# The "type: ignore" below silences the following error from mypy:
# error: Argument 1 to "classmethod" has incompatible
# type "Optional[Callable[[Any], Any]]";
# expected "Callable[..., Any]" [arg-type]
return classmethod(self.fget).__get__(None, owner)() # type: ignore
class SparkSession(SparkConversionMixin):
"""The entry point to programming Spark with the Dataset and DataFrame API.
A SparkSession can be used to create :class:`DataFrame`, register :class:`DataFrame` as
tables, execute SQL over tables, cache tables, and read parquet files.
To create a :class:`SparkSession`, use the following builder pattern:
.. versionchanged:: 3.4.0
Supports Spark Connect.
.. autoattribute:: builder
:annotation:
Examples
--------
Create a Spark session.
>>> spark = (
... SparkSession.builder
... .master("local")
... .appName("Word Count")
... .config("spark.some.config.option", "some-value")
... .getOrCreate()
... )
Create a Spark session with Spark Connect.
>>> spark = (
... SparkSession.builder
... .remote("sc://localhost")
... .appName("Word Count")
... .config("spark.some.config.option", "some-value")
... .getOrCreate()
... ) # doctest: +SKIP
"""
class Builder:
"""Builder for :class:`SparkSession`."""
_lock = RLock()
def __init__(self) -> None:
self._options: Dict[str, Any] = {}
@overload
def config(self, *, conf: SparkConf) -> "SparkSession.Builder":
...
@overload
def config(self, key: str, value: Any) -> "SparkSession.Builder":
...
@overload
def config(self, *, map: Dict[str, "OptionalPrimitiveType"]) -> "SparkSession.Builder":
...
def config(
self,
key: Optional[str] = None,
value: Optional[Any] = None,
conf: Optional[SparkConf] = None,
*,
map: Optional[Dict[str, "OptionalPrimitiveType"]] = None,
) -> "SparkSession.Builder":
"""Sets a config option. Options set using this method are automatically propagated to
both :class:`SparkConf` and :class:`SparkSession`'s own configuration.
.. versionadded:: 2.0.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
key : str, optional
a key name string for configuration property
value : str, optional
a value for configuration property
conf : :class:`SparkConf`, optional
an instance of :class:`SparkConf`
map: dictionary, optional
a dictionary of configurations to set
.. versionadded:: 3.4.0
Returns
-------
:class:`SparkSession.Builder`
See Also
--------
:class:`SparkConf`
Examples
--------
For an existing :class:`SparkConf`, use `conf` parameter.
>>> from pyspark.conf import SparkConf
>>> conf = SparkConf().setAppName("example").setMaster("local")
>>> SparkSession.builder.config(conf=conf)
<pyspark.sql.session.SparkSession.Builder...
For a (key, value) pair, you can omit parameter names.
>>> SparkSession.builder.config("spark.some.config.option", "some-value")
<pyspark.sql.session.SparkSession.Builder...
Set multiple configurations.
>>> SparkSession.builder.config(
... "spark.some.config.number", 123).config("spark.some.config.float", 0.123)
<pyspark.sql.session.SparkSession.Builder...
Set multiple configurations using a dictionary.
>>> SparkSession.builder.config(
... map={"spark.some.config.number": 123, "spark.some.config.float": 0.123})
<pyspark.sql.session.SparkSession.Builder...
"""
with self._lock:
if conf is not None:
for k, v in conf.getAll():
self._options[k] = v
self._validate_startup_urls()
elif map is not None:
for k, v in map.items(): # type: ignore[assignment]
v = to_str(v) # type: ignore[assignment]
self._options[k] = v
self._validate_startup_urls()
else:
value = to_str(value)
self._options[cast(str, key)] = value
self._validate_startup_urls()
return self
def _validate_startup_urls(
self,
) -> None:
"""
Helper function that validates the combination of startup URLs and raises an exception
if incompatible options are selected.
"""
if ("spark.master" in self._options or "MASTER" in os.environ) and (
"spark.remote" in self._options or "SPARK_REMOTE" in os.environ
):
raise PySparkRuntimeError(
errorClass="CANNOT_CONFIGURE_SPARK_CONNECT_MASTER",
messageParameters={
"master_url": self._options.get("spark.master", os.environ.get("MASTER")),
"connect_url": self._options.get(
"spark.remote", os.environ.get("SPARK_REMOTE")
),
},
)
if "spark.remote" in self._options:
remote = cast(str, self._options.get("spark.remote"))
if ("SPARK_REMOTE" in os.environ and os.environ["SPARK_REMOTE"] != remote) and (
"SPARK_LOCAL_REMOTE" in os.environ and not remote.startswith("local")
):
raise PySparkRuntimeError(
errorClass="CANNOT_CONFIGURE_SPARK_CONNECT",
messageParameters={
"existing_url": os.environ["SPARK_REMOTE"],
"new_url": remote,
},
)
def master(self, master: str) -> "SparkSession.Builder":
"""Sets the Spark master URL to connect to, such as "local" to run locally, "local[4]"
to run locally with 4 cores, or "spark://master:7077" to run on a Spark standalone
cluster.
.. versionadded:: 2.0.0
Parameters
----------
master : str
a url for spark master
Returns
-------
:class:`SparkSession.Builder`
Examples
--------
>>> SparkSession.builder.master("local")
<pyspark.sql.session.SparkSession.Builder...
"""
return self.config("spark.master", master)
def remote(self, url: str) -> "SparkSession.Builder":
"""Sets the Spark remote URL to connect to, such as "sc://host:port" to run
it via Spark Connect server.
.. versionadded:: 3.4.0
Parameters
----------
url : str
URL to Spark Connect server
Returns
-------
:class:`SparkSession.Builder`
Examples
--------
>>> SparkSession.builder.remote("sc://localhost") # doctest: +SKIP
<pyspark.sql.session.SparkSession.Builder...
"""
return self.config("spark.remote", url)
def appName(self, name: str) -> "SparkSession.Builder":
"""Sets a name for the application, which will be shown in the Spark web UI.
If no application name is set, a randomly generated name will be used.
.. versionadded:: 2.0.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
name : str
an application name
Returns
-------
:class:`SparkSession.Builder`
Examples
--------
>>> SparkSession.builder.appName("My app")
<pyspark.sql.session.SparkSession.Builder...
"""
return self.config("spark.app.name", name)
def enableHiveSupport(self) -> "SparkSession.Builder":
"""Enables Hive support, including connectivity to a persistent Hive metastore, support
for Hive SerDes, and Hive user-defined functions.
.. versionadded:: 2.0.0
Returns
-------
:class:`SparkSession.Builder`
Examples
--------
>>> SparkSession.builder.enableHiveSupport()
<pyspark.sql.session.SparkSession.Builder...
"""
return self.config("spark.sql.catalogImplementation", "hive")
def getOrCreate(self) -> "SparkSession":
"""Gets an existing :class:`SparkSession` or, if there is no existing one, creates a
new one based on the options set in this builder.
.. versionadded:: 2.0.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Returns
-------
:class:`SparkSession`
Examples
--------
This method first checks whether there is a valid global default SparkSession, and if
yes, return that one. If no valid global default SparkSession exists, the method
creates a new SparkSession and assigns the newly created SparkSession as the global
default.
>>> s1 = SparkSession.builder.config("k1", "v1").getOrCreate()
>>> s1.conf.get("k1") == "v1"
True
The configuration of the SparkSession can be changed afterwards
>>> s1.conf.set("k1", "v1_new")
>>> s1.conf.get("k1") == "v1_new"
True
In case an existing SparkSession is returned, the config options specified
in this builder will be applied to the existing SparkSession.
>>> s2 = SparkSession.builder.config("k2", "v2").getOrCreate()
>>> s1.conf.get("k1") == s2.conf.get("k1") == "v1_new"
True
>>> s1.conf.get("k2") == s2.conf.get("k2") == "v2"
True
"""
opts = dict(self._options)
if is_remote_only():
from pyspark.sql.connect.session import SparkSession as RemoteSparkSession
url = opts.get("spark.remote", os.environ.get("SPARK_REMOTE"))
if url is None:
raise PySparkRuntimeError(
errorClass="CONNECT_URL_NOT_SET",
messageParameters={},
)
os.environ["SPARK_CONNECT_MODE_ENABLED"] = "1"
opts["spark.remote"] = url
return RemoteSparkSession.builder.config(map=opts).getOrCreate() # type: ignore
from pyspark.core.context import SparkContext
with self._lock:
api_mode = opts.get("spark.api.mode", os.environ.get("SPARK_API_MODE", "")).lower()
if api_mode not in ("classic", "connect"):
api_mode = default_api_mode()
is_api_mode_connect = api_mode == "connect"
if (
"SPARK_CONNECT_MODE_ENABLED" in os.environ
or "SPARK_REMOTE" in os.environ
or "spark.remote" in opts
or is_api_mode_connect
):
with SparkContext._lock:
from pyspark.sql.connect.session import SparkSession as RemoteSparkSession
if (
SparkContext._active_spark_context is None
and SparkSession._instantiatedSession is None
):
url = opts.get("spark.remote", os.environ.get("SPARK_REMOTE"))
if url is None and is_api_mode_connect:
url = opts.get("spark.master", os.environ.get("MASTER", "local"))
if url.startswith("sc://"):
raise PySparkRuntimeError(
errorClass="MASTER_URL_INVALID",
messageParameters={},
)
if url is None:
raise PySparkRuntimeError(
errorClass="CONNECT_URL_NOT_SET",
messageParameters={},
)
if url.startswith("local") or (
is_api_mode_connect and not url.startswith("sc://")
):
os.environ["SPARK_LOCAL_REMOTE"] = "1"
RemoteSparkSession._start_connect_server(url, opts)
url = "sc://localhost"
os.environ["SPARK_CONNECT_MODE_ENABLED"] = "1"
opts["spark.remote"] = url
return cast(
SparkSession,
RemoteSparkSession.builder.config(map=opts).getOrCreate(),
)
elif "SPARK_LOCAL_REMOTE" in os.environ:
url = "sc://localhost"
os.environ["SPARK_CONNECT_MODE_ENABLED"] = "1"
opts["spark.remote"] = url
return cast(
SparkSession,
RemoteSparkSession.builder.config(map=opts).getOrCreate(),
)
else:
raise PySparkRuntimeError(
errorClass="SESSION_ALREADY_EXIST",
messageParameters={},
)
session = SparkSession._instantiatedSession
if session is None or session._sc._jsc is None:
sparkConf = SparkConf()
for key, value in self._options.items():
sparkConf.set(key, value)
# This SparkContext may be an existing one.
sc = SparkContext.getOrCreate(sparkConf)
# Do not update `SparkConf` for existing `SparkContext`, as it's shared
# by all sessions.
session = SparkSession(sc, options=self._options)
else:
module = SparkSession._get_j_spark_session_module(session._jvm)
module.applyModifiableSettings(session._jsparkSession, self._options)
return session
# Spark Connect-specific API
@remote_only
def create(self) -> "SparkSession":
"""Creates a new SparkSession. Can only be used in the context of Spark Connect
and will throw an exception otherwise.
.. versionadded:: 3.5.0
Returns
-------
:class:`SparkSession`
Notes
-----
This method will update the default and/or active session if they are not set.
"""
opts = dict(self._options)
if "SPARK_REMOTE" in os.environ or "spark.remote" in opts:
from pyspark.sql.connect.session import SparkSession as RemoteSparkSession
# Validate that no incompatible configuration options are selected.
self._validate_startup_urls()
url = opts.get("spark.remote", os.environ.get("SPARK_REMOTE"))
if url.startswith("local"):
raise PySparkRuntimeError(
errorClass="UNSUPPORTED_LOCAL_CONNECTION_STRING",
messageParameters={},
)
# Mark this Spark Session as Spark Connect. This prevents that local PySpark is
# used in conjunction with Spark Connect mode.
os.environ["SPARK_CONNECT_MODE_ENABLED"] = "1"
opts["spark.remote"] = url
return cast(SparkSession, RemoteSparkSession.builder.config(map=opts).create())
else:
raise PySparkRuntimeError(
errorClass="ONLY_SUPPORTED_WITH_SPARK_CONNECT",
messageParameters={"feature": "SparkSession.builder.create"},
)
# SPARK-47544: Explicitly declaring this as an identifier instead of a method.
# If changing, make sure this bug is not reintroduced.
builder: Builder = classproperty(lambda cls: cls.Builder()) # type: ignore
"""Creates a :class:`Builder` for constructing a :class:`SparkSession`.
.. versionchanged:: 3.4.0
Supports Spark Connect.
"""
_instantiatedSession: ClassVar[Optional["SparkSession"]] = None
_activeSession: ClassVar[Optional["SparkSession"]] = None
def __init__(
self,
sparkContext: "SparkContext",
jsparkSession: Optional["JavaObject"] = None,
options: Dict[str, Any] = {},
):
self._sc = sparkContext
self._jsc = self._sc._jsc
self._jvm = self._sc._jvm
assert self._jvm is not None
jSparkSessionClass = SparkSession._get_j_spark_session_class(self._jvm)
jSparkSessionModule = SparkSession._get_j_spark_session_module(self._jvm)
if jsparkSession is None:
if (
jSparkSessionClass.getDefaultSession().isDefined()
and not jSparkSessionClass.getDefaultSession().get().sparkContext().isStopped()
):
jsparkSession = jSparkSessionClass.getDefaultSession().get()
jSparkSessionModule.applyModifiableSettings(jsparkSession, options)
else:
jsparkSession = jSparkSessionClass(self._jsc.sc(), options)
else:
jSparkSessionModule.applyModifiableSettings(jsparkSession, options)
self._jsparkSession = jsparkSession
_monkey_patch_RDD(self)
install_exception_handler()
# If we had an instantiated SparkSession attached with a SparkContext
# which is stopped now, we need to renew the instantiated SparkSession.
# Otherwise, we will use invalid SparkSession when we call Builder.getOrCreate.
if SparkSession._should_update_active_session():
SparkSession._instantiatedSession = self
SparkSession._activeSession = self
assert self._jvm is not None
jSparkSessionClass.setDefaultSession(self._jsparkSession)
jSparkSessionClass.setActiveSession(self._jsparkSession)
self._profiler_collector = AccumulatorProfilerCollector()
@staticmethod
def _should_update_active_session() -> bool:
return (
SparkSession._instantiatedSession is None
or SparkSession._instantiatedSession._sc._jsc is None
)
@staticmethod
def _get_j_spark_session_class(jvm: "JVMView") -> "JavaClass":
return getattr(jvm, "org.apache.spark.sql.classic.SparkSession")
@staticmethod
def _get_j_spark_session_module(jvm: "JVMView") -> "JavaObject":
return getattr(getattr(jvm, "org.apache.spark.sql.classic.SparkSession$"), "MODULE$")
def _repr_html_(self) -> str:
return """
<div>
<p><b>SparkSession - {catalogImplementation}</b></p>
{sc_HTML}
</div>
""".format(
catalogImplementation=self.conf.get("spark.sql.catalogImplementation"),
sc_HTML=self.sparkContext._repr_html_(),
)
@property
def _jconf(self) -> "JavaObject":
"""Accessor for the JVM SQL-specific configurations"""
return self._jsparkSession.sessionState().conf()
if not is_remote_only():
def newSession(self) -> "SparkSession":
"""
Returns a new :class:`SparkSession` as new session, that has separate SQLConf,
registered temporary views and UDFs, but shared :class:`SparkContext` and
table cache.
.. versionadded:: 2.0.0
Returns
-------
:class:`SparkSession`
Spark session if an active session exists for the current thread
Examples
--------
>>> spark.newSession()
<...SparkSession object ...>
"""
return self.__class__(self._sc, self._jsparkSession.newSession())
@classmethod
@try_remote_session_classmethod
def getActiveSession(cls) -> Optional["SparkSession"]:
"""
Returns the active :class:`SparkSession` for the current thread, returned by the builder
.. versionadded:: 3.0.0
.. versionchanged:: 3.5.0
Supports Spark Connect.
Returns
-------
:class:`SparkSession`
Spark session if an active session exists for the current thread
Examples
--------
>>> s = SparkSession.getActiveSession()
>>> df = s.createDataFrame([('Alice', 1)], ['name', 'age'])
>>> df.select("age").show()
+---+
|age|
+---+
| 1|
+---+
"""
from pyspark import SparkContext
sc = SparkContext._active_spark_context
if sc is None:
return None
else:
assert sc._jvm is not None
jSparkSessionClass = SparkSession._get_j_spark_session_class(sc._jvm)
if jSparkSessionClass.getActiveSession().isDefined():
if SparkSession._should_update_active_session():
SparkSession(sc, jSparkSessionClass.getActiveSession().get())
return SparkSession._activeSession
else:
return None
@classmethod
@try_remote_session_classmethod
def active(cls) -> "SparkSession":
"""
Returns the active or default :class:`SparkSession` for the current thread, returned by
the builder.
.. versionadded:: 3.5.0
Returns
-------
:class:`SparkSession`
Spark session if an active or default session exists for the current thread.
"""
session = cls.getActiveSession()
if session is None:
session = cls._instantiatedSession
if session is None:
raise PySparkRuntimeError(
errorClass="NO_ACTIVE_OR_DEFAULT_SESSION",
messageParameters={},
)
return session
if not is_remote_only():
@property
def sparkContext(self) -> "SparkContext":
"""
Returns the underlying :class:`SparkContext`.
.. versionadded:: 2.0.0
Returns
-------
:class:`SparkContext`
Examples
--------
>>> spark.sparkContext
<SparkContext master=... appName=...>
Create an RDD from the Spark context
>>> rdd = spark.sparkContext.parallelize([1, 2, 3])
>>> rdd.collect()
[1, 2, 3]
"""
return self._sc
@cached_property
def version(self) -> str:
"""
The version of Spark on which this application is running.
.. versionadded:: 2.0.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Returns
-------
str
the version of Spark in string.
Examples
--------
>>> _ = spark.version
"""
return self._jsparkSession.version()
@cached_property
def conf(self) -> RuntimeConfig:
"""Runtime configuration interface for Spark.
This is the interface through which the user can get and set all Spark and Hadoop
configurations that are relevant to Spark SQL. When getting the value of a config,
this defaults to the value set in the underlying :class:`SparkContext`, if any.
.. versionadded:: 2.0.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Returns
-------
:class:`pyspark.sql.conf.RuntimeConfig`
Examples
--------
>>> spark.conf
<pyspark...RuntimeConf...>
Set a runtime configuration for the session
>>> spark.conf.set("key", "value")
>>> spark.conf.get("key")
'value'
"""
return RuntimeConfig(self._jsparkSession.conf())
@cached_property
def catalog(self) -> "Catalog":
"""Interface through which the user may create, drop, alter or query underlying
databases, tables, functions, etc.
.. versionadded:: 2.0.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Returns
-------
:class:`Catalog`
Examples
--------
>>> spark.catalog
<...Catalog object ...>
Create a temp view, show the list, and drop it.
>>> spark.range(1).createTempView("test_view")
>>> spark.catalog.listTables() # doctest: +SKIP
[Table(name='test_view', catalog=None, namespace=[], description=None, ...
>>> _ = spark.catalog.dropTempView("test_view")
"""
from pyspark.sql.catalog import Catalog
return Catalog(self)
@property
def udf(self) -> "UDFRegistration":
"""Returns a :class:`UDFRegistration` for UDF registration.
.. versionadded:: 2.0.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Returns
-------
:class:`UDFRegistration`
Examples
--------
Register a Python UDF, and use it in SQL.
>>> strlen = spark.udf.register("strlen", lambda x: len(x))
>>> spark.sql("SELECT strlen('test')").show()
+------------+
|strlen(test)|
+------------+
| 4|
+------------+
"""
from pyspark.sql.udf import UDFRegistration
return UDFRegistration(self)
@property
def udtf(self) -> "UDTFRegistration":
"""Returns a :class:`UDTFRegistration` for UDTF registration.
.. versionadded:: 3.5.0
Returns
-------
:class:`UDTFRegistration`
Notes
-----
Supports Spark Connect.
"""
from pyspark.sql.udtf import UDTFRegistration
return UDTFRegistration(self)
@property
def dataSource(self) -> "DataSourceRegistration":
"""Returns a :class:`DataSourceRegistration` for data source registration.
.. versionadded:: 4.0.0
Returns
-------
:class:`DataSourceRegistration`
Notes
-----
This feature is experimental and unstable.
"""
from pyspark.sql.datasource import DataSourceRegistration
return DataSourceRegistration(self)
@property
def profile(self) -> Profile:
"""Returns a :class:`Profile` for performance/memory profiling.
.. versionadded:: 4.0.0
Returns
-------
:class:`Profile`
Notes
-----
Supports Spark Connect.
"""
return Profile(self._profiler_collector)
def range(
self,
start: int,
end: Optional[int] = None,
step: int = 1,
numPartitions: Optional[int] = None,
) -> DataFrame:
"""
Create a :class:`DataFrame` with single :class:`pyspark.sql.types.LongType` column named
``id``, containing elements in a range from ``start`` to ``end`` (exclusive) with
step value ``step``.
.. versionadded:: 2.0.0
.. versionchanged:: 3.4.0
Supports Spark Connect.
Parameters
----------
start : int
the start value
end : int, optional
the end value (exclusive)
step : int, optional
the incremental step (default: 1)
numPartitions : int, optional
the number of partitions of the DataFrame
Returns
-------
:class:`DataFrame`
Examples
--------
>>> spark.range(1, 7, 2).show()