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.
|
|
|
|
#
|
|
|
|
|
|
|
|
from __future__ import print_function
|
|
|
|
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
|
|
|
|
|
|
|
if sys.version >= '3':
|
|
|
|
basestring = unicode = str
|
2017-11-07 15:32:37 -05:00
|
|
|
xrange = range
|
2016-04-28 13:55:48 -04:00
|
|
|
else:
|
2017-11-12 23:16:01 -05:00
|
|
|
from itertools import izip as zip, imap as map
|
2016-04-28 13:55:48 -04:00
|
|
|
|
|
|
|
from pyspark import since
|
|
|
|
from pyspark.rdd import RDD, ignore_unicode_prefix
|
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
|
2016-06-29 01:07:11 -04:00
|
|
|
from pyspark.sql.readwriter import DataFrameReader
|
|
|
|
from pyspark.sql.streaming import DataStreamReader
|
2017-11-28 03:45:22 -05:00
|
|
|
from pyspark.sql.types import Row, DataType, StringType, StructType, TimestampType, \
|
|
|
|
_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
|
2016-04-28 13:55:48 -04:00
|
|
|
:param samplingRatio: the sample ratio of rows used for inferring
|
|
|
|
:return: a DataFrame
|
|
|
|
|
|
|
|
>>> rdd.toDF().collect()
|
|
|
|
[Row(name=u'Alice', age=1)]
|
|
|
|
"""
|
|
|
|
return sparkSession.createDataFrame(self, schema, sampleRatio)
|
|
|
|
|
|
|
|
RDD.toDF = toDF
|
|
|
|
|
|
|
|
|
|
|
|
class SparkSession(object):
|
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
|
|
|
|
for Hive serdes, and Hive user-defined functions.
|
|
|
|
"""
|
|
|
|
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()
|
2017-11-15 11:59:29 -05: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
|
|
|
|
|
|
|
@ignore_unicode_prefix
|
|
|
|
def __init__(self, sparkContext, jsparkSession=None):
|
|
|
|
"""Creates a new SparkSession.
|
|
|
|
|
|
|
|
>>> from datetime import datetime
|
|
|
|
>>> 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()
|
|
|
|
[(1, u'string', 1.0, 1, True, datetime.datetime(2014, 8, 1, 14, 1, 5), 1, [1, 2, 3])]
|
|
|
|
"""
|
|
|
|
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)
|
2018-10-26 12:40:13 -04:00
|
|
|
self._jvm.SparkSession.setActiveSession(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):
|
|
|
|
"""
|
|
|
|
Returns the active SparkSession for the current thread, returned by the builder.
|
|
|
|
>>> 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
|
|
|
|
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
|
|
|
"""
|
|
|
|
Infer schema from list of Row or tuple.
|
|
|
|
|
|
|
|
:param data: list of Row 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")
|
|
|
|
first = data[0]
|
|
|
|
if type(first) is dict:
|
|
|
|
warnings.warn("inferring schema from dict is deprecated,"
|
|
|
|
"please use pyspark.sql.Row instead")
|
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
|
|
|
"""
|
|
|
|
Infer schema from an RDD of Row or tuple.
|
|
|
|
|
|
|
|
:param rdd: an RDD of Row or tuple
|
|
|
|
: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 type(first) is dict:
|
|
|
|
warnings.warn("Using RDD of dict to inferSchema is deprecated. "
|
|
|
|
"Use pyspark.sql.Row instead")
|
|
|
|
|
|
|
|
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
|
|
|
|
|
2017-11-12 23:16:01 -05:00
|
|
|
def _get_numpy_record_dtype(self, rec):
|
2017-11-07 15:32:37 -05:00
|
|
|
"""
|
|
|
|
Used when converting a pandas.DataFrame to Spark using to_records(), this will correct
|
2017-11-12 23:16:01 -05:00
|
|
|
the dtypes of fields in a record so they can be properly loaded into Spark.
|
|
|
|
:param rec: a numpy record to check field dtypes
|
|
|
|
:return corrected dtype for a numpy.record or None if no correction needed
|
2017-11-07 15:32:37 -05:00
|
|
|
"""
|
|
|
|
import numpy as np
|
|
|
|
cur_dtypes = rec.dtype
|
|
|
|
col_names = cur_dtypes.names
|
|
|
|
record_type_list = []
|
|
|
|
has_rec_fix = False
|
|
|
|
for i in xrange(len(cur_dtypes)):
|
|
|
|
curr_type = cur_dtypes[i]
|
|
|
|
# If type is a datetime64 timestamp, convert to microseconds
|
|
|
|
# NOTE: if dtype is datetime[ns] then np.record.tolist() will output values as longs,
|
|
|
|
# conversion from [us] or lower will lead to py datetime objects, see SPARK-22417
|
|
|
|
if curr_type == np.dtype('datetime64[ns]'):
|
|
|
|
curr_type = 'datetime64[us]'
|
|
|
|
has_rec_fix = True
|
|
|
|
record_type_list.append((str(col_names[i]), curr_type))
|
2017-11-12 23:16:01 -05:00
|
|
|
return np.dtype(record_type_list) if has_rec_fix else None
|
2017-11-07 15:32:37 -05:00
|
|
|
|
2017-11-28 03:45:22 -05:00
|
|
|
def _convert_from_pandas(self, pdf, schema, timezone):
|
2017-11-07 15:32:37 -05:00
|
|
|
"""
|
|
|
|
Convert a pandas.DataFrame to list of records that can be used to make a DataFrame
|
2017-11-12 23:16:01 -05:00
|
|
|
:return list of records
|
2017-11-07 15:32:37 -05:00
|
|
|
"""
|
2017-11-28 03:45:22 -05:00
|
|
|
if timezone is not None:
|
|
|
|
from pyspark.sql.types import _check_series_convert_timestamps_tz_local
|
|
|
|
copied = False
|
|
|
|
if isinstance(schema, StructType):
|
|
|
|
for field in schema:
|
|
|
|
# TODO: handle nested timestamps, such as ArrayType(TimestampType())?
|
|
|
|
if isinstance(field.dataType, TimestampType):
|
|
|
|
s = _check_series_convert_timestamps_tz_local(pdf[field.name], timezone)
|
2018-01-10 00:00:07 -05:00
|
|
|
if s is not pdf[field.name]:
|
|
|
|
if not copied:
|
|
|
|
# Copy once if the series is modified to prevent the original
|
|
|
|
# Pandas DataFrame from being updated
|
|
|
|
pdf = pdf.copy()
|
|
|
|
copied = True
|
|
|
|
pdf[field.name] = s
|
2017-11-28 03:45:22 -05:00
|
|
|
else:
|
|
|
|
for column, series in pdf.iteritems():
|
2018-01-10 00:00:07 -05:00
|
|
|
s = _check_series_convert_timestamps_tz_local(series, timezone)
|
|
|
|
if s is not series:
|
|
|
|
if not copied:
|
|
|
|
# Copy once if the series is modified to prevent the original
|
|
|
|
# Pandas DataFrame from being updated
|
|
|
|
pdf = pdf.copy()
|
|
|
|
copied = True
|
|
|
|
pdf[column] = s
|
2017-11-07 15:32:37 -05:00
|
|
|
|
|
|
|
# Convert pandas.DataFrame to list of numpy records
|
|
|
|
np_records = pdf.to_records(index=False)
|
|
|
|
|
|
|
|
# Check if any columns need to be fixed for Spark to infer properly
|
|
|
|
if len(np_records) > 0:
|
2017-11-12 23:16:01 -05:00
|
|
|
record_dtype = self._get_numpy_record_dtype(np_records[0])
|
|
|
|
if record_dtype is not None:
|
|
|
|
return [r.astype(record_dtype).tolist() for r in np_records]
|
2017-11-07 15:32:37 -05:00
|
|
|
|
|
|
|
# Convert list of numpy records to python lists
|
2017-11-12 23:16:01 -05:00
|
|
|
return [r.tolist() for r in np_records]
|
|
|
|
|
2017-11-28 03:45:22 -05:00
|
|
|
def _create_from_pandas_with_arrow(self, pdf, schema, timezone):
|
2017-11-12 23:16:01 -05:00
|
|
|
"""
|
|
|
|
Create a DataFrame from a given pandas.DataFrame by slicing it into partitions, converting
|
|
|
|
to Arrow data, then sending to the JVM to parallelize. If a schema is passed in, the
|
|
|
|
data types will be used to coerce the data in Pandas to Arrow conversion.
|
|
|
|
"""
|
[SPARK-23030][SQL][PYTHON] Use Arrow stream format for creating from and collecting Pandas DataFrames
## What changes were proposed in this pull request?
This changes the calls of `toPandas()` and `createDataFrame()` to use the Arrow stream format, when Arrow is enabled. Previously, Arrow data was written to byte arrays where each chunk is an output of the Arrow file format. This was mainly due to constraints at the time, and caused some overhead by writing the schema/footer on each chunk of data and then having to read multiple Arrow file inputs and concat them together.
Using the Arrow stream format has improved these by increasing performance, lower memory overhead for the average case, and simplified the code. Here are the details of this change:
**toPandas()**
_Before:_
Spark internal rows are converted to Arrow file format, each group of records is a complete Arrow file which contains the schema and other metadata. Next a collect is done and an Array of Arrow files is the result. After that each Arrow file is sent to Python driver which then loads each file and concats them to a single Arrow DataFrame.
_After:_
Spark internal rows are converted to ArrowRecordBatches directly, which is the simplest Arrow component for IPC data transfers. The driver JVM then immediately starts serving data to Python as an Arrow stream, sending the schema first. It then starts a Spark job with a custom handler that sends Arrow RecordBatches to Python. Partitions arriving in order are sent immediately, and out-of-order partitions are buffered until the ones that precede it come in. This improves performance, simplifies memory usage on executors, and improves the average memory usage on the JVM driver. Since the order of partitions must be preserved, the worst case is that the first partition will be the last to arrive all data must be buffered in memory until then. This case is no worse that before when doing a full collect.
**createDataFrame()**
_Before:_
A Pandas DataFrame is split into parts and each part is made into an Arrow file. Then each file is prefixed by the buffer size and written to a temp file. The temp file is read and each Arrow file is parallelized as a byte array.
_After:_
A Pandas DataFrame is split into parts, then an Arrow stream is written to a temp file where each part is an ArrowRecordBatch. The temp file is read as a stream and the Arrow messages are examined. If the message is an ArrowRecordBatch, the data is saved as a byte array. After reading the file, each ArrowRecordBatch is parallelized as a byte array. This has slightly more processing than before because we must look each Arrow message to extract the record batches, but performance ends up a litle better. It is cleaner in the sense that IPC from Python to JVM is done over a single Arrow stream.
## How was this patch tested?
Added new unit tests for the additions to ArrowConverters in Scala, existing tests for Python.
## Performance Tests - toPandas
Tests run on a 4 node standalone cluster with 32 cores total, 14.04.1-Ubuntu and OpenJDK 8
measured wall clock time to execute `toPandas()` and took the average best time of 5 runs/5 loops each.
Test code
```python
df = spark.range(1 << 25, numPartitions=32).toDF("id").withColumn("x1", rand()).withColumn("x2", rand()).withColumn("x3", rand()).withColumn("x4", rand())
for i in range(5):
start = time.time()
_ = df.toPandas()
elapsed = time.time() - start
```
Current Master | This PR
---------------------|------------
5.803557 | 5.16207
5.409119 | 5.133671
5.493509 | 5.147513
5.433107 | 5.105243
5.488757 | 5.018685
Avg Master | Avg This PR
------------------|--------------
5.5256098 | 5.1134364
Speedup of **1.08060595**
## Performance Tests - createDataFrame
Tests run on a 4 node standalone cluster with 32 cores total, 14.04.1-Ubuntu and OpenJDK 8
measured wall clock time to execute `createDataFrame()` and get the first record. Took the average best time of 5 runs/5 loops each.
Test code
```python
def run():
pdf = pd.DataFrame(np.random.rand(10000000, 10))
spark.createDataFrame(pdf).first()
for i in range(6):
start = time.time()
run()
elapsed = time.time() - start
gc.collect()
print("Run %d: %f" % (i, elapsed))
```
Current Master | This PR
--------------------|----------
6.234608 | 5.665641
6.32144 | 5.3475
6.527859 | 5.370803
6.95089 | 5.479151
6.235046 | 5.529167
Avg Master | Avg This PR
---------------|----------------
6.4539686 | 5.4784524
Speedup of **1.178064192**
## Memory Improvements
**toPandas()**
The most significant improvement is reduction of the upper bound space complexity in the JVM driver. Before, the entire dataset was collected in the JVM first before sending it to Python. With this change, as soon as a partition is collected, the result handler immediately sends it to Python, so the upper bound is the size of the largest partition. Also, using the Arrow stream format is more efficient because the schema is written once per stream, followed by record batches. The schema is now only send from driver JVM to Python. Before, multiple Arrow file formats were used that each contained the schema. This duplicated schema was created in the executors, sent to the driver JVM, and then Python where all but the first one received are discarded.
I verified the upper bound limit by running a test that would collect data that would exceed the amount of driver JVM memory available. Using these settings on a standalone cluster:
```
spark.driver.memory 1g
spark.executor.memory 5g
spark.sql.execution.arrow.enabled true
spark.sql.execution.arrow.fallback.enabled false
spark.sql.execution.arrow.maxRecordsPerBatch 0
spark.driver.maxResultSize 2g
```
Test code:
```python
from pyspark.sql.functions import rand
df = spark.range(1 << 25, numPartitions=32).toDF("id").withColumn("x1", rand()).withColumn("x2", rand()).withColumn("x3", rand())
df.toPandas()
```
This makes total data size of 33554432×8×4 = 1073741824
With the current master, it fails with OOM but passes using this PR.
**createDataFrame()**
No significant change in memory except that using the stream format instead of separate file formats avoids duplicated the schema, similar to toPandas above. The process of reading the stream and parallelizing the batches does cause the record batch message metadata to be copied, but it's size is insignificant.
Closes #21546 from BryanCutler/arrow-toPandas-stream-SPARK-23030.
Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-29 03:01:12 -04:00
|
|
|
from pyspark.serializers import ArrowStreamSerializer, _create_batch
|
2017-12-22 06:09:51 -05:00
|
|
|
from pyspark.sql.types import from_arrow_schema, to_arrow_type, TimestampType
|
|
|
|
from pyspark.sql.utils import require_minimum_pandas_version, \
|
|
|
|
require_minimum_pyarrow_version
|
|
|
|
|
|
|
|
require_minimum_pandas_version()
|
|
|
|
require_minimum_pyarrow_version()
|
|
|
|
|
|
|
|
from pandas.api.types import is_datetime64_dtype, is_datetime64tz_dtype
|
2017-12-21 06:43:56 -05:00
|
|
|
|
2017-11-12 23:16:01 -05:00
|
|
|
# Determine arrow types to coerce data when creating batches
|
|
|
|
if isinstance(schema, StructType):
|
|
|
|
arrow_types = [to_arrow_type(f.dataType) for f in schema.fields]
|
|
|
|
elif isinstance(schema, DataType):
|
|
|
|
raise ValueError("Single data type %s is not supported with Arrow" % str(schema))
|
|
|
|
else:
|
|
|
|
# Any timestamps must be coerced to be compatible with Spark
|
|
|
|
arrow_types = [to_arrow_type(TimestampType())
|
|
|
|
if is_datetime64_dtype(t) or is_datetime64tz_dtype(t) else None
|
|
|
|
for t in pdf.dtypes]
|
|
|
|
|
|
|
|
# Slice the DataFrame to be batched
|
|
|
|
step = -(-len(pdf) // self.sparkContext.defaultParallelism) # round int up
|
|
|
|
pdf_slices = (pdf[start:start + step] for start in xrange(0, len(pdf), step))
|
|
|
|
|
|
|
|
# Create Arrow record batches
|
2019-01-22 01:54:41 -05:00
|
|
|
safecheck = self._wrapped._conf.arrowSafeTypeConversion()
|
2019-03-07 11:52:24 -05:00
|
|
|
col_by_name = True # col by name only applies to StructType columns, can't happen here
|
2017-11-28 03:45:22 -05:00
|
|
|
batches = [_create_batch([(c, t) for (_, c), t in zip(pdf_slice.iteritems(), arrow_types)],
|
2019-03-07 11:52:24 -05:00
|
|
|
timezone, safecheck, col_by_name)
|
2017-11-12 23:16:01 -05:00
|
|
|
for pdf_slice in pdf_slices]
|
|
|
|
|
|
|
|
# Create the Spark schema from the first Arrow batch (always at least 1 batch after slicing)
|
|
|
|
if isinstance(schema, (list, tuple)):
|
|
|
|
struct = from_arrow_schema(batches[0].schema)
|
|
|
|
for i, name in enumerate(schema):
|
|
|
|
struct.fields[i].name = name
|
|
|
|
struct.names[i] = name
|
|
|
|
schema = struct
|
|
|
|
|
2018-08-13 22:35:34 -04:00
|
|
|
jsqlContext = self._wrapped._jsqlContext
|
|
|
|
|
[SPARK-23030][SQL][PYTHON] Use Arrow stream format for creating from and collecting Pandas DataFrames
## What changes were proposed in this pull request?
This changes the calls of `toPandas()` and `createDataFrame()` to use the Arrow stream format, when Arrow is enabled. Previously, Arrow data was written to byte arrays where each chunk is an output of the Arrow file format. This was mainly due to constraints at the time, and caused some overhead by writing the schema/footer on each chunk of data and then having to read multiple Arrow file inputs and concat them together.
Using the Arrow stream format has improved these by increasing performance, lower memory overhead for the average case, and simplified the code. Here are the details of this change:
**toPandas()**
_Before:_
Spark internal rows are converted to Arrow file format, each group of records is a complete Arrow file which contains the schema and other metadata. Next a collect is done and an Array of Arrow files is the result. After that each Arrow file is sent to Python driver which then loads each file and concats them to a single Arrow DataFrame.
_After:_
Spark internal rows are converted to ArrowRecordBatches directly, which is the simplest Arrow component for IPC data transfers. The driver JVM then immediately starts serving data to Python as an Arrow stream, sending the schema first. It then starts a Spark job with a custom handler that sends Arrow RecordBatches to Python. Partitions arriving in order are sent immediately, and out-of-order partitions are buffered until the ones that precede it come in. This improves performance, simplifies memory usage on executors, and improves the average memory usage on the JVM driver. Since the order of partitions must be preserved, the worst case is that the first partition will be the last to arrive all data must be buffered in memory until then. This case is no worse that before when doing a full collect.
**createDataFrame()**
_Before:_
A Pandas DataFrame is split into parts and each part is made into an Arrow file. Then each file is prefixed by the buffer size and written to a temp file. The temp file is read and each Arrow file is parallelized as a byte array.
_After:_
A Pandas DataFrame is split into parts, then an Arrow stream is written to a temp file where each part is an ArrowRecordBatch. The temp file is read as a stream and the Arrow messages are examined. If the message is an ArrowRecordBatch, the data is saved as a byte array. After reading the file, each ArrowRecordBatch is parallelized as a byte array. This has slightly more processing than before because we must look each Arrow message to extract the record batches, but performance ends up a litle better. It is cleaner in the sense that IPC from Python to JVM is done over a single Arrow stream.
## How was this patch tested?
Added new unit tests for the additions to ArrowConverters in Scala, existing tests for Python.
## Performance Tests - toPandas
Tests run on a 4 node standalone cluster with 32 cores total, 14.04.1-Ubuntu and OpenJDK 8
measured wall clock time to execute `toPandas()` and took the average best time of 5 runs/5 loops each.
Test code
```python
df = spark.range(1 << 25, numPartitions=32).toDF("id").withColumn("x1", rand()).withColumn("x2", rand()).withColumn("x3", rand()).withColumn("x4", rand())
for i in range(5):
start = time.time()
_ = df.toPandas()
elapsed = time.time() - start
```
Current Master | This PR
---------------------|------------
5.803557 | 5.16207
5.409119 | 5.133671
5.493509 | 5.147513
5.433107 | 5.105243
5.488757 | 5.018685
Avg Master | Avg This PR
------------------|--------------
5.5256098 | 5.1134364
Speedup of **1.08060595**
## Performance Tests - createDataFrame
Tests run on a 4 node standalone cluster with 32 cores total, 14.04.1-Ubuntu and OpenJDK 8
measured wall clock time to execute `createDataFrame()` and get the first record. Took the average best time of 5 runs/5 loops each.
Test code
```python
def run():
pdf = pd.DataFrame(np.random.rand(10000000, 10))
spark.createDataFrame(pdf).first()
for i in range(6):
start = time.time()
run()
elapsed = time.time() - start
gc.collect()
print("Run %d: %f" % (i, elapsed))
```
Current Master | This PR
--------------------|----------
6.234608 | 5.665641
6.32144 | 5.3475
6.527859 | 5.370803
6.95089 | 5.479151
6.235046 | 5.529167
Avg Master | Avg This PR
---------------|----------------
6.4539686 | 5.4784524
Speedup of **1.178064192**
## Memory Improvements
**toPandas()**
The most significant improvement is reduction of the upper bound space complexity in the JVM driver. Before, the entire dataset was collected in the JVM first before sending it to Python. With this change, as soon as a partition is collected, the result handler immediately sends it to Python, so the upper bound is the size of the largest partition. Also, using the Arrow stream format is more efficient because the schema is written once per stream, followed by record batches. The schema is now only send from driver JVM to Python. Before, multiple Arrow file formats were used that each contained the schema. This duplicated schema was created in the executors, sent to the driver JVM, and then Python where all but the first one received are discarded.
I verified the upper bound limit by running a test that would collect data that would exceed the amount of driver JVM memory available. Using these settings on a standalone cluster:
```
spark.driver.memory 1g
spark.executor.memory 5g
spark.sql.execution.arrow.enabled true
spark.sql.execution.arrow.fallback.enabled false
spark.sql.execution.arrow.maxRecordsPerBatch 0
spark.driver.maxResultSize 2g
```
Test code:
```python
from pyspark.sql.functions import rand
df = spark.range(1 << 25, numPartitions=32).toDF("id").withColumn("x1", rand()).withColumn("x2", rand()).withColumn("x3", rand())
df.toPandas()
```
This makes total data size of 33554432×8×4 = 1073741824
With the current master, it fails with OOM but passes using this PR.
**createDataFrame()**
No significant change in memory except that using the stream format instead of separate file formats avoids duplicated the schema, similar to toPandas above. The process of reading the stream and parallelizing the batches does cause the record batch message metadata to be copied, but it's size is insignificant.
Closes #21546 from BryanCutler/arrow-toPandas-stream-SPARK-23030.
Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-29 03:01:12 -04:00
|
|
|
def reader_func(temp_filename):
|
2018-08-13 22:35:34 -04:00
|
|
|
return self._jvm.PythonSQLUtils.readArrowStreamFromFile(jsqlContext, temp_filename)
|
|
|
|
|
|
|
|
def create_RDD_server():
|
|
|
|
return self._jvm.ArrowRDDServer(jsqlContext)
|
[SPARK-23030][SQL][PYTHON] Use Arrow stream format for creating from and collecting Pandas DataFrames
## What changes were proposed in this pull request?
This changes the calls of `toPandas()` and `createDataFrame()` to use the Arrow stream format, when Arrow is enabled. Previously, Arrow data was written to byte arrays where each chunk is an output of the Arrow file format. This was mainly due to constraints at the time, and caused some overhead by writing the schema/footer on each chunk of data and then having to read multiple Arrow file inputs and concat them together.
Using the Arrow stream format has improved these by increasing performance, lower memory overhead for the average case, and simplified the code. Here are the details of this change:
**toPandas()**
_Before:_
Spark internal rows are converted to Arrow file format, each group of records is a complete Arrow file which contains the schema and other metadata. Next a collect is done and an Array of Arrow files is the result. After that each Arrow file is sent to Python driver which then loads each file and concats them to a single Arrow DataFrame.
_After:_
Spark internal rows are converted to ArrowRecordBatches directly, which is the simplest Arrow component for IPC data transfers. The driver JVM then immediately starts serving data to Python as an Arrow stream, sending the schema first. It then starts a Spark job with a custom handler that sends Arrow RecordBatches to Python. Partitions arriving in order are sent immediately, and out-of-order partitions are buffered until the ones that precede it come in. This improves performance, simplifies memory usage on executors, and improves the average memory usage on the JVM driver. Since the order of partitions must be preserved, the worst case is that the first partition will be the last to arrive all data must be buffered in memory until then. This case is no worse that before when doing a full collect.
**createDataFrame()**
_Before:_
A Pandas DataFrame is split into parts and each part is made into an Arrow file. Then each file is prefixed by the buffer size and written to a temp file. The temp file is read and each Arrow file is parallelized as a byte array.
_After:_
A Pandas DataFrame is split into parts, then an Arrow stream is written to a temp file where each part is an ArrowRecordBatch. The temp file is read as a stream and the Arrow messages are examined. If the message is an ArrowRecordBatch, the data is saved as a byte array. After reading the file, each ArrowRecordBatch is parallelized as a byte array. This has slightly more processing than before because we must look each Arrow message to extract the record batches, but performance ends up a litle better. It is cleaner in the sense that IPC from Python to JVM is done over a single Arrow stream.
## How was this patch tested?
Added new unit tests for the additions to ArrowConverters in Scala, existing tests for Python.
## Performance Tests - toPandas
Tests run on a 4 node standalone cluster with 32 cores total, 14.04.1-Ubuntu and OpenJDK 8
measured wall clock time to execute `toPandas()` and took the average best time of 5 runs/5 loops each.
Test code
```python
df = spark.range(1 << 25, numPartitions=32).toDF("id").withColumn("x1", rand()).withColumn("x2", rand()).withColumn("x3", rand()).withColumn("x4", rand())
for i in range(5):
start = time.time()
_ = df.toPandas()
elapsed = time.time() - start
```
Current Master | This PR
---------------------|------------
5.803557 | 5.16207
5.409119 | 5.133671
5.493509 | 5.147513
5.433107 | 5.105243
5.488757 | 5.018685
Avg Master | Avg This PR
------------------|--------------
5.5256098 | 5.1134364
Speedup of **1.08060595**
## Performance Tests - createDataFrame
Tests run on a 4 node standalone cluster with 32 cores total, 14.04.1-Ubuntu and OpenJDK 8
measured wall clock time to execute `createDataFrame()` and get the first record. Took the average best time of 5 runs/5 loops each.
Test code
```python
def run():
pdf = pd.DataFrame(np.random.rand(10000000, 10))
spark.createDataFrame(pdf).first()
for i in range(6):
start = time.time()
run()
elapsed = time.time() - start
gc.collect()
print("Run %d: %f" % (i, elapsed))
```
Current Master | This PR
--------------------|----------
6.234608 | 5.665641
6.32144 | 5.3475
6.527859 | 5.370803
6.95089 | 5.479151
6.235046 | 5.529167
Avg Master | Avg This PR
---------------|----------------
6.4539686 | 5.4784524
Speedup of **1.178064192**
## Memory Improvements
**toPandas()**
The most significant improvement is reduction of the upper bound space complexity in the JVM driver. Before, the entire dataset was collected in the JVM first before sending it to Python. With this change, as soon as a partition is collected, the result handler immediately sends it to Python, so the upper bound is the size of the largest partition. Also, using the Arrow stream format is more efficient because the schema is written once per stream, followed by record batches. The schema is now only send from driver JVM to Python. Before, multiple Arrow file formats were used that each contained the schema. This duplicated schema was created in the executors, sent to the driver JVM, and then Python where all but the first one received are discarded.
I verified the upper bound limit by running a test that would collect data that would exceed the amount of driver JVM memory available. Using these settings on a standalone cluster:
```
spark.driver.memory 1g
spark.executor.memory 5g
spark.sql.execution.arrow.enabled true
spark.sql.execution.arrow.fallback.enabled false
spark.sql.execution.arrow.maxRecordsPerBatch 0
spark.driver.maxResultSize 2g
```
Test code:
```python
from pyspark.sql.functions import rand
df = spark.range(1 << 25, numPartitions=32).toDF("id").withColumn("x1", rand()).withColumn("x2", rand()).withColumn("x3", rand())
df.toPandas()
```
This makes total data size of 33554432×8×4 = 1073741824
With the current master, it fails with OOM but passes using this PR.
**createDataFrame()**
No significant change in memory except that using the stream format instead of separate file formats avoids duplicated the schema, similar to toPandas above. The process of reading the stream and parallelizing the batches does cause the record batch message metadata to be copied, but it's size is insignificant.
Closes #21546 from BryanCutler/arrow-toPandas-stream-SPARK-23030.
Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-29 03:01:12 -04:00
|
|
|
|
|
|
|
# Create Spark DataFrame from Arrow stream file, using one batch per partition
|
2018-08-13 22:35:34 -04:00
|
|
|
jrdd = self._sc._serialize_to_jvm(batches, ArrowStreamSerializer(), reader_func,
|
|
|
|
create_RDD_server)
|
|
|
|
jdf = self._jvm.PythonSQLUtils.toDataFrame(jrdd, schema.json(), jsqlContext)
|
2017-11-12 23:16:01 -05:00
|
|
|
df = DataFrame(jdf, self._wrapped)
|
|
|
|
df._schema = schema
|
|
|
|
return df
|
2017-11-07 15:32:37 -05:00
|
|
|
|
[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)
|
|
|
|
@ignore_unicode_prefix
|
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)
|
|
|
|
from ``data``, which should be an RDD of :class:`Row`,
|
|
|
|
or :class:`namedtuple`, or :class:`dict`.
|
|
|
|
|
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
|
|
|
|
: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``.
|
|
|
|
|
|
|
|
:param data: an RDD of any kind of SQL data representation(e.g. row, tuple, int, boolean,
|
|
|
|
etc.), or :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
|
|
|
|
2018-05-28 00:56:05 -04:00
|
|
|
.. note:: Usage with spark.sql.execution.arrow.enabled=True is experimental.
|
|
|
|
|
2016-04-28 13:55:48 -04:00
|
|
|
>>> l = [('Alice', 1)]
|
|
|
|
>>> spark.createDataFrame(l).collect()
|
|
|
|
[Row(_1=u'Alice', _2=1)]
|
|
|
|
>>> spark.createDataFrame(l, ['name', 'age']).collect()
|
|
|
|
[Row(name=u'Alice', age=1)]
|
|
|
|
|
|
|
|
>>> d = [{'name': 'Alice', 'age': 1}]
|
|
|
|
>>> spark.createDataFrame(d).collect()
|
|
|
|
[Row(age=1, name=u'Alice')]
|
|
|
|
|
|
|
|
>>> rdd = sc.parallelize(l)
|
|
|
|
>>> spark.createDataFrame(rdd).collect()
|
|
|
|
[Row(_1=u'Alice', _2=1)]
|
|
|
|
>>> df = spark.createDataFrame(rdd, ['name', 'age'])
|
|
|
|
>>> df.collect()
|
|
|
|
[Row(name=u'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=u'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=u'Alice', age=1)]
|
|
|
|
|
|
|
|
>>> spark.createDataFrame(df.toPandas()).collect() # doctest: +SKIP
|
|
|
|
[Row(name=u'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=u'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: ...
|
|
|
|
"""
|
2018-10-26 12:40:13 -04:00
|
|
|
SparkSession._activeSession = self
|
|
|
|
self._jvm.SparkSession.setActiveSession(self._jsparkSession)
|
2016-04-28 13:55:48 -04:00
|
|
|
if isinstance(data, DataFrame):
|
|
|
|
raise TypeError("data is already a DataFrame")
|
|
|
|
|
|
|
|
if isinstance(schema, basestring):
|
|
|
|
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-23319][TESTS] Explicitly specify Pandas and PyArrow versions in PySpark tests (to skip or test)
## What changes were proposed in this pull request?
This PR proposes to explicitly specify Pandas and PyArrow versions in PySpark tests to skip or test.
We declared the extra dependencies:
https://github.com/apache/spark/blob/b8bfce51abf28c66ba1fc67b0f25fe1617c81025/python/setup.py#L204
In case of PyArrow:
Currently we only check if pyarrow is installed or not without checking the version. It already fails to run tests. For example, if PyArrow 0.7.0 is installed:
```
======================================================================
ERROR: test_vectorized_udf_wrong_return_type (pyspark.sql.tests.ScalarPandasUDF)
----------------------------------------------------------------------
Traceback (most recent call last):
File "/.../spark/python/pyspark/sql/tests.py", line 4019, in test_vectorized_udf_wrong_return_type
f = pandas_udf(lambda x: x * 1.0, MapType(LongType(), LongType()))
File "/.../spark/python/pyspark/sql/functions.py", line 2309, in pandas_udf
return _create_udf(f=f, returnType=return_type, evalType=eval_type)
File "/.../spark/python/pyspark/sql/udf.py", line 47, in _create_udf
require_minimum_pyarrow_version()
File "/.../spark/python/pyspark/sql/utils.py", line 132, in require_minimum_pyarrow_version
"however, your version was %s." % pyarrow.__version__)
ImportError: pyarrow >= 0.8.0 must be installed on calling Python process; however, your version was 0.7.0.
----------------------------------------------------------------------
Ran 33 tests in 8.098s
FAILED (errors=33)
```
In case of Pandas:
There are few tests for old Pandas which were tested only when Pandas version was lower, and I rewrote them to be tested when both Pandas version is lower and missing.
## How was this patch tested?
Manually tested by modifying the condition:
```
test_createDataFrame_column_name_encoding (pyspark.sql.tests.ArrowTests) ... skipped 'Pandas >= 1.19.2 must be installed; however, your version was 0.19.2.'
test_createDataFrame_does_not_modify_input (pyspark.sql.tests.ArrowTests) ... skipped 'Pandas >= 1.19.2 must be installed; however, your version was 0.19.2.'
test_createDataFrame_respect_session_timezone (pyspark.sql.tests.ArrowTests) ... skipped 'Pandas >= 1.19.2 must be installed; however, your version was 0.19.2.'
```
```
test_createDataFrame_column_name_encoding (pyspark.sql.tests.ArrowTests) ... skipped 'Pandas >= 0.19.2 must be installed; however, it was not found.'
test_createDataFrame_does_not_modify_input (pyspark.sql.tests.ArrowTests) ... skipped 'Pandas >= 0.19.2 must be installed; however, it was not found.'
test_createDataFrame_respect_session_timezone (pyspark.sql.tests.ArrowTests) ... skipped 'Pandas >= 0.19.2 must be installed; however, it was not found.'
```
```
test_createDataFrame_column_name_encoding (pyspark.sql.tests.ArrowTests) ... skipped 'PyArrow >= 1.8.0 must be installed; however, your version was 0.8.0.'
test_createDataFrame_does_not_modify_input (pyspark.sql.tests.ArrowTests) ... skipped 'PyArrow >= 1.8.0 must be installed; however, your version was 0.8.0.'
test_createDataFrame_respect_session_timezone (pyspark.sql.tests.ArrowTests) ... skipped 'PyArrow >= 1.8.0 must be installed; however, your version was 0.8.0.'
```
```
test_createDataFrame_column_name_encoding (pyspark.sql.tests.ArrowTests) ... skipped 'PyArrow >= 0.8.0 must be installed; however, it was not found.'
test_createDataFrame_does_not_modify_input (pyspark.sql.tests.ArrowTests) ... skipped 'PyArrow >= 0.8.0 must be installed; however, it was not found.'
test_createDataFrame_respect_session_timezone (pyspark.sql.tests.ArrowTests) ... skipped 'PyArrow >= 0.8.0 must be installed; however, it was not found.'
```
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #20487 from HyukjinKwon/pyarrow-pandas-skip.
2018-02-07 09:28:10 -05:00
|
|
|
from pyspark.sql.utils import require_minimum_pandas_version
|
|
|
|
require_minimum_pandas_version()
|
|
|
|
|
2018-08-16 22:18:08 -04:00
|
|
|
if self._wrapped._conf.pandasRespectSessionTimeZone():
|
|
|
|
timezone = self._wrapped._conf.sessionLocalTimeZone()
|
2017-11-28 03:45:22 -05:00
|
|
|
else:
|
|
|
|
timezone = None
|
2017-11-12 23:16:01 -05:00
|
|
|
|
|
|
|
# If no schema supplied by user then get the names of columns only
|
|
|
|
if schema is None:
|
2018-01-10 00:55:24 -05:00
|
|
|
schema = [str(x) if not isinstance(x, basestring) else
|
|
|
|
(x.encode('utf-8') if not isinstance(x, str) else x)
|
|
|
|
for x in data.columns]
|
2017-11-12 23:16:01 -05:00
|
|
|
|
2018-08-16 22:18:08 -04:00
|
|
|
if self._wrapped._conf.arrowEnabled() and len(data) > 0:
|
2017-11-12 23:16:01 -05:00
|
|
|
try:
|
2017-11-28 03:45:22 -05:00
|
|
|
return self._create_from_pandas_with_arrow(data, schema, timezone)
|
2017-11-12 23:16:01 -05:00
|
|
|
except Exception as e:
|
[SPARK-23380][PYTHON] Adds a conf for Arrow fallback in toPandas/createDataFrame with Pandas DataFrame
## What changes were proposed in this pull request?
This PR adds a configuration to control the fallback of Arrow optimization for `toPandas` and `createDataFrame` with Pandas DataFrame.
## How was this patch tested?
Manually tested and unit tests added.
You can test this by:
**`createDataFrame`**
```python
spark.conf.set("spark.sql.execution.arrow.enabled", False)
pdf = spark.createDataFrame([[{'a': 1}]]).toPandas()
spark.conf.set("spark.sql.execution.arrow.enabled", True)
spark.conf.set("spark.sql.execution.arrow.fallback.enabled", True)
spark.createDataFrame(pdf, "a: map<string, int>")
```
```python
spark.conf.set("spark.sql.execution.arrow.enabled", False)
pdf = spark.createDataFrame([[{'a': 1}]]).toPandas()
spark.conf.set("spark.sql.execution.arrow.enabled", True)
spark.conf.set("spark.sql.execution.arrow.fallback.enabled", False)
spark.createDataFrame(pdf, "a: map<string, int>")
```
**`toPandas`**
```python
spark.conf.set("spark.sql.execution.arrow.enabled", True)
spark.conf.set("spark.sql.execution.arrow.fallback.enabled", True)
spark.createDataFrame([[{'a': 1}]]).toPandas()
```
```python
spark.conf.set("spark.sql.execution.arrow.enabled", True)
spark.conf.set("spark.sql.execution.arrow.fallback.enabled", False)
spark.createDataFrame([[{'a': 1}]]).toPandas()
```
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #20678 from HyukjinKwon/SPARK-23380-conf.
2018-03-08 06:22:07 -05:00
|
|
|
from pyspark.util import _exception_message
|
|
|
|
|
2018-08-16 22:18:08 -04:00
|
|
|
if self._wrapped._conf.arrowFallbackEnabled():
|
[SPARK-23380][PYTHON] Adds a conf for Arrow fallback in toPandas/createDataFrame with Pandas DataFrame
## What changes were proposed in this pull request?
This PR adds a configuration to control the fallback of Arrow optimization for `toPandas` and `createDataFrame` with Pandas DataFrame.
## How was this patch tested?
Manually tested and unit tests added.
You can test this by:
**`createDataFrame`**
```python
spark.conf.set("spark.sql.execution.arrow.enabled", False)
pdf = spark.createDataFrame([[{'a': 1}]]).toPandas()
spark.conf.set("spark.sql.execution.arrow.enabled", True)
spark.conf.set("spark.sql.execution.arrow.fallback.enabled", True)
spark.createDataFrame(pdf, "a: map<string, int>")
```
```python
spark.conf.set("spark.sql.execution.arrow.enabled", False)
pdf = spark.createDataFrame([[{'a': 1}]]).toPandas()
spark.conf.set("spark.sql.execution.arrow.enabled", True)
spark.conf.set("spark.sql.execution.arrow.fallback.enabled", False)
spark.createDataFrame(pdf, "a: map<string, int>")
```
**`toPandas`**
```python
spark.conf.set("spark.sql.execution.arrow.enabled", True)
spark.conf.set("spark.sql.execution.arrow.fallback.enabled", True)
spark.createDataFrame([[{'a': 1}]]).toPandas()
```
```python
spark.conf.set("spark.sql.execution.arrow.enabled", True)
spark.conf.set("spark.sql.execution.arrow.fallback.enabled", False)
spark.createDataFrame([[{'a': 1}]]).toPandas()
```
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #20678 from HyukjinKwon/SPARK-23380-conf.
2018-03-08 06:22:07 -05:00
|
|
|
msg = (
|
|
|
|
"createDataFrame attempted Arrow optimization because "
|
|
|
|
"'spark.sql.execution.arrow.enabled' is set to true; however, "
|
|
|
|
"failed by the reason below:\n %s\n"
|
2018-03-27 23:06:12 -04:00
|
|
|
"Attempting non-optimization as "
|
[SPARK-23380][PYTHON] Adds a conf for Arrow fallback in toPandas/createDataFrame with Pandas DataFrame
## What changes were proposed in this pull request?
This PR adds a configuration to control the fallback of Arrow optimization for `toPandas` and `createDataFrame` with Pandas DataFrame.
## How was this patch tested?
Manually tested and unit tests added.
You can test this by:
**`createDataFrame`**
```python
spark.conf.set("spark.sql.execution.arrow.enabled", False)
pdf = spark.createDataFrame([[{'a': 1}]]).toPandas()
spark.conf.set("spark.sql.execution.arrow.enabled", True)
spark.conf.set("spark.sql.execution.arrow.fallback.enabled", True)
spark.createDataFrame(pdf, "a: map<string, int>")
```
```python
spark.conf.set("spark.sql.execution.arrow.enabled", False)
pdf = spark.createDataFrame([[{'a': 1}]]).toPandas()
spark.conf.set("spark.sql.execution.arrow.enabled", True)
spark.conf.set("spark.sql.execution.arrow.fallback.enabled", False)
spark.createDataFrame(pdf, "a: map<string, int>")
```
**`toPandas`**
```python
spark.conf.set("spark.sql.execution.arrow.enabled", True)
spark.conf.set("spark.sql.execution.arrow.fallback.enabled", True)
spark.createDataFrame([[{'a': 1}]]).toPandas()
```
```python
spark.conf.set("spark.sql.execution.arrow.enabled", True)
spark.conf.set("spark.sql.execution.arrow.fallback.enabled", False)
spark.createDataFrame([[{'a': 1}]]).toPandas()
```
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #20678 from HyukjinKwon/SPARK-23380-conf.
2018-03-08 06:22:07 -05:00
|
|
|
"'spark.sql.execution.arrow.fallback.enabled' is set to "
|
|
|
|
"true." % _exception_message(e))
|
|
|
|
warnings.warn(msg)
|
|
|
|
else:
|
|
|
|
msg = (
|
|
|
|
"createDataFrame attempted Arrow optimization because "
|
2018-03-27 23:06:12 -04:00
|
|
|
"'spark.sql.execution.arrow.enabled' is set to true, but has reached "
|
|
|
|
"the error below and will not continue because automatic fallback "
|
|
|
|
"with 'spark.sql.execution.arrow.fallback.enabled' has been set to "
|
|
|
|
"false.\n %s" % _exception_message(e))
|
|
|
|
warnings.warn(msg)
|
|
|
|
raise
|
2017-11-28 03:45:22 -05:00
|
|
|
data = self._convert_from_pandas(data, schema, timezone)
|
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
|
|
|
|
|
|
|
|
@ignore_unicode_prefix
|
|
|
|
@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()
|
|
|
|
[Row(f1=1, f2=u'row1'), Row(f1=2, f2=u'row2'), Row(f1=3, f2=u'row3')]
|
|
|
|
"""
|
|
|
|
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
|
|
|
|
:class:`StreamingQuery` StreamingQueries 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`.
|
|
|
|
"""
|
|
|
|
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()
|
2018-10-26 12:40:13 -04:00
|
|
|
self._jvm.SparkSession.clearActiveSession()
|
2017-01-12 07:53:31 -05:00
|
|
|
SparkSession._instantiatedSession = None
|
2018-10-26 12:40:13 -04:00
|
|
|
SparkSession._activeSession = 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()
|