## 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.#importosimportsysimportwarningsfromcollections.abcimportSizedfromfunctoolsimportreducefromthreadingimportRLockfromtypesimportTracebackTypefromtypingimport(Any,ClassVar,Dict,Iterable,List,Optional,Tuple,Type,Union,Set,cast,no_type_check,overload,TYPE_CHECKING,)frompy4j.java_gatewayimportJavaObjectfrompysparkimportSparkConf,SparkContextfrompyspark.rddimportRDDfrompyspark.sql.columnimport_to_java_columnfrompyspark.sql.confimportRuntimeConfigfrompyspark.sql.dataframeimportDataFramefrompyspark.sql.functionsimportlitfrompyspark.sql.pandas.conversionimportSparkConversionMixinfrompyspark.sql.readwriterimportDataFrameReaderfrompyspark.sql.sql_formatterimportSQLStringFormatterfrompyspark.sql.streamingimportDataStreamReaderfrompyspark.sql.typesimport(AtomicType,DataType,StructField,StructType,_make_type_verifier,_infer_schema,_has_nulltype,_merge_type,_create_converter,_parse_datatype_string,_from_numpy_type,)frompyspark.errors.exceptions.capturedimportinstall_exception_handlerfrompyspark.sql.utilsimportis_timestamp_ntz_preferred,to_str,try_remote_session_classmethodfrompyspark.errorsimportPySparkValueError,PySparkTypeError,PySparkRuntimeErrorifTYPE_CHECKING:frompyspark.sql._typingimportAtomicValue,RowLike,OptionalPrimitiveTypefrompyspark.sql.catalogimportCatalogfrompyspark.sql.pandas._typingimportArrayLike,DataFrameLikeasPandasDataFrameLikefrompyspark.sql.streamingimportStreamingQueryManagerfrompyspark.sql.udfimportUDFRegistrationfrompyspark.sql.udtfimportUDTFRegistration# Running MyPy type checks will always require pandas and# other dependencies so importing here is fine.frompyspark.sql.connect.clientimportSparkConnectClient__all__=["SparkSession"]def_monkey_patch_RDD(sparkSession:"SparkSession")->None:@no_type_checkdeftoDF(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| +---+ """returnsparkSession.createDataFrame(self,schema,sampleRatio)RDD.toDF=toDF# type: ignore[assignment]# TODO(SPARK-38912): This method can be dropped once support for Python 3.8 is dropped# In Python 3.9, the @property decorator has been made compatible with the# @classmethod decorator (https://docs.python.org/3.9/library/functions.html#classmethod)## @classmethod + @property is also affected by a bug in Python's docstring which was backported# to Python 3.9.6 (https://github.com/python/cpython/pull/28838)classclassproperty(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]returnclassmethod(self.fget).__get__(None,owner)()# type: ignore
[docs]classSparkSession(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 """classBuilder:"""Builder for :class:`SparkSession`."""_lock=RLock()def__init__(self)->None:self._options:Dict[str,Any]={}@overloaddefconfig(self,*,conf:SparkConf)->"SparkSession.Builder":...@overloaddefconfig(self,key:str,value:Any)->"SparkSession.Builder":...@overloaddefconfig(self,*,map:Dict[str,"OptionalPrimitiveType"])->"SparkSession.Builder":...defconfig(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` Examples -------- For an existing class:`SparkConf`, use `conf` parameter. >>> from pyspark.conf import SparkConf >>> SparkSession.builder.config(conf=SparkConf()) <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... Additionally, you can pass a dictionary of configurations to set. >>> SparkSession.builder.config( ... map={"spark.some.config.number": 123, "spark.some.config.float": 0.123}) <pyspark.sql.session.SparkSession.Builder... """withself._lock:ifconfisnotNone:for(k,v)inconf.getAll():self._validate_startup_urls()self._options[k]=velifmapisnotNone:fork,vinmap.items():# type: ignore[assignment]v=to_str(v)# type: ignore[assignment]self._validate_startup_urls()self._options[k]=velse:value=to_str(value)self._validate_startup_urls()self._options[cast(str,key)]=valuereturnselfdef_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"inself._optionsand("spark.remote"inself._optionsor"SPARK_REMOTE"inos.environ):raiseRuntimeError("Spark master cannot be configured with Spark Connect server; ""however, found URL for Spark Connect [%s]"%self._options.get("spark.remote",os.environ.get("SPARK_REMOTE")))if"spark.remote"inself._optionsand("spark.master"inself._optionsor"MASTER"inos.environ):raiseRuntimeError("Spark Connect server cannot be configured with Spark master; ""however, found URL for Spark master [%s]"%self._options.get("spark.master",os.environ.get("MASTER")))if"spark.remote"inself._options:remote=cast(str,self._options.get("spark.remote"))if("SPARK_REMOTE"inos.environandos.environ["SPARK_REMOTE"]!=remote)and("SPARK_LOCAL_REMOTE"inos.environandnotremote.startswith("local")):raiseRuntimeError("Only one Spark Connect client URL can be set; however, got a ""different URL [%s] from the existing [%s]"%(os.environ["SPARK_REMOTE"],remote))defmaster(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... """returnself.config("spark.master",master)defremote(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... """returnself.config("spark.remote",url)defappName(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... """returnself.config("spark.app.name",name)defenableHiveSupport(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... """returnself.config("spark.sql.catalogImplementation","hive")defgetOrCreate(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 """frompyspark.contextimportSparkContextfrompyspark.confimportSparkConfopts=dict(self._options)withself._lock:if("SPARK_CONNECT_MODE_ENABLED"inos.environor"SPARK_REMOTE"inos.environor"spark.remote"inopts):withSparkContext._lock:frompyspark.sql.connect.sessionimportSparkSessionasRemoteSparkSessionif(SparkContext._active_spark_contextisNoneandSparkSession._instantiatedSessionisNone):url=opts.get("spark.remote",os.environ.get("SPARK_REMOTE"))ifurl.startswith("local"):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"]=urlreturnRemoteSparkSession.builder.config(map=opts).getOrCreate()elif"SPARK_LOCAL_REMOTE"inos.environ:url="sc://localhost"os.environ["SPARK_CONNECT_MODE_ENABLED"]="1"opts["spark.remote"]=urlreturnRemoteSparkSession.builder.config(map=opts).getOrCreate()else:raiseRuntimeError("Cannot start a remote Spark session because there ""is a regular Spark session already running.")session=SparkSession._instantiatedSessionifsessionisNoneorsession._sc._jscisNone:sparkConf=SparkConf()forkey,valueinself._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:getattr(getattr(session._jvm,"SparkSession$"),"MODULE$").applyModifiableSettings(session._jsparkSession,self._options)returnsession# Spark Connect-specific APIdefcreate(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"inos.environor"spark.remote"inopts:frompyspark.sql.connect.sessionimportSparkSessionasRemoteSparkSession# Validate that no incompatible configuration options are selected.self._validate_startup_urls()url=opts.get("spark.remote",os.environ.get("SPARK_REMOTE"))ifurl.startswith("local"):raiseRuntimeError("Creating new SparkSessions with `local` ""connection string is not supported.")# 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"]=urlreturnRemoteSparkSession.builder.config(map=opts).create()else:raiseRuntimeError("SparkSession.builder.create() can only be used with Spark Connect; ""however, spark.remote was not found.")# TODO(SPARK-38912): Replace @classproperty with @classmethod + @property once support for# Python 3.8 is dropped.## In Python 3.9, the @property decorator has been made compatible with the# @classmethod decorator (https://docs.python.org/3.9/library/functions.html#classmethod)## @classmethod + @property is also affected by a bug in Python's docstring which was backported# to Python 3.9.6 (https://github.com/python/cpython/pull/28838)
[docs]@classpropertydefbuilder(cls)->Builder:"""Creates a :class:`Builder` for constructing a :class:`SparkSession`. .. versionchanged:: 3.4.0 Supports Spark Connect. """returncls.Builder()
_instantiatedSession:ClassVar[Optional["SparkSession"]]=None_activeSession:ClassVar[Optional["SparkSession"]]=Nonedef__init__(self,sparkContext:SparkContext,jsparkSession:Optional[JavaObject]=None,options:Dict[str,Any]={},):self._sc=sparkContextself._jsc=self._sc._jscself._jvm=self._sc._jvmassertself._jvmisnotNoneifjsparkSessionisNone:if(self._jvm.SparkSession.getDefaultSession().isDefined()andnotself._jvm.SparkSession.getDefaultSession().get().sparkContext().isStopped()):jsparkSession=self._jvm.SparkSession.getDefaultSession().get()getattr(getattr(self._jvm,"SparkSession$"),"MODULE$").applyModifiableSettings(jsparkSession,options)else:jsparkSession=self._jvm.SparkSession(self._jsc.sc(),options)else:getattr(getattr(self._jvm,"SparkSession$"),"MODULE$").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._instantiatedSessionisNoneorSparkSession._instantiatedSession._sc._jscisNone):SparkSession._instantiatedSession=selfSparkSession._activeSession=selfassertself._jvmisnotNoneself._jvm.SparkSession.setDefaultSession(self._jsparkSession)self._jvm.SparkSession.setActiveSession(self._jsparkSession)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_(),)@propertydef_jconf(self)->"JavaObject":"""Accessor for the JVM SQL-specific configurations"""returnself._jsparkSession.sessionState().conf()
[docs]defnewSession(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 ...> """returnself.__class__(self._sc,self._jsparkSession.newSession())
[docs]@classmethod@try_remote_session_classmethoddefgetActiveSession(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| +---+ """frompysparkimportSparkContextsc=SparkContext._active_spark_contextifscisNone:returnNoneelse:assertsc._jvmisnotNoneifsc._jvm.SparkSession.getActiveSession().isDefined():SparkSession(sc,sc._jvm.SparkSession.getActiveSession().get())returnSparkSession._activeSessionelse:returnNone
[docs]@classmethod@try_remote_session_classmethoddefactive(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()ifsessionisNone:session=cls._instantiatedSessionifsessionisNone:raisePySparkRuntimeError(error_class="NO_ACTIVE_OR_DEFAULT_SESSION",message_parameters={},)returnsession
@propertydefsparkContext(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] """returnself._sc@propertydefversion(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 """returnself._jsparkSession.version()@propertydefconf(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' """ifnothasattr(self,"_conf"):self._conf=RuntimeConfig(self._jsparkSession.conf())returnself._conf@propertydefcatalog(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() [Table(name='test_view', catalog=None, namespace=[], description=None, ... >>> _ = spark.catalog.dropTempView("test_view") """frompyspark.sql.catalogimportCatalogifnothasattr(self,"_catalog"):self._catalog=Catalog(self)returnself._catalog@propertydefudf(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| +------------+ """frompyspark.sql.udfimportUDFRegistrationreturnUDFRegistration(self)@propertydefudtf(self)->"UDTFRegistration":"""Returns a :class:`UDTFRegistration` for UDTF registration. .. versionadded:: 3.5.0 Returns ------- :class:`UDTFRegistration` Notes ----- Supports Spark Connect. """frompyspark.sql.udtfimportUDTFRegistrationreturnUDTFRegistration(self)
[docs]defrange(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() +---+ | id| +---+ | 1| | 3| | 5| +---+ If only one argument is specified, it will be used as the end value. >>> spark.range(3).show() +---+ | id| +---+ | 0| | 1| | 2| +---+ """ifnumPartitionsisNone:numPartitions=self._sc.defaultParallelismifendisNone:jdf=self._jsparkSession.range(0,int(start),int(step),int(numPartitions))else:jdf=self._jsparkSession.range(int(start),int(end),int(step),int(numPartitions))returnDataFrame(jdf,self)
def_inferSchemaFromList(self,data:Iterable[Any],names:Optional[List[str]]=None)->StructType:""" Infer schema from list of Row, dict, or tuple. Parameters ---------- data : iterable list of Row, dict, or tuple names : list, optional list of column names Returns ------- :class:`pyspark.sql.types.StructType` """ifnotdata:raisePySparkValueError(error_class="CANNOT_INFER_EMPTY_SCHEMA",message_parameters={},)infer_dict_as_struct=self._jconf.inferDictAsStruct()infer_array_from_first_element=self._jconf.legacyInferArrayTypeFromFirstElement()prefer_timestamp_ntz=is_timestamp_ntz_preferred()schema=reduce(_merge_type,(_infer_schema(row,names,infer_dict_as_struct=infer_dict_as_struct,infer_array_from_first_element=infer_array_from_first_element,prefer_timestamp_ntz=prefer_timestamp_ntz,)forrowindata),)if_has_nulltype(schema):raisePySparkValueError(error_class="CANNOT_DETERMINE_TYPE",message_parameters={},)returnschemadef_inferSchema(self,rdd:RDD[Any],samplingRatio:Optional[float]=None,names:Optional[List[str]]=None,)->StructType:""" Infer schema from an RDD of Row, dict, or tuple. Parameters ---------- rdd : :class:`RDD` an RDD of Row, dict, or tuple samplingRatio : float, optional sampling ratio, or no sampling (default) names : list, optional Returns ------- :class:`pyspark.sql.types.StructType` """first=rdd.first()ifisinstance(first,Sized)andlen(first)==0:raisePySparkValueError(error_class="CANNOT_INFER_EMPTY_SCHEMA",message_parameters={},)infer_dict_as_struct=self._jconf.inferDictAsStruct()infer_array_from_first_element=self._jconf.legacyInferArrayTypeFromFirstElement()prefer_timestamp_ntz=is_timestamp_ntz_preferred()ifsamplingRatioisNone:schema=_infer_schema(first,names=names,infer_dict_as_struct=infer_dict_as_struct,prefer_timestamp_ntz=prefer_timestamp_ntz,)if_has_nulltype(schema):forrowinrdd.take(100)[1:]:schema=_merge_type(schema,_infer_schema(row,names=names,infer_dict_as_struct=infer_dict_as_struct,infer_array_from_first_element=infer_array_from_first_element,prefer_timestamp_ntz=prefer_timestamp_ntz,),)ifnot_has_nulltype(schema):breakelse:raisePySparkValueError(error_class="CANNOT_DETERMINE_TYPE",message_parameters={},)else:ifsamplingRatio<0.99:rdd=rdd.sample(False,float(samplingRatio))schema=rdd.map(lambdarow:_infer_schema(row,names,infer_dict_as_struct=infer_dict_as_struct,infer_array_from_first_element=infer_array_from_first_element,prefer_timestamp_ntz=prefer_timestamp_ntz,)).reduce(_merge_type)returnschemadef_createFromRDD(self,rdd:RDD[Any],schema:Optional[Union[DataType,List[str]]],samplingRatio:Optional[float],)->Tuple[RDD[Tuple],StructType]:""" Create an RDD for DataFrame from an existing RDD, returns the RDD and schema. """ifschemaisNoneorisinstance(schema,(list,tuple)):struct=self._inferSchema(rdd,samplingRatio,names=schema)converter=_create_converter(struct)tupled_rdd=rdd.map(converter)ifisinstance(schema,(list,tuple)):fori,nameinenumerate(schema):struct.fields[i].name=namestruct.names[i]=nameelifisinstance(schema,StructType):struct=schematupled_rdd=rddelse:raisePySparkTypeError(error_class="NOT_LIST_OR_NONE_OR_STRUCT",message_parameters={"arg_name":"schema","arg_type":type(schema).__name__,},)# convert python objects to sql datainternal_rdd=tupled_rdd.map(struct.toInternal)returninternal_rdd,structdef_createFromLocal(self,data:Iterable[Any],schema:Optional[Union[DataType,List[str]]])->Tuple[RDD[Tuple],StructType]:""" Create an RDD for DataFrame from a list or pandas.DataFrame, returns the RDD and schema. """# make sure data could consumed multiple timesifnotisinstance(data,list):data=list(data)ifschemaisNoneorisinstance(schema,(list,tuple)):struct=self._inferSchemaFromList(data,names=schema)converter=_create_converter(struct)tupled_data:Iterable[Tuple]=map(converter,data)ifisinstance(schema,(list,tuple)):fori,nameinenumerate(schema):struct.fields[i].name=namestruct.names[i]=nameelifisinstance(schema,StructType):struct=schematupled_data=dataelse:raisePySparkTypeError(error_class="NOT_LIST_OR_NONE_OR_STRUCT",message_parameters={"arg_name":"schema","arg_type":type(schema).__name__,},)# convert python objects to sql datainternal_data=[struct.toInternal(row)forrowintupled_data]returnself._sc.parallelize(internal_data),struct@staticmethoddef_create_shell_session()->"SparkSession":""" Initialize a :class:`SparkSession` for a pyspark shell session. This is called from shell.py to make error handling simpler without needing to declare local variables in that script, which would expose those to users. """importpy4jfrompyspark.confimportSparkConffrompyspark.contextimportSparkContexttry:# Try to access HiveConf, it will raise exception if Hive is not addedconf=SparkConf()assertSparkContext._jvmisnotNoneifconf.get("spark.sql.catalogImplementation","hive").lower()=="hive":SparkContext._jvm.org.apache.hadoop.hive.conf.HiveConf()returnSparkSession.builder.enableHiveSupport().getOrCreate()else:returnSparkSession._getActiveSessionOrCreate()except(py4j.protocol.Py4JError,TypeError):ifconf.get("spark.sql.catalogImplementation","").lower()=="hive":warnings.warn("Fall back to non-hive support because failing to access HiveConf, ""please make sure you build spark with hive")returnSparkSession._getActiveSessionOrCreate()@staticmethoddef_getActiveSessionOrCreate(**static_conf:Any)->"SparkSession":""" Returns the active :class:`SparkSession` for the current thread, returned by the builder, or if there is no existing one, creates a new one based on the options set in the builder. NOTE that 'static_conf' might not be set if there's an active or default Spark session running. """spark=SparkSession.getActiveSession()ifsparkisNone:builder=SparkSession.builderfork,vinstatic_conf.items():builder=builder.config(k,v)spark=builder.getOrCreate()returnspark@overloaddefcreateDataFrame(self,data:Iterable["RowLike"],schema:Union[List[str],Tuple[str,...]]=...,samplingRatio:Optional[float]=...,)->DataFrame:...@overloaddefcreateDataFrame(self,data:"RDD[RowLike]",schema:Union[List[str],Tuple[str,...]]=...,samplingRatio:Optional[float]=...,)->DataFrame:...@overloaddefcreateDataFrame(self,data:Iterable["RowLike"],schema:Union[StructType,str],*,verifySchema:bool=...,)->DataFrame:...@overloaddefcreateDataFrame(self,data:"RDD[RowLike]",schema:Union[StructType,str],*,verifySchema:bool=...,)->DataFrame:...@overloaddefcreateDataFrame(self,data:"RDD[AtomicValue]",schema:Union[AtomicType,str],verifySchema:bool=...,)->DataFrame:...@overloaddefcreateDataFrame(self,data:Iterable["AtomicValue"],schema:Union[AtomicType,str],verifySchema:bool=...,)->DataFrame:...@overloaddefcreateDataFrame(self,data:"PandasDataFrameLike",samplingRatio:Optional[float]=...)->DataFrame:...@overloaddefcreateDataFrame(self,data:"PandasDataFrameLike",schema:Union[StructType,str],verifySchema:bool=...,)->DataFrame:...
[docs]defcreateDataFrame(# type: ignore[misc]self,data:Union[RDD[Any],Iterable[Any],"PandasDataFrameLike","ArrayLike"],schema:Optional[Union[AtomicType,StructType,str]]=None,samplingRatio:Optional[float]=None,verifySchema:bool=True,)->DataFrame:""" Creates a :class:`DataFrame` from an :class:`RDD`, a list, a :class:`pandas.DataFrame` or a :class:`numpy.ndarray`. .. versionadded:: 2.0.0 .. versionchanged:: 3.4.0 Supports Spark Connect. Parameters ---------- data : :class:`RDD` or iterable an RDD of any kind of SQL data representation (:class:`Row`, :class:`tuple`, ``int``, ``boolean``, etc.), or :class:`list`, :class:`pandas.DataFrame` or :class:`numpy.ndarray`. 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<>``. When ``schema`` is a list of column names, the type of each column will be inferred from ``data``. When ``schema`` is ``None``, it will try to infer the schema (column names and types) from ``data``, which should be an RDD of either :class:`Row`, :class:`namedtuple`, or :class:`dict`. When ``schema`` is :class:`pyspark.sql.types.DataType` or a datatype string, it must match the real data, or an exception will be thrown at runtime. If the given schema is not :class:`pyspark.sql.types.StructType`, it will be wrapped into a :class:`pyspark.sql.types.StructType` as its only field, and the field name will be "value". Each record will also be wrapped into a tuple, which can be converted to row later. samplingRatio : float, optional the sample ratio of rows used for inferring. The first few rows will be used if ``samplingRatio`` is ``None``. verifySchema : bool, optional verify data types of every row against schema. Enabled by default. .. versionadded:: 2.1.0 Returns ------- :class:`DataFrame` Notes ----- Usage with `spark.sql.execution.arrow.pyspark.enabled=True` is experimental. Examples -------- Create a DataFrame from a list of tuples. >>> spark.createDataFrame([('Alice', 1)]).show() +-----+---+ | _1| _2| +-----+---+ |Alice| 1| +-----+---+ Create a DataFrame from a list of dictionaries. >>> d = [{'name': 'Alice', 'age': 1}] >>> spark.createDataFrame(d).show() +---+-----+ |age| name| +---+-----+ | 1|Alice| +---+-----+ Create a DataFrame with column names specified. >>> spark.createDataFrame([('Alice', 1)], ['name', 'age']).show() +-----+---+ | name|age| +-----+---+ |Alice| 1| +-----+---+ Create a DataFrame with the explicit schema specified. >>> from pyspark.sql.types import * >>> schema = StructType([ ... StructField("name", StringType(), True), ... StructField("age", IntegerType(), True)]) >>> spark.createDataFrame([('Alice', 1)], schema).show() +-----+---+ | name|age| +-----+---+ |Alice| 1| +-----+---+ Create a DataFrame with the schema in DDL formatted string. >>> spark.createDataFrame([('Alice', 1)], "name: string, age: int").show() +-----+---+ | name|age| +-----+---+ |Alice| 1| +-----+---+ Create an empty DataFrame. When initializing an empty DataFrame in PySpark, it's mandatory to specify its schema, as the DataFrame lacks data from which the schema can be inferred. >>> spark.createDataFrame([], "name: string, age: int").show() +----+---+ |name|age| +----+---+ +----+---+ Create a DataFrame from Row objects. >>> from pyspark.sql import Row >>> Person = Row('name', 'age') >>> df = spark.createDataFrame([Person("Alice", 1)]) >>> df.show() +-----+---+ | name|age| +-----+---+ |Alice| 1| +-----+---+ Create a DataFrame from a pandas DataFrame. >>> spark.createDataFrame(df.toPandas()).show() # doctest: +SKIP +-----+---+ | name|age| +-----+---+ |Alice| 1| +-----+---+ >>> spark.createDataFrame(pandas.DataFrame([[1, 2]])).collect() # doctest: +SKIP +---+---+ | 0| 1| +---+---+ | 1| 2| +---+---+ """SparkSession._activeSession=selfassertself._jvmisnotNoneself._jvm.SparkSession.setActiveSession(self._jsparkSession)ifisinstance(data,DataFrame):raisePySparkTypeError(error_class="SHOULD_NOT_DATAFRAME",message_parameters={"arg_name":"data"},)ifisinstance(schema,str):schema=cast(Union[AtomicType,StructType,str],_parse_datatype_string(schema))elifisinstance(schema,(list,tuple)):# Must re-encode any unicode strings to be consistent with StructField namesschema=[x.encode("utf-8")ifnotisinstance(x,str)elsexforxinschema]try:importpandasaspdhas_pandas=TrueexceptException:has_pandas=Falsetry:importnumpyasnphas_numpy=TrueexceptException:has_numpy=Falseifhas_numpyandisinstance(data,np.ndarray):# `data` of numpy.ndarray type will be converted to a pandas DataFrame,# so pandas is required.frompyspark.sql.pandas.utilsimportrequire_minimum_pandas_versionrequire_minimum_pandas_version()ifdata.ndimnotin[1,2]:raisePySparkValueError(error_class="INVALID_NDARRAY_DIMENSION",message_parameters={"dimensions":"1 or 2"},)ifdata.ndim==1ordata.shape[1]==1:column_names=["value"]else:column_names=["_%s"%iforiinrange(1,data.shape[1]+1)]ifschemaisNoneandnotself._jconf.arrowPySparkEnabled():# Construct `schema` from `np.dtype` of the input NumPy array# TODO: Apply the logic below when self._jconf.arrowPySparkEnabled() is Truespark_type=_from_numpy_type(data.dtype)ifspark_typeisnotNone:schema=StructType([StructField(name,spark_type,nullable=True)fornameincolumn_names])data=pd.DataFrame(data,columns=column_names)ifhas_pandasandisinstance(data,pd.DataFrame):# Create a DataFrame from pandas DataFrame.returnsuper(SparkSession,self).createDataFrame(# type: ignore[call-overload]data,schema,samplingRatio,verifySchema)returnself._create_dataframe(data,schema,samplingRatio,verifySchema# type: ignore[arg-type])
[docs]defsql(self,sqlQuery:str,args:Optional[Union[Dict[str,Any],List]]=None,**kwargs:Any)->DataFrame:"""Returns a :class:`DataFrame` representing the result of the given query. When ``kwargs`` is specified, this method formats the given string by using the Python standard formatter. The method binds named parameters to SQL literals or positional parameters from `args`. It doesn't support named and positional parameters in the same SQL query. .. versionadded:: 2.0.0 .. versionchanged:: 3.4.0 Supports Spark Connect and parameterized SQL. .. versionchanged:: 3.5.0 Added positional parameters. Parameters ---------- sqlQuery : str SQL query string. args : dict or list A dictionary of parameter names to Python objects or a list of Python objects that can be converted to SQL literal expressions. See <a href="https://spark.apache.org/docs/latest/sql-ref-datatypes.html"> Supported Data Types</a> for supported value types in Python. For example, dictionary keys: "rank", "name", "birthdate"; dictionary or list values: 1, "Steven", datetime.date(2023, 4, 2). A value can be also a `Column` of literal expression, in that case it is taken as is. .. versionadded:: 3.4.0 kwargs : dict Other variables that the user wants to set that can be referenced in the query .. versionchanged:: 3.3.0 Added optional argument ``kwargs`` to specify the mapping of variables in the query. This feature is experimental and unstable. Returns ------- :class:`DataFrame` Examples -------- Executing a SQL query. >>> spark.sql("SELECT * FROM range(10) where id > 7").show() +---+ | id| +---+ | 8| | 9| +---+ Executing a SQL query with variables as Python formatter standard. >>> spark.sql( ... "SELECT * FROM range(10) WHERE id > {bound1} AND id < {bound2}", bound1=7, bound2=9 ... ).show() +---+ | id| +---+ | 8| +---+ >>> mydf = spark.range(10) >>> spark.sql( ... "SELECT {col} FROM {mydf} WHERE id IN {x}", ... col=mydf.id, mydf=mydf, x=tuple(range(4))).show() +---+ | id| +---+ | 0| | 1| | 2| | 3| +---+ >>> spark.sql(''' ... SELECT m1.a, m2.b ... FROM {table1} m1 INNER JOIN {table2} m2 ... ON m1.key = m2.key ... ORDER BY m1.a, m2.b''', ... table1=spark.createDataFrame([(1, "a"), (2, "b")], ["a", "key"]), ... table2=spark.createDataFrame([(3, "a"), (4, "b"), (5, "b")], ["b", "key"])).show() +---+---+ | a| b| +---+---+ | 1| 3| | 2| 4| | 2| 5| +---+---+ Also, it is possible to query using class:`Column` from :class:`DataFrame`. >>> mydf = spark.createDataFrame([(1, 4), (2, 4), (3, 6)], ["A", "B"]) >>> spark.sql("SELECT {df.A}, {df[B]} FROM {df}", df=mydf).show() +---+---+ | A| B| +---+---+ | 1| 4| | 2| 4| | 3| 6| +---+---+ And substitude named parameters with the `:` prefix by SQL literals. >>> spark.sql("SELECT * FROM {df} WHERE {df[B]} > :minB", {"minB" : 5}, df=mydf).show() +---+---+ | A| B| +---+---+ | 3| 6| +---+---+ Or positional parameters marked by `?` in the SQL query by SQL literals. >>> spark.sql( ... "SELECT * FROM {df} WHERE {df[B]} > ? and ? < {df[A]}", ... args=[5, 2], df=mydf).show() +---+---+ | A| B| +---+---+ | 3| 6| +---+---+ """formatter=SQLStringFormatter(self)iflen(kwargs)>0:sqlQuery=formatter.format(sqlQuery,**kwargs)try:ifisinstance(args,Dict):litArgs={k:_to_java_column(lit(v))fork,vin(argsor{}).items()}else:assertself._jvmisnotNonelitArgs=self._jvm.PythonUtils.toArray([_to_java_column(lit(v))forvin(argsor[])])returnDataFrame(self._jsparkSession.sql(sqlQuery,litArgs),self)finally:iflen(kwargs)>0:formatter.clear()
[docs]deftable(self,tableName:str)->DataFrame:"""Returns the specified table as a :class:`DataFrame`. .. versionadded:: 2.0.0 .. versionchanged:: 3.4.0 Supports Spark Connect. Parameters ---------- tableName : str the table name to retrieve. Returns ------- :class:`DataFrame` Examples -------- >>> spark.range(5).createOrReplaceTempView("table1") >>> spark.table("table1").sort("id").show() +---+ | id| +---+ | 0| | 1| | 2| | 3| | 4| +---+ """returnDataFrame(self._jsparkSession.table(tableName),self)
@propertydefread(self)->DataFrameReader:""" Returns a :class:`DataFrameReader` that can be used to read data in as a :class:`DataFrame`. .. versionadded:: 2.0.0 .. versionchanged:: 3.4.0 Supports Spark Connect. Returns ------- :class:`DataFrameReader` Examples -------- >>> spark.read <...DataFrameReader object ...> Write a DataFrame into a JSON file and read it back. >>> import tempfile >>> with tempfile.TemporaryDirectory() as d: ... # Write a DataFrame into a JSON file ... spark.createDataFrame( ... [{"age": 100, "name": "Hyukjin Kwon"}] ... ).write.mode("overwrite").format("json").save(d) ... ... # Read the JSON file as a DataFrame. ... spark.read.format('json').load(d).show() +---+------------+ |age| name| +---+------------+ |100|Hyukjin Kwon| +---+------------+ """returnDataFrameReader(self)@propertydefreadStream(self)->DataStreamReader:""" Returns a :class:`DataStreamReader` that can be used to read data streams as a streaming :class:`DataFrame`. .. versionadded:: 2.0.0 .. versionchanged:: 3.5.0 Supports Spark Connect. Notes ----- This API is evolving. Returns ------- :class:`DataStreamReader` Examples -------- >>> spark.readStream <pyspark...DataStreamReader object ...> The example below uses Rate source that generates rows continuously. After that, we operate a modulo by 3, and then write the stream out to the console. The streaming query stops in 3 seconds. >>> import time >>> df = spark.readStream.format("rate").load() >>> df = df.selectExpr("value % 3 as v") >>> q = df.writeStream.format("console").start() >>> time.sleep(3) >>> q.stop() """returnDataStreamReader(self)@propertydefstreams(self)->"StreamingQueryManager":"""Returns a :class:`StreamingQueryManager` that allows managing all the :class:`StreamingQuery` instances active on `this` context. .. versionadded:: 2.0.0 .. versionchanged:: 3.5.0 Supports Spark Connect. Notes ----- This API is evolving. Returns ------- :class:`StreamingQueryManager` Examples -------- >>> spark.streams <pyspark...StreamingQueryManager object ...> Get the list of active streaming queries >>> sq = spark.readStream.format( ... "rate").load().writeStream.format('memory').queryName('this_query').start() >>> sqm = spark.streams >>> [q.name for q in sqm.active] ['this_query'] >>> sq.stop() """frompyspark.sql.streamingimportStreamingQueryManagerreturnStreamingQueryManager(self._jsparkSession.streams())
[docs]defstop(self)->None:""" Stop the underlying :class:`SparkContext`. .. versionadded:: 2.0.0 .. versionchanged:: 3.4.0 Supports Spark Connect. Examples -------- >>> spark.stop() # doctest: +SKIP """frompyspark.sql.contextimportSQLContextself._sc.stop()# We should clean the default session up. See SPARK-23228.assertself._jvmisnotNoneself._jvm.SparkSession.clearDefaultSession()self._jvm.SparkSession.clearActiveSession()SparkSession._instantiatedSession=NoneSparkSession._activeSession=NoneSQLContext._instantiatedContext=None
def__enter__(self)->"SparkSession":""" Enable 'with SparkSession.builder.(...).getOrCreate() as session: app' syntax. .. versionadded:: 2.0.0 Examples -------- >>> with SparkSession.builder.master("local").getOrCreate() as session: ... session.range(5).show() # doctest: +SKIP +---+ | id| +---+ | 0| | 1| | 2| | 3| | 4| +---+ """returnselfdef__exit__(self,exc_type:Optional[Type[BaseException]],exc_val:Optional[BaseException],exc_tb:Optional[TracebackType],)->None:""" Enable 'with SparkSession.builder.(...).getOrCreate() as session: app' syntax. Specifically stop the SparkSession on exit of the with block. .. versionadded:: 2.0.0 Examples -------- >>> with SparkSession.builder.master("local").getOrCreate() as session: ... session.range(5).show() # doctest: +SKIP +---+ | id| +---+ | 0| | 1| | 2| | 3| | 4| +---+ """self.stop()# SparkConnect-specific API@propertydefclient(self)->"SparkConnectClient":""" Gives access to the Spark Connect client. In normal cases this is not necessary to be used and only relevant for testing. .. versionadded:: 3.4.0 Returns ------- :class:`SparkConnectClient` Notes ----- This API is unstable, and a developer API. It returns non-API instance :class:`SparkConnectClient`. This is an API dedicated to Spark Connect client only. With regular Spark Session, it throws an exception. """raiseRuntimeError("SparkSession.client is only supported with Spark Connect; ""however, the current Spark session does not use Spark Connect.")
[docs]defaddArtifacts(self,*path:str,pyfile:bool=False,archive:bool=False,file:bool=False)->None:""" Add artifact(s) to the client session. Currently only local files are supported. .. versionadded:: 3.5.0 Parameters ---------- *path : tuple of str Artifact's URIs to add. pyfile : bool Whether to add them as Python dependencies such as .py, .egg, .zip or .jar files. The pyfiles are directly inserted into the path when executing Python functions in executors. archive : bool Whether to add them as archives such as .zip, .jar, .tar.gz, .tgz, or .tar files. The archives are unpacked on the executor side automatically. file : bool Add a file to be downloaded with this Spark job on every node. The ``path`` passed can only be a local file for now. Notes ----- This is an API dedicated to Spark Connect client only. With regular Spark Session, it throws an exception. """raiseRuntimeError("SparkSession.addArtifact(s) is only supported with Spark Connect; ""however, the current Spark session does not use Spark Connect.")
addArtifact=addArtifacts
[docs]defcopyFromLocalToFs(self,local_path:str,dest_path:str)->None:""" Copy file from local to cloud storage file system. If the file already exits in destination path, old file is overwritten. .. versionadded:: 3.5.0 Parameters ---------- local_path: str Path to a local file. Directories are not supported. The path can be either an absolute path or a relative path. dest_path: str The cloud storage path to the destination the file will be copied to. The path must be an an absolute path. Notes ----- This API is a developer API. Also, this is an API dedicated to Spark Connect client only. With regular Spark Session, it throws an exception. """raiseRuntimeError("SparkSession.copyFromLocalToFs is only supported with Spark Connect; ""however, the current Spark session does not use Spark Connect.")
[docs]definterruptAll(self)->List[str]:""" Interrupt all operations of this session currently running on the connected server. .. versionadded:: 3.5.0 Returns ------- list of str List of operationIds of interrupted operations. Notes ----- There is still a possibility of operation finishing just as it is interrupted. """raiseRuntimeError("SparkSession.interruptAll is only supported with Spark Connect; ""however, the current Spark session does not use Spark Connect.")
[docs]definterruptTag(self,tag:str)->List[str]:""" Interrupt all operations of this session with the given operation tag. .. versionadded:: 3.5.0 Returns ------- list of str List of operationIds of interrupted operations. Notes ----- There is still a possibility of operation finishing just as it is interrupted. """raiseRuntimeError("SparkSession.interruptTag is only supported with Spark Connect; ""however, the current Spark session does not use Spark Connect.")
[docs]definterruptOperation(self,op_id:str)->List[str]:""" Interrupt an operation of this session with the given operationId. .. versionadded:: 3.5.0 Returns ------- list of str List of operationIds of interrupted operations. Notes ----- There is still a possibility of operation finishing just as it is interrupted. """raiseRuntimeError("SparkSession.interruptOperation is only supported with Spark Connect; ""however, the current Spark session does not use Spark Connect.")
[docs]defaddTag(self,tag:str)->None:""" Add a tag to be assigned to all the operations started by this thread in this session. .. versionadded:: 3.5.0 Parameters ---------- tag : list of str The tag to be added. Cannot contain ',' (comma) character or be an empty string. """raiseRuntimeError("SparkSession.addTag is only supported with Spark Connect; ""however, the current Spark session does not use Spark Connect.")
[docs]defremoveTag(self,tag:str)->None:""" Remove a tag previously added to be assigned to all the operations started by this thread in this session. Noop if such a tag was not added earlier. .. versionadded:: 3.5.0 Parameters ---------- tag : list of str The tag to be removed. Cannot contain ',' (comma) character or be an empty string. """raiseRuntimeError("SparkSession.removeTag is only supported with Spark Connect; ""however, the current Spark session does not use Spark Connect.")
[docs]defgetTags(self)->Set[str]:""" Get the tags that are currently set to be assigned to all the operations started by this thread. .. versionadded:: 3.5.0 Returns ------- set of str Set of tags of interrupted operations. """raiseRuntimeError("SparkSession.getTags is only supported with Spark Connect; ""however, the current Spark session does not use Spark Connect.")
[docs]defclearTags(self)->None:""" Clear the current thread's operation tags. .. versionadded:: 3.5.0 """raiseRuntimeError("SparkSession.clearTags is only supported with Spark Connect; ""however, the current Spark session does not use Spark Connect.")