2015-02-09 23:49:22 -05:00
|
|
|
#
|
|
|
|
# Licensed to the Apache Software Foundation (ASF) under one or more
|
2015-05-02 18:04:13 -04:00
|
|
|
# contributor license agreements. See the NOTICE file distributed with
|
2015-02-09 23:49:22 -05:00
|
|
|
# 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 random
|
|
|
|
|
2015-04-16 19:20:57 -04:00
|
|
|
if sys.version >= '3':
|
|
|
|
basestring = unicode = str
|
|
|
|
long = int
|
2015-06-09 02:27:05 -04:00
|
|
|
from functools import reduce
|
2015-04-16 19:20:57 -04:00
|
|
|
else:
|
|
|
|
from itertools import imap as map
|
|
|
|
|
2017-04-05 14:47:40 -04:00
|
|
|
import warnings
|
|
|
|
|
2018-02-09 01:21:10 -05:00
|
|
|
from pyspark import copy_func, since, _NoValue
|
2015-04-16 19:20:57 -04:00
|
|
|
from pyspark.rdd import RDD, _load_from_socket, ignore_unicode_prefix
|
[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, BatchedSerializer, PickleSerializer, \
|
2017-07-10 18:21:03 -04:00
|
|
|
UTF8Deserializer
|
2015-02-09 23:49:22 -05:00
|
|
|
from pyspark.storagelevel import StorageLevel
|
|
|
|
from pyspark.traceback_utils import SCCallSiteSync
|
2015-07-09 17:43:38 -04:00
|
|
|
from pyspark.sql.types import _parse_datatype_json_string
|
2015-08-25 07:33:13 -04:00
|
|
|
from pyspark.sql.column import Column, _to_seq, _to_list, _to_java_column
|
2016-06-29 01:07:11 -04:00
|
|
|
from pyspark.sql.readwriter import DataFrameWriter
|
|
|
|
from pyspark.sql.streaming import DataStreamWriter
|
2017-09-22 09:39:47 -04:00
|
|
|
from pyspark.sql.types import IntegralType
|
2015-05-19 17:23:28 -04:00
|
|
|
from pyspark.sql.types import *
|
2017-11-28 03:45:22 -05:00
|
|
|
from pyspark.util import _exception_message
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2016-01-04 21:02:38 -05:00
|
|
|
__all__ = ["DataFrame", "DataFrameNaFunctions", "DataFrameStatFunctions"]
|
2015-02-09 23:49:22 -05:00
|
|
|
|
|
|
|
|
|
|
|
class DataFrame(object):
|
2015-03-31 21:31:36 -04:00
|
|
|
"""A distributed collection of data grouped into named columns.
|
2015-02-09 23:49:22 -05:00
|
|
|
|
|
|
|
A :class:`DataFrame` is equivalent to a relational table in Spark SQL,
|
2017-09-09 12:25:12 -04:00
|
|
|
and can be created using various functions in :class:`SparkSession`::
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2017-09-09 12:25:12 -04:00
|
|
|
people = spark.read.parquet("...")
|
2015-02-09 23:49:22 -05:00
|
|
|
|
|
|
|
Once created, it can be manipulated using the various domain-specific-language
|
|
|
|
(DSL) functions defined in: :class:`DataFrame`, :class:`Column`.
|
|
|
|
|
|
|
|
To select a column from the data frame, use the apply method::
|
|
|
|
|
|
|
|
ageCol = people.age
|
|
|
|
|
|
|
|
A more concrete example::
|
|
|
|
|
2017-09-09 12:25:12 -04:00
|
|
|
# To create DataFrame using SparkSession
|
|
|
|
people = spark.read.parquet("...")
|
|
|
|
department = spark.read.parquet("...")
|
2015-02-09 23:49:22 -05:00
|
|
|
|
[MINOR][PYSPARK][DOCS] Fix examples in PySpark documentation
## What changes were proposed in this pull request?
This PR proposes to fix wrongly indented examples in PySpark documentation
```
- >>> json_sdf = spark.readStream.format("json")\
- .schema(sdf_schema)\
- .load(tempfile.mkdtemp())
+ >>> json_sdf = spark.readStream.format("json") \\
+ ... .schema(sdf_schema) \\
+ ... .load(tempfile.mkdtemp())
```
```
- people.filter(people.age > 30).join(department, people.deptId == department.id)\
+ people.filter(people.age > 30).join(department, people.deptId == department.id) \\
```
```
- >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, 1.23), (2, 4.56)])), \
- LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))]
+ >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, 1.23), (2, 4.56)])),
+ ... LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))]
```
```
- >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, -1.23), (2, 4.56e-7)])), \
- LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))]
+ >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, -1.23), (2, 4.56e-7)])),
+ ... LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))]
```
```
- ... for x in iterator:
- ... print(x)
+ ... for x in iterator:
+ ... print(x)
```
## How was this patch tested?
Manually tested.
**Before**
![2016-09-26 8 36 02](https://cloud.githubusercontent.com/assets/6477701/18834471/05c7a478-8431-11e6-94bb-09aa37b12ddb.png)
![2016-09-26 9 22 16](https://cloud.githubusercontent.com/assets/6477701/18834472/06c8735c-8431-11e6-8775-78631eab0411.png)
<img width="601" alt="2016-09-27 2 29 27" src="https://cloud.githubusercontent.com/assets/6477701/18861294/29c0d5b4-84bf-11e6-99c5-3c9d913c125d.png">
<img width="1056" alt="2016-09-27 2 29 58" src="https://cloud.githubusercontent.com/assets/6477701/18861298/31694cd8-84bf-11e6-9e61-9888cb8c2089.png">
<img width="1079" alt="2016-09-27 2 30 05" src="https://cloud.githubusercontent.com/assets/6477701/18861301/359722da-84bf-11e6-97f9-5f5365582d14.png">
**After**
![2016-09-26 9 29 47](https://cloud.githubusercontent.com/assets/6477701/18834467/0367f9da-8431-11e6-86d9-a490d3297339.png)
![2016-09-26 9 30 24](https://cloud.githubusercontent.com/assets/6477701/18834463/f870fae0-8430-11e6-9482-01fc47898492.png)
<img width="515" alt="2016-09-27 2 28 19" src="https://cloud.githubusercontent.com/assets/6477701/18861305/3ff88b88-84bf-11e6-902c-9f725e8a8b10.png">
<img width="652" alt="2016-09-27 3 50 59" src="https://cloud.githubusercontent.com/assets/6477701/18863053/592fbc74-84ca-11e6-8dbf-99cf57947de8.png">
<img width="709" alt="2016-09-27 3 51 03" src="https://cloud.githubusercontent.com/assets/6477701/18863060/601607be-84ca-11e6-80aa-a401df41c321.png">
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #15242 from HyukjinKwon/minor-example-pyspark.
2016-09-28 06:19:04 -04:00
|
|
|
people.filter(people.age > 30).join(department, people.deptId == department.id) \\
|
2015-02-09 23:49:22 -05:00
|
|
|
.groupBy(department.name, "gender").agg({"salary": "avg", "age": "max"})
|
2015-05-21 02:05:54 -04:00
|
|
|
|
|
|
|
.. versionadded:: 1.3
|
2015-02-09 23:49:22 -05:00
|
|
|
"""
|
|
|
|
|
|
|
|
def __init__(self, jdf, sql_ctx):
|
|
|
|
self._jdf = jdf
|
|
|
|
self.sql_ctx = sql_ctx
|
|
|
|
self._sc = sql_ctx and sql_ctx._sc
|
|
|
|
self.is_cached = False
|
2015-02-14 02:03:22 -05:00
|
|
|
self._schema = None # initialized lazily
|
2015-04-16 19:20:57 -04:00
|
|
|
self._lazy_rdd = None
|
2018-06-04 21:23:08 -04:00
|
|
|
# Check whether _repr_html is supported or not, we use it to avoid calling _jdf twice
|
|
|
|
# by __repr__ and _repr_html_ while eager evaluation opened.
|
|
|
|
self._support_repr_html = False
|
2015-02-09 23:49:22 -05:00
|
|
|
|
|
|
|
@property
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-02-09 23:49:22 -05:00
|
|
|
def rdd(self):
|
2015-03-31 21:31:36 -04:00
|
|
|
"""Returns the content as an :class:`pyspark.RDD` of :class:`Row`.
|
2015-02-09 23:49:22 -05:00
|
|
|
"""
|
2015-04-16 19:20:57 -04:00
|
|
|
if self._lazy_rdd is None:
|
2015-02-09 23:49:22 -05:00
|
|
|
jrdd = self._jdf.javaToPython()
|
2015-07-09 17:43:38 -04:00
|
|
|
self._lazy_rdd = RDD(jrdd, self.sql_ctx._sc, BatchedSerializer(PickleSerializer()))
|
2015-02-09 23:49:22 -05:00
|
|
|
return self._lazy_rdd
|
|
|
|
|
2015-03-31 03:25:23 -04:00
|
|
|
@property
|
2015-05-21 02:05:54 -04:00
|
|
|
@since("1.3.1")
|
2015-03-31 03:25:23 -04:00
|
|
|
def na(self):
|
|
|
|
"""Returns a :class:`DataFrameNaFunctions` for handling missing values.
|
|
|
|
"""
|
|
|
|
return DataFrameNaFunctions(self)
|
|
|
|
|
2015-05-01 16:29:17 -04:00
|
|
|
@property
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.4)
|
2015-05-01 16:29:17 -04:00
|
|
|
def stat(self):
|
|
|
|
"""Returns a :class:`DataFrameStatFunctions` for statistic functions.
|
|
|
|
"""
|
|
|
|
return DataFrameStatFunctions(self)
|
|
|
|
|
2015-04-16 19:20:57 -04:00
|
|
|
@ignore_unicode_prefix
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-04-16 19:20:57 -04:00
|
|
|
def toJSON(self, use_unicode=True):
|
2015-03-31 21:31:36 -04:00
|
|
|
"""Converts a :class:`DataFrame` into a :class:`RDD` of string.
|
|
|
|
|
|
|
|
Each row is turned into a JSON document as one element in the returned RDD.
|
2015-02-09 23:49:22 -05:00
|
|
|
|
|
|
|
>>> df.toJSON().first()
|
2015-04-16 19:20:57 -04:00
|
|
|
u'{"age":2,"name":"Alice"}'
|
2015-02-09 23:49:22 -05:00
|
|
|
"""
|
|
|
|
rdd = self._jdf.toJSON()
|
|
|
|
return RDD(rdd.toJavaRDD(), self._sc, UTF8Deserializer(use_unicode))
|
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-02-09 23:49:22 -05:00
|
|
|
def registerTempTable(self, name):
|
2017-09-09 12:25:12 -04:00
|
|
|
"""Registers this DataFrame as a temporary table using the given name.
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2017-09-09 12:25:12 -04:00
|
|
|
The lifetime of this temporary table is tied to the :class:`SparkSession`
|
2015-03-31 21:31:36 -04:00
|
|
|
that was used to create this :class:`DataFrame`.
|
2015-02-09 23:49:22 -05:00
|
|
|
|
|
|
|
>>> df.registerTempTable("people")
|
2016-05-12 03:51:53 -04:00
|
|
|
>>> df2 = spark.sql("select * from people")
|
2015-02-09 23:49:22 -05:00
|
|
|
>>> sorted(df.collect()) == sorted(df2.collect())
|
|
|
|
True
|
2016-05-12 03:51:53 -04:00
|
|
|
>>> spark.catalog.dropTempView("people")
|
|
|
|
|
|
|
|
.. note:: Deprecated in 2.0, use createOrReplaceTempView instead.
|
|
|
|
"""
|
[SPARK-22313][PYTHON] Mark/print deprecation warnings as DeprecationWarning for deprecated APIs
## What changes were proposed in this pull request?
This PR proposes to mark the existing warnings as `DeprecationWarning` and print out warnings for deprecated functions.
This could be actually useful for Spark app developers. I use (old) PyCharm and this IDE can detect this specific `DeprecationWarning` in some cases:
**Before**
<img src="https://user-images.githubusercontent.com/6477701/31762664-df68d9f8-b4f6-11e7-8773-f0468f70a2cc.png" height="45" />
**After**
<img src="https://user-images.githubusercontent.com/6477701/31762662-de4d6868-b4f6-11e7-98dc-3c8446a0c28a.png" height="70" />
For console usage, `DeprecationWarning` is usually disabled (see https://docs.python.org/2/library/warnings.html#warning-categories and https://docs.python.org/3/library/warnings.html#warning-categories):
```
>>> import warnings
>>> filter(lambda f: f[2] == DeprecationWarning, warnings.filters)
[('ignore', <_sre.SRE_Pattern object at 0x10ba58c00>, <type 'exceptions.DeprecationWarning'>, <_sre.SRE_Pattern object at 0x10bb04138>, 0), ('ignore', None, <type 'exceptions.DeprecationWarning'>, None, 0)]
```
so, it won't actually mess up the terminal much unless it is intended.
If this is intendedly enabled, it'd should as below:
```
>>> import warnings
>>> warnings.simplefilter('always', DeprecationWarning)
>>>
>>> from pyspark.sql import functions
>>> functions.approxCountDistinct("a")
.../spark/python/pyspark/sql/functions.py:232: DeprecationWarning: Deprecated in 2.1, use approx_count_distinct instead.
"Deprecated in 2.1, use approx_count_distinct instead.", DeprecationWarning)
...
```
These instances were found by:
```
cd python/pyspark
grep -r "Deprecated" .
grep -r "deprecated" .
grep -r "deprecate" .
```
## How was this patch tested?
Manually tested.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #19535 from HyukjinKwon/deprecated-warning.
2017-10-23 23:44:47 -04:00
|
|
|
warnings.warn(
|
|
|
|
"Deprecated in 2.0, use createOrReplaceTempView instead.", DeprecationWarning)
|
2016-05-12 03:51:53 -04:00
|
|
|
self._jdf.createOrReplaceTempView(name)
|
|
|
|
|
|
|
|
@since(2.0)
|
|
|
|
def createTempView(self, name):
|
[SPARK-17338][SQL] add global temp view
## What changes were proposed in this pull request?
Global temporary view is a cross-session temporary view, which means it's shared among all sessions. Its lifetime is the lifetime of the Spark application, i.e. it will be automatically dropped when the application terminates. It's tied to a system preserved database `global_temp`(configurable via SparkConf), and we must use the qualified name to refer a global temp view, e.g. SELECT * FROM global_temp.view1.
changes for `SessionCatalog`:
1. add a new field `gloabalTempViews: GlobalTempViewManager`, to access the shared global temp views, and the global temp db name.
2. `createDatabase` will fail if users wanna create `global_temp`, which is system preserved.
3. `setCurrentDatabase` will fail if users wanna set `global_temp`, which is system preserved.
4. add `createGlobalTempView`, which is used in `CreateViewCommand` to create global temp views.
5. add `dropGlobalTempView`, which is used in `CatalogImpl` to drop global temp view.
6. add `alterTempViewDefinition`, which is used in `AlterViewAsCommand` to update the view definition for local/global temp views.
7. `renameTable`/`dropTable`/`isTemporaryTable`/`lookupRelation`/`getTempViewOrPermanentTableMetadata`/`refreshTable` will handle global temp views.
changes for SQL commands:
1. `CreateViewCommand`/`AlterViewAsCommand` is updated to support global temp views
2. `ShowTablesCommand` outputs a new column `database`, which is used to distinguish global and local temp views.
3. other commands can also handle global temp views if they call `SessionCatalog` APIs which accepts global temp views, e.g. `DropTableCommand`, `AlterTableRenameCommand`, `ShowColumnsCommand`, etc.
changes for other public API
1. add a new method `dropGlobalTempView` in `Catalog`
2. `Catalog.findTable` can find global temp view
3. add a new method `createGlobalTempView` in `Dataset`
## How was this patch tested?
new tests in `SQLViewSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes #14897 from cloud-fan/global-temp-view.
2016-10-10 03:48:57 -04:00
|
|
|
"""Creates a local temporary view with this DataFrame.
|
2016-05-12 03:51:53 -04:00
|
|
|
|
|
|
|
The lifetime of this temporary table is tied to the :class:`SparkSession`
|
|
|
|
that was used to create this :class:`DataFrame`.
|
|
|
|
throws :class:`TempTableAlreadyExistsException`, if the view name already exists in the
|
|
|
|
catalog.
|
|
|
|
|
|
|
|
>>> df.createTempView("people")
|
|
|
|
>>> df2 = spark.sql("select * from people")
|
|
|
|
>>> sorted(df.collect()) == sorted(df2.collect())
|
|
|
|
True
|
|
|
|
>>> df.createTempView("people") # doctest: +IGNORE_EXCEPTION_DETAIL
|
|
|
|
Traceback (most recent call last):
|
|
|
|
...
|
[SPARK-14603][SQL][FOLLOWUP] Verification of Metadata Operations by Session Catalog
#### What changes were proposed in this pull request?
This follow-up PR is to address the remaining comments in https://github.com/apache/spark/pull/12385
The major change in this PR is to issue better error messages in PySpark by using the mechanism that was proposed by davies in https://github.com/apache/spark/pull/7135
For example, in PySpark, if we input the following statement:
```python
>>> l = [('Alice', 1)]
>>> df = sqlContext.createDataFrame(l)
>>> df.createTempView("people")
>>> df.createTempView("people")
```
Before this PR, the exception we will get is like
```
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/Users/xiaoli/IdeaProjects/sparkDelivery/python/pyspark/sql/dataframe.py", line 152, in createTempView
self._jdf.createTempView(name)
File "/Users/xiaoli/IdeaProjects/sparkDelivery/python/lib/py4j-0.10.1-src.zip/py4j/java_gateway.py", line 933, in __call__
File "/Users/xiaoli/IdeaProjects/sparkDelivery/python/pyspark/sql/utils.py", line 63, in deco
return f(*a, **kw)
File "/Users/xiaoli/IdeaProjects/sparkDelivery/python/lib/py4j-0.10.1-src.zip/py4j/protocol.py", line 312, in get_return_value
py4j.protocol.Py4JJavaError: An error occurred while calling o35.createTempView.
: org.apache.spark.sql.catalyst.analysis.TempTableAlreadyExistsException: Temporary table 'people' already exists;
at org.apache.spark.sql.catalyst.catalog.SessionCatalog.createTempView(SessionCatalog.scala:324)
at org.apache.spark.sql.SparkSession.createTempView(SparkSession.scala:523)
at org.apache.spark.sql.Dataset.createTempView(Dataset.scala:2328)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:606)
at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:237)
at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)
at py4j.Gateway.invoke(Gateway.java:280)
at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:128)
at py4j.commands.CallCommand.execute(CallCommand.java:79)
at py4j.GatewayConnection.run(GatewayConnection.java:211)
at java.lang.Thread.run(Thread.java:745)
```
After this PR, the exception we will get become cleaner:
```
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/Users/xiaoli/IdeaProjects/sparkDelivery/python/pyspark/sql/dataframe.py", line 152, in createTempView
self._jdf.createTempView(name)
File "/Users/xiaoli/IdeaProjects/sparkDelivery/python/lib/py4j-0.10.1-src.zip/py4j/java_gateway.py", line 933, in __call__
File "/Users/xiaoli/IdeaProjects/sparkDelivery/python/pyspark/sql/utils.py", line 75, in deco
raise AnalysisException(s.split(': ', 1)[1], stackTrace)
pyspark.sql.utils.AnalysisException: u"Temporary table 'people' already exists;"
```
#### How was this patch tested?
Fixed an existing PySpark test case
Author: gatorsmile <gatorsmile@gmail.com>
Closes #13126 from gatorsmile/followup-14684.
2016-05-19 14:46:11 -04:00
|
|
|
AnalysisException: u"Temporary table 'people' already exists;"
|
2016-05-12 03:51:53 -04:00
|
|
|
>>> spark.catalog.dropTempView("people")
|
|
|
|
|
|
|
|
"""
|
|
|
|
self._jdf.createTempView(name)
|
|
|
|
|
|
|
|
@since(2.0)
|
|
|
|
def createOrReplaceTempView(self, name):
|
[SPARK-17338][SQL] add global temp view
## What changes were proposed in this pull request?
Global temporary view is a cross-session temporary view, which means it's shared among all sessions. Its lifetime is the lifetime of the Spark application, i.e. it will be automatically dropped when the application terminates. It's tied to a system preserved database `global_temp`(configurable via SparkConf), and we must use the qualified name to refer a global temp view, e.g. SELECT * FROM global_temp.view1.
changes for `SessionCatalog`:
1. add a new field `gloabalTempViews: GlobalTempViewManager`, to access the shared global temp views, and the global temp db name.
2. `createDatabase` will fail if users wanna create `global_temp`, which is system preserved.
3. `setCurrentDatabase` will fail if users wanna set `global_temp`, which is system preserved.
4. add `createGlobalTempView`, which is used in `CreateViewCommand` to create global temp views.
5. add `dropGlobalTempView`, which is used in `CatalogImpl` to drop global temp view.
6. add `alterTempViewDefinition`, which is used in `AlterViewAsCommand` to update the view definition for local/global temp views.
7. `renameTable`/`dropTable`/`isTemporaryTable`/`lookupRelation`/`getTempViewOrPermanentTableMetadata`/`refreshTable` will handle global temp views.
changes for SQL commands:
1. `CreateViewCommand`/`AlterViewAsCommand` is updated to support global temp views
2. `ShowTablesCommand` outputs a new column `database`, which is used to distinguish global and local temp views.
3. other commands can also handle global temp views if they call `SessionCatalog` APIs which accepts global temp views, e.g. `DropTableCommand`, `AlterTableRenameCommand`, `ShowColumnsCommand`, etc.
changes for other public API
1. add a new method `dropGlobalTempView` in `Catalog`
2. `Catalog.findTable` can find global temp view
3. add a new method `createGlobalTempView` in `Dataset`
## How was this patch tested?
new tests in `SQLViewSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes #14897 from cloud-fan/global-temp-view.
2016-10-10 03:48:57 -04:00
|
|
|
"""Creates or replaces a local temporary view with this DataFrame.
|
2016-05-12 03:51:53 -04:00
|
|
|
|
|
|
|
The lifetime of this temporary table is tied to the :class:`SparkSession`
|
|
|
|
that was used to create this :class:`DataFrame`.
|
|
|
|
|
|
|
|
>>> df.createOrReplaceTempView("people")
|
|
|
|
>>> df2 = df.filter(df.age > 3)
|
|
|
|
>>> df2.createOrReplaceTempView("people")
|
|
|
|
>>> df3 = spark.sql("select * from people")
|
|
|
|
>>> sorted(df3.collect()) == sorted(df2.collect())
|
|
|
|
True
|
|
|
|
>>> spark.catalog.dropTempView("people")
|
|
|
|
|
2015-02-09 23:49:22 -05:00
|
|
|
"""
|
2016-05-12 03:51:53 -04:00
|
|
|
self._jdf.createOrReplaceTempView(name)
|
2015-02-09 23:49:22 -05:00
|
|
|
|
[SPARK-17338][SQL] add global temp view
## What changes were proposed in this pull request?
Global temporary view is a cross-session temporary view, which means it's shared among all sessions. Its lifetime is the lifetime of the Spark application, i.e. it will be automatically dropped when the application terminates. It's tied to a system preserved database `global_temp`(configurable via SparkConf), and we must use the qualified name to refer a global temp view, e.g. SELECT * FROM global_temp.view1.
changes for `SessionCatalog`:
1. add a new field `gloabalTempViews: GlobalTempViewManager`, to access the shared global temp views, and the global temp db name.
2. `createDatabase` will fail if users wanna create `global_temp`, which is system preserved.
3. `setCurrentDatabase` will fail if users wanna set `global_temp`, which is system preserved.
4. add `createGlobalTempView`, which is used in `CreateViewCommand` to create global temp views.
5. add `dropGlobalTempView`, which is used in `CatalogImpl` to drop global temp view.
6. add `alterTempViewDefinition`, which is used in `AlterViewAsCommand` to update the view definition for local/global temp views.
7. `renameTable`/`dropTable`/`isTemporaryTable`/`lookupRelation`/`getTempViewOrPermanentTableMetadata`/`refreshTable` will handle global temp views.
changes for SQL commands:
1. `CreateViewCommand`/`AlterViewAsCommand` is updated to support global temp views
2. `ShowTablesCommand` outputs a new column `database`, which is used to distinguish global and local temp views.
3. other commands can also handle global temp views if they call `SessionCatalog` APIs which accepts global temp views, e.g. `DropTableCommand`, `AlterTableRenameCommand`, `ShowColumnsCommand`, etc.
changes for other public API
1. add a new method `dropGlobalTempView` in `Catalog`
2. `Catalog.findTable` can find global temp view
3. add a new method `createGlobalTempView` in `Dataset`
## How was this patch tested?
new tests in `SQLViewSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes #14897 from cloud-fan/global-temp-view.
2016-10-10 03:48:57 -04:00
|
|
|
@since(2.1)
|
|
|
|
def createGlobalTempView(self, name):
|
|
|
|
"""Creates a global temporary view with this DataFrame.
|
|
|
|
|
|
|
|
The lifetime of this temporary view is tied to this Spark application.
|
|
|
|
throws :class:`TempTableAlreadyExistsException`, if the view name already exists in the
|
|
|
|
catalog.
|
|
|
|
|
|
|
|
>>> df.createGlobalTempView("people")
|
|
|
|
>>> df2 = spark.sql("select * from global_temp.people")
|
|
|
|
>>> sorted(df.collect()) == sorted(df2.collect())
|
|
|
|
True
|
|
|
|
>>> df.createGlobalTempView("people") # doctest: +IGNORE_EXCEPTION_DETAIL
|
|
|
|
Traceback (most recent call last):
|
|
|
|
...
|
|
|
|
AnalysisException: u"Temporary table 'people' already exists;"
|
|
|
|
>>> spark.catalog.dropGlobalTempView("people")
|
|
|
|
|
|
|
|
"""
|
|
|
|
self._jdf.createGlobalTempView(name)
|
|
|
|
|
2017-05-31 14:38:43 -04:00
|
|
|
@since(2.2)
|
|
|
|
def createOrReplaceGlobalTempView(self, name):
|
|
|
|
"""Creates or replaces a global temporary view using the given name.
|
|
|
|
|
|
|
|
The lifetime of this temporary view is tied to this Spark application.
|
|
|
|
|
|
|
|
>>> df.createOrReplaceGlobalTempView("people")
|
|
|
|
>>> df2 = df.filter(df.age > 3)
|
|
|
|
>>> df2.createOrReplaceGlobalTempView("people")
|
|
|
|
>>> df3 = spark.sql("select * from global_temp.people")
|
|
|
|
>>> sorted(df3.collect()) == sorted(df2.collect())
|
|
|
|
True
|
|
|
|
>>> spark.catalog.dropGlobalTempView("people")
|
|
|
|
|
|
|
|
"""
|
|
|
|
self._jdf.createOrReplaceGlobalTempView(name)
|
|
|
|
|
2015-05-19 17:23:28 -04:00
|
|
|
@property
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.4)
|
2015-05-19 17:23:28 -04:00
|
|
|
def write(self):
|
|
|
|
"""
|
2016-06-14 20:58:45 -04:00
|
|
|
Interface for saving the content of the non-streaming :class:`DataFrame` out into external
|
|
|
|
storage.
|
2015-05-19 17:23:28 -04:00
|
|
|
|
2015-06-03 03:23:34 -04:00
|
|
|
:return: :class:`DataFrameWriter`
|
2015-05-19 17:23:28 -04:00
|
|
|
"""
|
|
|
|
return DataFrameWriter(self)
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2016-06-14 20:58:45 -04:00
|
|
|
@property
|
|
|
|
@since(2.0)
|
|
|
|
def writeStream(self):
|
|
|
|
"""
|
|
|
|
Interface for saving the content of the streaming :class:`DataFrame` out into external
|
|
|
|
storage.
|
|
|
|
|
2017-05-26 16:33:23 -04:00
|
|
|
.. note:: Evolving.
|
2016-06-14 20:58:45 -04:00
|
|
|
|
|
|
|
:return: :class:`DataStreamWriter`
|
|
|
|
"""
|
|
|
|
return DataStreamWriter(self)
|
|
|
|
|
2015-02-14 02:03:22 -05:00
|
|
|
@property
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-02-09 23:49:22 -05:00
|
|
|
def schema(self):
|
2016-07-28 17:57:15 -04:00
|
|
|
"""Returns the schema of this :class:`DataFrame` as a :class:`pyspark.sql.types.StructType`.
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2015-02-14 02:03:22 -05:00
|
|
|
>>> df.schema
|
2015-02-09 23:49:22 -05:00
|
|
|
StructType(List(StructField(age,IntegerType,true),StructField(name,StringType,true)))
|
|
|
|
"""
|
2015-02-14 02:03:22 -05:00
|
|
|
if self._schema is None:
|
2015-06-17 14:10:16 -04:00
|
|
|
try:
|
|
|
|
self._schema = _parse_datatype_json_string(self._jdf.schema().json())
|
|
|
|
except AttributeError as e:
|
|
|
|
raise Exception(
|
|
|
|
"Unable to parse datatype from schema. %s" % e)
|
2015-02-14 02:03:22 -05:00
|
|
|
return self._schema
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-02-09 23:49:22 -05:00
|
|
|
def printSchema(self):
|
|
|
|
"""Prints out the schema in the tree format.
|
|
|
|
|
|
|
|
>>> df.printSchema()
|
|
|
|
root
|
|
|
|
|-- age: integer (nullable = true)
|
|
|
|
|-- name: string (nullable = true)
|
|
|
|
<BLANKLINE>
|
|
|
|
"""
|
2015-04-16 19:20:57 -04:00
|
|
|
print(self._jdf.schema().treeString())
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-02-17 13:22:48 -05:00
|
|
|
def explain(self, extended=False):
|
2015-03-31 21:31:36 -04:00
|
|
|
"""Prints the (logical and physical) plans to the console for debugging purpose.
|
2015-02-17 13:22:48 -05:00
|
|
|
|
2015-03-31 21:31:36 -04:00
|
|
|
:param extended: boolean, default ``False``. If ``False``, prints only the physical plan.
|
2015-02-17 16:48:38 -05:00
|
|
|
|
|
|
|
>>> df.explain()
|
2015-10-21 02:58:27 -04:00
|
|
|
== Physical Plan ==
|
2016-03-12 03:48:36 -05:00
|
|
|
Scan ExistingRDD[age#0,name#1]
|
2015-02-17 16:48:38 -05:00
|
|
|
|
|
|
|
>>> df.explain(True)
|
|
|
|
== Parsed Logical Plan ==
|
|
|
|
...
|
|
|
|
== Analyzed Logical Plan ==
|
|
|
|
...
|
|
|
|
== Optimized Logical Plan ==
|
|
|
|
...
|
|
|
|
== Physical Plan ==
|
|
|
|
...
|
|
|
|
"""
|
|
|
|
if extended:
|
2015-04-16 19:20:57 -04:00
|
|
|
print(self._jdf.queryExecution().toString())
|
2015-02-17 16:48:38 -05:00
|
|
|
else:
|
2015-10-21 02:58:27 -04:00
|
|
|
print(self._jdf.queryExecution().simpleString())
|
2015-02-17 13:22:48 -05:00
|
|
|
|
2018-07-27 16:47:33 -04:00
|
|
|
@since(2.4)
|
|
|
|
def exceptAll(self, other):
|
|
|
|
"""Return a new :class:`DataFrame` containing rows in this :class:`DataFrame` but
|
|
|
|
not in another :class:`DataFrame` while preserving duplicates.
|
|
|
|
|
|
|
|
This is equivalent to `EXCEPT ALL` in SQL.
|
|
|
|
|
|
|
|
>>> df1 = spark.createDataFrame(
|
|
|
|
... [("a", 1), ("a", 1), ("a", 1), ("a", 2), ("b", 3), ("c", 4)], ["C1", "C2"])
|
|
|
|
>>> df2 = spark.createDataFrame([("a", 1), ("b", 3)], ["C1", "C2"])
|
|
|
|
|
|
|
|
>>> df1.exceptAll(df2).show()
|
|
|
|
+---+---+
|
|
|
|
| C1| C2|
|
|
|
|
+---+---+
|
|
|
|
| a| 1|
|
|
|
|
| a| 1|
|
|
|
|
| a| 2|
|
|
|
|
| c| 4|
|
|
|
|
+---+---+
|
|
|
|
|
|
|
|
Also as standard in SQL, this function resolves columns by position (not by name).
|
|
|
|
"""
|
|
|
|
return DataFrame(self._jdf.exceptAll(other._jdf), self.sql_ctx)
|
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-02-17 13:22:48 -05:00
|
|
|
def isLocal(self):
|
2015-03-31 21:31:36 -04:00
|
|
|
"""Returns ``True`` if the :func:`collect` and :func:`take` methods can be run locally
|
2015-02-17 13:22:48 -05:00
|
|
|
(without any Spark executors).
|
|
|
|
"""
|
|
|
|
return self._jdf.isLocal()
|
|
|
|
|
2016-04-20 13:32:01 -04:00
|
|
|
@property
|
|
|
|
@since(2.0)
|
|
|
|
def isStreaming(self):
|
|
|
|
"""Returns true if this :class:`Dataset` contains one or more sources that continuously
|
|
|
|
return data as it arrives. A :class:`Dataset` that reads data from a streaming source
|
2016-06-28 20:02:16 -04:00
|
|
|
must be executed as a :class:`StreamingQuery` using the :func:`start` method in
|
|
|
|
:class:`DataStreamWriter`. Methods that return a single answer, (e.g., :func:`count` or
|
2016-04-20 13:32:01 -04:00
|
|
|
:func:`collect`) will throw an :class:`AnalysisException` when there is a streaming
|
|
|
|
source present.
|
2016-05-11 18:12:27 -04:00
|
|
|
|
2017-05-26 16:33:23 -04:00
|
|
|
.. note:: Evolving
|
2016-04-20 13:32:01 -04:00
|
|
|
"""
|
|
|
|
return self._jdf.isStreaming()
|
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2017-04-27 01:18:01 -04:00
|
|
|
def show(self, n=20, truncate=True, vertical=False):
|
2015-03-31 21:31:36 -04:00
|
|
|
"""Prints the first ``n`` rows to the console.
|
2015-02-11 15:13:16 -05:00
|
|
|
|
2015-06-30 02:44:11 -04:00
|
|
|
:param n: Number of rows to show.
|
2016-06-28 07:41:06 -04:00
|
|
|
:param truncate: If set to True, truncate strings longer than 20 chars by default.
|
|
|
|
If set to a number greater than one, truncates long strings to length ``truncate``
|
|
|
|
and align cells right.
|
2017-04-27 01:18:01 -04:00
|
|
|
:param vertical: If set to True, print output rows vertically (one line
|
|
|
|
per column value).
|
2015-06-30 02:44:11 -04:00
|
|
|
|
2015-02-24 23:51:55 -05:00
|
|
|
>>> df
|
|
|
|
DataFrame[age: int, name: string]
|
2015-02-11 15:13:16 -05:00
|
|
|
>>> df.show()
|
2015-05-04 15:08:38 -04:00
|
|
|
+---+-----+
|
|
|
|
|age| name|
|
|
|
|
+---+-----+
|
|
|
|
| 2|Alice|
|
|
|
|
| 5| Bob|
|
|
|
|
+---+-----+
|
2016-06-28 07:41:06 -04:00
|
|
|
>>> df.show(truncate=3)
|
|
|
|
+---+----+
|
|
|
|
|age|name|
|
|
|
|
+---+----+
|
|
|
|
| 2| Ali|
|
|
|
|
| 5| Bob|
|
|
|
|
+---+----+
|
2017-04-27 01:18:01 -04:00
|
|
|
>>> df.show(vertical=True)
|
|
|
|
-RECORD 0-----
|
|
|
|
age | 2
|
|
|
|
name | Alice
|
|
|
|
-RECORD 1-----
|
|
|
|
age | 5
|
|
|
|
name | Bob
|
2015-02-11 15:13:16 -05:00
|
|
|
"""
|
2016-06-28 07:41:06 -04:00
|
|
|
if isinstance(truncate, bool) and truncate:
|
2017-04-27 01:18:01 -04:00
|
|
|
print(self._jdf.showString(n, 20, vertical))
|
2016-06-28 07:41:06 -04:00
|
|
|
else:
|
2017-04-27 01:18:01 -04:00
|
|
|
print(self._jdf.showString(n, int(truncate), vertical))
|
2015-02-11 15:13:16 -05:00
|
|
|
|
|
|
|
def __repr__(self):
|
2018-07-02 02:35:37 -04:00
|
|
|
if not self._support_repr_html and self.sql_ctx._conf.isReplEagerEvalEnabled():
|
2018-06-04 21:23:08 -04:00
|
|
|
vertical = False
|
|
|
|
return self._jdf.showString(
|
2018-07-02 02:35:37 -04:00
|
|
|
self.sql_ctx._conf.replEagerEvalMaxNumRows(),
|
|
|
|
self.sql_ctx._conf.replEagerEvalTruncate(), vertical)
|
2018-06-04 21:23:08 -04:00
|
|
|
else:
|
|
|
|
return "DataFrame[%s]" % (", ".join("%s: %s" % c for c in self.dtypes))
|
|
|
|
|
|
|
|
def _repr_html_(self):
|
|
|
|
"""Returns a dataframe with html code when you enabled eager evaluation
|
|
|
|
by 'spark.sql.repl.eagerEval.enabled', this only called by REPL you are
|
|
|
|
using support eager evaluation with HTML.
|
|
|
|
"""
|
|
|
|
import cgi
|
|
|
|
if not self._support_repr_html:
|
|
|
|
self._support_repr_html = True
|
2018-07-02 02:35:37 -04:00
|
|
|
if self.sql_ctx._conf.isReplEagerEvalEnabled():
|
|
|
|
max_num_rows = max(self.sql_ctx._conf.replEagerEvalMaxNumRows(), 0)
|
2018-06-04 21:23:08 -04:00
|
|
|
sock_info = self._jdf.getRowsToPython(
|
2018-07-02 02:35:37 -04:00
|
|
|
max_num_rows, self.sql_ctx._conf.replEagerEvalTruncate())
|
2018-06-04 21:23:08 -04:00
|
|
|
rows = list(_load_from_socket(sock_info, BatchedSerializer(PickleSerializer())))
|
|
|
|
head = rows[0]
|
|
|
|
row_data = rows[1:]
|
|
|
|
has_more_data = len(row_data) > max_num_rows
|
|
|
|
row_data = row_data[:max_num_rows]
|
|
|
|
|
|
|
|
html = "<table border='1'>\n"
|
|
|
|
# generate table head
|
|
|
|
html += "<tr><th>%s</th></tr>\n" % "</th><th>".join(map(lambda x: cgi.escape(x), head))
|
|
|
|
# generate table rows
|
|
|
|
for row in row_data:
|
|
|
|
html += "<tr><td>%s</td></tr>\n" % "</td><td>".join(
|
|
|
|
map(lambda x: cgi.escape(x), row))
|
|
|
|
html += "</table>\n"
|
|
|
|
if has_more_data:
|
|
|
|
html += "only showing top %d %s\n" % (
|
|
|
|
max_num_rows, "row" if max_num_rows == 1 else "rows")
|
|
|
|
return html
|
|
|
|
else:
|
|
|
|
return None
|
2015-02-11 15:13:16 -05:00
|
|
|
|
2016-11-21 20:24:02 -05:00
|
|
|
@since(2.1)
|
|
|
|
def checkpoint(self, eager=True):
|
|
|
|
"""Returns a checkpointed version of this Dataset. Checkpointing can be used to truncate the
|
|
|
|
logical plan of this DataFrame, which is especially useful in iterative algorithms where the
|
|
|
|
plan may grow exponentially. It will be saved to files inside the checkpoint
|
|
|
|
directory set with L{SparkContext.setCheckpointDir()}.
|
|
|
|
|
|
|
|
:param eager: Whether to checkpoint this DataFrame immediately
|
|
|
|
|
|
|
|
.. note:: Experimental
|
|
|
|
"""
|
|
|
|
jdf = self._jdf.checkpoint(eager)
|
|
|
|
return DataFrame(jdf, self.sql_ctx)
|
|
|
|
|
[SPARK-22649][PYTHON][SQL] Adding localCheckpoint to Dataset API
## What changes were proposed in this pull request?
This change adds local checkpoint support to datasets and respective bind from Python Dataframe API.
If reliability requirements can be lowered to favor performance, as in cases of further quick transformations followed by a reliable save, localCheckpoints() fit very well.
Furthermore, at the moment Reliable checkpoints still incur double computation (see #9428)
In general it makes the API more complete as well.
## How was this patch tested?
Python land quick use case:
```python
>>> from time import sleep
>>> from pyspark.sql import types as T
>>> from pyspark.sql import functions as F
>>> def f(x):
sleep(1)
return x*2
...:
>>> df1 = spark.range(30, numPartitions=6)
>>> df2 = df1.select(F.udf(f, T.LongType())("id"))
>>> %time _ = df2.collect()
CPU times: user 7.79 ms, sys: 5.84 ms, total: 13.6 ms
Wall time: 12.2 s
>>> %time df3 = df2.localCheckpoint()
CPU times: user 2.38 ms, sys: 2.3 ms, total: 4.68 ms
Wall time: 10.3 s
>>> %time _ = df3.collect()
CPU times: user 5.09 ms, sys: 410 µs, total: 5.5 ms
Wall time: 148 ms
>>> sc.setCheckpointDir(".")
>>> %time df3 = df2.checkpoint()
CPU times: user 4.04 ms, sys: 1.63 ms, total: 5.67 ms
Wall time: 20.3 s
```
Author: Fernando Pereira <fernando.pereira@epfl.ch>
Closes #19805 from ferdonline/feature_dataset_localCheckpoint.
2017-12-19 23:47:12 -05:00
|
|
|
@since(2.3)
|
|
|
|
def localCheckpoint(self, eager=True):
|
|
|
|
"""Returns a locally checkpointed version of this Dataset. Checkpointing can be used to
|
|
|
|
truncate the logical plan of this DataFrame, which is especially useful in iterative
|
|
|
|
algorithms where the plan may grow exponentially. Local checkpoints are stored in the
|
|
|
|
executors using the caching subsystem and therefore they are not reliable.
|
|
|
|
|
|
|
|
:param eager: Whether to checkpoint this DataFrame immediately
|
|
|
|
|
|
|
|
.. note:: Experimental
|
|
|
|
"""
|
|
|
|
jdf = self._jdf.localCheckpoint(eager)
|
|
|
|
return DataFrame(jdf, self.sql_ctx)
|
|
|
|
|
2016-11-21 20:24:02 -05:00
|
|
|
@since(2.1)
|
|
|
|
def withWatermark(self, eventTime, delayThreshold):
|
|
|
|
"""Defines an event time watermark for this :class:`DataFrame`. A watermark tracks a point
|
|
|
|
in time before which we assume no more late data is going to arrive.
|
|
|
|
|
|
|
|
Spark will use this watermark for several purposes:
|
|
|
|
- To know when a given time window aggregation can be finalized and thus can be emitted
|
|
|
|
when using output modes that do not allow updates.
|
|
|
|
|
|
|
|
- To minimize the amount of state that we need to keep for on-going aggregations.
|
|
|
|
|
|
|
|
The current watermark is computed by looking at the `MAX(eventTime)` seen across
|
|
|
|
all of the partitions in the query minus a user specified `delayThreshold`. Due to the cost
|
|
|
|
of coordinating this value across partitions, the actual watermark used is only guaranteed
|
|
|
|
to be at least `delayThreshold` behind the actual event time. In some cases we may still
|
|
|
|
process records that arrive more than `delayThreshold` late.
|
|
|
|
|
|
|
|
:param eventTime: the name of the column that contains the event time of the row.
|
|
|
|
:param delayThreshold: the minimum delay to wait to data to arrive late, relative to the
|
|
|
|
latest record that has been processed in the form of an interval
|
|
|
|
(e.g. "1 minute" or "5 hours").
|
|
|
|
|
2017-05-26 16:33:23 -04:00
|
|
|
.. note:: Evolving
|
2016-11-21 20:24:02 -05:00
|
|
|
|
|
|
|
>>> sdf.select('name', sdf.time.cast('timestamp')).withWatermark('time', '10 minutes')
|
|
|
|
DataFrame[name: string, time: timestamp]
|
|
|
|
"""
|
|
|
|
if not eventTime or type(eventTime) is not str:
|
|
|
|
raise TypeError("eventTime should be provided as a string")
|
|
|
|
if not delayThreshold or type(delayThreshold) is not str:
|
|
|
|
raise TypeError("delayThreshold should be provided as a string interval")
|
|
|
|
jdf = self._jdf.withWatermark(eventTime, delayThreshold)
|
|
|
|
return DataFrame(jdf, self.sql_ctx)
|
|
|
|
|
2017-05-03 22:15:28 -04:00
|
|
|
@since(2.2)
|
|
|
|
def hint(self, name, *parameters):
|
|
|
|
"""Specifies some hint on the current DataFrame.
|
|
|
|
|
|
|
|
:param name: A name of the hint.
|
|
|
|
:param parameters: Optional parameters.
|
|
|
|
:return: :class:`DataFrame`
|
|
|
|
|
|
|
|
>>> df.join(df2.hint("broadcast"), "name").show()
|
|
|
|
+----+---+------+
|
|
|
|
|name|age|height|
|
|
|
|
+----+---+------+
|
|
|
|
| Bob| 5| 85|
|
|
|
|
+----+---+------+
|
|
|
|
"""
|
|
|
|
if len(parameters) == 1 and isinstance(parameters[0], list):
|
|
|
|
parameters = parameters[0]
|
|
|
|
|
|
|
|
if not isinstance(name, str):
|
|
|
|
raise TypeError("name should be provided as str, got {0}".format(type(name)))
|
|
|
|
|
|
|
|
for p in parameters:
|
|
|
|
if not isinstance(p, str):
|
|
|
|
raise TypeError(
|
|
|
|
"all parameters should be str, got {0} of type {1}".format(p, type(p)))
|
|
|
|
|
|
|
|
jdf = self._jdf.hint(name, self._jseq(parameters))
|
|
|
|
return DataFrame(jdf, self.sql_ctx)
|
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-02-09 23:49:22 -05:00
|
|
|
def count(self):
|
2015-03-31 21:31:36 -04:00
|
|
|
"""Returns the number of rows in this :class:`DataFrame`.
|
2015-02-09 23:49:22 -05:00
|
|
|
|
|
|
|
>>> df.count()
|
2015-04-16 19:20:57 -04:00
|
|
|
2
|
2015-02-09 23:49:22 -05:00
|
|
|
"""
|
2015-04-16 19:20:57 -04:00
|
|
|
return int(self._jdf.count())
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2015-04-16 19:20:57 -04:00
|
|
|
@ignore_unicode_prefix
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-02-09 23:49:22 -05:00
|
|
|
def collect(self):
|
2015-03-31 21:31:36 -04:00
|
|
|
"""Returns all the records as a list of :class:`Row`.
|
2015-02-09 23:49:22 -05:00
|
|
|
|
|
|
|
>>> df.collect()
|
|
|
|
[Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')]
|
|
|
|
"""
|
|
|
|
with SCCallSiteSync(self._sc) as css:
|
2018-04-13 17:28:24 -04:00
|
|
|
sock_info = self._jdf.collectToPython()
|
|
|
|
return list(_load_from_socket(sock_info, BatchedSerializer(PickleSerializer())))
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2016-04-04 16:31:44 -04:00
|
|
|
@ignore_unicode_prefix
|
|
|
|
@since(2.0)
|
|
|
|
def toLocalIterator(self):
|
|
|
|
"""
|
|
|
|
Returns an iterator that contains all of the rows in this :class:`DataFrame`.
|
|
|
|
The iterator will consume as much memory as the largest partition in this DataFrame.
|
|
|
|
|
|
|
|
>>> list(df.toLocalIterator())
|
|
|
|
[Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')]
|
|
|
|
"""
|
|
|
|
with SCCallSiteSync(self._sc) as css:
|
2018-04-13 17:28:24 -04:00
|
|
|
sock_info = self._jdf.toPythonIterator()
|
|
|
|
return _load_from_socket(sock_info, BatchedSerializer(PickleSerializer()))
|
2016-04-04 16:31:44 -04:00
|
|
|
|
2015-04-16 19:20:57 -04:00
|
|
|
@ignore_unicode_prefix
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-02-09 23:49:22 -05:00
|
|
|
def limit(self, num):
|
2015-03-31 21:31:36 -04:00
|
|
|
"""Limits the result count to the number specified.
|
2015-02-09 23:49:22 -05:00
|
|
|
|
|
|
|
>>> df.limit(1).collect()
|
|
|
|
[Row(age=2, name=u'Alice')]
|
|
|
|
>>> df.limit(0).collect()
|
|
|
|
[]
|
|
|
|
"""
|
|
|
|
jdf = self._jdf.limit(num)
|
|
|
|
return DataFrame(jdf, self.sql_ctx)
|
|
|
|
|
2015-04-16 19:20:57 -04:00
|
|
|
@ignore_unicode_prefix
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-02-09 23:49:22 -05:00
|
|
|
def take(self, num):
|
2015-03-31 21:31:36 -04:00
|
|
|
"""Returns the first ``num`` rows as a :class:`list` of :class:`Row`.
|
2015-02-09 23:49:22 -05:00
|
|
|
|
|
|
|
>>> df.take(2)
|
|
|
|
[Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')]
|
|
|
|
"""
|
[SPARK-17514] df.take(1) and df.limit(1).collect() should perform the same in Python
## What changes were proposed in this pull request?
In PySpark, `df.take(1)` runs a single-stage job which computes only one partition of the DataFrame, while `df.limit(1).collect()` computes all partitions and runs a two-stage job. This difference in performance is confusing.
The reason why `limit(1).collect()` is so much slower is that `collect()` internally maps to `df.rdd.<some-pyspark-conversions>.toLocalIterator`, which causes Spark SQL to build a query where a global limit appears in the middle of the plan; this, in turn, ends up being executed inefficiently because limits in the middle of plans are now implemented by repartitioning to a single task rather than by running a `take()` job on the driver (this was done in #7334, a patch which was a prerequisite to allowing partition-local limits to be pushed beneath unions, etc.).
In order to fix this performance problem I think that we should generalize the fix from SPARK-10731 / #8876 so that `DataFrame.collect()` also delegates to the Scala implementation and shares the same performance properties. This patch modifies `DataFrame.collect()` to first collect all results to the driver and then pass them to Python, allowing this query to be planned using Spark's `CollectLimit` optimizations.
## How was this patch tested?
Added a regression test in `sql/tests.py` which asserts that the expected number of jobs, stages, and tasks are run for both queries.
Author: Josh Rosen <joshrosen@databricks.com>
Closes #15068 from JoshRosen/pyspark-collect-limit.
2016-09-14 13:10:01 -04:00
|
|
|
return self.limit(num).collect()
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-02-17 13:22:48 -05:00
|
|
|
def foreach(self, f):
|
2015-03-31 21:31:36 -04:00
|
|
|
"""Applies the ``f`` function to all :class:`Row` of this :class:`DataFrame`.
|
2015-02-17 13:22:48 -05:00
|
|
|
|
2015-03-31 21:31:36 -04:00
|
|
|
This is a shorthand for ``df.rdd.foreach()``.
|
2015-02-17 13:22:48 -05:00
|
|
|
|
|
|
|
>>> def f(person):
|
2015-04-16 19:20:57 -04:00
|
|
|
... print(person.name)
|
2015-02-17 13:22:48 -05:00
|
|
|
>>> df.foreach(f)
|
|
|
|
"""
|
2016-03-02 18:26:34 -05:00
|
|
|
self.rdd.foreach(f)
|
2015-02-17 13:22:48 -05:00
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-02-17 13:22:48 -05:00
|
|
|
def foreachPartition(self, f):
|
2015-03-31 21:31:36 -04:00
|
|
|
"""Applies the ``f`` function to each partition of this :class:`DataFrame`.
|
2015-02-17 13:22:48 -05:00
|
|
|
|
2015-03-31 21:31:36 -04:00
|
|
|
This a shorthand for ``df.rdd.foreachPartition()``.
|
2015-02-17 13:22:48 -05:00
|
|
|
|
|
|
|
>>> def f(people):
|
|
|
|
... for person in people:
|
2015-04-16 19:20:57 -04:00
|
|
|
... print(person.name)
|
2015-02-17 13:22:48 -05:00
|
|
|
>>> df.foreachPartition(f)
|
|
|
|
"""
|
2016-03-02 18:26:34 -05:00
|
|
|
self.rdd.foreachPartition(f)
|
2015-02-17 13:22:48 -05:00
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-02-09 23:49:22 -05:00
|
|
|
def cache(self):
|
2016-10-14 18:07:32 -04:00
|
|
|
"""Persists the :class:`DataFrame` with the default storage level (C{MEMORY_AND_DISK}).
|
|
|
|
|
2016-11-22 06:40:18 -05:00
|
|
|
.. note:: The default storage level has changed to C{MEMORY_AND_DISK} to match Scala in 2.0.
|
2015-02-09 23:49:22 -05:00
|
|
|
"""
|
|
|
|
self.is_cached = True
|
|
|
|
self._jdf.cache()
|
|
|
|
return self
|
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2016-10-14 18:07:32 -04:00
|
|
|
def persist(self, storageLevel=StorageLevel.MEMORY_AND_DISK):
|
|
|
|
"""Sets the storage level to persist the contents of the :class:`DataFrame` across
|
|
|
|
operations after the first time it is computed. This can only be used to assign
|
|
|
|
a new storage level if the :class:`DataFrame` does not have a storage level set yet.
|
|
|
|
If no storage level is specified defaults to (C{MEMORY_AND_DISK}).
|
|
|
|
|
2016-11-22 06:40:18 -05:00
|
|
|
.. note:: The default storage level has changed to C{MEMORY_AND_DISK} to match Scala in 2.0.
|
2015-02-09 23:49:22 -05:00
|
|
|
"""
|
|
|
|
self.is_cached = True
|
|
|
|
javaStorageLevel = self._sc._getJavaStorageLevel(storageLevel)
|
|
|
|
self._jdf.persist(javaStorageLevel)
|
|
|
|
return self
|
|
|
|
|
2016-10-14 18:07:32 -04:00
|
|
|
@property
|
|
|
|
@since(2.1)
|
|
|
|
def storageLevel(self):
|
|
|
|
"""Get the :class:`DataFrame`'s current storage level.
|
|
|
|
|
|
|
|
>>> df.storageLevel
|
|
|
|
StorageLevel(False, False, False, False, 1)
|
|
|
|
>>> df.cache().storageLevel
|
|
|
|
StorageLevel(True, True, False, True, 1)
|
|
|
|
>>> df2.persist(StorageLevel.DISK_ONLY_2).storageLevel
|
|
|
|
StorageLevel(True, False, False, False, 2)
|
|
|
|
"""
|
|
|
|
java_storage_level = self._jdf.storageLevel()
|
|
|
|
storage_level = StorageLevel(java_storage_level.useDisk(),
|
|
|
|
java_storage_level.useMemory(),
|
|
|
|
java_storage_level.useOffHeap(),
|
|
|
|
java_storage_level.deserialized(),
|
|
|
|
java_storage_level.replication())
|
|
|
|
return storage_level
|
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2016-04-19 20:29:28 -04:00
|
|
|
def unpersist(self, blocking=False):
|
2015-03-31 21:31:36 -04:00
|
|
|
"""Marks the :class:`DataFrame` as non-persistent, and remove all blocks for it from
|
2015-02-09 23:49:22 -05:00
|
|
|
memory and disk.
|
2016-04-19 20:29:28 -04:00
|
|
|
|
|
|
|
.. note:: `blocking` default has changed to False to match Scala in 2.0.
|
2015-02-09 23:49:22 -05:00
|
|
|
"""
|
|
|
|
self.is_cached = False
|
|
|
|
self._jdf.unpersist(blocking)
|
|
|
|
return self
|
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.4)
|
2015-05-19 17:23:28 -04:00
|
|
|
def coalesce(self, numPartitions):
|
|
|
|
"""
|
|
|
|
Returns a new :class:`DataFrame` that has exactly `numPartitions` partitions.
|
|
|
|
|
2018-03-05 09:17:16 -05:00
|
|
|
:param numPartitions: int, to specify the target number of partitions
|
|
|
|
|
2015-05-19 17:23:28 -04:00
|
|
|
Similar to coalesce defined on an :class:`RDD`, this operation results in a
|
|
|
|
narrow dependency, e.g. if you go from 1000 partitions to 100 partitions,
|
|
|
|
there will not be a shuffle, instead each of the 100 new partitions will
|
2017-02-15 13:45:37 -05:00
|
|
|
claim 10 of the current partitions. If a larger number of partitions is requested,
|
|
|
|
it will stay at the current number of partitions.
|
|
|
|
|
|
|
|
However, if you're doing a drastic coalesce, e.g. to numPartitions = 1,
|
|
|
|
this may result in your computation taking place on fewer nodes than
|
|
|
|
you like (e.g. one node in the case of numPartitions = 1). To avoid this,
|
|
|
|
you can call repartition(). This will add a shuffle step, but means the
|
|
|
|
current upstream partitions will be executed in parallel (per whatever
|
|
|
|
the current partitioning is).
|
2015-05-19 17:23:28 -04:00
|
|
|
|
|
|
|
>>> df.coalesce(1).rdd.getNumPartitions()
|
|
|
|
1
|
|
|
|
"""
|
|
|
|
return DataFrame(self._jdf.coalesce(numPartitions), self.sql_ctx)
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2015-11-06 18:48:20 -05:00
|
|
|
@since(1.3)
|
|
|
|
def repartition(self, numPartitions, *cols):
|
|
|
|
"""
|
|
|
|
Returns a new :class:`DataFrame` partitioned by the given partitioning expressions. The
|
|
|
|
resulting DataFrame is hash partitioned.
|
|
|
|
|
2018-03-05 09:17:16 -05:00
|
|
|
:param numPartitions:
|
|
|
|
can be an int to specify the target number of partitions or a Column.
|
|
|
|
If it is a Column, it will be used as the first partitioning column. If not specified,
|
|
|
|
the default number of partitions is used.
|
2015-11-06 18:48:20 -05:00
|
|
|
|
|
|
|
.. versionchanged:: 1.6
|
|
|
|
Added optional arguments to specify the partitioning columns. Also made numPartitions
|
|
|
|
optional if partitioning columns are specified.
|
|
|
|
|
|
|
|
>>> df.repartition(10).rdd.getNumPartitions()
|
|
|
|
10
|
2016-03-25 01:34:55 -04:00
|
|
|
>>> data = df.union(df).repartition("age")
|
2015-11-06 18:48:20 -05:00
|
|
|
>>> data.show()
|
|
|
|
+---+-----+
|
|
|
|
|age| name|
|
|
|
|
+---+-----+
|
|
|
|
| 5| Bob|
|
|
|
|
| 5| Bob|
|
2016-01-14 01:43:28 -05:00
|
|
|
| 2|Alice|
|
|
|
|
| 2|Alice|
|
2015-11-06 18:48:20 -05:00
|
|
|
+---+-----+
|
|
|
|
>>> data = data.repartition(7, "age")
|
|
|
|
>>> data.show()
|
|
|
|
+---+-----+
|
|
|
|
|age| name|
|
|
|
|
+---+-----+
|
|
|
|
| 2|Alice|
|
[SPARK-16052][SQL] Improve `CollapseRepartition` optimizer for Repartition/RepartitionBy
## What changes were proposed in this pull request?
This PR improves `CollapseRepartition` to optimize the adjacent combinations of **Repartition** and **RepartitionBy**. Also, this PR adds a testsuite for this optimizer.
**Target Scenario**
```scala
scala> val dsView1 = spark.range(8).repartition(8, $"id")
scala> dsView1.createOrReplaceTempView("dsView1")
scala> sql("select id from dsView1 distribute by id").explain(true)
```
**Before**
```scala
scala> sql("select id from dsView1 distribute by id").explain(true)
== Parsed Logical Plan ==
'RepartitionByExpression ['id]
+- 'Project ['id]
+- 'UnresolvedRelation `dsView1`
== Analyzed Logical Plan ==
id: bigint
RepartitionByExpression [id#0L]
+- Project [id#0L]
+- SubqueryAlias dsview1
+- RepartitionByExpression [id#0L], 8
+- Range (0, 8, splits=8)
== Optimized Logical Plan ==
RepartitionByExpression [id#0L]
+- RepartitionByExpression [id#0L], 8
+- Range (0, 8, splits=8)
== Physical Plan ==
Exchange hashpartitioning(id#0L, 200)
+- Exchange hashpartitioning(id#0L, 8)
+- *Range (0, 8, splits=8)
```
**After**
```scala
scala> sql("select id from dsView1 distribute by id").explain(true)
== Parsed Logical Plan ==
'RepartitionByExpression ['id]
+- 'Project ['id]
+- 'UnresolvedRelation `dsView1`
== Analyzed Logical Plan ==
id: bigint
RepartitionByExpression [id#0L]
+- Project [id#0L]
+- SubqueryAlias dsview1
+- RepartitionByExpression [id#0L], 8
+- Range (0, 8, splits=8)
== Optimized Logical Plan ==
RepartitionByExpression [id#0L]
+- Range (0, 8, splits=8)
== Physical Plan ==
Exchange hashpartitioning(id#0L, 200)
+- *Range (0, 8, splits=8)
```
## How was this patch tested?
Pass the Jenkins tests (including a new testsuite).
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes #13765 from dongjoon-hyun/SPARK-16052.
2016-07-08 04:44:53 -04:00
|
|
|
| 5| Bob|
|
2015-11-06 18:48:20 -05:00
|
|
|
| 2|Alice|
|
[SPARK-16052][SQL] Improve `CollapseRepartition` optimizer for Repartition/RepartitionBy
## What changes were proposed in this pull request?
This PR improves `CollapseRepartition` to optimize the adjacent combinations of **Repartition** and **RepartitionBy**. Also, this PR adds a testsuite for this optimizer.
**Target Scenario**
```scala
scala> val dsView1 = spark.range(8).repartition(8, $"id")
scala> dsView1.createOrReplaceTempView("dsView1")
scala> sql("select id from dsView1 distribute by id").explain(true)
```
**Before**
```scala
scala> sql("select id from dsView1 distribute by id").explain(true)
== Parsed Logical Plan ==
'RepartitionByExpression ['id]
+- 'Project ['id]
+- 'UnresolvedRelation `dsView1`
== Analyzed Logical Plan ==
id: bigint
RepartitionByExpression [id#0L]
+- Project [id#0L]
+- SubqueryAlias dsview1
+- RepartitionByExpression [id#0L], 8
+- Range (0, 8, splits=8)
== Optimized Logical Plan ==
RepartitionByExpression [id#0L]
+- RepartitionByExpression [id#0L], 8
+- Range (0, 8, splits=8)
== Physical Plan ==
Exchange hashpartitioning(id#0L, 200)
+- Exchange hashpartitioning(id#0L, 8)
+- *Range (0, 8, splits=8)
```
**After**
```scala
scala> sql("select id from dsView1 distribute by id").explain(true)
== Parsed Logical Plan ==
'RepartitionByExpression ['id]
+- 'Project ['id]
+- 'UnresolvedRelation `dsView1`
== Analyzed Logical Plan ==
id: bigint
RepartitionByExpression [id#0L]
+- Project [id#0L]
+- SubqueryAlias dsview1
+- RepartitionByExpression [id#0L], 8
+- Range (0, 8, splits=8)
== Optimized Logical Plan ==
RepartitionByExpression [id#0L]
+- Range (0, 8, splits=8)
== Physical Plan ==
Exchange hashpartitioning(id#0L, 200)
+- *Range (0, 8, splits=8)
```
## How was this patch tested?
Pass the Jenkins tests (including a new testsuite).
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes #13765 from dongjoon-hyun/SPARK-16052.
2016-07-08 04:44:53 -04:00
|
|
|
| 5| Bob|
|
2015-11-06 18:48:20 -05:00
|
|
|
+---+-----+
|
|
|
|
>>> data.rdd.getNumPartitions()
|
|
|
|
7
|
|
|
|
>>> data = data.repartition("name", "age")
|
|
|
|
>>> data.show()
|
|
|
|
+---+-----+
|
|
|
|
|age| name|
|
|
|
|
+---+-----+
|
|
|
|
| 5| Bob|
|
|
|
|
| 5| Bob|
|
|
|
|
| 2|Alice|
|
|
|
|
| 2|Alice|
|
|
|
|
+---+-----+
|
|
|
|
"""
|
|
|
|
if isinstance(numPartitions, int):
|
|
|
|
if len(cols) == 0:
|
|
|
|
return DataFrame(self._jdf.repartition(numPartitions), self.sql_ctx)
|
|
|
|
else:
|
|
|
|
return DataFrame(
|
|
|
|
self._jdf.repartition(numPartitions, self._jcols(*cols)), self.sql_ctx)
|
|
|
|
elif isinstance(numPartitions, (basestring, Column)):
|
|
|
|
cols = (numPartitions, ) + cols
|
|
|
|
return DataFrame(self._jdf.repartition(self._jcols(*cols)), self.sql_ctx)
|
|
|
|
else:
|
|
|
|
raise TypeError("numPartitions should be an int or Column")
|
|
|
|
|
2018-02-11 05:23:15 -05:00
|
|
|
@since("2.4.0")
|
|
|
|
def repartitionByRange(self, numPartitions, *cols):
|
|
|
|
"""
|
|
|
|
Returns a new :class:`DataFrame` partitioned by the given partitioning expressions. The
|
|
|
|
resulting DataFrame is range partitioned.
|
|
|
|
|
2018-03-05 09:17:16 -05:00
|
|
|
:param numPartitions:
|
|
|
|
can be an int to specify the target number of partitions or a Column.
|
|
|
|
If it is a Column, it will be used as the first partitioning column. If not specified,
|
|
|
|
the default number of partitions is used.
|
2018-02-11 05:23:15 -05:00
|
|
|
|
|
|
|
At least one partition-by expression must be specified.
|
|
|
|
When no explicit sort order is specified, "ascending nulls first" is assumed.
|
|
|
|
|
|
|
|
>>> df.repartitionByRange(2, "age").rdd.getNumPartitions()
|
|
|
|
2
|
|
|
|
>>> df.show()
|
|
|
|
+---+-----+
|
|
|
|
|age| name|
|
|
|
|
+---+-----+
|
|
|
|
| 2|Alice|
|
|
|
|
| 5| Bob|
|
|
|
|
+---+-----+
|
|
|
|
>>> df.repartitionByRange(1, "age").rdd.getNumPartitions()
|
|
|
|
1
|
|
|
|
>>> data = df.repartitionByRange("age")
|
|
|
|
>>> df.show()
|
|
|
|
+---+-----+
|
|
|
|
|age| name|
|
|
|
|
+---+-----+
|
|
|
|
| 2|Alice|
|
|
|
|
| 5| Bob|
|
|
|
|
+---+-----+
|
|
|
|
"""
|
|
|
|
if isinstance(numPartitions, int):
|
|
|
|
if len(cols) == 0:
|
|
|
|
return ValueError("At least one partition-by expression must be specified.")
|
|
|
|
else:
|
|
|
|
return DataFrame(
|
|
|
|
self._jdf.repartitionByRange(numPartitions, self._jcols(*cols)), self.sql_ctx)
|
|
|
|
elif isinstance(numPartitions, (basestring, Column)):
|
|
|
|
cols = (numPartitions,) + cols
|
|
|
|
return DataFrame(self._jdf.repartitionByRange(self._jcols(*cols)), self.sql_ctx)
|
|
|
|
else:
|
|
|
|
raise TypeError("numPartitions should be an int, string or Column")
|
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-02-17 13:22:48 -05:00
|
|
|
def distinct(self):
|
2015-03-31 21:31:36 -04:00
|
|
|
"""Returns a new :class:`DataFrame` containing the distinct rows in this :class:`DataFrame`.
|
2015-02-18 04:00:54 -05:00
|
|
|
|
|
|
|
>>> df.distinct().count()
|
2015-04-16 19:20:57 -04:00
|
|
|
2
|
2015-02-17 13:22:48 -05:00
|
|
|
"""
|
|
|
|
return DataFrame(self._jdf.distinct(), self.sql_ctx)
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
[SPARK-21779][PYTHON] Simpler DataFrame.sample API in Python
## What changes were proposed in this pull request?
This PR make `DataFrame.sample(...)` can omit `withReplacement` defaulting `False`, consistently with equivalent Scala / Java API.
In short, the following examples are allowed:
```python
>>> df = spark.range(10)
>>> df.sample(0.5).count()
7
>>> df.sample(fraction=0.5).count()
3
>>> df.sample(0.5, seed=42).count()
5
>>> df.sample(fraction=0.5, seed=42).count()
5
```
In addition, this PR also adds some type checking logics as below:
```python
>>> df = spark.range(10)
>>> df.sample().count()
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [].
>>> df.sample(True).count()
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'bool'>].
>>> df.sample(42).count()
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'int'>].
>>> df.sample(fraction=False, seed="a").count()
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'bool'>, <type 'str'>].
>>> df.sample(seed=[1]).count()
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'list'>].
>>> df.sample(withReplacement="a", fraction=0.5, seed=1)
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'str'>, <type 'float'>, <type 'int'>].
```
## How was this patch tested?
Manually tested, unit tests added in doc tests and manually checked the built documentation for Python.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #18999 from HyukjinKwon/SPARK-21779.
2017-09-01 00:01:23 -04:00
|
|
|
def sample(self, withReplacement=None, fraction=None, seed=None):
|
2015-03-31 21:31:36 -04:00
|
|
|
"""Returns a sampled subset of this :class:`DataFrame`.
|
2015-02-09 23:49:22 -05:00
|
|
|
|
[SPARK-21779][PYTHON] Simpler DataFrame.sample API in Python
## What changes were proposed in this pull request?
This PR make `DataFrame.sample(...)` can omit `withReplacement` defaulting `False`, consistently with equivalent Scala / Java API.
In short, the following examples are allowed:
```python
>>> df = spark.range(10)
>>> df.sample(0.5).count()
7
>>> df.sample(fraction=0.5).count()
3
>>> df.sample(0.5, seed=42).count()
5
>>> df.sample(fraction=0.5, seed=42).count()
5
```
In addition, this PR also adds some type checking logics as below:
```python
>>> df = spark.range(10)
>>> df.sample().count()
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [].
>>> df.sample(True).count()
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'bool'>].
>>> df.sample(42).count()
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'int'>].
>>> df.sample(fraction=False, seed="a").count()
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'bool'>, <type 'str'>].
>>> df.sample(seed=[1]).count()
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'list'>].
>>> df.sample(withReplacement="a", fraction=0.5, seed=1)
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'str'>, <type 'float'>, <type 'int'>].
```
## How was this patch tested?
Manually tested, unit tests added in doc tests and manually checked the built documentation for Python.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #18999 from HyukjinKwon/SPARK-21779.
2017-09-01 00:01:23 -04:00
|
|
|
:param withReplacement: Sample with replacement or not (default False).
|
|
|
|
:param fraction: Fraction of rows to generate, range [0.0, 1.0].
|
|
|
|
:param seed: Seed for sampling (default a random seed).
|
|
|
|
|
2016-11-22 06:40:18 -05:00
|
|
|
.. note:: This is not guaranteed to provide exactly the fraction specified of the total
|
|
|
|
count of the given :class:`DataFrame`.
|
2016-11-17 06:34:55 -05:00
|
|
|
|
[SPARK-21779][PYTHON] Simpler DataFrame.sample API in Python
## What changes were proposed in this pull request?
This PR make `DataFrame.sample(...)` can omit `withReplacement` defaulting `False`, consistently with equivalent Scala / Java API.
In short, the following examples are allowed:
```python
>>> df = spark.range(10)
>>> df.sample(0.5).count()
7
>>> df.sample(fraction=0.5).count()
3
>>> df.sample(0.5, seed=42).count()
5
>>> df.sample(fraction=0.5, seed=42).count()
5
```
In addition, this PR also adds some type checking logics as below:
```python
>>> df = spark.range(10)
>>> df.sample().count()
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [].
>>> df.sample(True).count()
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'bool'>].
>>> df.sample(42).count()
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'int'>].
>>> df.sample(fraction=False, seed="a").count()
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'bool'>, <type 'str'>].
>>> df.sample(seed=[1]).count()
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'list'>].
>>> df.sample(withReplacement="a", fraction=0.5, seed=1)
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'str'>, <type 'float'>, <type 'int'>].
```
## How was this patch tested?
Manually tested, unit tests added in doc tests and manually checked the built documentation for Python.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #18999 from HyukjinKwon/SPARK-21779.
2017-09-01 00:01:23 -04:00
|
|
|
.. note:: `fraction` is required and, `withReplacement` and `seed` are optional.
|
|
|
|
|
|
|
|
>>> df = spark.range(10)
|
|
|
|
>>> df.sample(0.5, 3).count()
|
|
|
|
4
|
|
|
|
>>> df.sample(fraction=0.5, seed=3).count()
|
|
|
|
4
|
|
|
|
>>> df.sample(withReplacement=True, fraction=0.5, seed=3).count()
|
|
|
|
1
|
|
|
|
>>> df.sample(1.0).count()
|
|
|
|
10
|
|
|
|
>>> df.sample(fraction=1.0).count()
|
|
|
|
10
|
|
|
|
>>> df.sample(False, fraction=1.0).count()
|
|
|
|
10
|
2015-02-09 23:49:22 -05:00
|
|
|
"""
|
[SPARK-21779][PYTHON] Simpler DataFrame.sample API in Python
## What changes were proposed in this pull request?
This PR make `DataFrame.sample(...)` can omit `withReplacement` defaulting `False`, consistently with equivalent Scala / Java API.
In short, the following examples are allowed:
```python
>>> df = spark.range(10)
>>> df.sample(0.5).count()
7
>>> df.sample(fraction=0.5).count()
3
>>> df.sample(0.5, seed=42).count()
5
>>> df.sample(fraction=0.5, seed=42).count()
5
```
In addition, this PR also adds some type checking logics as below:
```python
>>> df = spark.range(10)
>>> df.sample().count()
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [].
>>> df.sample(True).count()
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'bool'>].
>>> df.sample(42).count()
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'int'>].
>>> df.sample(fraction=False, seed="a").count()
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'bool'>, <type 'str'>].
>>> df.sample(seed=[1]).count()
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'list'>].
>>> df.sample(withReplacement="a", fraction=0.5, seed=1)
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'str'>, <type 'float'>, <type 'int'>].
```
## How was this patch tested?
Manually tested, unit tests added in doc tests and manually checked the built documentation for Python.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #18999 from HyukjinKwon/SPARK-21779.
2017-09-01 00:01:23 -04:00
|
|
|
|
|
|
|
# For the cases below:
|
|
|
|
# sample(True, 0.5 [, seed])
|
|
|
|
# sample(True, fraction=0.5 [, seed])
|
|
|
|
# sample(withReplacement=False, fraction=0.5 [, seed])
|
|
|
|
is_withReplacement_set = \
|
|
|
|
type(withReplacement) == bool and isinstance(fraction, float)
|
|
|
|
|
|
|
|
# For the case below:
|
|
|
|
# sample(faction=0.5 [, seed])
|
|
|
|
is_withReplacement_omitted_kwargs = \
|
|
|
|
withReplacement is None and isinstance(fraction, float)
|
|
|
|
|
|
|
|
# For the case below:
|
|
|
|
# sample(0.5 [, seed])
|
|
|
|
is_withReplacement_omitted_args = isinstance(withReplacement, float)
|
|
|
|
|
|
|
|
if not (is_withReplacement_set
|
|
|
|
or is_withReplacement_omitted_kwargs
|
|
|
|
or is_withReplacement_omitted_args):
|
|
|
|
argtypes = [
|
|
|
|
str(type(arg)) for arg in [withReplacement, fraction, seed] if arg is not None]
|
|
|
|
raise TypeError(
|
|
|
|
"withReplacement (optional), fraction (required) and seed (optional)"
|
|
|
|
" should be a bool, float and number; however, "
|
|
|
|
"got [%s]." % ", ".join(argtypes))
|
|
|
|
|
|
|
|
if is_withReplacement_omitted_args:
|
|
|
|
if fraction is not None:
|
|
|
|
seed = fraction
|
|
|
|
fraction = withReplacement
|
|
|
|
withReplacement = None
|
|
|
|
|
|
|
|
seed = long(seed) if seed is not None else None
|
|
|
|
args = [arg for arg in [withReplacement, fraction, seed] if arg is not None]
|
|
|
|
jdf = self._jdf.sample(*args)
|
|
|
|
return DataFrame(jdf, self.sql_ctx)
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2015-07-30 20:16:03 -04:00
|
|
|
@since(1.5)
|
|
|
|
def sampleBy(self, col, fractions, seed=None):
|
|
|
|
"""
|
|
|
|
Returns a stratified sample without replacement based on the
|
|
|
|
fraction given on each stratum.
|
|
|
|
|
|
|
|
:param col: column that defines strata
|
|
|
|
:param fractions:
|
|
|
|
sampling fraction for each stratum. If a stratum is not
|
|
|
|
specified, we treat its fraction as zero.
|
|
|
|
:param seed: random seed
|
|
|
|
:return: a new DataFrame that represents the stratified sample
|
|
|
|
|
|
|
|
>>> from pyspark.sql.functions import col
|
|
|
|
>>> dataset = sqlContext.range(0, 100).select((col("id") % 3).alias("key"))
|
|
|
|
>>> sampled = dataset.sampleBy("key", fractions={0: 0.1, 1: 0.2}, seed=0)
|
|
|
|
>>> sampled.groupBy("key").count().orderBy("key").show()
|
|
|
|
+---+-----+
|
|
|
|
|key|count|
|
|
|
|
+---+-----+
|
2015-11-06 15:06:24 -05:00
|
|
|
| 0| 5|
|
|
|
|
| 1| 9|
|
2015-07-30 20:16:03 -04:00
|
|
|
+---+-----+
|
[SPARK-25381][SQL] Stratified sampling by Column argument
## What changes were proposed in this pull request?
In the PR, I propose to add an overloaded method for `sampleBy` which accepts the first argument of the `Column` type. This will allow to sample by any complex columns as well as sampling by multiple columns. For example:
```Scala
spark.createDataFrame(Seq(("Bob", 17), ("Alice", 10), ("Nico", 8), ("Bob", 17),
("Alice", 10))).toDF("name", "age")
.stat
.sampleBy(struct($"name", $"age"), Map(Row("Alice", 10) -> 0.3, Row("Nico", 8) -> 1.0), 36L)
.show()
+-----+---+
| name|age|
+-----+---+
| Nico| 8|
|Alice| 10|
+-----+---+
```
## How was this patch tested?
Added new test for sampling by multiple columns for Scala and test for Java, Python to check that `sampleBy` is able to sample by `Column` type argument.
Closes #22365 from MaxGekk/sample-by-column.
Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-09-20 13:11:40 -04:00
|
|
|
>>> dataset.sampleBy(col("key"), fractions={2: 1.0}, seed=0).count()
|
|
|
|
33
|
2015-07-30 20:16:03 -04:00
|
|
|
|
2018-10-02 11:48:24 -04:00
|
|
|
.. versionchanged:: 3.0
|
[SPARK-25381][SQL] Stratified sampling by Column argument
## What changes were proposed in this pull request?
In the PR, I propose to add an overloaded method for `sampleBy` which accepts the first argument of the `Column` type. This will allow to sample by any complex columns as well as sampling by multiple columns. For example:
```Scala
spark.createDataFrame(Seq(("Bob", 17), ("Alice", 10), ("Nico", 8), ("Bob", 17),
("Alice", 10))).toDF("name", "age")
.stat
.sampleBy(struct($"name", $"age"), Map(Row("Alice", 10) -> 0.3, Row("Nico", 8) -> 1.0), 36L)
.show()
+-----+---+
| name|age|
+-----+---+
| Nico| 8|
|Alice| 10|
+-----+---+
```
## How was this patch tested?
Added new test for sampling by multiple columns for Scala and test for Java, Python to check that `sampleBy` is able to sample by `Column` type argument.
Closes #22365 from MaxGekk/sample-by-column.
Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-09-20 13:11:40 -04:00
|
|
|
Added sampling by a column of :class:`Column`
|
2015-07-30 20:16:03 -04:00
|
|
|
"""
|
[SPARK-25381][SQL] Stratified sampling by Column argument
## What changes were proposed in this pull request?
In the PR, I propose to add an overloaded method for `sampleBy` which accepts the first argument of the `Column` type. This will allow to sample by any complex columns as well as sampling by multiple columns. For example:
```Scala
spark.createDataFrame(Seq(("Bob", 17), ("Alice", 10), ("Nico", 8), ("Bob", 17),
("Alice", 10))).toDF("name", "age")
.stat
.sampleBy(struct($"name", $"age"), Map(Row("Alice", 10) -> 0.3, Row("Nico", 8) -> 1.0), 36L)
.show()
+-----+---+
| name|age|
+-----+---+
| Nico| 8|
|Alice| 10|
+-----+---+
```
## How was this patch tested?
Added new test for sampling by multiple columns for Scala and test for Java, Python to check that `sampleBy` is able to sample by `Column` type argument.
Closes #22365 from MaxGekk/sample-by-column.
Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-09-20 13:11:40 -04:00
|
|
|
if isinstance(col, basestring):
|
|
|
|
col = Column(col)
|
|
|
|
elif not isinstance(col, Column):
|
|
|
|
raise ValueError("col must be a string or a column, but got %r" % type(col))
|
2015-07-30 20:16:03 -04:00
|
|
|
if not isinstance(fractions, dict):
|
|
|
|
raise ValueError("fractions must be a dict but got %r" % type(fractions))
|
|
|
|
for k, v in fractions.items():
|
|
|
|
if not isinstance(k, (float, int, long, basestring)):
|
|
|
|
raise ValueError("key must be float, int, long, or string, but got %r" % type(k))
|
|
|
|
fractions[k] = float(v)
|
[SPARK-25381][SQL] Stratified sampling by Column argument
## What changes were proposed in this pull request?
In the PR, I propose to add an overloaded method for `sampleBy` which accepts the first argument of the `Column` type. This will allow to sample by any complex columns as well as sampling by multiple columns. For example:
```Scala
spark.createDataFrame(Seq(("Bob", 17), ("Alice", 10), ("Nico", 8), ("Bob", 17),
("Alice", 10))).toDF("name", "age")
.stat
.sampleBy(struct($"name", $"age"), Map(Row("Alice", 10) -> 0.3, Row("Nico", 8) -> 1.0), 36L)
.show()
+-----+---+
| name|age|
+-----+---+
| Nico| 8|
|Alice| 10|
+-----+---+
```
## How was this patch tested?
Added new test for sampling by multiple columns for Scala and test for Java, Python to check that `sampleBy` is able to sample by `Column` type argument.
Closes #22365 from MaxGekk/sample-by-column.
Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-09-20 13:11:40 -04:00
|
|
|
col = col._jc
|
2015-07-30 20:16:03 -04:00
|
|
|
seed = seed if seed is not None else random.randint(0, sys.maxsize)
|
|
|
|
return DataFrame(self._jdf.stat().sampleBy(col, self._jmap(fractions), seed), self.sql_ctx)
|
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.4)
|
2015-04-29 18:34:05 -04:00
|
|
|
def randomSplit(self, weights, seed=None):
|
|
|
|
"""Randomly splits this :class:`DataFrame` with the provided weights.
|
|
|
|
|
2015-04-29 22:13:47 -04:00
|
|
|
:param weights: list of doubles as weights with which to split the DataFrame. Weights will
|
|
|
|
be normalized if they don't sum up to 1.0.
|
|
|
|
:param seed: The seed for sampling.
|
|
|
|
|
2015-04-29 18:34:05 -04:00
|
|
|
>>> splits = df4.randomSplit([1.0, 2.0], 24)
|
|
|
|
>>> splits[0].count()
|
|
|
|
1
|
|
|
|
|
|
|
|
>>> splits[1].count()
|
|
|
|
3
|
|
|
|
"""
|
|
|
|
for w in weights:
|
2015-04-29 22:13:47 -04:00
|
|
|
if w < 0.0:
|
|
|
|
raise ValueError("Weights must be positive. Found weight value: %s" % w)
|
2015-04-29 18:34:05 -04:00
|
|
|
seed = seed if seed is not None else random.randint(0, sys.maxsize)
|
2015-08-25 07:33:13 -04:00
|
|
|
rdd_array = self._jdf.randomSplit(_to_list(self.sql_ctx._sc, weights), long(seed))
|
2015-04-29 18:34:05 -04:00
|
|
|
return [DataFrame(rdd, self.sql_ctx) for rdd in rdd_array]
|
|
|
|
|
2015-02-09 23:49:22 -05:00
|
|
|
@property
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-02-09 23:49:22 -05:00
|
|
|
def dtypes(self):
|
2015-03-31 21:31:36 -04:00
|
|
|
"""Returns all column names and their data types as a list.
|
2015-02-09 23:49:22 -05:00
|
|
|
|
|
|
|
>>> df.dtypes
|
2015-02-11 15:13:16 -05:00
|
|
|
[('age', 'int'), ('name', 'string')]
|
2015-02-09 23:49:22 -05:00
|
|
|
"""
|
2015-02-14 02:03:22 -05:00
|
|
|
return [(str(f.name), f.dataType.simpleString()) for f in self.schema.fields]
|
2015-02-09 23:49:22 -05:00
|
|
|
|
|
|
|
@property
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-02-09 23:49:22 -05:00
|
|
|
def columns(self):
|
2015-03-31 21:31:36 -04:00
|
|
|
"""Returns all column names as a list.
|
2015-02-09 23:49:22 -05:00
|
|
|
|
|
|
|
>>> df.columns
|
2015-07-01 19:43:18 -04:00
|
|
|
['age', 'name']
|
2015-02-09 23:49:22 -05:00
|
|
|
"""
|
2015-02-14 02:03:22 -05:00
|
|
|
return [f.name for f in self.schema.fields]
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2018-01-25 17:50:48 -05:00
|
|
|
@since(2.3)
|
|
|
|
def colRegex(self, colName):
|
|
|
|
"""
|
|
|
|
Selects column based on the column name specified as a regex and returns it
|
|
|
|
as :class:`Column`.
|
|
|
|
|
|
|
|
:param colName: string, column name specified as a regex.
|
|
|
|
|
|
|
|
>>> df = spark.createDataFrame([("a", 1), ("b", 2), ("c", 3)], ["Col1", "Col2"])
|
|
|
|
>>> df.select(df.colRegex("`(Col1)?+.+`")).show()
|
|
|
|
+----+
|
|
|
|
|Col2|
|
|
|
|
+----+
|
|
|
|
| 1|
|
|
|
|
| 2|
|
|
|
|
| 3|
|
|
|
|
+----+
|
|
|
|
"""
|
|
|
|
if not isinstance(colName, basestring):
|
|
|
|
raise ValueError("colName should be provided as string")
|
|
|
|
jc = self._jdf.colRegex(colName)
|
|
|
|
return Column(jc)
|
|
|
|
|
2015-04-23 21:52:55 -04:00
|
|
|
@ignore_unicode_prefix
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-04-23 21:52:55 -04:00
|
|
|
def alias(self, alias):
|
|
|
|
"""Returns a new :class:`DataFrame` with an alias set.
|
|
|
|
|
2018-03-05 09:17:16 -05:00
|
|
|
:param alias: string, an alias name to be set for the DataFrame.
|
|
|
|
|
2015-04-23 21:52:55 -04:00
|
|
|
>>> from pyspark.sql.functions import *
|
|
|
|
>>> df_as1 = df.alias("df_as1")
|
|
|
|
>>> df_as2 = df.alias("df_as2")
|
|
|
|
>>> joined_df = df_as1.join(df_as2, col("df_as1.name") == col("df_as2.name"), 'inner')
|
[SPARK-13329] [SQL] considering output for statistics of logical plan
The current implementation of statistics of UnaryNode does not considering output (for example, Project may product much less columns than it's child), we should considering it to have a better guess.
We usually only join with few columns from a parquet table, the size of projected plan could be much smaller than the original parquet files. Having a better guess of size help we choose between broadcast join or sort merge join.
After this PR, I saw a few queries choose broadcast join other than sort merge join without turning spark.sql.autoBroadcastJoinThreshold for every query, ended up with about 6-8X improvements on end-to-end time.
We use `defaultSize` of DataType to estimate the size of a column, currently For DecimalType/StringType/BinaryType and UDT, we are over-estimate too much (4096 Bytes), so this PR change them to some more reasonable values. Here are the new defaultSize for them:
DecimalType: 8 or 16 bytes, based on the precision
StringType: 20 bytes
BinaryType: 100 bytes
UDF: default size of SQL type
These numbers are not perfect (hard to have a perfect number for them), but should be better than 4096.
Author: Davies Liu <davies@databricks.com>
Closes #11210 from davies/statics.
2016-02-23 15:55:44 -05:00
|
|
|
>>> joined_df.select("df_as1.name", "df_as2.name", "df_as2.age").collect()
|
2016-05-19 15:12:42 -04:00
|
|
|
[Row(name=u'Bob', name=u'Bob', age=5), Row(name=u'Alice', name=u'Alice', age=2)]
|
2015-04-23 21:52:55 -04:00
|
|
|
"""
|
|
|
|
assert isinstance(alias, basestring), "alias should be a string"
|
|
|
|
return DataFrame(getattr(self._jdf, "as")(alias), self.sql_ctx)
|
|
|
|
|
2016-10-14 21:24:47 -04:00
|
|
|
@ignore_unicode_prefix
|
|
|
|
@since(2.1)
|
|
|
|
def crossJoin(self, other):
|
|
|
|
"""Returns the cartesian product with another :class:`DataFrame`.
|
|
|
|
|
|
|
|
:param other: Right side of the cartesian product.
|
|
|
|
|
|
|
|
>>> df.select("age", "name").collect()
|
|
|
|
[Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')]
|
|
|
|
>>> df2.select("name", "height").collect()
|
|
|
|
[Row(name=u'Tom', height=80), Row(name=u'Bob', height=85)]
|
|
|
|
>>> df.crossJoin(df2.select("height")).select("age", "name", "height").collect()
|
|
|
|
[Row(age=2, name=u'Alice', height=80), Row(age=2, name=u'Alice', height=85),
|
|
|
|
Row(age=5, name=u'Bob', height=80), Row(age=5, name=u'Bob', height=85)]
|
|
|
|
"""
|
|
|
|
|
|
|
|
jdf = self._jdf.crossJoin(other._jdf)
|
|
|
|
return DataFrame(jdf, self.sql_ctx)
|
|
|
|
|
2015-04-16 19:20:57 -04:00
|
|
|
@ignore_unicode_prefix
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-06-09 02:27:05 -04:00
|
|
|
def join(self, other, on=None, how=None):
|
2015-03-31 21:31:36 -04:00
|
|
|
"""Joins with another :class:`DataFrame`, using the given join expression.
|
|
|
|
|
2015-02-09 23:49:22 -05:00
|
|
|
:param other: Right side of the join
|
2016-07-19 02:49:47 -04:00
|
|
|
:param on: a string for the join column name, a list of column names,
|
|
|
|
a join expression (Column), or a list of Columns.
|
|
|
|
If `on` is a string or a list of strings indicating the name of the join column(s),
|
[SPARK-12520] [PYSPARK] Correct Descriptions and Add Use Cases in Equi-Join
After reading the JIRA https://issues.apache.org/jira/browse/SPARK-12520, I double checked the code.
For example, users can do the Equi-Join like
```df.join(df2, 'name', 'outer').select('name', 'height').collect()```
- There exists a bug in 1.5 and 1.4. The code just ignores the third parameter (join type) users pass. However, the join type we called is `Inner`, even if the user-specified type is the other type (e.g., `Outer`).
- After a PR: https://github.com/apache/spark/pull/8600, the 1.6 does not have such an issue, but the description has not been updated.
Plan to submit another PR to fix 1.5 and issue an error message if users specify a non-inner join type when using Equi-Join.
Author: gatorsmile <gatorsmile@gmail.com>
Closes #10477 from gatorsmile/pyOuterJoin.
2015-12-28 02:18:48 -05:00
|
|
|
the column(s) must exist on both sides, and this performs an equi-join.
|
2017-01-08 23:37:46 -05:00
|
|
|
:param how: str, default ``inner``. Must be one of: ``inner``, ``cross``, ``outer``,
|
|
|
|
``full``, ``full_outer``, ``left``, ``left_outer``, ``right``, ``right_outer``,
|
|
|
|
``left_semi``, and ``left_anti``.
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2016-07-19 02:49:47 -04:00
|
|
|
The following performs a full outer join between ``df1`` and ``df2``.
|
|
|
|
|
2015-02-09 23:49:22 -05:00
|
|
|
>>> df.join(df2, df.name == df2.name, 'outer').select(df.name, df2.height).collect()
|
2016-01-14 01:43:28 -05:00
|
|
|
[Row(name=None, height=80), Row(name=u'Bob', height=85), Row(name=u'Alice', height=None)]
|
2015-04-22 18:26:58 -04:00
|
|
|
|
[SPARK-12520] [PYSPARK] Correct Descriptions and Add Use Cases in Equi-Join
After reading the JIRA https://issues.apache.org/jira/browse/SPARK-12520, I double checked the code.
For example, users can do the Equi-Join like
```df.join(df2, 'name', 'outer').select('name', 'height').collect()```
- There exists a bug in 1.5 and 1.4. The code just ignores the third parameter (join type) users pass. However, the join type we called is `Inner`, even if the user-specified type is the other type (e.g., `Outer`).
- After a PR: https://github.com/apache/spark/pull/8600, the 1.6 does not have such an issue, but the description has not been updated.
Plan to submit another PR to fix 1.5 and issue an error message if users specify a non-inner join type when using Equi-Join.
Author: gatorsmile <gatorsmile@gmail.com>
Closes #10477 from gatorsmile/pyOuterJoin.
2015-12-28 02:18:48 -05:00
|
|
|
>>> df.join(df2, 'name', 'outer').select('name', 'height').collect()
|
2016-01-14 01:43:28 -05:00
|
|
|
[Row(name=u'Tom', height=80), Row(name=u'Bob', height=85), Row(name=u'Alice', height=None)]
|
[SPARK-12520] [PYSPARK] Correct Descriptions and Add Use Cases in Equi-Join
After reading the JIRA https://issues.apache.org/jira/browse/SPARK-12520, I double checked the code.
For example, users can do the Equi-Join like
```df.join(df2, 'name', 'outer').select('name', 'height').collect()```
- There exists a bug in 1.5 and 1.4. The code just ignores the third parameter (join type) users pass. However, the join type we called is `Inner`, even if the user-specified type is the other type (e.g., `Outer`).
- After a PR: https://github.com/apache/spark/pull/8600, the 1.6 does not have such an issue, but the description has not been updated.
Plan to submit another PR to fix 1.5 and issue an error message if users specify a non-inner join type when using Equi-Join.
Author: gatorsmile <gatorsmile@gmail.com>
Closes #10477 from gatorsmile/pyOuterJoin.
2015-12-28 02:18:48 -05:00
|
|
|
|
2015-06-09 02:27:05 -04:00
|
|
|
>>> cond = [df.name == df3.name, df.age == df3.age]
|
|
|
|
>>> df.join(df3, cond, 'outer').select(df.name, df3.age).collect()
|
2016-01-14 01:43:28 -05:00
|
|
|
[Row(name=u'Alice', age=2), Row(name=u'Bob', age=5)]
|
2015-06-09 02:27:05 -04:00
|
|
|
|
2015-04-22 18:26:58 -04:00
|
|
|
>>> df.join(df2, 'name').select(df.name, df2.height).collect()
|
|
|
|
[Row(name=u'Bob', height=85)]
|
2015-06-09 02:27:05 -04:00
|
|
|
|
|
|
|
>>> df.join(df4, ['name', 'age']).select(df.name, df.age).collect()
|
|
|
|
[Row(name=u'Bob', age=5)]
|
2015-02-09 23:49:22 -05:00
|
|
|
"""
|
|
|
|
|
2015-06-09 02:27:05 -04:00
|
|
|
if on is not None and not isinstance(on, list):
|
|
|
|
on = [on]
|
|
|
|
|
2016-10-12 13:09:49 -04:00
|
|
|
if on is not None:
|
|
|
|
if isinstance(on[0], basestring):
|
|
|
|
on = self._jseq(on)
|
2015-09-22 02:46:00 -04:00
|
|
|
else:
|
2016-10-12 13:09:49 -04:00
|
|
|
assert isinstance(on[0], Column), "on should be Column or list of Column"
|
2016-10-14 21:24:47 -04:00
|
|
|
on = reduce(lambda x, y: x.__and__(y), on)
|
2016-10-12 13:09:49 -04:00
|
|
|
on = on._jc
|
|
|
|
|
|
|
|
if on is None and how is None:
|
2016-10-14 21:24:47 -04:00
|
|
|
jdf = self._jdf.join(other._jdf)
|
2015-02-09 23:49:22 -05:00
|
|
|
else:
|
2015-06-09 02:27:05 -04:00
|
|
|
if how is None:
|
2016-10-12 13:09:49 -04:00
|
|
|
how = "inner"
|
[SPARK-21264][PYTHON] Call cross join path in join without 'on' and with 'how'
## What changes were proposed in this pull request?
Currently, it throws a NPE when missing columns but join type is speicified in join at PySpark as below:
```python
spark.conf.set("spark.sql.crossJoin.enabled", "false")
spark.range(1).join(spark.range(1), how="inner").show()
```
```
Traceback (most recent call last):
...
py4j.protocol.Py4JJavaError: An error occurred while calling o66.join.
: java.lang.NullPointerException
at org.apache.spark.sql.Dataset.join(Dataset.scala:931)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
...
```
```python
spark.conf.set("spark.sql.crossJoin.enabled", "true")
spark.range(1).join(spark.range(1), how="inner").show()
```
```
...
py4j.protocol.Py4JJavaError: An error occurred while calling o84.join.
: java.lang.NullPointerException
at org.apache.spark.sql.Dataset.join(Dataset.scala:931)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
...
```
This PR suggests to follow Scala's one as below:
```scala
scala> spark.conf.set("spark.sql.crossJoin.enabled", "false")
scala> spark.range(1).join(spark.range(1), Seq.empty[String], "inner").show()
```
```
org.apache.spark.sql.AnalysisException: Detected cartesian product for INNER join between logical plans
Range (0, 1, step=1, splits=Some(8))
and
Range (0, 1, step=1, splits=Some(8))
Join condition is missing or trivial.
Use the CROSS JOIN syntax to allow cartesian products between these relations.;
...
```
```scala
scala> spark.conf.set("spark.sql.crossJoin.enabled", "true")
scala> spark.range(1).join(spark.range(1), Seq.empty[String], "inner").show()
```
```
+---+---+
| id| id|
+---+---+
| 0| 0|
+---+---+
```
**After**
```python
spark.conf.set("spark.sql.crossJoin.enabled", "false")
spark.range(1).join(spark.range(1), how="inner").show()
```
```
Traceback (most recent call last):
...
pyspark.sql.utils.AnalysisException: u'Detected cartesian product for INNER join between logical plans\nRange (0, 1, step=1, splits=Some(8))\nand\nRange (0, 1, step=1, splits=Some(8))\nJoin condition is missing or trivial.\nUse the CROSS JOIN syntax to allow cartesian products between these relations.;'
```
```python
spark.conf.set("spark.sql.crossJoin.enabled", "true")
spark.range(1).join(spark.range(1), how="inner").show()
```
```
+---+---+
| id| id|
+---+---+
| 0| 0|
+---+---+
```
## How was this patch tested?
Added tests in `python/pyspark/sql/tests.py`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #18484 from HyukjinKwon/SPARK-21264.
2017-07-03 22:35:08 -04:00
|
|
|
if on is None:
|
|
|
|
on = self._jseq([])
|
2016-10-12 13:09:49 -04:00
|
|
|
assert isinstance(how, basestring), "how should be basestring"
|
|
|
|
jdf = self._jdf.join(other._jdf, on, how)
|
2015-02-09 23:49:22 -05:00
|
|
|
return DataFrame(jdf, self.sql_ctx)
|
|
|
|
|
2015-11-06 18:48:20 -05:00
|
|
|
@since(1.6)
|
|
|
|
def sortWithinPartitions(self, *cols, **kwargs):
|
|
|
|
"""Returns a new :class:`DataFrame` with each partition sorted by the specified column(s).
|
|
|
|
|
|
|
|
:param cols: list of :class:`Column` or column names to sort by.
|
|
|
|
:param ascending: boolean or list of boolean (default True).
|
|
|
|
Sort ascending vs. descending. Specify list for multiple sort orders.
|
|
|
|
If a list is specified, length of the list must equal length of the `cols`.
|
|
|
|
|
|
|
|
>>> df.sortWithinPartitions("age", ascending=False).show()
|
|
|
|
+---+-----+
|
|
|
|
|age| name|
|
|
|
|
+---+-----+
|
|
|
|
| 2|Alice|
|
|
|
|
| 5| Bob|
|
|
|
|
+---+-----+
|
|
|
|
"""
|
|
|
|
jdf = self._jdf.sortWithinPartitions(self._sort_cols(cols, kwargs))
|
|
|
|
return DataFrame(jdf, self.sql_ctx)
|
|
|
|
|
2015-04-16 19:20:57 -04:00
|
|
|
@ignore_unicode_prefix
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-04-17 12:29:27 -04:00
|
|
|
def sort(self, *cols, **kwargs):
|
2015-03-31 21:31:36 -04:00
|
|
|
"""Returns a new :class:`DataFrame` sorted by the specified column(s).
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2015-04-17 12:29:27 -04:00
|
|
|
:param cols: list of :class:`Column` or column names to sort by.
|
2015-04-17 17:30:13 -04:00
|
|
|
:param ascending: boolean or list of boolean (default True).
|
|
|
|
Sort ascending vs. descending. Specify list for multiple sort orders.
|
|
|
|
If a list is specified, length of the list must equal length of the `cols`.
|
2015-02-09 23:49:22 -05:00
|
|
|
|
|
|
|
>>> df.sort(df.age.desc()).collect()
|
|
|
|
[Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')]
|
2015-04-17 12:29:27 -04:00
|
|
|
>>> df.sort("age", ascending=False).collect()
|
|
|
|
[Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')]
|
2015-02-24 21:59:23 -05:00
|
|
|
>>> df.orderBy(df.age.desc()).collect()
|
|
|
|
[Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')]
|
|
|
|
>>> from pyspark.sql.functions import *
|
|
|
|
>>> df.sort(asc("age")).collect()
|
|
|
|
[Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')]
|
|
|
|
>>> df.orderBy(desc("age"), "name").collect()
|
2015-02-09 23:49:22 -05:00
|
|
|
[Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')]
|
2015-04-17 12:29:27 -04:00
|
|
|
>>> df.orderBy(["age", "name"], ascending=[0, 1]).collect()
|
|
|
|
[Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')]
|
2015-02-09 23:49:22 -05:00
|
|
|
"""
|
2015-11-06 18:48:20 -05:00
|
|
|
jdf = self._jdf.sort(self._sort_cols(cols, kwargs))
|
2015-02-09 23:49:22 -05:00
|
|
|
return DataFrame(jdf, self.sql_ctx)
|
|
|
|
|
2015-02-24 21:59:23 -05:00
|
|
|
orderBy = sort
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2015-04-17 12:29:27 -04:00
|
|
|
def _jseq(self, cols, converter=None):
|
|
|
|
"""Return a JVM Seq of Columns from a list of Column or names"""
|
|
|
|
return _to_seq(self.sql_ctx._sc, cols, converter)
|
|
|
|
|
2015-05-12 13:23:41 -04:00
|
|
|
def _jmap(self, jm):
|
|
|
|
"""Return a JVM Scala Map from a dict"""
|
|
|
|
return _to_scala_map(self.sql_ctx._sc, jm)
|
|
|
|
|
2015-04-17 12:29:27 -04:00
|
|
|
def _jcols(self, *cols):
|
|
|
|
"""Return a JVM Seq of Columns from a list of Column or column names
|
|
|
|
|
|
|
|
If `cols` has only one list in it, cols[0] will be used as the list.
|
|
|
|
"""
|
|
|
|
if len(cols) == 1 and isinstance(cols[0], list):
|
|
|
|
cols = cols[0]
|
|
|
|
return self._jseq(cols, _to_java_column)
|
|
|
|
|
2015-11-06 18:48:20 -05:00
|
|
|
def _sort_cols(self, cols, kwargs):
|
|
|
|
""" Return a JVM Seq of Columns that describes the sort order
|
|
|
|
"""
|
|
|
|
if not cols:
|
|
|
|
raise ValueError("should sort by at least one column")
|
|
|
|
if len(cols) == 1 and isinstance(cols[0], list):
|
|
|
|
cols = cols[0]
|
|
|
|
jcols = [_to_java_column(c) for c in cols]
|
|
|
|
ascending = kwargs.get('ascending', True)
|
|
|
|
if isinstance(ascending, (bool, int)):
|
|
|
|
if not ascending:
|
|
|
|
jcols = [jc.desc() for jc in jcols]
|
|
|
|
elif isinstance(ascending, list):
|
|
|
|
jcols = [jc if asc else jc.desc()
|
|
|
|
for asc, jc in zip(ascending, jcols)]
|
|
|
|
else:
|
|
|
|
raise TypeError("ascending can only be boolean or list, but got %s" % type(ascending))
|
|
|
|
return self._jseq(jcols)
|
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since("1.3.1")
|
2015-03-26 15:26:13 -04:00
|
|
|
def describe(self, *cols):
|
2017-08-18 21:10:54 -04:00
|
|
|
"""Computes basic statistics for numeric and string columns.
|
2015-03-26 15:26:13 -04:00
|
|
|
|
|
|
|
This include count, mean, stddev, min, and max. If no columns are
|
2016-07-08 17:36:50 -04:00
|
|
|
given, this function computes statistics for all numerical or string columns.
|
2015-03-26 15:26:13 -04:00
|
|
|
|
2016-11-22 06:40:18 -05:00
|
|
|
.. note:: This function is meant for exploratory data analysis, as we make no
|
|
|
|
guarantee about the backward compatibility of the schema of the resulting DataFrame.
|
2015-06-03 03:23:34 -04:00
|
|
|
|
2016-07-08 17:36:50 -04:00
|
|
|
>>> df.describe(['age']).show()
|
2015-09-12 13:17:15 -04:00
|
|
|
+-------+------------------+
|
|
|
|
|summary| age|
|
|
|
|
+-------+------------------+
|
|
|
|
| count| 2|
|
|
|
|
| mean| 3.5|
|
|
|
|
| stddev|2.1213203435596424|
|
|
|
|
| min| 2|
|
|
|
|
| max| 5|
|
|
|
|
+-------+------------------+
|
2016-07-08 17:36:50 -04:00
|
|
|
>>> df.describe().show()
|
2015-09-12 13:17:15 -04:00
|
|
|
+-------+------------------+-----+
|
|
|
|
|summary| age| name|
|
|
|
|
+-------+------------------+-----+
|
|
|
|
| count| 2| 2|
|
|
|
|
| mean| 3.5| null|
|
2015-11-18 16:03:37 -05:00
|
|
|
| stddev|2.1213203435596424| null|
|
2015-09-12 13:17:15 -04:00
|
|
|
| min| 2|Alice|
|
|
|
|
| max| 5| Bob|
|
|
|
|
+-------+------------------+-----+
|
2017-08-18 21:10:54 -04:00
|
|
|
|
|
|
|
Use summary for expanded statistics and control over which statistics to compute.
|
2015-03-26 15:26:13 -04:00
|
|
|
"""
|
2015-06-05 16:49:33 -04:00
|
|
|
if len(cols) == 1 and isinstance(cols[0], list):
|
|
|
|
cols = cols[0]
|
2015-04-17 12:29:27 -04:00
|
|
|
jdf = self._jdf.describe(self._jseq(cols))
|
2015-03-26 15:26:13 -04:00
|
|
|
return DataFrame(jdf, self.sql_ctx)
|
|
|
|
|
2017-08-18 21:10:54 -04:00
|
|
|
@since("2.3.0")
|
|
|
|
def summary(self, *statistics):
|
|
|
|
"""Computes specified statistics for numeric and string columns. Available statistics are:
|
|
|
|
- count
|
|
|
|
- mean
|
|
|
|
- stddev
|
|
|
|
- min
|
|
|
|
- max
|
|
|
|
- arbitrary approximate percentiles specified as a percentage (eg, 75%)
|
|
|
|
|
|
|
|
If no statistics are given, this function computes count, mean, stddev, min,
|
|
|
|
approximate quartiles (percentiles at 25%, 50%, and 75%), and max.
|
|
|
|
|
|
|
|
.. note:: This function is meant for exploratory data analysis, as we make no
|
|
|
|
guarantee about the backward compatibility of the schema of the resulting DataFrame.
|
|
|
|
|
|
|
|
>>> df.summary().show()
|
|
|
|
+-------+------------------+-----+
|
|
|
|
|summary| age| name|
|
|
|
|
+-------+------------------+-----+
|
|
|
|
| count| 2| 2|
|
|
|
|
| mean| 3.5| null|
|
|
|
|
| stddev|2.1213203435596424| null|
|
|
|
|
| min| 2|Alice|
|
[SPARK-22208][SQL] Improve percentile_approx by not rounding up targetError and starting from index 0
## What changes were proposed in this pull request?
Currently percentile_approx never returns the first element when percentile is in (relativeError, 1/N], where relativeError default 1/10000, and N is the total number of elements. But ideally, percentiles in [0, 1/N] should all return the first element as the answer.
For example, given input data 1 to 10, if a user queries 10% (or even less) percentile, it should return 1, because the first value 1 already reaches 10%. Currently it returns 2.
Based on the paper, targetError is not rounded up, and searching index should start from 0 instead of 1. By following the paper, we should be able to fix the cases mentioned above.
## How was this patch tested?
Added a new test case and fix existing test cases.
Author: Zhenhua Wang <wzh_zju@163.com>
Closes #19438 from wzhfy/improve_percentile_approx.
2017-10-11 03:16:12 -04:00
|
|
|
| 25%| 2| null|
|
|
|
|
| 50%| 2| null|
|
2017-09-25 12:28:42 -04:00
|
|
|
| 75%| 5| null|
|
2017-08-18 21:10:54 -04:00
|
|
|
| max| 5| Bob|
|
|
|
|
+-------+------------------+-----+
|
|
|
|
|
|
|
|
>>> df.summary("count", "min", "25%", "75%", "max").show()
|
|
|
|
+-------+---+-----+
|
|
|
|
|summary|age| name|
|
|
|
|
+-------+---+-----+
|
|
|
|
| count| 2| 2|
|
|
|
|
| min| 2|Alice|
|
[SPARK-22208][SQL] Improve percentile_approx by not rounding up targetError and starting from index 0
## What changes were proposed in this pull request?
Currently percentile_approx never returns the first element when percentile is in (relativeError, 1/N], where relativeError default 1/10000, and N is the total number of elements. But ideally, percentiles in [0, 1/N] should all return the first element as the answer.
For example, given input data 1 to 10, if a user queries 10% (or even less) percentile, it should return 1, because the first value 1 already reaches 10%. Currently it returns 2.
Based on the paper, targetError is not rounded up, and searching index should start from 0 instead of 1. By following the paper, we should be able to fix the cases mentioned above.
## How was this patch tested?
Added a new test case and fix existing test cases.
Author: Zhenhua Wang <wzh_zju@163.com>
Closes #19438 from wzhfy/improve_percentile_approx.
2017-10-11 03:16:12 -04:00
|
|
|
| 25%| 2| null|
|
2017-09-25 12:28:42 -04:00
|
|
|
| 75%| 5| null|
|
2017-08-18 21:10:54 -04:00
|
|
|
| max| 5| Bob|
|
|
|
|
+-------+---+-----+
|
|
|
|
|
|
|
|
To do a summary for specific columns first select them:
|
|
|
|
|
|
|
|
>>> df.select("age", "name").summary("count").show()
|
|
|
|
+-------+---+----+
|
|
|
|
|summary|age|name|
|
|
|
|
+-------+---+----+
|
|
|
|
| count| 2| 2|
|
|
|
|
+-------+---+----+
|
|
|
|
|
|
|
|
See also describe for basic statistics.
|
|
|
|
"""
|
|
|
|
if len(statistics) == 1 and isinstance(statistics[0], list):
|
|
|
|
statistics = statistics[0]
|
|
|
|
jdf = self._jdf.summary(self._jseq(statistics))
|
|
|
|
return DataFrame(jdf, self.sql_ctx)
|
|
|
|
|
2015-04-16 19:20:57 -04:00
|
|
|
@ignore_unicode_prefix
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-02-09 23:49:22 -05:00
|
|
|
def head(self, n=None):
|
2015-06-03 03:23:34 -04:00
|
|
|
"""Returns the first ``n`` rows.
|
|
|
|
|
2016-11-22 06:40:18 -05:00
|
|
|
.. note:: This method should only be used if the resulting array is expected
|
|
|
|
to be small, as all the data is loaded into the driver's memory.
|
2016-02-06 12:29:09 -05:00
|
|
|
|
2015-06-03 03:23:34 -04:00
|
|
|
:param n: int, default 1. Number of rows to return.
|
|
|
|
:return: If n is greater than 1, return a list of :class:`Row`.
|
|
|
|
If n is 1, return a single Row.
|
2015-02-09 23:49:22 -05:00
|
|
|
|
|
|
|
>>> df.head()
|
|
|
|
Row(age=2, name=u'Alice')
|
|
|
|
>>> df.head(1)
|
|
|
|
[Row(age=2, name=u'Alice')]
|
|
|
|
"""
|
|
|
|
if n is None:
|
|
|
|
rs = self.head(1)
|
|
|
|
return rs[0] if rs else None
|
|
|
|
return self.take(n)
|
|
|
|
|
2015-04-16 19:20:57 -04:00
|
|
|
@ignore_unicode_prefix
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-02-09 23:49:22 -05:00
|
|
|
def first(self):
|
2015-03-31 21:31:36 -04:00
|
|
|
"""Returns the first row as a :class:`Row`.
|
2015-02-09 23:49:22 -05:00
|
|
|
|
|
|
|
>>> df.first()
|
|
|
|
Row(age=2, name=u'Alice')
|
|
|
|
"""
|
|
|
|
return self.head()
|
|
|
|
|
2015-04-16 19:20:57 -04:00
|
|
|
@ignore_unicode_prefix
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-02-09 23:49:22 -05:00
|
|
|
def __getitem__(self, item):
|
2015-03-31 21:31:36 -04:00
|
|
|
"""Returns the column as a :class:`Column`.
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2015-02-19 15:09:44 -05:00
|
|
|
>>> df.select(df['age']).collect()
|
2015-02-09 23:49:22 -05:00
|
|
|
[Row(age=2), Row(age=5)]
|
|
|
|
>>> df[ ["name", "age"]].collect()
|
|
|
|
[Row(name=u'Alice', age=2), Row(name=u'Bob', age=5)]
|
|
|
|
>>> df[ df.age > 3 ].collect()
|
|
|
|
[Row(age=5, name=u'Bob')]
|
2015-04-16 20:33:57 -04:00
|
|
|
>>> df[df[0] > 3].collect()
|
|
|
|
[Row(age=5, name=u'Bob')]
|
2015-02-09 23:49:22 -05:00
|
|
|
"""
|
|
|
|
if isinstance(item, basestring):
|
|
|
|
jc = self._jdf.apply(item)
|
2015-02-19 15:09:44 -05:00
|
|
|
return Column(jc)
|
2015-02-09 23:49:22 -05:00
|
|
|
elif isinstance(item, Column):
|
|
|
|
return self.filter(item)
|
2015-04-16 20:33:57 -04:00
|
|
|
elif isinstance(item, (list, tuple)):
|
2015-02-09 23:49:22 -05:00
|
|
|
return self.select(*item)
|
2015-04-16 20:33:57 -04:00
|
|
|
elif isinstance(item, int):
|
|
|
|
jc = self._jdf.apply(self.columns[item])
|
|
|
|
return Column(jc)
|
2015-02-09 23:49:22 -05:00
|
|
|
else:
|
2015-04-20 13:44:09 -04:00
|
|
|
raise TypeError("unexpected item type: %s" % type(item))
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-02-09 23:49:22 -05:00
|
|
|
def __getattr__(self, name):
|
2015-03-31 21:31:36 -04:00
|
|
|
"""Returns the :class:`Column` denoted by ``name``.
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2015-02-19 15:09:44 -05:00
|
|
|
>>> df.select(df.age).collect()
|
2015-02-09 23:49:22 -05:00
|
|
|
[Row(age=2), Row(age=5)]
|
|
|
|
"""
|
2015-04-16 20:33:57 -04:00
|
|
|
if name not in self.columns:
|
2015-04-29 12:48:47 -04:00
|
|
|
raise AttributeError(
|
|
|
|
"'%s' object has no attribute '%s'" % (self.__class__.__name__, name))
|
2015-02-09 23:49:22 -05:00
|
|
|
jc = self._jdf.apply(name)
|
2015-02-19 15:09:44 -05:00
|
|
|
return Column(jc)
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2015-04-16 19:20:57 -04:00
|
|
|
@ignore_unicode_prefix
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-02-09 23:49:22 -05:00
|
|
|
def select(self, *cols):
|
2015-03-31 21:31:36 -04:00
|
|
|
"""Projects a set of expressions and returns a new :class:`DataFrame`.
|
|
|
|
|
|
|
|
:param cols: list of column names (string) or expressions (:class:`Column`).
|
|
|
|
If one of the column names is '*', that column is expanded to include all columns
|
|
|
|
in the current DataFrame.
|
2015-02-09 23:49:22 -05:00
|
|
|
|
|
|
|
>>> df.select('*').collect()
|
|
|
|
[Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')]
|
|
|
|
>>> df.select('name', 'age').collect()
|
|
|
|
[Row(name=u'Alice', age=2), Row(name=u'Bob', age=5)]
|
|
|
|
>>> df.select(df.name, (df.age + 10).alias('age')).collect()
|
|
|
|
[Row(name=u'Alice', age=12), Row(name=u'Bob', age=15)]
|
|
|
|
"""
|
2015-04-17 12:29:27 -04:00
|
|
|
jdf = self._jdf.select(self._jcols(*cols))
|
2015-02-09 23:49:22 -05:00
|
|
|
return DataFrame(jdf, self.sql_ctx)
|
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-02-09 23:49:22 -05:00
|
|
|
def selectExpr(self, *expr):
|
2015-03-31 21:31:36 -04:00
|
|
|
"""Projects a set of SQL expressions and returns a new :class:`DataFrame`.
|
|
|
|
|
|
|
|
This is a variant of :func:`select` that accepts SQL expressions.
|
2015-02-09 23:49:22 -05:00
|
|
|
|
|
|
|
>>> df.selectExpr("age * 2", "abs(age)").collect()
|
2015-11-10 14:06:29 -05:00
|
|
|
[Row((age * 2)=4, abs(age)=2), Row((age * 2)=10, abs(age)=5)]
|
2015-02-09 23:49:22 -05:00
|
|
|
"""
|
2015-04-17 12:29:27 -04:00
|
|
|
if len(expr) == 1 and isinstance(expr[0], list):
|
|
|
|
expr = expr[0]
|
|
|
|
jdf = self._jdf.selectExpr(self._jseq(expr))
|
2015-02-09 23:49:22 -05:00
|
|
|
return DataFrame(jdf, self.sql_ctx)
|
|
|
|
|
2015-04-16 19:20:57 -04:00
|
|
|
@ignore_unicode_prefix
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-02-09 23:49:22 -05:00
|
|
|
def filter(self, condition):
|
2015-03-31 21:31:36 -04:00
|
|
|
"""Filters rows using the given condition.
|
|
|
|
|
|
|
|
:func:`where` is an alias for :func:`filter`.
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2015-03-31 21:31:36 -04:00
|
|
|
:param condition: a :class:`Column` of :class:`types.BooleanType`
|
|
|
|
or a string of SQL expression.
|
2015-02-09 23:49:22 -05:00
|
|
|
|
|
|
|
>>> df.filter(df.age > 3).collect()
|
|
|
|
[Row(age=5, name=u'Bob')]
|
|
|
|
>>> df.where(df.age == 2).collect()
|
|
|
|
[Row(age=2, name=u'Alice')]
|
|
|
|
|
|
|
|
>>> df.filter("age > 3").collect()
|
|
|
|
[Row(age=5, name=u'Bob')]
|
|
|
|
>>> df.where("age = 2").collect()
|
|
|
|
[Row(age=2, name=u'Alice')]
|
|
|
|
"""
|
|
|
|
if isinstance(condition, basestring):
|
|
|
|
jdf = self._jdf.filter(condition)
|
|
|
|
elif isinstance(condition, Column):
|
|
|
|
jdf = self._jdf.filter(condition._jc)
|
|
|
|
else:
|
|
|
|
raise TypeError("condition should be string or Column")
|
|
|
|
return DataFrame(jdf, self.sql_ctx)
|
|
|
|
|
2015-04-16 19:20:57 -04:00
|
|
|
@ignore_unicode_prefix
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-02-09 23:49:22 -05:00
|
|
|
def groupBy(self, *cols):
|
2015-03-31 21:31:36 -04:00
|
|
|
"""Groups the :class:`DataFrame` using the specified columns,
|
2015-02-09 23:49:22 -05:00
|
|
|
so we can run aggregation on them. See :class:`GroupedData`
|
|
|
|
for all the available aggregate functions.
|
|
|
|
|
2015-04-17 12:29:27 -04:00
|
|
|
:func:`groupby` is an alias for :func:`groupBy`.
|
|
|
|
|
2015-03-31 21:31:36 -04:00
|
|
|
:param cols: list of columns to group by.
|
|
|
|
Each element should be a column name (string) or an expression (:class:`Column`).
|
|
|
|
|
2015-02-09 23:49:22 -05:00
|
|
|
>>> df.groupBy().avg().collect()
|
2015-07-02 00:14:13 -04:00
|
|
|
[Row(avg(age)=3.5)]
|
2016-01-14 01:43:28 -05:00
|
|
|
>>> sorted(df.groupBy('name').agg({'age': 'mean'}).collect())
|
2015-07-02 00:14:13 -04:00
|
|
|
[Row(name=u'Alice', avg(age)=2.0), Row(name=u'Bob', avg(age)=5.0)]
|
2016-01-14 01:43:28 -05:00
|
|
|
>>> sorted(df.groupBy(df.name).avg().collect())
|
2015-07-02 00:14:13 -04:00
|
|
|
[Row(name=u'Alice', avg(age)=2.0), Row(name=u'Bob', avg(age)=5.0)]
|
2016-02-10 23:13:38 -05:00
|
|
|
>>> sorted(df.groupBy(['name', df.age]).count().collect())
|
|
|
|
[Row(name=u'Alice', age=2, count=1), Row(name=u'Bob', age=5, count=1)]
|
2015-02-09 23:49:22 -05:00
|
|
|
"""
|
2015-05-21 20:43:08 -04:00
|
|
|
jgd = self._jdf.groupBy(self._jcols(*cols))
|
2015-05-15 23:09:15 -04:00
|
|
|
from pyspark.sql.group import GroupedData
|
2017-10-10 18:32:01 -04:00
|
|
|
return GroupedData(jgd, self)
|
2015-05-21 20:43:08 -04:00
|
|
|
|
|
|
|
@since(1.4)
|
|
|
|
def rollup(self, *cols):
|
|
|
|
"""
|
|
|
|
Create a multi-dimensional rollup for the current :class:`DataFrame` using
|
|
|
|
the specified columns, so we can run aggregation on them.
|
|
|
|
|
2016-02-10 23:13:38 -05:00
|
|
|
>>> df.rollup("name", df.age).count().orderBy("name", "age").show()
|
2015-05-21 20:43:08 -04:00
|
|
|
+-----+----+-----+
|
|
|
|
| name| age|count|
|
|
|
|
+-----+----+-----+
|
|
|
|
| null|null| 2|
|
2016-01-14 01:43:28 -05:00
|
|
|
|Alice|null| 1|
|
2016-02-10 23:13:38 -05:00
|
|
|
|Alice| 2| 1|
|
|
|
|
| Bob|null| 1|
|
|
|
|
| Bob| 5| 1|
|
2015-05-21 20:43:08 -04:00
|
|
|
+-----+----+-----+
|
|
|
|
"""
|
|
|
|
jgd = self._jdf.rollup(self._jcols(*cols))
|
|
|
|
from pyspark.sql.group import GroupedData
|
2017-10-10 18:32:01 -04:00
|
|
|
return GroupedData(jgd, self)
|
2015-05-21 20:43:08 -04:00
|
|
|
|
|
|
|
@since(1.4)
|
|
|
|
def cube(self, *cols):
|
|
|
|
"""
|
|
|
|
Create a multi-dimensional cube for the current :class:`DataFrame` using
|
|
|
|
the specified columns, so we can run aggregation on them.
|
|
|
|
|
2016-02-10 23:13:38 -05:00
|
|
|
>>> df.cube("name", df.age).count().orderBy("name", "age").show()
|
2015-05-21 20:43:08 -04:00
|
|
|
+-----+----+-----+
|
|
|
|
| name| age|count|
|
|
|
|
+-----+----+-----+
|
2016-02-10 23:13:38 -05:00
|
|
|
| null|null| 2|
|
2015-05-21 20:43:08 -04:00
|
|
|
| null| 2| 1|
|
|
|
|
| null| 5| 1|
|
2016-01-14 01:43:28 -05:00
|
|
|
|Alice|null| 1|
|
2016-02-10 23:13:38 -05:00
|
|
|
|Alice| 2| 1|
|
|
|
|
| Bob|null| 1|
|
|
|
|
| Bob| 5| 1|
|
2015-05-21 20:43:08 -04:00
|
|
|
+-----+----+-----+
|
|
|
|
"""
|
|
|
|
jgd = self._jdf.cube(self._jcols(*cols))
|
|
|
|
from pyspark.sql.group import GroupedData
|
2017-10-10 18:32:01 -04:00
|
|
|
return GroupedData(jgd, self)
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-02-09 23:49:22 -05:00
|
|
|
def agg(self, *exprs):
|
|
|
|
""" Aggregate on the entire :class:`DataFrame` without groups
|
2015-03-31 21:31:36 -04:00
|
|
|
(shorthand for ``df.groupBy.agg()``).
|
2015-02-09 23:49:22 -05:00
|
|
|
|
|
|
|
>>> df.agg({"age": "max"}).collect()
|
2015-07-02 00:14:13 -04:00
|
|
|
[Row(max(age)=5)]
|
2015-02-14 02:03:22 -05:00
|
|
|
>>> from pyspark.sql import functions as F
|
|
|
|
>>> df.agg(F.min(df.age)).collect()
|
2015-07-02 00:14:13 -04:00
|
|
|
[Row(min(age)=2)]
|
2015-02-09 23:49:22 -05:00
|
|
|
"""
|
|
|
|
return self.groupBy().agg(*exprs)
|
|
|
|
|
2016-03-23 02:43:09 -04:00
|
|
|
@since(2.0)
|
|
|
|
def union(self, other):
|
2017-06-09 21:29:33 -04:00
|
|
|
""" Return a new :class:`DataFrame` containing union of rows in this and another frame.
|
2016-03-23 02:43:09 -04:00
|
|
|
|
|
|
|
This is equivalent to `UNION ALL` in SQL. To do a SQL-style set union
|
[SPARK-21897][PYTHON][R] Add unionByName API to DataFrame in Python and R
## What changes were proposed in this pull request?
This PR proposes to add a wrapper for `unionByName` API to R and Python as well.
**Python**
```python
df1 = spark.createDataFrame([[1, 2, 3]], ["col0", "col1", "col2"])
df2 = spark.createDataFrame([[4, 5, 6]], ["col1", "col2", "col0"])
df1.unionByName(df2).show()
```
```
+----+----+----+
|col0|col1|col3|
+----+----+----+
| 1| 2| 3|
| 6| 4| 5|
+----+----+----+
```
**R**
```R
df1 <- select(createDataFrame(mtcars), "carb", "am", "gear")
df2 <- select(createDataFrame(mtcars), "am", "gear", "carb")
head(unionByName(limit(df1, 2), limit(df2, 2)))
```
```
carb am gear
1 4 1 4
2 4 1 4
3 4 1 4
4 4 1 4
```
## How was this patch tested?
Doctests for Python and unit test added in `test_sparkSQL.R` for R.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #19105 from HyukjinKwon/unionByName-r-python.
2017-09-03 08:03:21 -04:00
|
|
|
(that does deduplication of elements), use this function followed by :func:`distinct`.
|
2017-06-09 21:29:33 -04:00
|
|
|
|
|
|
|
Also as standard in SQL, this function resolves columns by position (not by name).
|
2016-03-23 02:43:09 -04:00
|
|
|
"""
|
2016-03-25 01:34:55 -04:00
|
|
|
return DataFrame(self._jdf.union(other._jdf), self.sql_ctx)
|
2016-03-23 02:43:09 -04:00
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-02-09 23:49:22 -05:00
|
|
|
def unionAll(self, other):
|
2017-06-09 21:29:33 -04:00
|
|
|
""" Return a new :class:`DataFrame` containing union of rows in this and another frame.
|
|
|
|
|
|
|
|
This is equivalent to `UNION ALL` in SQL. To do a SQL-style set union
|
[SPARK-21897][PYTHON][R] Add unionByName API to DataFrame in Python and R
## What changes were proposed in this pull request?
This PR proposes to add a wrapper for `unionByName` API to R and Python as well.
**Python**
```python
df1 = spark.createDataFrame([[1, 2, 3]], ["col0", "col1", "col2"])
df2 = spark.createDataFrame([[4, 5, 6]], ["col1", "col2", "col0"])
df1.unionByName(df2).show()
```
```
+----+----+----+
|col0|col1|col3|
+----+----+----+
| 1| 2| 3|
| 6| 4| 5|
+----+----+----+
```
**R**
```R
df1 <- select(createDataFrame(mtcars), "carb", "am", "gear")
df2 <- select(createDataFrame(mtcars), "am", "gear", "carb")
head(unionByName(limit(df1, 2), limit(df2, 2)))
```
```
carb am gear
1 4 1 4
2 4 1 4
3 4 1 4
4 4 1 4
```
## How was this patch tested?
Doctests for Python and unit test added in `test_sparkSQL.R` for R.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #19105 from HyukjinKwon/unionByName-r-python.
2017-09-03 08:03:21 -04:00
|
|
|
(that does deduplication of elements), use this function followed by :func:`distinct`.
|
2017-06-09 21:29:33 -04:00
|
|
|
|
|
|
|
Also as standard in SQL, this function resolves columns by position (not by name).
|
2015-02-09 23:49:22 -05:00
|
|
|
|
[SPARK-21897][PYTHON][R] Add unionByName API to DataFrame in Python and R
## What changes were proposed in this pull request?
This PR proposes to add a wrapper for `unionByName` API to R and Python as well.
**Python**
```python
df1 = spark.createDataFrame([[1, 2, 3]], ["col0", "col1", "col2"])
df2 = spark.createDataFrame([[4, 5, 6]], ["col1", "col2", "col0"])
df1.unionByName(df2).show()
```
```
+----+----+----+
|col0|col1|col3|
+----+----+----+
| 1| 2| 3|
| 6| 4| 5|
+----+----+----+
```
**R**
```R
df1 <- select(createDataFrame(mtcars), "carb", "am", "gear")
df2 <- select(createDataFrame(mtcars), "am", "gear", "carb")
head(unionByName(limit(df1, 2), limit(df2, 2)))
```
```
carb am gear
1 4 1 4
2 4 1 4
3 4 1 4
4 4 1 4
```
## How was this patch tested?
Doctests for Python and unit test added in `test_sparkSQL.R` for R.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #19105 from HyukjinKwon/unionByName-r-python.
2017-09-03 08:03:21 -04:00
|
|
|
.. note:: Deprecated in 2.0, use :func:`union` instead.
|
2015-02-09 23:49:22 -05:00
|
|
|
"""
|
[SPARK-22313][PYTHON] Mark/print deprecation warnings as DeprecationWarning for deprecated APIs
## What changes were proposed in this pull request?
This PR proposes to mark the existing warnings as `DeprecationWarning` and print out warnings for deprecated functions.
This could be actually useful for Spark app developers. I use (old) PyCharm and this IDE can detect this specific `DeprecationWarning` in some cases:
**Before**
<img src="https://user-images.githubusercontent.com/6477701/31762664-df68d9f8-b4f6-11e7-8773-f0468f70a2cc.png" height="45" />
**After**
<img src="https://user-images.githubusercontent.com/6477701/31762662-de4d6868-b4f6-11e7-98dc-3c8446a0c28a.png" height="70" />
For console usage, `DeprecationWarning` is usually disabled (see https://docs.python.org/2/library/warnings.html#warning-categories and https://docs.python.org/3/library/warnings.html#warning-categories):
```
>>> import warnings
>>> filter(lambda f: f[2] == DeprecationWarning, warnings.filters)
[('ignore', <_sre.SRE_Pattern object at 0x10ba58c00>, <type 'exceptions.DeprecationWarning'>, <_sre.SRE_Pattern object at 0x10bb04138>, 0), ('ignore', None, <type 'exceptions.DeprecationWarning'>, None, 0)]
```
so, it won't actually mess up the terminal much unless it is intended.
If this is intendedly enabled, it'd should as below:
```
>>> import warnings
>>> warnings.simplefilter('always', DeprecationWarning)
>>>
>>> from pyspark.sql import functions
>>> functions.approxCountDistinct("a")
.../spark/python/pyspark/sql/functions.py:232: DeprecationWarning: Deprecated in 2.1, use approx_count_distinct instead.
"Deprecated in 2.1, use approx_count_distinct instead.", DeprecationWarning)
...
```
These instances were found by:
```
cd python/pyspark
grep -r "Deprecated" .
grep -r "deprecated" .
grep -r "deprecate" .
```
## How was this patch tested?
Manually tested.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #19535 from HyukjinKwon/deprecated-warning.
2017-10-23 23:44:47 -04:00
|
|
|
warnings.warn("Deprecated in 2.0, use union instead.", DeprecationWarning)
|
2016-03-23 02:43:09 -04:00
|
|
|
return self.union(other)
|
2015-02-09 23:49:22 -05:00
|
|
|
|
[SPARK-21897][PYTHON][R] Add unionByName API to DataFrame in Python and R
## What changes were proposed in this pull request?
This PR proposes to add a wrapper for `unionByName` API to R and Python as well.
**Python**
```python
df1 = spark.createDataFrame([[1, 2, 3]], ["col0", "col1", "col2"])
df2 = spark.createDataFrame([[4, 5, 6]], ["col1", "col2", "col0"])
df1.unionByName(df2).show()
```
```
+----+----+----+
|col0|col1|col3|
+----+----+----+
| 1| 2| 3|
| 6| 4| 5|
+----+----+----+
```
**R**
```R
df1 <- select(createDataFrame(mtcars), "carb", "am", "gear")
df2 <- select(createDataFrame(mtcars), "am", "gear", "carb")
head(unionByName(limit(df1, 2), limit(df2, 2)))
```
```
carb am gear
1 4 1 4
2 4 1 4
3 4 1 4
4 4 1 4
```
## How was this patch tested?
Doctests for Python and unit test added in `test_sparkSQL.R` for R.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #19105 from HyukjinKwon/unionByName-r-python.
2017-09-03 08:03:21 -04:00
|
|
|
@since(2.3)
|
|
|
|
def unionByName(self, other):
|
|
|
|
""" Returns a new :class:`DataFrame` containing union of rows in this and another frame.
|
|
|
|
|
|
|
|
This is different from both `UNION ALL` and `UNION DISTINCT` in SQL. To do a SQL-style set
|
|
|
|
union (that does deduplication of elements), use this function followed by :func:`distinct`.
|
|
|
|
|
|
|
|
The difference between this function and :func:`union` is that this function
|
|
|
|
resolves columns by name (not by position):
|
|
|
|
|
|
|
|
>>> df1 = spark.createDataFrame([[1, 2, 3]], ["col0", "col1", "col2"])
|
|
|
|
>>> df2 = spark.createDataFrame([[4, 5, 6]], ["col1", "col2", "col0"])
|
|
|
|
>>> df1.unionByName(df2).show()
|
|
|
|
+----+----+----+
|
|
|
|
|col0|col1|col2|
|
|
|
|
+----+----+----+
|
|
|
|
| 1| 2| 3|
|
|
|
|
| 6| 4| 5|
|
|
|
|
+----+----+----+
|
|
|
|
"""
|
|
|
|
return DataFrame(self._jdf.unionByName(other._jdf), self.sql_ctx)
|
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-02-09 23:49:22 -05:00
|
|
|
def intersect(self, other):
|
|
|
|
""" Return a new :class:`DataFrame` containing rows only in
|
|
|
|
both this frame and another frame.
|
|
|
|
|
|
|
|
This is equivalent to `INTERSECT` in SQL.
|
|
|
|
"""
|
|
|
|
return DataFrame(self._jdf.intersect(other._jdf), self.sql_ctx)
|
|
|
|
|
[SPARK-21274][SQL] Implement INTERSECT ALL clause
## What changes were proposed in this pull request?
Implements INTERSECT ALL clause through query rewrites using existing operators in Spark. Please refer to [Link](https://drive.google.com/open?id=1nyW0T0b_ajUduQoPgZLAsyHK8s3_dko3ulQuxaLpUXE) for the design.
Input Query
``` SQL
SELECT c1 FROM ut1 INTERSECT ALL SELECT c1 FROM ut2
```
Rewritten Query
```SQL
SELECT c1
FROM (
SELECT replicate_row(min_count, c1)
FROM (
SELECT c1,
IF (vcol1_cnt > vcol2_cnt, vcol2_cnt, vcol1_cnt) AS min_count
FROM (
SELECT c1, count(vcol1) as vcol1_cnt, count(vcol2) as vcol2_cnt
FROM (
SELECT c1, true as vcol1, null as vcol2 FROM ut1
UNION ALL
SELECT c1, null as vcol1, true as vcol2 FROM ut2
) AS union_all
GROUP BY c1
HAVING vcol1_cnt >= 1 AND vcol2_cnt >= 1
)
)
)
```
## How was this patch tested?
Added test cases in SQLQueryTestSuite, DataFrameSuite, SetOperationSuite
Author: Dilip Biswal <dbiswal@us.ibm.com>
Closes #21886 from dilipbiswal/dkb_intersect_all_final.
2018-07-30 01:11:01 -04:00
|
|
|
@since(2.4)
|
|
|
|
def intersectAll(self, other):
|
|
|
|
""" Return a new :class:`DataFrame` containing rows in both this dataframe and other
|
|
|
|
dataframe while preserving duplicates.
|
|
|
|
|
|
|
|
This is equivalent to `INTERSECT ALL` in SQL.
|
|
|
|
>>> df1 = spark.createDataFrame([("a", 1), ("a", 1), ("b", 3), ("c", 4)], ["C1", "C2"])
|
|
|
|
>>> df2 = spark.createDataFrame([("a", 1), ("a", 1), ("b", 3)], ["C1", "C2"])
|
|
|
|
|
|
|
|
>>> df1.intersectAll(df2).sort("C1", "C2").show()
|
|
|
|
+---+---+
|
|
|
|
| C1| C2|
|
|
|
|
+---+---+
|
|
|
|
| a| 1|
|
|
|
|
| a| 1|
|
|
|
|
| b| 3|
|
|
|
|
+---+---+
|
|
|
|
|
|
|
|
Also as standard in SQL, this function resolves columns by position (not by name).
|
|
|
|
"""
|
|
|
|
return DataFrame(self._jdf.intersectAll(other._jdf), self.sql_ctx)
|
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-02-09 23:49:22 -05:00
|
|
|
def subtract(self, other):
|
|
|
|
""" Return a new :class:`DataFrame` containing rows in this frame
|
|
|
|
but not in another frame.
|
|
|
|
|
2018-01-17 03:01:41 -05:00
|
|
|
This is equivalent to `EXCEPT DISTINCT` in SQL.
|
|
|
|
|
2015-02-09 23:49:22 -05:00
|
|
|
"""
|
|
|
|
return DataFrame(getattr(self._jdf, "except")(other._jdf), self.sql_ctx)
|
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.4)
|
2015-05-11 22:15:14 -04:00
|
|
|
def dropDuplicates(self, subset=None):
|
|
|
|
"""Return a new :class:`DataFrame` with duplicate rows removed,
|
|
|
|
optionally only considering certain columns.
|
|
|
|
|
2017-02-23 14:25:39 -05:00
|
|
|
For a static batch :class:`DataFrame`, it just drops duplicate rows. For a streaming
|
|
|
|
:class:`DataFrame`, it will keep all data across triggers as intermediate state to drop
|
|
|
|
duplicates rows. You can use :func:`withWatermark` to limit how late the duplicate data can
|
|
|
|
be and system will accordingly limit the state. In addition, too late data older than
|
|
|
|
watermark will be dropped to avoid any possibility of duplicates.
|
|
|
|
|
2015-09-29 17:45:18 -04:00
|
|
|
:func:`drop_duplicates` is an alias for :func:`dropDuplicates`.
|
|
|
|
|
2015-05-11 22:15:14 -04:00
|
|
|
>>> from pyspark.sql import Row
|
2016-07-06 13:45:51 -04:00
|
|
|
>>> df = sc.parallelize([ \\
|
|
|
|
... Row(name='Alice', age=5, height=80), \\
|
|
|
|
... Row(name='Alice', age=5, height=80), \\
|
|
|
|
... Row(name='Alice', age=10, height=80)]).toDF()
|
2015-05-11 22:15:14 -04:00
|
|
|
>>> df.dropDuplicates().show()
|
|
|
|
+---+------+-----+
|
|
|
|
|age|height| name|
|
|
|
|
+---+------+-----+
|
|
|
|
| 5| 80|Alice|
|
|
|
|
| 10| 80|Alice|
|
|
|
|
+---+------+-----+
|
|
|
|
|
|
|
|
>>> df.dropDuplicates(['name', 'height']).show()
|
|
|
|
+---+------+-----+
|
|
|
|
|age|height| name|
|
|
|
|
+---+------+-----+
|
|
|
|
| 5| 80|Alice|
|
|
|
|
+---+------+-----+
|
|
|
|
"""
|
|
|
|
if subset is None:
|
|
|
|
jdf = self._jdf.dropDuplicates()
|
|
|
|
else:
|
|
|
|
jdf = self._jdf.dropDuplicates(self._jseq(subset))
|
|
|
|
return DataFrame(jdf, self.sql_ctx)
|
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since("1.3.1")
|
2015-03-30 23:47:10 -04:00
|
|
|
def dropna(self, how='any', thresh=None, subset=None):
|
|
|
|
"""Returns a new :class:`DataFrame` omitting rows with null values.
|
2015-06-07 04:21:02 -04:00
|
|
|
:func:`DataFrame.dropna` and :func:`DataFrameNaFunctions.drop` are aliases of each other.
|
2015-03-31 03:25:23 -04:00
|
|
|
|
2015-03-30 23:47:10 -04:00
|
|
|
:param how: 'any' or 'all'.
|
|
|
|
If 'any', drop a row if it contains any nulls.
|
|
|
|
If 'all', drop a row only if all its values are null.
|
|
|
|
:param thresh: int, default None
|
|
|
|
If specified, drop rows that have less than `thresh` non-null values.
|
|
|
|
This overwrites the `how` parameter.
|
|
|
|
:param subset: optional list of column names to consider.
|
|
|
|
|
2015-03-31 03:25:23 -04:00
|
|
|
>>> df4.na.drop().show()
|
2015-05-04 15:08:38 -04:00
|
|
|
+---+------+-----+
|
|
|
|
|age|height| name|
|
|
|
|
+---+------+-----+
|
|
|
|
| 10| 80|Alice|
|
|
|
|
+---+------+-----+
|
2015-03-30 23:47:10 -04:00
|
|
|
"""
|
|
|
|
if how is not None and how not in ['any', 'all']:
|
|
|
|
raise ValueError("how ('" + how + "') should be 'any' or 'all'")
|
|
|
|
|
|
|
|
if subset is None:
|
|
|
|
subset = self.columns
|
|
|
|
elif isinstance(subset, basestring):
|
|
|
|
subset = [subset]
|
|
|
|
elif not isinstance(subset, (list, tuple)):
|
|
|
|
raise ValueError("subset should be a list or tuple of column names")
|
|
|
|
|
|
|
|
if thresh is None:
|
|
|
|
thresh = len(subset) if how == 'any' else 1
|
|
|
|
|
2015-04-17 12:29:27 -04:00
|
|
|
return DataFrame(self._jdf.na().drop(thresh, self._jseq(subset)), self.sql_ctx)
|
2015-03-30 23:47:10 -04:00
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since("1.3.1")
|
2015-03-30 23:47:10 -04:00
|
|
|
def fillna(self, value, subset=None):
|
2015-03-31 21:31:36 -04:00
|
|
|
"""Replace null values, alias for ``na.fill()``.
|
2015-06-07 04:21:02 -04:00
|
|
|
:func:`DataFrame.fillna` and :func:`DataFrameNaFunctions.fill` are aliases of each other.
|
2015-03-30 23:47:10 -04:00
|
|
|
|
2017-06-03 01:56:42 -04:00
|
|
|
:param value: int, long, float, string, bool or dict.
|
2015-03-30 23:47:10 -04:00
|
|
|
Value to replace null values with.
|
|
|
|
If the value is a dict, then `subset` is ignored and `value` must be a mapping
|
|
|
|
from column name (string) to replacement value. The replacement value must be
|
2017-05-01 00:42:05 -04:00
|
|
|
an int, long, float, boolean, or string.
|
2015-03-30 23:47:10 -04:00
|
|
|
:param subset: optional list of column names to consider.
|
|
|
|
Columns specified in subset that do not have matching data type are ignored.
|
|
|
|
For example, if `value` is a string, and subset contains a non-string column,
|
|
|
|
then the non-string column is simply ignored.
|
|
|
|
|
2015-06-07 04:21:02 -04:00
|
|
|
>>> df4.na.fill(50).show()
|
2015-05-04 15:08:38 -04:00
|
|
|
+---+------+-----+
|
|
|
|
|age|height| name|
|
|
|
|
+---+------+-----+
|
|
|
|
| 10| 80|Alice|
|
|
|
|
| 5| 50| Bob|
|
|
|
|
| 50| 50| Tom|
|
|
|
|
| 50| 50| null|
|
|
|
|
+---+------+-----+
|
2015-03-30 23:47:10 -04:00
|
|
|
|
2017-06-03 01:56:42 -04:00
|
|
|
>>> df5.na.fill(False).show()
|
|
|
|
+----+-------+-----+
|
|
|
|
| age| name| spy|
|
|
|
|
+----+-------+-----+
|
|
|
|
| 10| Alice|false|
|
|
|
|
| 5| Bob|false|
|
|
|
|
|null|Mallory| true|
|
|
|
|
+----+-------+-----+
|
|
|
|
|
2015-03-31 03:25:23 -04:00
|
|
|
>>> df4.na.fill({'age': 50, 'name': 'unknown'}).show()
|
2015-05-04 15:08:38 -04:00
|
|
|
+---+------+-------+
|
|
|
|
|age|height| name|
|
|
|
|
+---+------+-------+
|
|
|
|
| 10| 80| Alice|
|
|
|
|
| 5| null| Bob|
|
|
|
|
| 50| null| Tom|
|
|
|
|
| 50| null|unknown|
|
|
|
|
+---+------+-------+
|
2015-03-30 23:47:10 -04:00
|
|
|
"""
|
2017-06-03 01:56:42 -04:00
|
|
|
if not isinstance(value, (float, int, long, basestring, bool, dict)):
|
|
|
|
raise ValueError("value should be a float, int, long, string, bool or dict")
|
|
|
|
|
|
|
|
# Note that bool validates isinstance(int), but we don't want to
|
|
|
|
# convert bools to floats
|
2015-03-30 23:47:10 -04:00
|
|
|
|
2017-06-03 01:56:42 -04:00
|
|
|
if not isinstance(value, bool) and isinstance(value, (int, long)):
|
2015-03-30 23:47:10 -04:00
|
|
|
value = float(value)
|
|
|
|
|
|
|
|
if isinstance(value, dict):
|
|
|
|
return DataFrame(self._jdf.na().fill(value), self.sql_ctx)
|
|
|
|
elif subset is None:
|
|
|
|
return DataFrame(self._jdf.na().fill(value), self.sql_ctx)
|
|
|
|
else:
|
|
|
|
if isinstance(subset, basestring):
|
|
|
|
subset = [subset]
|
|
|
|
elif not isinstance(subset, (list, tuple)):
|
|
|
|
raise ValueError("subset should be a list or tuple of column names")
|
|
|
|
|
2015-04-17 12:29:27 -04:00
|
|
|
return DataFrame(self._jdf.na().fill(value, self._jseq(subset)), self.sql_ctx)
|
2015-03-30 23:47:10 -04:00
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.4)
|
2018-02-09 01:21:10 -05:00
|
|
|
def replace(self, to_replace, value=_NoValue, subset=None):
|
2015-05-12 13:23:41 -04:00
|
|
|
"""Returns a new :class:`DataFrame` replacing a value with another value.
|
2015-06-07 04:21:02 -04:00
|
|
|
:func:`DataFrame.replace` and :func:`DataFrameNaFunctions.replace` are
|
|
|
|
aliases of each other.
|
[SPARK-14932][SQL] Allow DataFrame.replace() to replace values with None
## What changes were proposed in this pull request?
Currently `df.na.replace("*", Map[String, String]("NULL" -> null))` will produce exception.
This PR enables passing null/None as value in the replacement map in DataFrame.replace().
Note that the replacement map keys and values should still be the same type, while the values can have a mix of null/None and that type.
This PR enables following operations for example:
`df.na.replace("*", Map[String, String]("NULL" -> null))`(scala)
`df.na.replace("*", Map[Any, Any](60 -> null, 70 -> 80))`(scala)
`df.na.replace('Alice', None)`(python)
`df.na.replace([10, 20])`(python, replacing with None is by default)
One use case could be: I want to replace all the empty strings with null/None because they were incorrectly generated and then drop all null/None data
`df.na.replace("*", Map("" -> null)).na.drop()`(scala)
`df.replace(u'', None).dropna()`(python)
## How was this patch tested?
Scala unit test.
Python doctest and unit test.
Author: bravo-zhang <mzhang1230@gmail.com>
Closes #18820 from bravo-zhang/spark-14932.
2017-08-09 20:42:21 -04:00
|
|
|
Values to_replace and value must have the same type and can only be numerics, booleans,
|
|
|
|
or strings. Value can have None. When replacing, the new value will be cast
|
2017-02-14 12:42:24 -05:00
|
|
|
to the type of the existing column.
|
|
|
|
For numeric replacements all values to be replaced should have unique
|
|
|
|
floating point representation. In case of conflicts (for example with `{42: -1, 42.0: 1}`)
|
|
|
|
and arbitrary replacement will be used.
|
|
|
|
|
|
|
|
:param to_replace: bool, int, long, float, string, list or dict.
|
2015-05-12 13:23:41 -04:00
|
|
|
Value to be replaced.
|
2018-02-09 01:21:10 -05:00
|
|
|
If the value is a dict, then `value` is ignored or can be omitted, and `to_replace`
|
|
|
|
must be a mapping between a value and a replacement.
|
[SPARK-14932][SQL] Allow DataFrame.replace() to replace values with None
## What changes were proposed in this pull request?
Currently `df.na.replace("*", Map[String, String]("NULL" -> null))` will produce exception.
This PR enables passing null/None as value in the replacement map in DataFrame.replace().
Note that the replacement map keys and values should still be the same type, while the values can have a mix of null/None and that type.
This PR enables following operations for example:
`df.na.replace("*", Map[String, String]("NULL" -> null))`(scala)
`df.na.replace("*", Map[Any, Any](60 -> null, 70 -> 80))`(scala)
`df.na.replace('Alice', None)`(python)
`df.na.replace([10, 20])`(python, replacing with None is by default)
One use case could be: I want to replace all the empty strings with null/None because they were incorrectly generated and then drop all null/None data
`df.na.replace("*", Map("" -> null)).na.drop()`(scala)
`df.replace(u'', None).dropna()`(python)
## How was this patch tested?
Scala unit test.
Python doctest and unit test.
Author: bravo-zhang <mzhang1230@gmail.com>
Closes #18820 from bravo-zhang/spark-14932.
2017-08-09 20:42:21 -04:00
|
|
|
:param value: bool, int, long, float, string, list or None.
|
|
|
|
The replacement value must be a bool, int, long, float, string or None. If `value` is a
|
2017-02-14 12:42:24 -05:00
|
|
|
list, `value` should be of the same length and type as `to_replace`.
|
|
|
|
If `value` is a scalar and `to_replace` is a sequence, then `value` is
|
|
|
|
used as a replacement for each item in `to_replace`.
|
2015-05-12 13:23:41 -04:00
|
|
|
:param subset: optional list of column names to consider.
|
|
|
|
Columns specified in subset that do not have matching data type are ignored.
|
|
|
|
For example, if `value` is a string, and subset contains a non-string column,
|
|
|
|
then the non-string column is simply ignored.
|
2015-05-18 11:35:14 -04:00
|
|
|
|
2015-06-07 04:21:02 -04:00
|
|
|
>>> df4.na.replace(10, 20).show()
|
2015-05-12 13:23:41 -04:00
|
|
|
+----+------+-----+
|
|
|
|
| age|height| name|
|
|
|
|
+----+------+-----+
|
|
|
|
| 20| 80|Alice|
|
|
|
|
| 5| null| Bob|
|
|
|
|
|null| null| Tom|
|
|
|
|
|null| null| null|
|
|
|
|
+----+------+-----+
|
|
|
|
|
[SPARK-14932][SQL] Allow DataFrame.replace() to replace values with None
## What changes were proposed in this pull request?
Currently `df.na.replace("*", Map[String, String]("NULL" -> null))` will produce exception.
This PR enables passing null/None as value in the replacement map in DataFrame.replace().
Note that the replacement map keys and values should still be the same type, while the values can have a mix of null/None and that type.
This PR enables following operations for example:
`df.na.replace("*", Map[String, String]("NULL" -> null))`(scala)
`df.na.replace("*", Map[Any, Any](60 -> null, 70 -> 80))`(scala)
`df.na.replace('Alice', None)`(python)
`df.na.replace([10, 20])`(python, replacing with None is by default)
One use case could be: I want to replace all the empty strings with null/None because they were incorrectly generated and then drop all null/None data
`df.na.replace("*", Map("" -> null)).na.drop()`(scala)
`df.replace(u'', None).dropna()`(python)
## How was this patch tested?
Scala unit test.
Python doctest and unit test.
Author: bravo-zhang <mzhang1230@gmail.com>
Closes #18820 from bravo-zhang/spark-14932.
2017-08-09 20:42:21 -04:00
|
|
|
>>> df4.na.replace('Alice', None).show()
|
|
|
|
+----+------+----+
|
|
|
|
| age|height|name|
|
|
|
|
+----+------+----+
|
|
|
|
| 10| 80|null|
|
|
|
|
| 5| null| Bob|
|
|
|
|
|null| null| Tom|
|
|
|
|
|null| null|null|
|
|
|
|
+----+------+----+
|
|
|
|
|
2018-02-09 01:21:10 -05:00
|
|
|
>>> df4.na.replace({'Alice': None}).show()
|
|
|
|
+----+------+----+
|
|
|
|
| age|height|name|
|
|
|
|
+----+------+----+
|
|
|
|
| 10| 80|null|
|
|
|
|
| 5| null| Bob|
|
|
|
|
|null| null| Tom|
|
|
|
|
|null| null|null|
|
|
|
|
+----+------+----+
|
|
|
|
|
2015-06-07 04:21:02 -04:00
|
|
|
>>> df4.na.replace(['Alice', 'Bob'], ['A', 'B'], 'name').show()
|
2015-05-12 13:23:41 -04:00
|
|
|
+----+------+----+
|
|
|
|
| age|height|name|
|
|
|
|
+----+------+----+
|
|
|
|
| 10| 80| A|
|
|
|
|
| 5| null| B|
|
|
|
|
|null| null| Tom|
|
|
|
|
|null| null|null|
|
|
|
|
+----+------+----+
|
|
|
|
"""
|
2018-02-09 01:21:10 -05:00
|
|
|
if value is _NoValue:
|
|
|
|
if isinstance(to_replace, dict):
|
|
|
|
value = None
|
|
|
|
else:
|
|
|
|
raise TypeError("value argument is required when to_replace is not a dictionary.")
|
|
|
|
|
2017-04-05 14:47:40 -04:00
|
|
|
# Helper functions
|
|
|
|
def all_of(types):
|
|
|
|
"""Given a type or tuple of types and a sequence of xs
|
|
|
|
check if each x is instance of type(s)
|
|
|
|
|
|
|
|
>>> all_of(bool)([True, False])
|
|
|
|
True
|
|
|
|
>>> all_of(basestring)(["a", 1])
|
|
|
|
False
|
|
|
|
"""
|
|
|
|
def all_of_(xs):
|
|
|
|
return all(isinstance(x, types) for x in xs)
|
|
|
|
return all_of_
|
|
|
|
|
|
|
|
all_of_bool = all_of(bool)
|
|
|
|
all_of_str = all_of(basestring)
|
|
|
|
all_of_numeric = all_of((float, int, long))
|
|
|
|
|
|
|
|
# Validate input types
|
|
|
|
valid_types = (bool, float, int, long, basestring, list, tuple)
|
|
|
|
if not isinstance(to_replace, valid_types + (dict, )):
|
2015-05-12 13:23:41 -04:00
|
|
|
raise ValueError(
|
[SPARK-14932][SQL] Allow DataFrame.replace() to replace values with None
## What changes were proposed in this pull request?
Currently `df.na.replace("*", Map[String, String]("NULL" -> null))` will produce exception.
This PR enables passing null/None as value in the replacement map in DataFrame.replace().
Note that the replacement map keys and values should still be the same type, while the values can have a mix of null/None and that type.
This PR enables following operations for example:
`df.na.replace("*", Map[String, String]("NULL" -> null))`(scala)
`df.na.replace("*", Map[Any, Any](60 -> null, 70 -> 80))`(scala)
`df.na.replace('Alice', None)`(python)
`df.na.replace([10, 20])`(python, replacing with None is by default)
One use case could be: I want to replace all the empty strings with null/None because they were incorrectly generated and then drop all null/None data
`df.na.replace("*", Map("" -> null)).na.drop()`(scala)
`df.replace(u'', None).dropna()`(python)
## How was this patch tested?
Scala unit test.
Python doctest and unit test.
Author: bravo-zhang <mzhang1230@gmail.com>
Closes #18820 from bravo-zhang/spark-14932.
2017-08-09 20:42:21 -04:00
|
|
|
"to_replace should be a bool, float, int, long, string, list, tuple, or dict. "
|
2017-04-05 14:47:40 -04:00
|
|
|
"Got {0}".format(type(to_replace)))
|
|
|
|
|
[SPARK-14932][SQL] Allow DataFrame.replace() to replace values with None
## What changes were proposed in this pull request?
Currently `df.na.replace("*", Map[String, String]("NULL" -> null))` will produce exception.
This PR enables passing null/None as value in the replacement map in DataFrame.replace().
Note that the replacement map keys and values should still be the same type, while the values can have a mix of null/None and that type.
This PR enables following operations for example:
`df.na.replace("*", Map[String, String]("NULL" -> null))`(scala)
`df.na.replace("*", Map[Any, Any](60 -> null, 70 -> 80))`(scala)
`df.na.replace('Alice', None)`(python)
`df.na.replace([10, 20])`(python, replacing with None is by default)
One use case could be: I want to replace all the empty strings with null/None because they were incorrectly generated and then drop all null/None data
`df.na.replace("*", Map("" -> null)).na.drop()`(scala)
`df.replace(u'', None).dropna()`(python)
## How was this patch tested?
Scala unit test.
Python doctest and unit test.
Author: bravo-zhang <mzhang1230@gmail.com>
Closes #18820 from bravo-zhang/spark-14932.
2017-08-09 20:42:21 -04:00
|
|
|
if not isinstance(value, valid_types) and value is not None \
|
|
|
|
and not isinstance(to_replace, dict):
|
2017-04-05 14:47:40 -04:00
|
|
|
raise ValueError("If to_replace is not a dict, value should be "
|
[SPARK-14932][SQL] Allow DataFrame.replace() to replace values with None
## What changes were proposed in this pull request?
Currently `df.na.replace("*", Map[String, String]("NULL" -> null))` will produce exception.
This PR enables passing null/None as value in the replacement map in DataFrame.replace().
Note that the replacement map keys and values should still be the same type, while the values can have a mix of null/None and that type.
This PR enables following operations for example:
`df.na.replace("*", Map[String, String]("NULL" -> null))`(scala)
`df.na.replace("*", Map[Any, Any](60 -> null, 70 -> 80))`(scala)
`df.na.replace('Alice', None)`(python)
`df.na.replace([10, 20])`(python, replacing with None is by default)
One use case could be: I want to replace all the empty strings with null/None because they were incorrectly generated and then drop all null/None data
`df.na.replace("*", Map("" -> null)).na.drop()`(scala)
`df.replace(u'', None).dropna()`(python)
## How was this patch tested?
Scala unit test.
Python doctest and unit test.
Author: bravo-zhang <mzhang1230@gmail.com>
Closes #18820 from bravo-zhang/spark-14932.
2017-08-09 20:42:21 -04:00
|
|
|
"a bool, float, int, long, string, list, tuple or None. "
|
2017-04-05 14:47:40 -04:00
|
|
|
"Got {0}".format(type(value)))
|
2015-05-12 13:23:41 -04:00
|
|
|
|
2017-04-05 14:47:40 -04:00
|
|
|
if isinstance(to_replace, (list, tuple)) and isinstance(value, (list, tuple)):
|
|
|
|
if len(to_replace) != len(value):
|
|
|
|
raise ValueError("to_replace and value lists should be of the same length. "
|
|
|
|
"Got {0} and {1}".format(len(to_replace), len(value)))
|
2015-05-12 13:23:41 -04:00
|
|
|
|
2017-04-05 14:47:40 -04:00
|
|
|
if not (subset is None or isinstance(subset, (list, tuple, basestring))):
|
|
|
|
raise ValueError("subset should be a list or tuple of column names, "
|
|
|
|
"column name or None. Got {0}".format(type(subset)))
|
2015-05-12 13:23:41 -04:00
|
|
|
|
2017-04-05 14:47:40 -04:00
|
|
|
# Reshape input arguments if necessary
|
2015-05-12 13:23:41 -04:00
|
|
|
if isinstance(to_replace, (float, int, long, basestring)):
|
|
|
|
to_replace = [to_replace]
|
|
|
|
|
2017-04-05 14:47:40 -04:00
|
|
|
if isinstance(to_replace, dict):
|
2015-05-12 13:23:41 -04:00
|
|
|
rep_dict = to_replace
|
2017-04-05 14:47:40 -04:00
|
|
|
if value is not None:
|
|
|
|
warnings.warn("to_replace is a dict and value is not None. value will be ignored.")
|
|
|
|
else:
|
[SPARK-14932][SQL] Allow DataFrame.replace() to replace values with None
## What changes were proposed in this pull request?
Currently `df.na.replace("*", Map[String, String]("NULL" -> null))` will produce exception.
This PR enables passing null/None as value in the replacement map in DataFrame.replace().
Note that the replacement map keys and values should still be the same type, while the values can have a mix of null/None and that type.
This PR enables following operations for example:
`df.na.replace("*", Map[String, String]("NULL" -> null))`(scala)
`df.na.replace("*", Map[Any, Any](60 -> null, 70 -> 80))`(scala)
`df.na.replace('Alice', None)`(python)
`df.na.replace([10, 20])`(python, replacing with None is by default)
One use case could be: I want to replace all the empty strings with null/None because they were incorrectly generated and then drop all null/None data
`df.na.replace("*", Map("" -> null)).na.drop()`(scala)
`df.replace(u'', None).dropna()`(python)
## How was this patch tested?
Scala unit test.
Python doctest and unit test.
Author: bravo-zhang <mzhang1230@gmail.com>
Closes #18820 from bravo-zhang/spark-14932.
2017-08-09 20:42:21 -04:00
|
|
|
if isinstance(value, (float, int, long, basestring)) or value is None:
|
|
|
|
value = [value for _ in range(len(to_replace))]
|
2017-04-05 14:47:40 -04:00
|
|
|
rep_dict = dict(zip(to_replace, value))
|
2015-05-12 13:23:41 -04:00
|
|
|
|
2017-04-05 14:47:40 -04:00
|
|
|
if isinstance(subset, basestring):
|
2015-05-12 13:23:41 -04:00
|
|
|
subset = [subset]
|
|
|
|
|
[SPARK-14932][SQL] Allow DataFrame.replace() to replace values with None
## What changes were proposed in this pull request?
Currently `df.na.replace("*", Map[String, String]("NULL" -> null))` will produce exception.
This PR enables passing null/None as value in the replacement map in DataFrame.replace().
Note that the replacement map keys and values should still be the same type, while the values can have a mix of null/None and that type.
This PR enables following operations for example:
`df.na.replace("*", Map[String, String]("NULL" -> null))`(scala)
`df.na.replace("*", Map[Any, Any](60 -> null, 70 -> 80))`(scala)
`df.na.replace('Alice', None)`(python)
`df.na.replace([10, 20])`(python, replacing with None is by default)
One use case could be: I want to replace all the empty strings with null/None because they were incorrectly generated and then drop all null/None data
`df.na.replace("*", Map("" -> null)).na.drop()`(scala)
`df.replace(u'', None).dropna()`(python)
## How was this patch tested?
Scala unit test.
Python doctest and unit test.
Author: bravo-zhang <mzhang1230@gmail.com>
Closes #18820 from bravo-zhang/spark-14932.
2017-08-09 20:42:21 -04:00
|
|
|
# Verify we were not passed in mixed type generics.
|
|
|
|
if not any(all_of_type(rep_dict.keys())
|
|
|
|
and all_of_type(x for x in rep_dict.values() if x is not None)
|
2017-04-05 14:47:40 -04:00
|
|
|
for all_of_type in [all_of_bool, all_of_str, all_of_numeric]):
|
|
|
|
raise ValueError("Mixed type replacements are not supported")
|
2015-05-12 13:23:41 -04:00
|
|
|
|
2017-04-05 14:47:40 -04:00
|
|
|
if subset is None:
|
|
|
|
return DataFrame(self._jdf.na().replace('*', rep_dict), self.sql_ctx)
|
|
|
|
else:
|
|
|
|
return DataFrame(
|
|
|
|
self._jdf.na().replace(self._jseq(subset), self._jmap(rep_dict)), self.sql_ctx)
|
2015-05-12 13:23:41 -04:00
|
|
|
|
2016-02-25 02:15:36 -05:00
|
|
|
@since(2.0)
|
|
|
|
def approxQuantile(self, col, probabilities, relativeError):
|
|
|
|
"""
|
2017-02-01 17:11:28 -05:00
|
|
|
Calculates the approximate quantiles of numerical columns of a
|
2016-02-25 02:15:36 -05:00
|
|
|
DataFrame.
|
|
|
|
|
|
|
|
The result of this algorithm has the following deterministic bound:
|
|
|
|
If the DataFrame has N elements and if we request the quantile at
|
|
|
|
probability `p` up to error `err`, then the algorithm will return
|
|
|
|
a sample `x` from the DataFrame so that the *exact* rank of `x` is
|
|
|
|
close to (p * N). More precisely,
|
|
|
|
|
|
|
|
floor((p - err) * N) <= rank(x) <= ceil((p + err) * N).
|
|
|
|
|
|
|
|
This method implements a variation of the Greenwald-Khanna
|
|
|
|
algorithm (with some speed optimizations). The algorithm was first
|
|
|
|
present in [[http://dx.doi.org/10.1145/375663.375670
|
|
|
|
Space-efficient Online Computation of Quantile Summaries]]
|
|
|
|
by Greenwald and Khanna.
|
|
|
|
|
2017-03-21 11:45:59 -04:00
|
|
|
Note that null values will be ignored in numerical columns before calculation.
|
|
|
|
For columns only containing null values, an empty list is returned.
|
2017-02-01 17:11:28 -05:00
|
|
|
|
|
|
|
:param col: str, list.
|
|
|
|
Can be a single column name, or a list of names for multiple columns.
|
2016-02-25 02:15:36 -05:00
|
|
|
:param probabilities: a list of quantile probabilities
|
|
|
|
Each number must belong to [0, 1].
|
|
|
|
For example 0 is the minimum, 0.5 is the median, 1 is the maximum.
|
|
|
|
:param relativeError: The relative target precision to achieve
|
|
|
|
(>= 0). If set to zero, the exact quantiles are computed, which
|
|
|
|
could be very expensive. Note that values greater than 1 are
|
|
|
|
accepted but give the same result as 1.
|
2017-02-01 17:11:28 -05:00
|
|
|
:return: the approximate quantiles at the given probabilities. If
|
|
|
|
the input `col` is a string, the output is a list of floats. If the
|
|
|
|
input `col` is a list or tuple of strings, the output is also a
|
|
|
|
list, but each element in it is a list of floats, i.e., the output
|
|
|
|
is a list of list of floats.
|
|
|
|
|
|
|
|
.. versionchanged:: 2.2
|
|
|
|
Added support for multiple columns.
|
2016-02-25 02:15:36 -05:00
|
|
|
"""
|
2017-02-01 17:11:28 -05:00
|
|
|
|
2017-09-08 14:57:33 -04:00
|
|
|
if not isinstance(col, (basestring, list, tuple)):
|
2017-02-01 17:11:28 -05:00
|
|
|
raise ValueError("col should be a string, list or tuple, but got %r" % type(col))
|
|
|
|
|
2017-09-08 14:57:33 -04:00
|
|
|
isStr = isinstance(col, basestring)
|
2017-02-01 17:11:28 -05:00
|
|
|
|
|
|
|
if isinstance(col, tuple):
|
|
|
|
col = list(col)
|
2017-09-08 14:57:33 -04:00
|
|
|
elif isStr:
|
2017-02-01 17:11:28 -05:00
|
|
|
col = [col]
|
|
|
|
|
|
|
|
for c in col:
|
2017-09-08 14:57:33 -04:00
|
|
|
if not isinstance(c, basestring):
|
2017-02-01 17:11:28 -05:00
|
|
|
raise ValueError("columns should be strings, but got %r" % type(c))
|
|
|
|
col = _to_list(self._sc, col)
|
2016-02-25 02:15:36 -05:00
|
|
|
|
|
|
|
if not isinstance(probabilities, (list, tuple)):
|
|
|
|
raise ValueError("probabilities should be a list or tuple")
|
|
|
|
if isinstance(probabilities, tuple):
|
|
|
|
probabilities = list(probabilities)
|
|
|
|
for p in probabilities:
|
|
|
|
if not isinstance(p, (float, int, long)) or p < 0 or p > 1:
|
|
|
|
raise ValueError("probabilities should be numerical (float, int, long) in [0,1].")
|
|
|
|
probabilities = _to_list(self._sc, probabilities)
|
|
|
|
|
|
|
|
if not isinstance(relativeError, (float, int, long)) or relativeError < 0:
|
|
|
|
raise ValueError("relativeError should be numerical (float, int, long) >= 0.")
|
|
|
|
relativeError = float(relativeError)
|
|
|
|
|
|
|
|
jaq = self._jdf.stat().approxQuantile(col, probabilities, relativeError)
|
2017-02-01 17:11:28 -05:00
|
|
|
jaq_list = [list(j) for j in jaq]
|
|
|
|
return jaq_list[0] if isStr else jaq_list
|
2016-02-25 02:15:36 -05:00
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.4)
|
2015-05-04 00:44:39 -04:00
|
|
|
def corr(self, col1, col2, method=None):
|
|
|
|
"""
|
2015-06-07 04:21:02 -04:00
|
|
|
Calculates the correlation of two columns of a DataFrame as a double value.
|
|
|
|
Currently only supports the Pearson Correlation Coefficient.
|
|
|
|
:func:`DataFrame.corr` and :func:`DataFrameStatFunctions.corr` are aliases of each other.
|
2015-05-04 00:44:39 -04:00
|
|
|
|
|
|
|
:param col1: The name of the first column
|
|
|
|
:param col2: The name of the second column
|
|
|
|
:param method: The correlation method. Currently only supports "pearson"
|
|
|
|
"""
|
2017-09-08 14:57:33 -04:00
|
|
|
if not isinstance(col1, basestring):
|
2015-05-04 00:44:39 -04:00
|
|
|
raise ValueError("col1 should be a string.")
|
2017-09-08 14:57:33 -04:00
|
|
|
if not isinstance(col2, basestring):
|
2015-05-04 00:44:39 -04:00
|
|
|
raise ValueError("col2 should be a string.")
|
|
|
|
if not method:
|
|
|
|
method = "pearson"
|
|
|
|
if not method == "pearson":
|
|
|
|
raise ValueError("Currently only the calculation of the Pearson Correlation " +
|
|
|
|
"coefficient is supported.")
|
|
|
|
return self._jdf.stat().corr(col1, col2, method)
|
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.4)
|
2015-05-01 16:29:17 -04:00
|
|
|
def cov(self, col1, col2):
|
|
|
|
"""
|
|
|
|
Calculate the sample covariance for the given columns, specified by their names, as a
|
|
|
|
double value. :func:`DataFrame.cov` and :func:`DataFrameStatFunctions.cov` are aliases.
|
|
|
|
|
|
|
|
:param col1: The name of the first column
|
|
|
|
:param col2: The name of the second column
|
|
|
|
"""
|
2017-09-08 14:57:33 -04:00
|
|
|
if not isinstance(col1, basestring):
|
2015-05-01 16:29:17 -04:00
|
|
|
raise ValueError("col1 should be a string.")
|
2017-09-08 14:57:33 -04:00
|
|
|
if not isinstance(col2, basestring):
|
2015-05-01 16:29:17 -04:00
|
|
|
raise ValueError("col2 should be a string.")
|
|
|
|
return self._jdf.stat().cov(col1, col2)
|
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.4)
|
2015-05-04 20:02:49 -04:00
|
|
|
def crosstab(self, col1, col2):
|
|
|
|
"""
|
|
|
|
Computes a pair-wise frequency table of the given columns. Also known as a contingency
|
2015-05-05 14:01:25 -04:00
|
|
|
table. The number of distinct values for each column should be less than 1e4. At most 1e6
|
|
|
|
non-zero pair frequencies will be returned.
|
|
|
|
The first column of each row will be the distinct values of `col1` and the column names
|
|
|
|
will be the distinct values of `col2`. The name of the first column will be `$col1_$col2`.
|
2015-07-23 13:32:11 -04:00
|
|
|
Pairs that have no occurrences will have zero as their counts.
|
2015-05-04 20:02:49 -04:00
|
|
|
:func:`DataFrame.crosstab` and :func:`DataFrameStatFunctions.crosstab` are aliases.
|
|
|
|
|
|
|
|
:param col1: The name of the first column. Distinct items will make the first item of
|
|
|
|
each row.
|
|
|
|
:param col2: The name of the second column. Distinct items will make the column names
|
|
|
|
of the DataFrame.
|
|
|
|
"""
|
2017-09-08 14:57:33 -04:00
|
|
|
if not isinstance(col1, basestring):
|
2015-05-04 20:02:49 -04:00
|
|
|
raise ValueError("col1 should be a string.")
|
2017-09-08 14:57:33 -04:00
|
|
|
if not isinstance(col2, basestring):
|
2015-05-04 20:02:49 -04:00
|
|
|
raise ValueError("col2 should be a string.")
|
|
|
|
return DataFrame(self._jdf.stat().crosstab(col1, col2), self.sql_ctx)
|
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.4)
|
2015-05-02 02:43:24 -04:00
|
|
|
def freqItems(self, cols, support=None):
|
|
|
|
"""
|
|
|
|
Finding frequent items for columns, possibly with false positives. Using the
|
|
|
|
frequent element count algorithm described in
|
|
|
|
"http://dx.doi.org/10.1145/762471.762473, proposed by Karp, Schenker, and Papadimitriou".
|
|
|
|
:func:`DataFrame.freqItems` and :func:`DataFrameStatFunctions.freqItems` are aliases.
|
|
|
|
|
2016-11-22 06:40:18 -05:00
|
|
|
.. note:: This function is meant for exploratory data analysis, as we make no
|
|
|
|
guarantee about the backward compatibility of the schema of the resulting DataFrame.
|
2015-06-02 00:29:39 -04:00
|
|
|
|
2015-05-02 02:43:24 -04:00
|
|
|
:param cols: Names of the columns to calculate frequent items for as a list or tuple of
|
|
|
|
strings.
|
|
|
|
:param support: The frequency with which to consider an item 'frequent'. Default is 1%.
|
|
|
|
The support must be greater than 1e-4.
|
|
|
|
"""
|
|
|
|
if isinstance(cols, tuple):
|
|
|
|
cols = list(cols)
|
|
|
|
if not isinstance(cols, list):
|
|
|
|
raise ValueError("cols must be a list or tuple of column names as strings.")
|
|
|
|
if not support:
|
|
|
|
support = 0.01
|
|
|
|
return DataFrame(self._jdf.stat().freqItems(_to_seq(self._sc, cols), support), self.sql_ctx)
|
|
|
|
|
2015-04-16 19:20:57 -04:00
|
|
|
@ignore_unicode_prefix
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-02-14 02:03:22 -05:00
|
|
|
def withColumn(self, colName, col):
|
2015-08-19 16:56:40 -04:00
|
|
|
"""
|
|
|
|
Returns a new :class:`DataFrame` by adding a column or replacing the
|
|
|
|
existing column that has the same name.
|
2015-03-31 21:31:36 -04:00
|
|
|
|
2018-01-31 21:15:17 -05:00
|
|
|
The column expression must be an expression over this DataFrame; attempting to add
|
2018-01-30 01:19:59 -05:00
|
|
|
a column from some other dataframe will raise an error.
|
|
|
|
|
2015-03-31 21:31:36 -04:00
|
|
|
:param colName: string, name of the new column.
|
|
|
|
:param col: a :class:`Column` expression for the new column.
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2015-02-14 02:03:22 -05:00
|
|
|
>>> df.withColumn('age2', df.age + 2).collect()
|
2015-02-09 23:49:22 -05:00
|
|
|
[Row(age=2, name=u'Alice', age2=4), Row(age=5, name=u'Bob', age2=7)]
|
2018-01-30 01:19:59 -05:00
|
|
|
|
2015-02-09 23:49:22 -05:00
|
|
|
"""
|
2015-08-19 16:56:40 -04:00
|
|
|
assert isinstance(col, Column), "col should be Column"
|
|
|
|
return DataFrame(self._jdf.withColumn(colName, col._jc), self.sql_ctx)
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2015-04-16 19:20:57 -04:00
|
|
|
@ignore_unicode_prefix
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-02-14 02:03:22 -05:00
|
|
|
def withColumnRenamed(self, existing, new):
|
2015-05-11 23:04:36 -04:00
|
|
|
"""Returns a new :class:`DataFrame` by renaming an existing column.
|
2016-07-22 08:20:06 -04:00
|
|
|
This is a no-op if schema doesn't contain the given column name.
|
2015-03-31 21:31:36 -04:00
|
|
|
|
|
|
|
:param existing: string, name of the existing column to rename.
|
2018-03-05 09:17:16 -05:00
|
|
|
:param new: string, new name of the column.
|
2015-02-11 15:13:16 -05:00
|
|
|
|
2015-02-14 02:03:22 -05:00
|
|
|
>>> df.withColumnRenamed('age', 'age2').collect()
|
2015-02-11 15:13:16 -05:00
|
|
|
[Row(age2=2, name=u'Alice'), Row(age2=5, name=u'Bob')]
|
|
|
|
"""
|
2015-08-19 16:56:40 -04:00
|
|
|
return DataFrame(self._jdf.withColumnRenamed(existing, new), self.sql_ctx)
|
2015-02-11 15:13:16 -05:00
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.4)
|
2015-05-11 23:04:36 -04:00
|
|
|
@ignore_unicode_prefix
|
2016-07-15 01:55:49 -04:00
|
|
|
def drop(self, *cols):
|
2015-05-11 23:04:36 -04:00
|
|
|
"""Returns a new :class:`DataFrame` that drops the specified column.
|
2016-07-22 08:20:06 -04:00
|
|
|
This is a no-op if schema doesn't contain the given column name(s).
|
2015-05-11 23:04:36 -04:00
|
|
|
|
2016-07-15 01:55:49 -04:00
|
|
|
:param cols: a string name of the column to drop, or a
|
|
|
|
:class:`Column` to drop, or a list of string name of the columns to drop.
|
2015-05-11 23:04:36 -04:00
|
|
|
|
|
|
|
>>> df.drop('age').collect()
|
|
|
|
[Row(name=u'Alice'), Row(name=u'Bob')]
|
2015-06-04 14:30:07 -04:00
|
|
|
|
|
|
|
>>> df.drop(df.age).collect()
|
|
|
|
[Row(name=u'Alice'), Row(name=u'Bob')]
|
|
|
|
|
|
|
|
>>> df.join(df2, df.name == df2.name, 'inner').drop(df.name).collect()
|
|
|
|
[Row(age=5, height=85, name=u'Bob')]
|
|
|
|
|
|
|
|
>>> df.join(df2, df.name == df2.name, 'inner').drop(df2.name).collect()
|
|
|
|
[Row(age=5, name=u'Bob', height=85)]
|
2016-07-15 01:55:49 -04:00
|
|
|
|
|
|
|
>>> df.join(df2, 'name', 'inner').drop('age', 'height').collect()
|
|
|
|
[Row(name=u'Bob')]
|
2015-05-11 23:04:36 -04:00
|
|
|
"""
|
2016-07-15 01:55:49 -04:00
|
|
|
if len(cols) == 1:
|
|
|
|
col = cols[0]
|
|
|
|
if isinstance(col, basestring):
|
|
|
|
jdf = self._jdf.drop(col)
|
|
|
|
elif isinstance(col, Column):
|
|
|
|
jdf = self._jdf.drop(col._jc)
|
|
|
|
else:
|
|
|
|
raise TypeError("col should be a string or a Column")
|
2015-06-04 14:30:07 -04:00
|
|
|
else:
|
2016-07-15 01:55:49 -04:00
|
|
|
for col in cols:
|
|
|
|
if not isinstance(col, basestring):
|
|
|
|
raise TypeError("each col in the param list should be a string")
|
|
|
|
jdf = self._jdf.drop(self._jseq(cols))
|
|
|
|
|
2015-05-11 23:04:36 -04:00
|
|
|
return DataFrame(jdf, self.sql_ctx)
|
|
|
|
|
2015-10-26 04:25:19 -04:00
|
|
|
@ignore_unicode_prefix
|
|
|
|
def toDF(self, *cols):
|
|
|
|
"""Returns a new class:`DataFrame` that with new specified column names
|
|
|
|
|
|
|
|
:param cols: list of new column names (string)
|
|
|
|
|
|
|
|
>>> df.toDF('f1', 'f2').collect()
|
|
|
|
[Row(f1=2, f2=u'Alice'), Row(f1=5, f2=u'Bob')]
|
|
|
|
"""
|
|
|
|
jdf = self._jdf.toDF(self._jseq(cols))
|
|
|
|
return DataFrame(jdf, self.sql_ctx)
|
|
|
|
|
2015-05-21 02:05:54 -04:00
|
|
|
@since(1.3)
|
2015-02-14 02:03:22 -05:00
|
|
|
def toPandas(self):
|
2017-07-10 18:21:03 -04:00
|
|
|
"""
|
|
|
|
Returns the contents of this :class:`DataFrame` as Pandas ``pandas.DataFrame``.
|
2015-03-31 21:31:36 -04:00
|
|
|
|
|
|
|
This is only available if Pandas is installed and available.
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2016-11-22 06:40:18 -05:00
|
|
|
.. note:: This method should only be used if the resulting Pandas's DataFrame is expected
|
|
|
|
to be small, as all the data is loaded into the driver's memory.
|
|
|
|
|
2018-05-28 00:56:05 -04:00
|
|
|
.. note:: Usage with spark.sql.execution.arrow.enabled=True is experimental.
|
|
|
|
|
2015-02-14 02:03:22 -05:00
|
|
|
>>> df.toPandas() # doctest: +SKIP
|
2015-02-09 23:49:22 -05:00
|
|
|
age name
|
|
|
|
0 2 Alice
|
|
|
|
1 5 Bob
|
|
|
|
"""
|
[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()
|
|
|
|
|
2015-02-09 23:49:22 -05:00
|
|
|
import pandas as pd
|
2017-11-28 03:45:22 -05:00
|
|
|
|
2018-07-02 02:35:37 -04:00
|
|
|
if self.sql_ctx._conf.pandasRespectSessionTimeZone():
|
|
|
|
timezone = self.sql_ctx._conf.sessionLocalTimeZone()
|
2017-11-28 03:45:22 -05:00
|
|
|
else:
|
|
|
|
timezone = None
|
|
|
|
|
2018-07-02 02:35:37 -04:00
|
|
|
if self.sql_ctx._conf.arrowEnabled():
|
[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
|
|
|
use_arrow = True
|
2017-07-10 18:21:03 -04:00
|
|
|
try:
|
[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.sql.types import to_arrow_schema
|
2017-12-22 06:09:51 -05:00
|
|
|
from pyspark.sql.utils import require_minimum_pyarrow_version
|
[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
|
|
|
|
2017-12-22 06:09:51 -05:00
|
|
|
require_minimum_pyarrow_version()
|
2018-02-16 12:41:17 -05:00
|
|
|
to_arrow_schema(self.schema)
|
|
|
|
except Exception as e:
|
2017-09-22 09:39:47 -04:00
|
|
|
|
2018-07-02 02:35:37 -04:00
|
|
|
if self.sql_ctx._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 = (
|
|
|
|
"toPandas 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)
|
|
|
|
use_arrow = False
|
|
|
|
else:
|
|
|
|
msg = (
|
|
|
|
"toPandas 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
|
[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
|
|
|
|
|
|
|
# Try to use Arrow optimization when the schema is supported and the required version
|
|
|
|
# of PyArrow is found, if 'spark.sql.execution.arrow.enabled' is enabled.
|
|
|
|
if use_arrow:
|
|
|
|
try:
|
|
|
|
from pyspark.sql.types import _check_dataframe_convert_date, \
|
|
|
|
_check_dataframe_localize_timestamps
|
|
|
|
import pyarrow
|
[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
|
|
|
batches = self._collectAsArrow()
|
|
|
|
if len(batches) > 0:
|
|
|
|
table = pyarrow.Table.from_batches(batches)
|
[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
|
|
|
pdf = table.to_pandas()
|
|
|
|
pdf = _check_dataframe_convert_date(pdf, self.schema)
|
|
|
|
return _check_dataframe_localize_timestamps(pdf, timezone)
|
|
|
|
else:
|
|
|
|
return pd.DataFrame.from_records([], columns=self.columns)
|
|
|
|
except Exception as e:
|
|
|
|
# We might have to allow fallback here as well but multiple Spark jobs can
|
|
|
|
# be executed. So, simply fail in this case for now.
|
|
|
|
msg = (
|
|
|
|
"toPandas 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 can not continue. Note that "
|
|
|
|
"'spark.sql.execution.arrow.fallback.enabled' does not have an effect "
|
|
|
|
"on failures in the middle of computation.\n %s" % _exception_message(e))
|
|
|
|
warnings.warn(msg)
|
|
|
|
raise
|
[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
|
|
|
|
|
|
|
# Below is toPandas without Arrow optimization.
|
|
|
|
pdf = pd.DataFrame.from_records(self.collect(), columns=self.columns)
|
|
|
|
|
|
|
|
dtype = {}
|
|
|
|
for field in self.schema:
|
|
|
|
pandas_type = _to_corrected_pandas_type(field.dataType)
|
|
|
|
# SPARK-21766: if an integer field is nullable and has null values, it can be
|
|
|
|
# inferred by pandas as float column. Once we convert the column with NaN back
|
|
|
|
# to integer type e.g., np.int16, we will hit exception. So we use the inferred
|
|
|
|
# float type, not the corrected type from the schema in this case.
|
|
|
|
if pandas_type is not None and \
|
|
|
|
not(isinstance(field.dataType, IntegralType) and field.nullable and
|
|
|
|
pdf[field.name].isnull().any()):
|
|
|
|
dtype[field.name] = pandas_type
|
|
|
|
|
|
|
|
for f, t in dtype.items():
|
|
|
|
pdf[f] = pdf[f].astype(t, copy=False)
|
|
|
|
|
|
|
|
if timezone is None:
|
|
|
|
return pdf
|
|
|
|
else:
|
|
|
|
from pyspark.sql.types import _check_series_convert_timestamps_local_tz
|
2017-07-10 18:21:03 -04:00
|
|
|
for field in self.schema:
|
[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
|
|
|
# TODO: handle nested timestamps, such as ArrayType(TimestampType())?
|
|
|
|
if isinstance(field.dataType, TimestampType):
|
|
|
|
pdf[field.name] = \
|
|
|
|
_check_series_convert_timestamps_local_tz(pdf[field.name], timezone)
|
|
|
|
return pdf
|
2017-06-22 04:22:02 -04:00
|
|
|
|
2017-07-10 18:21:03 -04:00
|
|
|
def _collectAsArrow(self):
|
|
|
|
"""
|
[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
|
|
|
Returns all records as a list of ArrowRecordBatches, pyarrow must be installed
|
|
|
|
and available on driver and worker Python environments.
|
2017-07-10 18:21:03 -04:00
|
|
|
|
|
|
|
.. note:: Experimental.
|
|
|
|
"""
|
|
|
|
with SCCallSiteSync(self._sc) as css:
|
2018-04-13 17:28:24 -04:00
|
|
|
sock_info = self._jdf.collectAsArrowToPython()
|
[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
|
|
|
return list(_load_from_socket(sock_info, ArrowStreamSerializer()))
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2015-06-07 04:21:02 -04:00
|
|
|
##########################################################################################
|
2015-05-11 22:15:14 -04:00
|
|
|
# Pandas compatibility
|
2015-06-07 04:21:02 -04:00
|
|
|
##########################################################################################
|
|
|
|
|
2016-03-14 22:25:49 -04:00
|
|
|
groupby = copy_func(
|
|
|
|
groupBy,
|
|
|
|
sinceversion=1.4,
|
|
|
|
doc=":func:`groupby` is an alias for :func:`groupBy`.")
|
|
|
|
|
|
|
|
drop_duplicates = copy_func(
|
|
|
|
dropDuplicates,
|
|
|
|
sinceversion=1.4,
|
|
|
|
doc=":func:`drop_duplicates` is an alias for :func:`dropDuplicates`.")
|
|
|
|
|
|
|
|
where = copy_func(
|
|
|
|
filter,
|
|
|
|
sinceversion=1.3,
|
|
|
|
doc=":func:`where` is an alias for :func:`filter`.")
|
2015-05-11 22:15:14 -04:00
|
|
|
|
2015-02-09 23:49:22 -05:00
|
|
|
|
2015-05-12 13:23:41 -04:00
|
|
|
def _to_scala_map(sc, jm):
|
|
|
|
"""
|
|
|
|
Convert a dict into a JVM Map.
|
|
|
|
"""
|
|
|
|
return sc._jvm.PythonUtils.toScalaMap(jm)
|
|
|
|
|
|
|
|
|
2017-06-22 04:22:02 -04:00
|
|
|
def _to_corrected_pandas_type(dt):
|
|
|
|
"""
|
|
|
|
When converting Spark SQL records to Pandas DataFrame, the inferred data type may be wrong.
|
|
|
|
This method gets the corrected data type for Pandas if that type may be inferred uncorrectly.
|
|
|
|
"""
|
|
|
|
import numpy as np
|
|
|
|
if type(dt) == ByteType:
|
|
|
|
return np.int8
|
|
|
|
elif type(dt) == ShortType:
|
|
|
|
return np.int16
|
|
|
|
elif type(dt) == IntegerType:
|
|
|
|
return np.int32
|
|
|
|
elif type(dt) == FloatType:
|
|
|
|
return np.float32
|
|
|
|
else:
|
|
|
|
return None
|
|
|
|
|
|
|
|
|
2015-03-31 03:25:23 -04:00
|
|
|
class DataFrameNaFunctions(object):
|
|
|
|
"""Functionality for working with missing data in :class:`DataFrame`.
|
2015-06-07 04:21:02 -04:00
|
|
|
|
|
|
|
.. versionadded:: 1.4
|
2015-03-31 03:25:23 -04:00
|
|
|
"""
|
|
|
|
|
|
|
|
def __init__(self, df):
|
|
|
|
self.df = df
|
|
|
|
|
|
|
|
def drop(self, how='any', thresh=None, subset=None):
|
|
|
|
return self.df.dropna(how=how, thresh=thresh, subset=subset)
|
|
|
|
|
|
|
|
drop.__doc__ = DataFrame.dropna.__doc__
|
|
|
|
|
|
|
|
def fill(self, value, subset=None):
|
|
|
|
return self.df.fillna(value=value, subset=subset)
|
|
|
|
|
|
|
|
fill.__doc__ = DataFrame.fillna.__doc__
|
|
|
|
|
2018-02-09 01:21:10 -05:00
|
|
|
def replace(self, to_replace, value=_NoValue, subset=None):
|
2015-06-07 04:21:02 -04:00
|
|
|
return self.df.replace(to_replace, value, subset)
|
|
|
|
|
|
|
|
replace.__doc__ = DataFrame.replace.__doc__
|
|
|
|
|
2015-03-31 03:25:23 -04:00
|
|
|
|
2015-05-01 16:29:17 -04:00
|
|
|
class DataFrameStatFunctions(object):
|
|
|
|
"""Functionality for statistic functions with :class:`DataFrame`.
|
2015-06-07 04:21:02 -04:00
|
|
|
|
|
|
|
.. versionadded:: 1.4
|
2015-05-01 16:29:17 -04:00
|
|
|
"""
|
|
|
|
|
|
|
|
def __init__(self, df):
|
|
|
|
self.df = df
|
|
|
|
|
2016-02-25 02:15:36 -05:00
|
|
|
def approxQuantile(self, col, probabilities, relativeError):
|
|
|
|
return self.df.approxQuantile(col, probabilities, relativeError)
|
|
|
|
|
|
|
|
approxQuantile.__doc__ = DataFrame.approxQuantile.__doc__
|
|
|
|
|
2015-05-04 00:44:39 -04:00
|
|
|
def corr(self, col1, col2, method=None):
|
|
|
|
return self.df.corr(col1, col2, method)
|
|
|
|
|
|
|
|
corr.__doc__ = DataFrame.corr.__doc__
|
|
|
|
|
2015-05-01 16:29:17 -04:00
|
|
|
def cov(self, col1, col2):
|
|
|
|
return self.df.cov(col1, col2)
|
|
|
|
|
|
|
|
cov.__doc__ = DataFrame.cov.__doc__
|
|
|
|
|
2015-05-04 20:02:49 -04:00
|
|
|
def crosstab(self, col1, col2):
|
|
|
|
return self.df.crosstab(col1, col2)
|
|
|
|
|
|
|
|
crosstab.__doc__ = DataFrame.crosstab.__doc__
|
|
|
|
|
2015-05-02 02:43:24 -04:00
|
|
|
def freqItems(self, cols, support=None):
|
|
|
|
return self.df.freqItems(cols, support)
|
|
|
|
|
|
|
|
freqItems.__doc__ = DataFrame.freqItems.__doc__
|
|
|
|
|
2015-07-30 20:16:03 -04:00
|
|
|
def sampleBy(self, col, fractions, seed=None):
|
|
|
|
return self.df.sampleBy(col, fractions, seed)
|
|
|
|
|
|
|
|
sampleBy.__doc__ = DataFrame.sampleBy.__doc__
|
|
|
|
|
2015-05-01 16:29:17 -04:00
|
|
|
|
2015-02-09 23:49:22 -05:00
|
|
|
def _test():
|
|
|
|
import doctest
|
|
|
|
from pyspark.context import SparkContext
|
2016-05-12 03:51:53 -04:00
|
|
|
from pyspark.sql import Row, SQLContext, SparkSession
|
2015-02-09 23:49:22 -05:00
|
|
|
import pyspark.sql.dataframe
|
2016-11-21 20:24:02 -05:00
|
|
|
from pyspark.sql.functions import from_unixtime
|
2015-02-09 23:49:22 -05:00
|
|
|
globs = pyspark.sql.dataframe.__dict__.copy()
|
|
|
|
sc = SparkContext('local[4]', 'PythonTest')
|
|
|
|
globs['sc'] = sc
|
2015-04-08 16:31:45 -04:00
|
|
|
globs['sqlContext'] = SQLContext(sc)
|
2016-05-12 03:51:53 -04:00
|
|
|
globs['spark'] = SparkSession(sc)
|
2015-02-18 17:17:04 -05:00
|
|
|
globs['df'] = sc.parallelize([(2, 'Alice'), (5, 'Bob')])\
|
|
|
|
.toDF(StructType([StructField('age', IntegerType()),
|
|
|
|
StructField('name', StringType())]))
|
2015-02-14 02:03:22 -05:00
|
|
|
globs['df2'] = sc.parallelize([Row(name='Tom', height=80), Row(name='Bob', height=85)]).toDF()
|
2015-06-09 02:27:05 -04:00
|
|
|
globs['df3'] = sc.parallelize([Row(name='Alice', age=2),
|
|
|
|
Row(name='Bob', age=5)]).toDF()
|
2015-03-30 23:47:10 -04:00
|
|
|
globs['df4'] = sc.parallelize([Row(name='Alice', age=10, height=80),
|
2016-11-21 20:24:02 -05:00
|
|
|
Row(name='Bob', age=5, height=None),
|
|
|
|
Row(name='Tom', age=None, height=None),
|
|
|
|
Row(name=None, age=None, height=None)]).toDF()
|
2017-06-03 01:56:42 -04:00
|
|
|
globs['df5'] = sc.parallelize([Row(name='Alice', spy=False, age=10),
|
|
|
|
Row(name='Bob', spy=None, age=5),
|
|
|
|
Row(name='Mallory', spy=True, age=None)]).toDF()
|
2016-11-21 20:24:02 -05:00
|
|
|
globs['sdf'] = sc.parallelize([Row(name='Tom', time=1479441846),
|
|
|
|
Row(name='Bob', time=1479442946)]).toDF()
|
2015-03-30 23:47:10 -04:00
|
|
|
|
2015-02-09 23:49:22 -05:00
|
|
|
(failure_count, test_count) = doctest.testmod(
|
2015-02-11 15:13:16 -05:00
|
|
|
pyspark.sql.dataframe, globs=globs,
|
2015-02-17 16:48:38 -05:00
|
|
|
optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE | doctest.REPORT_NDIFF)
|
2015-02-09 23:49:22 -05:00
|
|
|
globs['sc'].stop()
|
|
|
|
if failure_count:
|
2018-03-08 06:38:34 -05:00
|
|
|
sys.exit(-1)
|
2015-02-09 23:49:22 -05:00
|
|
|
|
|
|
|
|
|
|
|
if __name__ == "__main__":
|
|
|
|
_test()
|