Hot-keys on this page
r m x p toggle line displays
j k next/prev highlighted chunk
0 (zero) top of page
1 (one) first highlighted chunk
# # 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. #
_make_type_verifier, _infer_schema, _has_nulltype, _merge_type, _create_converter, \ _parse_datatype_string
""" 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.toDF().collect() [Row(name='Alice', age=1)] """
"""The entry point to programming Spark with the Dataset and DataFrame API.
A SparkSession can be used 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:
.. autoattribute:: builder :annotation:
Examples -------- >>> spark = SparkSession.builder \\ ... .master("local") \\ ... .appName("Word Count") \\ ... .config("spark.some.config.option", "some-value") \\ ... .getOrCreate()
>>> from datetime import datetime >>> from pyspark.sql import Row >>> spark = SparkSession(sc) >>> allTypes = sc.parallelize([Row(i=1, s="string", d=1.0, l=1, ... b=True, list=[1, 2, 3], dict={"s": 0}, row=Row(a=1), ... time=datetime(2014, 8, 1, 14, 1, 5))]) >>> df = allTypes.toDF() >>> df.createOrReplaceTempView("allTypes") >>> spark.sql('select i+1, d+1, not b, list[1], dict["s"], time, row.a ' ... 'from allTypes where b and i > 0').collect() [Row((i + 1)=2, (d + 1)=2.0, (NOT b)=False, list[1]=2, \ dict[s]=0, time=datetime.datetime(2014, 8, 1, 14, 1, 5), a=1)] >>> df.rdd.map(lambda x: (x.i, x.s, x.d, x.l, x.b, x.time, x.row.a, x.list)).collect() [(1, 'string', 1.0, 1, True, datetime.datetime(2014, 8, 1, 14, 1, 5), 1, [1, 2, 3])] """
"""Builder for :class:`SparkSession`. """
"""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
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`
Examples -------- For an existing SparkConf, use `conf` parameter.
>>> from pyspark.conf import SparkConf >>> SparkSession.builder.config(conf=SparkConf()) <pyspark.sql.session...
For a (key, value) pair, you can omit parameter names.
>>> SparkSession.builder.config("spark.some.config.option", "some-value") <pyspark.sql.session...
""" else:
"""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 """
"""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
Parameters ---------- name : str an application name """
def enableHiveSupport(self): """Enables Hive support, including connectivity to a persistent Hive metastore, support for Hive SerDes, and Hive user-defined functions. """
with self._lock: self._sc = sc return self
"""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
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
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") True >>> s1.conf.get("k2") == s2.conf.get("k2") True """ sc = self._sc else: # This SparkContext may be an existing one. # Do not update `SparkConf` for existing `SparkContext`, as it's shared # by all sessions.
"""A class attribute having a :class:`Builder` to construct :class:`SparkSession` instances."""
and not self._jvm.SparkSession.getDefaultSession().get() \ .sparkContext().isStopped(): else: # 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. or SparkSession._instantiatedSession._sc._jsc is None:
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_() )
def newSession(self): """ Returns a new :class:`SparkSession` as new session, that has separate SQLConf, registered temporary views and UDFs, but shared :class:`SparkContext` and table cache. """
def getActiveSession(cls): """ Returns the active :class:`SparkSession` for the current thread, returned by the builder
.. versionadded:: 3.0.0
Returns ------- :class:`SparkSession` Spark session if an active session exists for the current thread
Examples -------- >>> s = SparkSession.getActiveSession() >>> l = [('Alice', 1)] >>> rdd = s.sparkContext.parallelize(l) >>> df = s.createDataFrame(rdd, ['name', 'age']) >>> df.select("age").collect() [Row(age=1)] """ else: else: return None
def sparkContext(self): """Returns the underlying :class:`SparkContext`."""
def version(self): """The version of Spark on which this application is running."""
def conf(self): """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.
Returns ------- :class:`pyspark.sql.conf.RuntimeConfig` """
def catalog(self): """Interface through which the user may create, drop, alter or query underlying databases, tables, functions, etc.
.. versionadded:: 2.0.0
Returns ------- :class:`Catalog` """
def udf(self): """Returns a :class:`UDFRegistration` for UDF registration.
.. versionadded:: 2.0.0
Returns ------- :class:`UDFRegistration` """
""" 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
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).collect() [Row(id=1), Row(id=3), Row(id=5)]
If only one argument is specified, it will be used as the end value.
>>> spark.range(3).collect() [Row(id=0), Row(id=1), Row(id=2)] """
else:
""" 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` """ raise ValueError("can not infer schema from empty dataset") for row in data)) raise ValueError("Some of types cannot be determined after inferring")
""" 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` """ raise ValueError("The first row in RDD is empty, " "can not infer schema")
row, names=names, infer_dict_as_struct=infer_dict_as_struct)) else: raise ValueError("Some of types cannot be determined by the " "first 100 rows, please try again with sampling") else: rdd = rdd.sample(False, float(samplingRatio)) row, names, infer_dict_as_struct=infer_dict_as_struct)).reduce(_merge_type)
""" Create an RDD for DataFrame from an existing RDD, returns the RDD and schema. """
raise TypeError("schema should be StructType or list or None, but got: %s" % schema)
# convert python objects to sql data
""" Create an RDD for DataFrame from a list or pandas.DataFrame, returns the RDD and schema. """ # make sure data could consumed multiple times
raise TypeError("schema should be StructType or list or None, but got: %s" % schema)
# convert python objects to sql data
def _create_shell_session(): """ 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. """ import py4j from pyspark.conf import SparkConf from pyspark.context import SparkContext try: # Try to access HiveConf, it will raise exception if Hive is not added conf = SparkConf() if conf.get('spark.sql.catalogImplementation', 'hive').lower() == 'hive': SparkContext._jvm.org.apache.hadoop.hive.conf.HiveConf() return SparkSession.builder\ .enableHiveSupport()\ .getOrCreate() else: return SparkSession.builder.getOrCreate() except (py4j.protocol.Py4JError, TypeError): if conf.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")
return SparkSession.builder.getOrCreate()
""" Creates a :class:`DataFrame` from an :class:`RDD`, a list or a :class:`pandas.DataFrame`.
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.
If schema inference is needed, ``samplingRatio`` is used to determined the ratio of rows used for schema inference. The first row will be used if ``samplingRatio`` is ``None``.
.. versionadded:: 2.0.0
.. versionchanged:: 2.1.0 Added verifySchema.
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`, or :class:`pandas.DataFrame`. 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`. samplingRatio : float, optional the sample ratio of rows used for inferring verifySchema : bool, optional verify data types of every row against schema. Enabled by default.
Returns ------- :class:`DataFrame`
Notes ----- Usage with spark.sql.execution.arrow.pyspark.enabled=True is experimental.
Examples -------- >>> l = [('Alice', 1)] >>> spark.createDataFrame(l).collect() [Row(_1='Alice', _2=1)] >>> spark.createDataFrame(l, ['name', 'age']).collect() [Row(name='Alice', age=1)]
>>> d = [{'name': 'Alice', 'age': 1}] >>> spark.createDataFrame(d).collect() [Row(age=1, name='Alice')]
>>> rdd = sc.parallelize(l) >>> spark.createDataFrame(rdd).collect() [Row(_1='Alice', _2=1)] >>> df = spark.createDataFrame(rdd, ['name', 'age']) >>> df.collect() [Row(name='Alice', age=1)]
>>> from pyspark.sql import Row >>> Person = Row('name', 'age') >>> person = rdd.map(lambda r: Person(*r)) >>> df2 = spark.createDataFrame(person) >>> df2.collect() [Row(name='Alice', age=1)]
>>> from pyspark.sql.types import * >>> schema = StructType([ ... StructField("name", StringType(), True), ... StructField("age", IntegerType(), True)]) >>> df3 = spark.createDataFrame(rdd, schema) >>> df3.collect() [Row(name='Alice', age=1)]
>>> spark.createDataFrame(df.toPandas()).collect() # doctest: +SKIP [Row(name='Alice', age=1)] >>> spark.createDataFrame(pandas.DataFrame([[1, 2]])).collect() # doctest: +SKIP [Row(0=1, 1=2)]
>>> spark.createDataFrame(rdd, "a: string, b: int").collect() [Row(a='Alice', b=1)] >>> rdd = rdd.map(lambda row: row[1]) >>> spark.createDataFrame(rdd, "int").collect() [Row(value=1)] >>> spark.createDataFrame(rdd, "boolean").collect() # doctest: +IGNORE_EXCEPTION_DETAIL Traceback (most recent call last): ... Py4JJavaError: ... """ raise TypeError("data is already a DataFrame")
# Must re-encode any unicode strings to be consistent with StructField names
# Create a DataFrame from pandas DataFrame. data, schema, samplingRatio, verifySchema)
dataType, name="field value") if verifySchema else lambda _: True
else:
else:
"""Returns a :class:`DataFrame` representing the result of the given query.
.. versionadded:: 2.0.0
Returns ------- :class:`DataFrame`
Examples -------- >>> df.createOrReplaceTempView("table1") >>> df2 = spark.sql("SELECT field1 AS f1, field2 as f2 from table1") >>> df2.collect() [Row(f1=1, f2='row1'), Row(f1=2, f2='row2'), Row(f1=3, f2='row3')] """
"""Returns the specified table as a :class:`DataFrame`.
.. versionadded:: 2.0.0
Returns ------- :class:`DataFrame`
Examples -------- >>> df.createOrReplaceTempView("table1") >>> df2 = spark.table("table1") >>> sorted(df.collect()) == sorted(df2.collect()) True """
def read(self): """ Returns a :class:`DataFrameReader` that can be used to read data in as a :class:`DataFrame`.
.. versionadded:: 2.0.0
Returns ------- :class:`DataFrameReader` """
def readStream(self): """ Returns a :class:`DataStreamReader` that can be used to read data streams as a streaming :class:`DataFrame`.
.. versionadded:: 2.0.0
Notes ----- This API is evolving.
Returns ------- :class:`DataStreamReader` """
def streams(self): """Returns a :class:`StreamingQueryManager` that allows managing all the :class:`StreamingQuery` instances active on `this` context.
.. versionadded:: 2.0.0
Notes ----- This API is evolving.
Returns ------- :class:`StreamingQueryManager` """
def stop(self): """Stop the underlying :class:`SparkContext`. """ # We should clean the default session up. See SPARK-23228.
def __enter__(self): """ Enable 'with SparkSession.builder.(...).getOrCreate() as session: app' syntax. """ return self
def __exit__(self, exc_type, exc_val, exc_tb): """ Enable 'with SparkSession.builder.(...).getOrCreate() as session: app' syntax.
Specifically stop the SparkSession on exit of the with block. """ self.stop()
[Row(field1=1, field2="row1"), Row(field1=2, field2="row2"), Row(field1=3, field2="row3")]) pyspark.sql.session, globs=globs, optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE) sys.exit(-1)
|