2016-04-28 13:55:48 -04:00
|
|
|
#
|
|
|
|
# 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 sys
|
|
|
|
import warnings
|
|
|
|
from functools import reduce
|
2016-05-03 21:05:40 -04:00
|
|
|
from threading import RLock
|
2016-04-28 13:55:48 -04:00
|
|
|
|
|
|
|
from pyspark import since
|
2020-07-13 22:22:44 -04:00
|
|
|
from pyspark.rdd import RDD
|
2016-04-29 12:34:10 -04:00
|
|
|
from pyspark.sql.conf import RuntimeConfig
|
2016-04-28 13:55:48 -04:00
|
|
|
from pyspark.sql.dataframe import DataFrame
|
[SPARK-30434][PYTHON][SQL] Move pandas related functionalities into 'pandas' sub-package
### What changes were proposed in this pull request?
This PR proposes to move pandas related functionalities into pandas package. Namely:
```bash
pyspark/sql/pandas
├── __init__.py
├── conversion.py # Conversion between pandas <> PySpark DataFrames
├── functions.py # pandas_udf
├── group_ops.py # Grouped UDF / Cogrouped UDF + groupby.apply, groupby.cogroup.apply
├── map_ops.py # Map Iter UDF + mapInPandas
├── serializers.py # pandas <> PyArrow serializers
├── types.py # Type utils between pandas <> PyArrow
└── utils.py # Version requirement checks
```
In order to separately locate `groupby.apply`, `groupby.cogroup.apply`, `mapInPandas`, `toPandas`, and `createDataFrame(pdf)` under `pandas` sub-package, I had to use a mix-in approach which Scala side uses often by `trait`, and also pandas itself uses this approach (see `IndexOpsMixin` as an example) to group related functionalities. Currently, you can think it's like Scala's self typed trait. See the structure below:
```python
class PandasMapOpsMixin(object):
def mapInPandas(self, ...):
...
return ...
# other Pandas <> PySpark APIs
```
```python
class DataFrame(PandasMapOpsMixin):
# other DataFrame APIs equivalent to Scala side.
```
Yes, This is a big PR but they are mostly just moving around except one case `createDataFrame` which I had to split the methods.
### Why are the changes needed?
There are pandas functionalities here and there and I myself gets lost where it was. Also, when you have to make a change commonly for all of pandas related features, it's almost impossible now.
Also, after this change, `DataFrame` and `SparkSession` become more consistent with Scala side since pandas is specific to Python, and this change separates pandas-specific APIs away from `DataFrame` or `SparkSession`.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Existing tests should cover. Also, I manually built the PySpark API documentation and checked.
Closes #27109 from HyukjinKwon/pandas-refactoring.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-08 20:22:50 -05:00
|
|
|
from pyspark.sql.pandas.conversion import SparkConversionMixin
|
2016-06-29 01:07:11 -04:00
|
|
|
from pyspark.sql.readwriter import DataFrameReader
|
|
|
|
from pyspark.sql.streaming import DataStreamReader
|
2020-08-08 11:51:57 -04:00
|
|
|
from pyspark.sql.types import DataType, StructType, \
|
2017-11-28 03:45:22 -05:00
|
|
|
_make_type_verifier, _infer_schema, _has_nulltype, _merge_type, _create_converter, \
|
|
|
|
_parse_datatype_string
|
2016-04-28 13:55:48 -04:00
|
|
|
from pyspark.sql.utils import install_exception_handler
|
|
|
|
|
|
|
|
__all__ = ["SparkSession"]
|
|
|
|
|
|
|
|
|
|
|
|
def _monkey_patch_RDD(sparkSession):
|
|
|
|
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)``
|
|
|
|
|
2016-07-28 17:57:15 -04:00
|
|
|
:param schema: a :class:`pyspark.sql.types.StructType` or list of names of columns
|
2020-08-26 13:34:49 -04:00
|
|
|
:param sampleRatio: the sample ratio of rows used for inferring
|
2016-04-28 13:55:48 -04:00
|
|
|
:return: a DataFrame
|
|
|
|
|
|
|
|
>>> rdd.toDF().collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(name='Alice', age=1)]
|
2016-04-28 13:55:48 -04:00
|
|
|
"""
|
|
|
|
return sparkSession.createDataFrame(self, schema, sampleRatio)
|
|
|
|
|
|
|
|
RDD.toDF = toDF
|
|
|
|
|
|
|
|
|
[SPARK-30434][PYTHON][SQL] Move pandas related functionalities into 'pandas' sub-package
### What changes were proposed in this pull request?
This PR proposes to move pandas related functionalities into pandas package. Namely:
```bash
pyspark/sql/pandas
├── __init__.py
├── conversion.py # Conversion between pandas <> PySpark DataFrames
├── functions.py # pandas_udf
├── group_ops.py # Grouped UDF / Cogrouped UDF + groupby.apply, groupby.cogroup.apply
├── map_ops.py # Map Iter UDF + mapInPandas
├── serializers.py # pandas <> PyArrow serializers
├── types.py # Type utils between pandas <> PyArrow
└── utils.py # Version requirement checks
```
In order to separately locate `groupby.apply`, `groupby.cogroup.apply`, `mapInPandas`, `toPandas`, and `createDataFrame(pdf)` under `pandas` sub-package, I had to use a mix-in approach which Scala side uses often by `trait`, and also pandas itself uses this approach (see `IndexOpsMixin` as an example) to group related functionalities. Currently, you can think it's like Scala's self typed trait. See the structure below:
```python
class PandasMapOpsMixin(object):
def mapInPandas(self, ...):
...
return ...
# other Pandas <> PySpark APIs
```
```python
class DataFrame(PandasMapOpsMixin):
# other DataFrame APIs equivalent to Scala side.
```
Yes, This is a big PR but they are mostly just moving around except one case `createDataFrame` which I had to split the methods.
### Why are the changes needed?
There are pandas functionalities here and there and I myself gets lost where it was. Also, when you have to make a change commonly for all of pandas related features, it's almost impossible now.
Also, after this change, `DataFrame` and `SparkSession` become more consistent with Scala side since pandas is specific to Python, and this change separates pandas-specific APIs away from `DataFrame` or `SparkSession`.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Existing tests should cover. Also, I manually built the PySpark API documentation and checked.
Closes #27109 from HyukjinKwon/pandas-refactoring.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-08 20:22:50 -05:00
|
|
|
class SparkSession(SparkConversionMixin):
|
2016-05-03 21:05:40 -04:00
|
|
|
"""The entry point to programming Spark with the Dataset and DataFrame API.
|
2016-04-28 13:55:48 -04:00
|
|
|
|
|
|
|
A SparkSession can be used create :class:`DataFrame`, register :class:`DataFrame` as
|
|
|
|
tables, execute SQL over tables, cache tables, and read parquet files.
|
2016-05-03 21:05:40 -04:00
|
|
|
To create a SparkSession, use the following builder pattern:
|
|
|
|
|
2016-07-06 13:45:51 -04:00
|
|
|
>>> spark = SparkSession.builder \\
|
|
|
|
... .master("local") \\
|
|
|
|
... .appName("Word Count") \\
|
|
|
|
... .config("spark.some.config.option", "some-value") \\
|
|
|
|
... .getOrCreate()
|
2017-11-15 11:59:29 -05:00
|
|
|
|
|
|
|
.. autoattribute:: builder
|
|
|
|
:annotation:
|
2016-04-28 13:55:48 -04:00
|
|
|
"""
|
|
|
|
|
2016-05-03 21:05:40 -04:00
|
|
|
class Builder(object):
|
|
|
|
"""Builder for :class:`SparkSession`.
|
|
|
|
"""
|
|
|
|
|
|
|
|
_lock = RLock()
|
|
|
|
_options = {}
|
2018-09-26 21:51:20 -04:00
|
|
|
_sc = None
|
2016-05-03 21:05:40 -04:00
|
|
|
|
|
|
|
@since(2.0)
|
|
|
|
def config(self, key=None, value=None, conf=None):
|
|
|
|
"""Sets a config option. Options set using this method are automatically propagated to
|
|
|
|
both :class:`SparkConf` and :class:`SparkSession`'s own configuration.
|
|
|
|
|
|
|
|
For an existing SparkConf, use `conf` parameter.
|
2016-07-06 13:45:51 -04:00
|
|
|
|
2016-05-03 21:05:40 -04:00
|
|
|
>>> from pyspark.conf import SparkConf
|
|
|
|
>>> SparkSession.builder.config(conf=SparkConf())
|
|
|
|
<pyspark.sql.session...
|
|
|
|
|
|
|
|
For a (key, value) pair, you can omit parameter names.
|
2016-07-06 13:45:51 -04:00
|
|
|
|
2016-05-03 21:05:40 -04:00
|
|
|
>>> SparkSession.builder.config("spark.some.config.option", "some-value")
|
|
|
|
<pyspark.sql.session...
|
|
|
|
|
|
|
|
:param key: a key name string for configuration property
|
|
|
|
:param value: a value for configuration property
|
|
|
|
:param conf: an instance of :class:`SparkConf`
|
|
|
|
"""
|
|
|
|
with self._lock:
|
|
|
|
if conf is None:
|
|
|
|
self._options[key] = str(value)
|
|
|
|
else:
|
|
|
|
for (k, v) in conf.getAll():
|
|
|
|
self._options[k] = v
|
|
|
|
return self
|
|
|
|
|
|
|
|
@since(2.0)
|
|
|
|
def master(self, master):
|
|
|
|
"""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.
|
|
|
|
|
|
|
|
:param master: a url for spark master
|
|
|
|
"""
|
|
|
|
return self.config("spark.master", master)
|
|
|
|
|
|
|
|
@since(2.0)
|
|
|
|
def appName(self, name):
|
|
|
|
"""Sets a name for the application, which will be shown in the Spark web UI.
|
|
|
|
|
2016-05-20 00:53:26 -04:00
|
|
|
If no application name is set, a randomly generated name will be used.
|
|
|
|
|
2016-05-03 21:05:40 -04:00
|
|
|
:param name: an application name
|
|
|
|
"""
|
|
|
|
return self.config("spark.app.name", name)
|
|
|
|
|
|
|
|
@since(2.0)
|
|
|
|
def enableHiveSupport(self):
|
|
|
|
"""Enables Hive support, including connectivity to a persistent Hive metastore, support
|
2019-11-01 14:55:29 -04:00
|
|
|
for Hive SerDes, and Hive user-defined functions.
|
2016-05-03 21:05:40 -04:00
|
|
|
"""
|
|
|
|
return self.config("spark.sql.catalogImplementation", "hive")
|
|
|
|
|
2018-09-26 21:51:20 -04:00
|
|
|
def _sparkContext(self, sc):
|
|
|
|
with self._lock:
|
|
|
|
self._sc = sc
|
|
|
|
return self
|
|
|
|
|
2016-05-03 21:05:40 -04:00
|
|
|
@since(2.0)
|
|
|
|
def getOrCreate(self):
|
2016-05-20 00:53:26 -04:00
|
|
|
"""Gets an existing :class:`SparkSession` or, if there is no existing one, creates a
|
|
|
|
new one based on the options set in this builder.
|
|
|
|
|
2016-05-25 13:49:11 -04:00
|
|
|
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()
|
2018-09-27 00:37:03 -04:00
|
|
|
>>> s1.conf.get("k1") == "v1"
|
2016-05-25 13:49:11 -04:00
|
|
|
True
|
2016-05-20 00:53:26 -04:00
|
|
|
|
|
|
|
In case an existing SparkSession is returned, the config options specified
|
|
|
|
in this builder will be applied to the existing SparkSession.
|
2016-05-25 13:49:11 -04:00
|
|
|
|
|
|
|
>>> 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
|
2016-05-03 21:05:40 -04:00
|
|
|
"""
|
|
|
|
with self._lock:
|
|
|
|
from pyspark.context import SparkContext
|
2016-05-25 13:49:11 -04:00
|
|
|
from pyspark.conf import SparkConf
|
2017-01-12 07:53:31 -05:00
|
|
|
session = SparkSession._instantiatedSession
|
|
|
|
if session is None or session._sc._jsc is None:
|
2018-09-26 21:51:20 -04:00
|
|
|
if self._sc is not None:
|
|
|
|
sc = self._sc
|
|
|
|
else:
|
|
|
|
sparkConf = SparkConf()
|
|
|
|
for key, value in self._options.items():
|
|
|
|
sparkConf.set(key, value)
|
|
|
|
# This SparkContext may be an existing one.
|
2018-09-27 00:37:03 -04:00
|
|
|
sc = SparkContext.getOrCreate(sparkConf)
|
|
|
|
# Do not update `SparkConf` for existing `SparkContext`, as it's shared
|
|
|
|
# by all sessions.
|
2016-05-25 13:49:11 -04:00
|
|
|
session = SparkSession(sc)
|
2016-05-03 21:05:40 -04:00
|
|
|
for key, value in self._options.items():
|
2016-10-11 23:27:08 -04:00
|
|
|
session._jsparkSession.sessionState().conf().setConfString(key, value)
|
2016-05-25 13:49:11 -04:00
|
|
|
return session
|
2016-05-03 21:05:40 -04:00
|
|
|
|
|
|
|
builder = Builder()
|
2019-11-01 14:55:29 -04:00
|
|
|
"""A class attribute having a :class:`Builder` to construct :class:`SparkSession` instances."""
|
2016-05-03 21:05:40 -04:00
|
|
|
|
2017-01-12 07:53:31 -05:00
|
|
|
_instantiatedSession = None
|
2018-10-26 12:40:13 -04:00
|
|
|
_activeSession = None
|
2016-04-28 13:55:48 -04:00
|
|
|
|
|
|
|
def __init__(self, sparkContext, jsparkSession=None):
|
|
|
|
"""Creates a new SparkSession.
|
|
|
|
|
|
|
|
>>> from datetime import datetime
|
2020-08-08 11:51:57 -04:00
|
|
|
>>> from pyspark.sql import Row
|
2016-04-28 13:55:48 -04:00
|
|
|
>>> 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()
|
2016-05-12 03:51:53 -04:00
|
|
|
>>> df.createOrReplaceTempView("allTypes")
|
2016-04-28 13:55:48 -04:00
|
|
|
>>> 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 + CAST(1 AS BIGINT))=2, (d + CAST(1 AS DOUBLE))=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()
|
2020-07-13 22:22:44 -04:00
|
|
|
[(1, 'string', 1.0, 1, True, datetime.datetime(2014, 8, 1, 14, 1, 5), 1, [1, 2, 3])]
|
2016-04-28 13:55:48 -04:00
|
|
|
"""
|
|
|
|
from pyspark.sql.context import SQLContext
|
|
|
|
self._sc = sparkContext
|
|
|
|
self._jsc = self._sc._jsc
|
|
|
|
self._jvm = self._sc._jvm
|
|
|
|
if jsparkSession is None:
|
2018-01-31 06:04:51 -05:00
|
|
|
if self._jvm.SparkSession.getDefaultSession().isDefined() \
|
|
|
|
and not self._jvm.SparkSession.getDefaultSession().get() \
|
|
|
|
.sparkContext().isStopped():
|
|
|
|
jsparkSession = self._jvm.SparkSession.getDefaultSession().get()
|
|
|
|
else:
|
|
|
|
jsparkSession = self._jvm.SparkSession(self._jsc.sc())
|
2016-04-28 13:55:48 -04:00
|
|
|
self._jsparkSession = jsparkSession
|
2016-05-20 00:53:26 -04:00
|
|
|
self._jwrapped = self._jsparkSession.sqlContext()
|
2016-04-28 13:55:48 -04:00
|
|
|
self._wrapped = SQLContext(self._sc, self, self._jwrapped)
|
|
|
|
_monkey_patch_RDD(self)
|
|
|
|
install_exception_handler()
|
2017-01-12 07:53:31 -05:00
|
|
|
# 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._instantiatedSession is None \
|
|
|
|
or SparkSession._instantiatedSession._sc._jsc is None:
|
|
|
|
SparkSession._instantiatedSession = self
|
2018-10-26 12:40:13 -04:00
|
|
|
SparkSession._activeSession = self
|
2018-01-31 06:04:51 -05:00
|
|
|
self._jvm.SparkSession.setDefaultSession(self._jsparkSession)
|
2020-10-16 02:05:17 -04:00
|
|
|
self._jvm.java.lang.Class.forName("org.apache.spark.sql.SparkSession$")\
|
|
|
|
.getDeclaredField("MODULE$")\
|
|
|
|
.get(None)\
|
|
|
|
.setActiveSessionInternal(self._jsparkSession)
|
2016-04-28 13:55:48 -04:00
|
|
|
|
2017-04-18 15:35:27 -04:00
|
|
|
def _repr_html_(self):
|
|
|
|
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_()
|
|
|
|
)
|
|
|
|
|
2016-04-28 13:55:48 -04:00
|
|
|
@since(2.0)
|
|
|
|
def newSession(self):
|
|
|
|
"""
|
|
|
|
Returns a new SparkSession as new session, that has separate SQLConf,
|
2016-05-17 21:01:59 -04:00
|
|
|
registered temporary views and UDFs, but shared SparkContext and
|
2016-04-28 13:55:48 -04:00
|
|
|
table cache.
|
|
|
|
"""
|
|
|
|
return self.__class__(self._sc, self._jsparkSession.newSession())
|
|
|
|
|
2018-10-26 12:40:13 -04:00
|
|
|
@classmethod
|
|
|
|
@since(3.0)
|
|
|
|
def getActiveSession(cls):
|
|
|
|
"""
|
2020-07-02 08:02:00 -04:00
|
|
|
Returns the active SparkSession for the current thread, returned by the builder
|
|
|
|
|
|
|
|
:return: :class:`SparkSession` if an active session exists for the current thread
|
|
|
|
|
2018-10-26 12:40:13 -04:00
|
|
|
>>> s = SparkSession.getActiveSession()
|
|
|
|
>>> l = [('Alice', 1)]
|
|
|
|
>>> rdd = s.sparkContext.parallelize(l)
|
|
|
|
>>> df = s.createDataFrame(rdd, ['name', 'age'])
|
|
|
|
>>> df.select("age").collect()
|
|
|
|
[Row(age=1)]
|
|
|
|
"""
|
|
|
|
from pyspark import SparkContext
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
if sc is None:
|
|
|
|
return None
|
|
|
|
else:
|
|
|
|
if sc._jvm.SparkSession.getActiveSession().isDefined():
|
|
|
|
SparkSession(sc, sc._jvm.SparkSession.getActiveSession().get())
|
|
|
|
return SparkSession._activeSession
|
|
|
|
else:
|
|
|
|
return None
|
|
|
|
|
2016-05-20 02:44:10 -04:00
|
|
|
@property
|
|
|
|
@since(2.0)
|
|
|
|
def sparkContext(self):
|
|
|
|
"""Returns the underlying :class:`SparkContext`."""
|
|
|
|
return self._sc
|
|
|
|
|
2016-07-14 01:30:46 -04:00
|
|
|
@property
|
|
|
|
@since(2.0)
|
|
|
|
def version(self):
|
|
|
|
"""The version of Spark on which this application is running."""
|
|
|
|
return self._jsparkSession.version()
|
|
|
|
|
2016-04-29 12:34:10 -04:00
|
|
|
@property
|
|
|
|
@since(2.0)
|
|
|
|
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.
|
|
|
|
"""
|
|
|
|
if not hasattr(self, "_conf"):
|
|
|
|
self._conf = RuntimeConfig(self._jsparkSession.conf())
|
|
|
|
return self._conf
|
|
|
|
|
|
|
|
@property
|
|
|
|
@since(2.0)
|
|
|
|
def catalog(self):
|
|
|
|
"""Interface through which the user may create, drop, alter or query underlying
|
2019-11-01 14:55:29 -04:00
|
|
|
databases, tables, functions, etc.
|
2017-11-02 10:22:52 -04:00
|
|
|
|
|
|
|
:return: :class:`Catalog`
|
2016-04-29 12:34:10 -04:00
|
|
|
"""
|
2018-01-18 00:51:05 -05:00
|
|
|
from pyspark.sql.catalog import Catalog
|
2016-04-29 12:34:10 -04:00
|
|
|
if not hasattr(self, "_catalog"):
|
|
|
|
self._catalog = Catalog(self)
|
|
|
|
return self._catalog
|
|
|
|
|
2016-04-28 13:55:48 -04:00
|
|
|
@property
|
|
|
|
@since(2.0)
|
|
|
|
def udf(self):
|
|
|
|
"""Returns a :class:`UDFRegistration` for UDF registration.
|
|
|
|
|
|
|
|
:return: :class:`UDFRegistration`
|
|
|
|
"""
|
2018-01-18 00:51:05 -05:00
|
|
|
from pyspark.sql.udf import UDFRegistration
|
|
|
|
return UDFRegistration(self)
|
2016-04-28 13:55:48 -04:00
|
|
|
|
|
|
|
@since(2.0)
|
|
|
|
def range(self, start, end=None, step=1, numPartitions=None):
|
|
|
|
"""
|
2016-07-28 17:57:15 -04:00
|
|
|
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``.
|
2016-04-28 13:55:48 -04:00
|
|
|
|
|
|
|
:param start: the start value
|
|
|
|
:param end: the end value (exclusive)
|
|
|
|
:param step: the incremental step (default: 1)
|
|
|
|
:param numPartitions: the number of partitions of the DataFrame
|
|
|
|
:return: :class:`DataFrame`
|
|
|
|
|
|
|
|
>>> 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)]
|
|
|
|
"""
|
|
|
|
if numPartitions is None:
|
|
|
|
numPartitions = self._sc.defaultParallelism
|
|
|
|
|
|
|
|
if end is None:
|
|
|
|
jdf = self._jsparkSession.range(0, int(start), int(step), int(numPartitions))
|
|
|
|
else:
|
|
|
|
jdf = self._jsparkSession.range(int(start), int(end), int(step), int(numPartitions))
|
|
|
|
|
|
|
|
return DataFrame(jdf, self._wrapped)
|
|
|
|
|
2018-01-08 00:32:05 -05:00
|
|
|
def _inferSchemaFromList(self, data, names=None):
|
2016-04-28 13:55:48 -04:00
|
|
|
"""
|
2020-08-24 17:55:11 -04:00
|
|
|
Infer schema from list of Row, dict, or tuple.
|
2016-04-28 13:55:48 -04:00
|
|
|
|
2020-08-24 17:55:11 -04:00
|
|
|
:param data: list of Row, dict, or tuple
|
2018-01-08 00:32:05 -05:00
|
|
|
:param names: list of column names
|
2016-07-28 17:57:15 -04:00
|
|
|
:return: :class:`pyspark.sql.types.StructType`
|
2016-04-28 13:55:48 -04:00
|
|
|
"""
|
|
|
|
if not data:
|
|
|
|
raise ValueError("can not infer schema from empty dataset")
|
2018-01-08 00:32:05 -05:00
|
|
|
schema = reduce(_merge_type, (_infer_schema(row, names) for row in data))
|
2016-04-28 13:55:48 -04:00
|
|
|
if _has_nulltype(schema):
|
|
|
|
raise ValueError("Some of types cannot be determined after inferring")
|
|
|
|
return schema
|
|
|
|
|
2018-01-08 00:32:05 -05:00
|
|
|
def _inferSchema(self, rdd, samplingRatio=None, names=None):
|
2016-04-28 13:55:48 -04:00
|
|
|
"""
|
2020-08-24 17:55:11 -04:00
|
|
|
Infer schema from an RDD of Row, dict, or tuple.
|
2016-04-28 13:55:48 -04:00
|
|
|
|
2020-08-24 17:55:11 -04:00
|
|
|
:param rdd: an RDD of Row, dict, or tuple
|
2016-04-28 13:55:48 -04:00
|
|
|
:param samplingRatio: sampling ratio, or no sampling (default)
|
2016-07-28 17:57:15 -04:00
|
|
|
:return: :class:`pyspark.sql.types.StructType`
|
2016-04-28 13:55:48 -04:00
|
|
|
"""
|
|
|
|
first = rdd.first()
|
|
|
|
if not first:
|
|
|
|
raise ValueError("The first row in RDD is empty, "
|
|
|
|
"can not infer schema")
|
|
|
|
|
|
|
|
if samplingRatio is None:
|
2018-01-08 00:32:05 -05:00
|
|
|
schema = _infer_schema(first, names=names)
|
2016-04-28 13:55:48 -04:00
|
|
|
if _has_nulltype(schema):
|
|
|
|
for row in rdd.take(100)[1:]:
|
2018-01-08 00:32:05 -05:00
|
|
|
schema = _merge_type(schema, _infer_schema(row, names=names))
|
2016-04-28 13:55:48 -04:00
|
|
|
if not _has_nulltype(schema):
|
|
|
|
break
|
|
|
|
else:
|
|
|
|
raise ValueError("Some of types cannot be determined by the "
|
|
|
|
"first 100 rows, please try again with sampling")
|
|
|
|
else:
|
|
|
|
if samplingRatio < 0.99:
|
|
|
|
rdd = rdd.sample(False, float(samplingRatio))
|
2018-01-08 00:32:05 -05:00
|
|
|
schema = rdd.map(lambda row: _infer_schema(row, names)).reduce(_merge_type)
|
2016-04-28 13:55:48 -04:00
|
|
|
return schema
|
|
|
|
|
|
|
|
def _createFromRDD(self, rdd, schema, samplingRatio):
|
|
|
|
"""
|
|
|
|
Create an RDD for DataFrame from an existing RDD, returns the RDD and schema.
|
|
|
|
"""
|
|
|
|
if schema is None or isinstance(schema, (list, tuple)):
|
2018-01-08 00:32:05 -05:00
|
|
|
struct = self._inferSchema(rdd, samplingRatio, names=schema)
|
2016-04-28 13:55:48 -04:00
|
|
|
converter = _create_converter(struct)
|
|
|
|
rdd = rdd.map(converter)
|
|
|
|
if isinstance(schema, (list, tuple)):
|
|
|
|
for i, name in enumerate(schema):
|
|
|
|
struct.fields[i].name = name
|
|
|
|
struct.names[i] = name
|
|
|
|
schema = struct
|
|
|
|
|
|
|
|
elif not isinstance(schema, StructType):
|
|
|
|
raise TypeError("schema should be StructType or list or None, but got: %s" % schema)
|
|
|
|
|
|
|
|
# convert python objects to sql data
|
|
|
|
rdd = rdd.map(schema.toInternal)
|
|
|
|
return rdd, schema
|
|
|
|
|
|
|
|
def _createFromLocal(self, data, schema):
|
|
|
|
"""
|
2016-06-06 04:35:47 -04:00
|
|
|
Create an RDD for DataFrame from a list or pandas.DataFrame, returns
|
2016-04-28 13:55:48 -04:00
|
|
|
the RDD and schema.
|
|
|
|
"""
|
|
|
|
# make sure data could consumed multiple times
|
|
|
|
if not isinstance(data, list):
|
|
|
|
data = list(data)
|
|
|
|
|
|
|
|
if schema is None or isinstance(schema, (list, tuple)):
|
2018-01-08 00:32:05 -05:00
|
|
|
struct = self._inferSchemaFromList(data, names=schema)
|
2016-08-15 15:41:27 -04:00
|
|
|
converter = _create_converter(struct)
|
|
|
|
data = map(converter, data)
|
2016-04-28 13:55:48 -04:00
|
|
|
if isinstance(schema, (list, tuple)):
|
|
|
|
for i, name in enumerate(schema):
|
|
|
|
struct.fields[i].name = name
|
|
|
|
struct.names[i] = name
|
|
|
|
schema = struct
|
|
|
|
|
2016-08-15 15:41:27 -04:00
|
|
|
elif not isinstance(schema, StructType):
|
2016-04-28 13:55:48 -04:00
|
|
|
raise TypeError("schema should be StructType or list or None, but got: %s" % schema)
|
|
|
|
|
|
|
|
# convert python objects to sql data
|
|
|
|
data = [schema.toInternal(row) for row in data]
|
|
|
|
return self._sc.parallelize(data), schema
|
|
|
|
|
[SPARK-16451][REPL] Fail shell if SparkSession fails to start.
Currently, in spark-shell, if the session fails to start, the
user sees a bunch of unrelated errors which are caused by code
in the shell initialization that references the "spark" variable,
which does not exist in that case. Things like:
```
<console>:14: error: not found: value spark
import spark.sql
```
The user is also left with a non-working shell (unless they want
to just write non-Spark Scala or Python code, that is).
This change fails the whole shell session at the point where the
failure occurs, so that the last error message is the one with
the actual information about the failure.
For the python error handling, I moved the session initialization code
to session.py, so that traceback.print_exc() only shows the last error.
Otherwise, the printed exception would contain all previous exceptions
with a message "During handling of the above exception, another
exception occurred", making the actual error kinda hard to parse.
Tested with spark-shell, pyspark (with 2.7 and 3.5), by forcing an
error during SparkContext initialization.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes #21368 from vanzin/SPARK-16451.
2018-06-04 21:29:29 -04:00
|
|
|
@staticmethod
|
|
|
|
def _create_shell_session():
|
|
|
|
"""
|
|
|
|
Initialize a 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()
|
[SPARK-24563][PYTHON] Catch TypeError when testing existence of HiveConf when creating pysp…
…ark shell
## What changes were proposed in this pull request?
This PR catches TypeError when testing existence of HiveConf when creating pyspark shell
## How was this patch tested?
Manually tested. Here are the manual test cases:
Build with hive:
```
(pyarrow-dev) Lis-MacBook-Pro:spark icexelloss$ bin/pyspark
Python 3.6.5 | packaged by conda-forge | (default, Apr 6 2018, 13:44:09)
[GCC 4.2.1 Compatible Apple LLVM 6.1.0 (clang-602.0.53)] on darwin
Type "help", "copyright", "credits" or "license" for more information.
18/06/14 14:55:41 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
Welcome to
____ __
/ __/__ ___ _____/ /__
_\ \/ _ \/ _ `/ __/ '_/
/__ / .__/\_,_/_/ /_/\_\ version 2.4.0-SNAPSHOT
/_/
Using Python version 3.6.5 (default, Apr 6 2018 13:44:09)
SparkSession available as 'spark'.
>>> spark.conf.get('spark.sql.catalogImplementation')
'hive'
```
Build without hive:
```
(pyarrow-dev) Lis-MacBook-Pro:spark icexelloss$ bin/pyspark
Python 3.6.5 | packaged by conda-forge | (default, Apr 6 2018, 13:44:09)
[GCC 4.2.1 Compatible Apple LLVM 6.1.0 (clang-602.0.53)] on darwin
Type "help", "copyright", "credits" or "license" for more information.
18/06/14 15:04:52 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
Welcome to
____ __
/ __/__ ___ _____/ /__
_\ \/ _ \/ _ `/ __/ '_/
/__ / .__/\_,_/_/ /_/\_\ version 2.4.0-SNAPSHOT
/_/
Using Python version 3.6.5 (default, Apr 6 2018 13:44:09)
SparkSession available as 'spark'.
>>> spark.conf.get('spark.sql.catalogImplementation')
'in-memory'
```
Failed to start shell:
```
(pyarrow-dev) Lis-MacBook-Pro:spark icexelloss$ bin/pyspark
Python 3.6.5 | packaged by conda-forge | (default, Apr 6 2018, 13:44:09)
[GCC 4.2.1 Compatible Apple LLVM 6.1.0 (clang-602.0.53)] on darwin
Type "help", "copyright", "credits" or "license" for more information.
18/06/14 15:07:53 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
/Users/icexelloss/workspace/spark/python/pyspark/shell.py:45: UserWarning: Failed to initialize Spark session.
warnings.warn("Failed to initialize Spark session.")
Traceback (most recent call last):
File "/Users/icexelloss/workspace/spark/python/pyspark/shell.py", line 41, in <module>
spark = SparkSession._create_shell_session()
File "/Users/icexelloss/workspace/spark/python/pyspark/sql/session.py", line 581, in _create_shell_session
return SparkSession.builder.getOrCreate()
File "/Users/icexelloss/workspace/spark/python/pyspark/sql/session.py", line 168, in getOrCreate
raise py4j.protocol.Py4JError("Fake Py4JError")
py4j.protocol.Py4JError: Fake Py4JError
(pyarrow-dev) Lis-MacBook-Pro:spark icexelloss$
```
Author: Li Jin <ice.xelloss@gmail.com>
Closes #21569 from icexelloss/SPARK-24563-fix-pyspark-shell-without-hive.
2018-06-14 16:16:20 -04:00
|
|
|
except (py4j.protocol.Py4JError, TypeError):
|
[SPARK-16451][REPL] Fail shell if SparkSession fails to start.
Currently, in spark-shell, if the session fails to start, the
user sees a bunch of unrelated errors which are caused by code
in the shell initialization that references the "spark" variable,
which does not exist in that case. Things like:
```
<console>:14: error: not found: value spark
import spark.sql
```
The user is also left with a non-working shell (unless they want
to just write non-Spark Scala or Python code, that is).
This change fails the whole shell session at the point where the
failure occurs, so that the last error message is the one with
the actual information about the failure.
For the python error handling, I moved the session initialization code
to session.py, so that traceback.print_exc() only shows the last error.
Otherwise, the printed exception would contain all previous exceptions
with a message "During handling of the above exception, another
exception occurred", making the actual error kinda hard to parse.
Tested with spark-shell, pyspark (with 2.7 and 3.5), by forcing an
error during SparkContext initialization.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes #21368 from vanzin/SPARK-16451.
2018-06-04 21:29:29 -04:00
|
|
|
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()
|
|
|
|
|
2016-04-28 13:55:48 -04:00
|
|
|
@since(2.0)
|
2016-08-15 15:41:27 -04:00
|
|
|
def createDataFrame(self, data, schema=None, samplingRatio=None, verifySchema=True):
|
2016-04-28 13:55:48 -04:00
|
|
|
"""
|
|
|
|
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)
|
2019-11-01 14:55:29 -04:00
|
|
|
from ``data``, which should be an RDD of either :class:`Row`,
|
|
|
|
:class:`namedtuple`, or :class:`dict`.
|
2016-04-28 13:55:48 -04:00
|
|
|
|
2016-07-29 17:07:03 -04:00
|
|
|
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
|
2016-07-28 17:57:15 -04:00
|
|
|
:class:`pyspark.sql.types.StructType`, it will be wrapped into a
|
2019-11-01 14:55:29 -04:00
|
|
|
: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.
|
2016-04-28 13:55:48 -04:00
|
|
|
|
|
|
|
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``.
|
|
|
|
|
2019-11-01 14:55:29 -04:00
|
|
|
:param data: an RDD of any kind of SQL data representation (e.g. row, tuple, int, boolean,
|
|
|
|
etc.), :class:`list`, or :class:`pandas.DataFrame`.
|
2016-07-29 17:07:03 -04:00
|
|
|
:param schema: a :class:`pyspark.sql.types.DataType` or a datatype string or a list of
|
2016-07-28 17:57:15 -04:00
|
|
|
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 ``IntegerType``.
|
2016-04-28 13:55:48 -04:00
|
|
|
:param samplingRatio: the sample ratio of rows used for inferring
|
2016-08-15 15:41:27 -04:00
|
|
|
:param verifySchema: verify data types of every row against schema.
|
2016-04-28 13:55:48 -04:00
|
|
|
:return: :class:`DataFrame`
|
|
|
|
|
2016-08-15 15:41:27 -04:00
|
|
|
.. versionchanged:: 2.1
|
|
|
|
Added verifySchema.
|
2016-04-28 13:55:48 -04:00
|
|
|
|
2019-06-02 21:01:37 -04:00
|
|
|
.. note:: Usage with spark.sql.execution.arrow.pyspark.enabled=True is experimental.
|
2018-05-28 00:56:05 -04:00
|
|
|
|
2016-04-28 13:55:48 -04:00
|
|
|
>>> l = [('Alice', 1)]
|
|
|
|
>>> spark.createDataFrame(l).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(_1='Alice', _2=1)]
|
2016-04-28 13:55:48 -04:00
|
|
|
>>> spark.createDataFrame(l, ['name', 'age']).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(name='Alice', age=1)]
|
2016-04-28 13:55:48 -04:00
|
|
|
|
|
|
|
>>> d = [{'name': 'Alice', 'age': 1}]
|
|
|
|
>>> spark.createDataFrame(d).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(age=1, name='Alice')]
|
2016-04-28 13:55:48 -04:00
|
|
|
|
|
|
|
>>> rdd = sc.parallelize(l)
|
|
|
|
>>> spark.createDataFrame(rdd).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(_1='Alice', _2=1)]
|
2016-04-28 13:55:48 -04:00
|
|
|
>>> df = spark.createDataFrame(rdd, ['name', 'age'])
|
|
|
|
>>> df.collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(name='Alice', age=1)]
|
2016-04-28 13:55:48 -04:00
|
|
|
|
|
|
|
>>> from pyspark.sql import Row
|
|
|
|
>>> Person = Row('name', 'age')
|
|
|
|
>>> person = rdd.map(lambda r: Person(*r))
|
|
|
|
>>> df2 = spark.createDataFrame(person)
|
|
|
|
>>> df2.collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(name='Alice', age=1)]
|
2016-04-28 13:55:48 -04:00
|
|
|
|
|
|
|
>>> from pyspark.sql.types import *
|
|
|
|
>>> schema = StructType([
|
|
|
|
... StructField("name", StringType(), True),
|
|
|
|
... StructField("age", IntegerType(), True)])
|
|
|
|
>>> df3 = spark.createDataFrame(rdd, schema)
|
|
|
|
>>> df3.collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(name='Alice', age=1)]
|
2016-04-28 13:55:48 -04:00
|
|
|
|
|
|
|
>>> spark.createDataFrame(df.toPandas()).collect() # doctest: +SKIP
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(name='Alice', age=1)]
|
2016-04-28 13:55:48 -04:00
|
|
|
>>> spark.createDataFrame(pandas.DataFrame([[1, 2]])).collect() # doctest: +SKIP
|
|
|
|
[Row(0=1, 1=2)]
|
|
|
|
|
|
|
|
>>> spark.createDataFrame(rdd, "a: string, b: int").collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(a='Alice', b=1)]
|
2016-04-28 13:55:48 -04:00
|
|
|
>>> 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: ...
|
|
|
|
"""
|
2018-10-26 12:40:13 -04:00
|
|
|
SparkSession._activeSession = self
|
2020-10-16 02:05:17 -04:00
|
|
|
self._jvm.java.lang.Class.forName("org.apache.spark.sql.SparkSession$")\
|
|
|
|
.getDeclaredField("MODULE$")\
|
|
|
|
.get(None)\
|
|
|
|
.setActiveSessionInternal(self._jsparkSession)
|
2016-04-28 13:55:48 -04:00
|
|
|
if isinstance(data, DataFrame):
|
|
|
|
raise TypeError("data is already a DataFrame")
|
|
|
|
|
2020-07-13 22:22:44 -04:00
|
|
|
if isinstance(schema, str):
|
2016-04-28 13:55:48 -04:00
|
|
|
schema = _parse_datatype_string(schema)
|
2017-11-15 09:35:13 -05:00
|
|
|
elif isinstance(schema, (list, tuple)):
|
|
|
|
# Must re-encode any unicode strings to be consistent with StructField names
|
|
|
|
schema = [x.encode('utf-8') if not isinstance(x, str) else x for x in schema]
|
2016-04-28 13:55:48 -04:00
|
|
|
|
|
|
|
try:
|
|
|
|
import pandas
|
|
|
|
has_pandas = True
|
|
|
|
except Exception:
|
|
|
|
has_pandas = False
|
|
|
|
if has_pandas and isinstance(data, pandas.DataFrame):
|
[SPARK-30434][PYTHON][SQL] Move pandas related functionalities into 'pandas' sub-package
### What changes were proposed in this pull request?
This PR proposes to move pandas related functionalities into pandas package. Namely:
```bash
pyspark/sql/pandas
├── __init__.py
├── conversion.py # Conversion between pandas <> PySpark DataFrames
├── functions.py # pandas_udf
├── group_ops.py # Grouped UDF / Cogrouped UDF + groupby.apply, groupby.cogroup.apply
├── map_ops.py # Map Iter UDF + mapInPandas
├── serializers.py # pandas <> PyArrow serializers
├── types.py # Type utils between pandas <> PyArrow
└── utils.py # Version requirement checks
```
In order to separately locate `groupby.apply`, `groupby.cogroup.apply`, `mapInPandas`, `toPandas`, and `createDataFrame(pdf)` under `pandas` sub-package, I had to use a mix-in approach which Scala side uses often by `trait`, and also pandas itself uses this approach (see `IndexOpsMixin` as an example) to group related functionalities. Currently, you can think it's like Scala's self typed trait. See the structure below:
```python
class PandasMapOpsMixin(object):
def mapInPandas(self, ...):
...
return ...
# other Pandas <> PySpark APIs
```
```python
class DataFrame(PandasMapOpsMixin):
# other DataFrame APIs equivalent to Scala side.
```
Yes, This is a big PR but they are mostly just moving around except one case `createDataFrame` which I had to split the methods.
### Why are the changes needed?
There are pandas functionalities here and there and I myself gets lost where it was. Also, when you have to make a change commonly for all of pandas related features, it's almost impossible now.
Also, after this change, `DataFrame` and `SparkSession` become more consistent with Scala side since pandas is specific to Python, and this change separates pandas-specific APIs away from `DataFrame` or `SparkSession`.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Existing tests should cover. Also, I manually built the PySpark API documentation and checked.
Closes #27109 from HyukjinKwon/pandas-refactoring.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-08 20:22:50 -05:00
|
|
|
# Create a DataFrame from pandas DataFrame.
|
|
|
|
return super(SparkSession, self).createDataFrame(
|
2020-01-15 22:39:44 -05:00
|
|
|
data, schema, samplingRatio, verifySchema)
|
|
|
|
return self._create_dataframe(data, schema, samplingRatio, verifySchema)
|
2016-04-28 13:55:48 -04:00
|
|
|
|
2020-01-15 22:39:44 -05:00
|
|
|
def _create_dataframe(self, data, schema, samplingRatio, verifySchema):
|
2016-04-28 13:55:48 -04:00
|
|
|
if isinstance(schema, StructType):
|
[SPARK-19507][SPARK-21296][PYTHON] Avoid per-record type dispatch in schema verification and improve exception message
## What changes were proposed in this pull request?
**Context**
While reviewing https://github.com/apache/spark/pull/17227, I realised here we type-dispatch per record. The PR itself is fine in terms of performance as is but this prints a prefix, `"obj"` in exception message as below:
```
from pyspark.sql.types import *
schema = StructType([StructField('s', IntegerType(), nullable=False)])
spark.createDataFrame([["1"]], schema)
...
TypeError: obj.s: IntegerType can not accept object '1' in type <type 'str'>
```
I suggested to get rid of this but during investigating this, I realised my approach might bring a performance regression as it is a hot path.
Only for SPARK-19507 and https://github.com/apache/spark/pull/17227, It needs more changes to cleanly get rid of the prefix and I rather decided to fix both issues together.
**Propersal**
This PR tried to
- get rid of per-record type dispatch as we do in many code paths in Scala so that it improves the performance (roughly ~25% improvement) - SPARK-21296
This was tested with a simple code `spark.createDataFrame(range(1000000), "int")`. However, I am quite sure the actual improvement in practice is larger than this, in particular, when the schema is complicated.
- improve error message in exception describing field information as prose - SPARK-19507
## How was this patch tested?
Manually tested and unit tests were added in `python/pyspark/sql/tests.py`.
Benchmark - codes: https://gist.github.com/HyukjinKwon/c3397469c56cb26c2d7dd521ed0bc5a3
Error message - codes: https://gist.github.com/HyukjinKwon/b1b2c7f65865444c4a8836435100e398
**Before**
Benchmark:
- Results: https://gist.github.com/HyukjinKwon/4a291dab45542106301a0c1abcdca924
Error message
- Results: https://gist.github.com/HyukjinKwon/57b1916395794ce924faa32b14a3fe19
**After**
Benchmark
- Results: https://gist.github.com/HyukjinKwon/21496feecc4a920e50c4e455f836266e
Error message
- Results: https://gist.github.com/HyukjinKwon/7a494e4557fe32a652ce1236e504a395
Closes #17227
Author: hyukjinkwon <gurwls223@gmail.com>
Author: David Gingrich <david@textio.com>
Closes #18521 from HyukjinKwon/python-type-dispatch.
2017-07-04 08:45:58 -04:00
|
|
|
verify_func = _make_type_verifier(schema) if verifySchema else lambda _: True
|
|
|
|
|
2016-04-28 13:55:48 -04:00
|
|
|
def prepare(obj):
|
[SPARK-19507][SPARK-21296][PYTHON] Avoid per-record type dispatch in schema verification and improve exception message
## What changes were proposed in this pull request?
**Context**
While reviewing https://github.com/apache/spark/pull/17227, I realised here we type-dispatch per record. The PR itself is fine in terms of performance as is but this prints a prefix, `"obj"` in exception message as below:
```
from pyspark.sql.types import *
schema = StructType([StructField('s', IntegerType(), nullable=False)])
spark.createDataFrame([["1"]], schema)
...
TypeError: obj.s: IntegerType can not accept object '1' in type <type 'str'>
```
I suggested to get rid of this but during investigating this, I realised my approach might bring a performance regression as it is a hot path.
Only for SPARK-19507 and https://github.com/apache/spark/pull/17227, It needs more changes to cleanly get rid of the prefix and I rather decided to fix both issues together.
**Propersal**
This PR tried to
- get rid of per-record type dispatch as we do in many code paths in Scala so that it improves the performance (roughly ~25% improvement) - SPARK-21296
This was tested with a simple code `spark.createDataFrame(range(1000000), "int")`. However, I am quite sure the actual improvement in practice is larger than this, in particular, when the schema is complicated.
- improve error message in exception describing field information as prose - SPARK-19507
## How was this patch tested?
Manually tested and unit tests were added in `python/pyspark/sql/tests.py`.
Benchmark - codes: https://gist.github.com/HyukjinKwon/c3397469c56cb26c2d7dd521ed0bc5a3
Error message - codes: https://gist.github.com/HyukjinKwon/b1b2c7f65865444c4a8836435100e398
**Before**
Benchmark:
- Results: https://gist.github.com/HyukjinKwon/4a291dab45542106301a0c1abcdca924
Error message
- Results: https://gist.github.com/HyukjinKwon/57b1916395794ce924faa32b14a3fe19
**After**
Benchmark
- Results: https://gist.github.com/HyukjinKwon/21496feecc4a920e50c4e455f836266e
Error message
- Results: https://gist.github.com/HyukjinKwon/7a494e4557fe32a652ce1236e504a395
Closes #17227
Author: hyukjinkwon <gurwls223@gmail.com>
Author: David Gingrich <david@textio.com>
Closes #18521 from HyukjinKwon/python-type-dispatch.
2017-07-04 08:45:58 -04:00
|
|
|
verify_func(obj)
|
2016-04-28 13:55:48 -04:00
|
|
|
return obj
|
|
|
|
elif isinstance(schema, DataType):
|
2016-08-15 15:41:27 -04:00
|
|
|
dataType = schema
|
|
|
|
schema = StructType().add("value", schema)
|
2016-04-28 13:55:48 -04:00
|
|
|
|
[SPARK-19507][SPARK-21296][PYTHON] Avoid per-record type dispatch in schema verification and improve exception message
## What changes were proposed in this pull request?
**Context**
While reviewing https://github.com/apache/spark/pull/17227, I realised here we type-dispatch per record. The PR itself is fine in terms of performance as is but this prints a prefix, `"obj"` in exception message as below:
```
from pyspark.sql.types import *
schema = StructType([StructField('s', IntegerType(), nullable=False)])
spark.createDataFrame([["1"]], schema)
...
TypeError: obj.s: IntegerType can not accept object '1' in type <type 'str'>
```
I suggested to get rid of this but during investigating this, I realised my approach might bring a performance regression as it is a hot path.
Only for SPARK-19507 and https://github.com/apache/spark/pull/17227, It needs more changes to cleanly get rid of the prefix and I rather decided to fix both issues together.
**Propersal**
This PR tried to
- get rid of per-record type dispatch as we do in many code paths in Scala so that it improves the performance (roughly ~25% improvement) - SPARK-21296
This was tested with a simple code `spark.createDataFrame(range(1000000), "int")`. However, I am quite sure the actual improvement in practice is larger than this, in particular, when the schema is complicated.
- improve error message in exception describing field information as prose - SPARK-19507
## How was this patch tested?
Manually tested and unit tests were added in `python/pyspark/sql/tests.py`.
Benchmark - codes: https://gist.github.com/HyukjinKwon/c3397469c56cb26c2d7dd521ed0bc5a3
Error message - codes: https://gist.github.com/HyukjinKwon/b1b2c7f65865444c4a8836435100e398
**Before**
Benchmark:
- Results: https://gist.github.com/HyukjinKwon/4a291dab45542106301a0c1abcdca924
Error message
- Results: https://gist.github.com/HyukjinKwon/57b1916395794ce924faa32b14a3fe19
**After**
Benchmark
- Results: https://gist.github.com/HyukjinKwon/21496feecc4a920e50c4e455f836266e
Error message
- Results: https://gist.github.com/HyukjinKwon/7a494e4557fe32a652ce1236e504a395
Closes #17227
Author: hyukjinkwon <gurwls223@gmail.com>
Author: David Gingrich <david@textio.com>
Closes #18521 from HyukjinKwon/python-type-dispatch.
2017-07-04 08:45:58 -04:00
|
|
|
verify_func = _make_type_verifier(
|
|
|
|
dataType, name="field value") if verifySchema else lambda _: True
|
|
|
|
|
2016-04-28 13:55:48 -04:00
|
|
|
def prepare(obj):
|
[SPARK-19507][SPARK-21296][PYTHON] Avoid per-record type dispatch in schema verification and improve exception message
## What changes were proposed in this pull request?
**Context**
While reviewing https://github.com/apache/spark/pull/17227, I realised here we type-dispatch per record. The PR itself is fine in terms of performance as is but this prints a prefix, `"obj"` in exception message as below:
```
from pyspark.sql.types import *
schema = StructType([StructField('s', IntegerType(), nullable=False)])
spark.createDataFrame([["1"]], schema)
...
TypeError: obj.s: IntegerType can not accept object '1' in type <type 'str'>
```
I suggested to get rid of this but during investigating this, I realised my approach might bring a performance regression as it is a hot path.
Only for SPARK-19507 and https://github.com/apache/spark/pull/17227, It needs more changes to cleanly get rid of the prefix and I rather decided to fix both issues together.
**Propersal**
This PR tried to
- get rid of per-record type dispatch as we do in many code paths in Scala so that it improves the performance (roughly ~25% improvement) - SPARK-21296
This was tested with a simple code `spark.createDataFrame(range(1000000), "int")`. However, I am quite sure the actual improvement in practice is larger than this, in particular, when the schema is complicated.
- improve error message in exception describing field information as prose - SPARK-19507
## How was this patch tested?
Manually tested and unit tests were added in `python/pyspark/sql/tests.py`.
Benchmark - codes: https://gist.github.com/HyukjinKwon/c3397469c56cb26c2d7dd521ed0bc5a3
Error message - codes: https://gist.github.com/HyukjinKwon/b1b2c7f65865444c4a8836435100e398
**Before**
Benchmark:
- Results: https://gist.github.com/HyukjinKwon/4a291dab45542106301a0c1abcdca924
Error message
- Results: https://gist.github.com/HyukjinKwon/57b1916395794ce924faa32b14a3fe19
**After**
Benchmark
- Results: https://gist.github.com/HyukjinKwon/21496feecc4a920e50c4e455f836266e
Error message
- Results: https://gist.github.com/HyukjinKwon/7a494e4557fe32a652ce1236e504a395
Closes #17227
Author: hyukjinkwon <gurwls223@gmail.com>
Author: David Gingrich <david@textio.com>
Closes #18521 from HyukjinKwon/python-type-dispatch.
2017-07-04 08:45:58 -04:00
|
|
|
verify_func(obj)
|
2016-08-15 15:41:27 -04:00
|
|
|
return obj,
|
2016-04-28 13:55:48 -04:00
|
|
|
else:
|
|
|
|
prepare = lambda obj: obj
|
|
|
|
|
|
|
|
if isinstance(data, RDD):
|
|
|
|
rdd, schema = self._createFromRDD(data.map(prepare), schema, samplingRatio)
|
|
|
|
else:
|
|
|
|
rdd, schema = self._createFromLocal(map(prepare, data), schema)
|
|
|
|
jrdd = self._jvm.SerDeUtil.toJavaArray(rdd._to_java_object_rdd())
|
|
|
|
jdf = self._jsparkSession.applySchemaToPythonRDD(jrdd.rdd(), schema.json())
|
|
|
|
df = DataFrame(jdf, self._wrapped)
|
|
|
|
df._schema = schema
|
|
|
|
return df
|
|
|
|
|
|
|
|
@since(2.0)
|
|
|
|
def sql(self, sqlQuery):
|
|
|
|
"""Returns a :class:`DataFrame` representing the result of the given query.
|
|
|
|
|
|
|
|
:return: :class:`DataFrame`
|
|
|
|
|
2016-05-12 03:51:53 -04:00
|
|
|
>>> df.createOrReplaceTempView("table1")
|
2016-04-28 13:55:48 -04:00
|
|
|
>>> df2 = spark.sql("SELECT field1 AS f1, field2 as f2 from table1")
|
|
|
|
>>> df2.collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(f1=1, f2='row1'), Row(f1=2, f2='row2'), Row(f1=3, f2='row3')]
|
2016-04-28 13:55:48 -04:00
|
|
|
"""
|
|
|
|
return DataFrame(self._jsparkSession.sql(sqlQuery), self._wrapped)
|
|
|
|
|
|
|
|
@since(2.0)
|
|
|
|
def table(self, tableName):
|
|
|
|
"""Returns the specified table as a :class:`DataFrame`.
|
|
|
|
|
|
|
|
:return: :class:`DataFrame`
|
|
|
|
|
2016-05-12 03:51:53 -04:00
|
|
|
>>> df.createOrReplaceTempView("table1")
|
2016-04-28 13:55:48 -04:00
|
|
|
>>> df2 = spark.table("table1")
|
|
|
|
>>> sorted(df.collect()) == sorted(df2.collect())
|
|
|
|
True
|
|
|
|
"""
|
|
|
|
return DataFrame(self._jsparkSession.table(tableName), self._wrapped)
|
|
|
|
|
|
|
|
@property
|
|
|
|
@since(2.0)
|
|
|
|
def read(self):
|
|
|
|
"""
|
|
|
|
Returns a :class:`DataFrameReader` that can be used to read data
|
|
|
|
in as a :class:`DataFrame`.
|
|
|
|
|
|
|
|
:return: :class:`DataFrameReader`
|
|
|
|
"""
|
|
|
|
return DataFrameReader(self._wrapped)
|
2016-04-29 12:34:10 -04:00
|
|
|
|
2016-06-14 20:58:45 -04:00
|
|
|
@property
|
|
|
|
@since(2.0)
|
|
|
|
def readStream(self):
|
|
|
|
"""
|
|
|
|
Returns a :class:`DataStreamReader` that can be used to read data streams
|
|
|
|
as a streaming :class:`DataFrame`.
|
|
|
|
|
2017-05-26 16:33:23 -04:00
|
|
|
.. note:: Evolving.
|
2016-06-14 20:58:45 -04:00
|
|
|
|
|
|
|
:return: :class:`DataStreamReader`
|
|
|
|
"""
|
|
|
|
return DataStreamReader(self._wrapped)
|
|
|
|
|
2016-06-14 05:12:29 -04:00
|
|
|
@property
|
|
|
|
@since(2.0)
|
|
|
|
def streams(self):
|
2016-06-15 13:46:02 -04:00
|
|
|
"""Returns a :class:`StreamingQueryManager` that allows managing all the
|
2019-11-01 14:55:29 -04:00
|
|
|
:class:`StreamingQuery` instances active on `this` context.
|
2016-06-14 05:12:29 -04:00
|
|
|
|
2017-05-26 16:33:23 -04:00
|
|
|
.. note:: Evolving.
|
2016-06-14 20:58:45 -04:00
|
|
|
|
2016-06-15 13:46:02 -04:00
|
|
|
:return: :class:`StreamingQueryManager`
|
2016-06-14 05:12:29 -04:00
|
|
|
"""
|
2016-06-15 13:46:02 -04:00
|
|
|
from pyspark.sql.streaming import StreamingQueryManager
|
|
|
|
return StreamingQueryManager(self._jsparkSession.streams())
|
2016-06-14 05:12:29 -04:00
|
|
|
|
2016-05-03 21:05:40 -04:00
|
|
|
@since(2.0)
|
|
|
|
def stop(self):
|
|
|
|
"""Stop the underlying :class:`SparkContext`.
|
|
|
|
"""
|
2020-02-19 22:21:24 -05:00
|
|
|
from pyspark.sql.context import SQLContext
|
2016-05-03 21:05:40 -04:00
|
|
|
self._sc.stop()
|
2018-01-31 06:04:51 -05:00
|
|
|
# We should clean the default session up. See SPARK-23228.
|
|
|
|
self._jvm.SparkSession.clearDefaultSession()
|
2020-10-16 02:05:17 -04:00
|
|
|
self._jvm.java.lang.Class.forName("org.apache.spark.sql.SparkSession$")\
|
|
|
|
.getDeclaredField("MODULE$")\
|
|
|
|
.get(None)\
|
|
|
|
.clearActiveSessionInternal()
|
2017-01-12 07:53:31 -05:00
|
|
|
SparkSession._instantiatedSession = None
|
2018-10-26 12:40:13 -04:00
|
|
|
SparkSession._activeSession = None
|
2020-02-19 22:21:24 -05:00
|
|
|
SQLContext._instantiatedContext = None
|
2016-05-03 21:05:40 -04:00
|
|
|
|
2016-06-18 01:57:38 -04:00
|
|
|
@since(2.0)
|
|
|
|
def __enter__(self):
|
|
|
|
"""
|
|
|
|
Enable 'with SparkSession.builder.(...).getOrCreate() as session: app' syntax.
|
|
|
|
"""
|
|
|
|
return self
|
|
|
|
|
|
|
|
@since(2.0)
|
|
|
|
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()
|
|
|
|
|
2016-04-29 12:34:10 -04:00
|
|
|
|
|
|
|
def _test():
|
|
|
|
import os
|
|
|
|
import doctest
|
|
|
|
from pyspark.context import SparkContext
|
|
|
|
from pyspark.sql import Row
|
|
|
|
import pyspark.sql.session
|
|
|
|
|
|
|
|
os.chdir(os.environ["SPARK_HOME"])
|
|
|
|
|
|
|
|
globs = pyspark.sql.session.__dict__.copy()
|
|
|
|
sc = SparkContext('local[4]', 'PythonTest')
|
|
|
|
globs['sc'] = sc
|
|
|
|
globs['spark'] = SparkSession(sc)
|
|
|
|
globs['rdd'] = rdd = sc.parallelize(
|
|
|
|
[Row(field1=1, field2="row1"),
|
|
|
|
Row(field1=2, field2="row2"),
|
|
|
|
Row(field1=3, field2="row3")])
|
|
|
|
globs['df'] = rdd.toDF()
|
|
|
|
(failure_count, test_count) = doctest.testmod(
|
|
|
|
pyspark.sql.session, globs=globs,
|
|
|
|
optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE)
|
|
|
|
globs['sc'].stop()
|
|
|
|
if failure_count:
|
2018-03-08 06:38:34 -05:00
|
|
|
sys.exit(-1)
|
2016-04-29 12:34:10 -04:00
|
|
|
|
|
|
|
if __name__ == "__main__":
|
|
|
|
_test()
|