2015-02-14 02:03:22 -05:00
|
|
|
#
|
|
|
|
# Licensed to the Apache Software Foundation (ASF) under one or more
|
|
|
|
# contributor license agreements. See the NOTICE file distributed with
|
|
|
|
# this work for additional information regarding copyright ownership.
|
|
|
|
# The ASF licenses this file to You under the Apache License, Version 2.0
|
|
|
|
# (the "License"); you may not use this file except in compliance with
|
|
|
|
# the License. You may obtain a copy of the License at
|
|
|
|
#
|
|
|
|
# http://www.apache.org/licenses/LICENSE-2.0
|
|
|
|
#
|
|
|
|
# Unless required by applicable law or agreed to in writing, software
|
|
|
|
# distributed under the License is distributed on an "AS IS" BASIS,
|
|
|
|
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
|
|
# See the License for the specific language governing permissions and
|
|
|
|
# limitations under the License.
|
|
|
|
#
|
|
|
|
|
|
|
|
"""
|
|
|
|
A collections of builtin functions
|
|
|
|
"""
|
2015-04-16 19:20:57 -04:00
|
|
|
import sys
|
2017-02-15 13:16:34 -05:00
|
|
|
import functools
|
[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
|
|
|
import warnings
|
2015-02-14 02:03:22 -05:00
|
|
|
|
2015-09-08 23:56:22 -04:00
|
|
|
from pyspark import since, SparkContext
|
2020-07-13 22:22:44 -04:00
|
|
|
from pyspark.rdd import PythonEvalType
|
2020-10-26 22:05:53 -04:00
|
|
|
from pyspark.sql.column import Column, _to_java_column, _to_seq, _create_column_from_literal
|
2015-09-22 02:36:41 -04:00
|
|
|
from pyspark.sql.dataframe import DataFrame
|
2017-11-17 10:43:08 -05:00
|
|
|
from pyspark.sql.types import StringType, DataType
|
2018-03-25 23:42:32 -04:00
|
|
|
# Keep UserDefinedFunction import for backwards compatible import; moved in SPARK-22409
|
2020-08-08 11:51:57 -04:00
|
|
|
from pyspark.sql.udf import UserDefinedFunction, _create_udf # noqa: F401
|
|
|
|
from pyspark.sql.udf import _create_udf
|
[SPARK-30434][PYTHON][SQL] Move pandas related functionalities into 'pandas' sub-package
### What changes were proposed in this pull request?
This PR proposes to move pandas related functionalities into pandas package. Namely:
```bash
pyspark/sql/pandas
├── __init__.py
├── conversion.py # Conversion between pandas <> PySpark DataFrames
├── functions.py # pandas_udf
├── group_ops.py # Grouped UDF / Cogrouped UDF + groupby.apply, groupby.cogroup.apply
├── map_ops.py # Map Iter UDF + mapInPandas
├── serializers.py # pandas <> PyArrow serializers
├── types.py # Type utils between pandas <> PyArrow
└── utils.py # Version requirement checks
```
In order to separately locate `groupby.apply`, `groupby.cogroup.apply`, `mapInPandas`, `toPandas`, and `createDataFrame(pdf)` under `pandas` sub-package, I had to use a mix-in approach which Scala side uses often by `trait`, and also pandas itself uses this approach (see `IndexOpsMixin` as an example) to group related functionalities. Currently, you can think it's like Scala's self typed trait. See the structure below:
```python
class PandasMapOpsMixin(object):
def mapInPandas(self, ...):
...
return ...
# other Pandas <> PySpark APIs
```
```python
class DataFrame(PandasMapOpsMixin):
# other DataFrame APIs equivalent to Scala side.
```
Yes, This is a big PR but they are mostly just moving around except one case `createDataFrame` which I had to split the methods.
### Why are the changes needed?
There are pandas functionalities here and there and I myself gets lost where it was. Also, when you have to make a change commonly for all of pandas related features, it's almost impossible now.
Also, after this change, `DataFrame` and `SparkSession` become more consistent with Scala side since pandas is specific to Python, and this change separates pandas-specific APIs away from `DataFrame` or `SparkSession`.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Existing tests should cover. Also, I manually built the PySpark API documentation and checked.
Closes #27109 from HyukjinKwon/pandas-refactoring.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-08 20:22:50 -05:00
|
|
|
# Keep pandas_udf and PandasUDFType import for backwards compatible import; moved in SPARK-28264
|
2020-08-08 11:51:57 -04:00
|
|
|
from pyspark.sql.pandas.functions import pandas_udf, PandasUDFType # noqa: F401
|
2019-07-18 00:37:03 -04:00
|
|
|
from pyspark.sql.utils import to_str
|
2015-02-14 02:03:22 -05:00
|
|
|
|
[SPARK-26979][PYTHON][FOLLOW-UP] Make binary math/string functions take string as columns as well
## What changes were proposed in this pull request?
This is a followup of https://github.com/apache/spark/pull/23882 to handle binary math/string functions. For instance, see the cases below:
**Before:**
```python
>>> from pyspark.sql.functions import lit, ascii
>>> spark.range(1).select(lit('a').alias("value")).select(ascii("value"))
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../spark/python/pyspark/sql/functions.py", line 51, in _
jc = getattr(sc._jvm.functions, name)(col._jc if isinstance(col, Column) else col)
File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1286, in __call__
File "/.../spark/python/pyspark/sql/utils.py", line 63, in deco
return f(*a, **kw)
File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/protocol.py", line 332, in get_return_value
py4j.protocol.Py4JError: An error occurred while calling z:org.apache.spark.sql.functions.ascii. Trace:
py4j.Py4JException: Method ascii([class java.lang.String]) does not exist
at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:318)
at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:339)
at py4j.Gateway.invoke(Gateway.java:276)
at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
at py4j.commands.CallCommand.execute(CallCommand.java:79)
at py4j.GatewayConnection.run(GatewayConnection.java:238)
at java.lang.Thread.run(Thread.java:748)
```
```python
>>> from pyspark.sql.functions import atan2
>>> spark.range(1).select(atan2("id", "id"))
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../spark/python/pyspark/sql/functions.py", line 78, in _
jc = getattr(sc._jvm.functions, name)(col1._jc if isinstance(col1, Column) else float(col1),
ValueError: could not convert string to float: id
```
**After:**
```python
>>> from pyspark.sql.functions import lit, ascii
>>> spark.range(1).select(lit('a').alias("value")).select(ascii("value"))
DataFrame[ascii(value): int]
```
```python
>>> from pyspark.sql.functions import atan2
>>> spark.range(1).select(atan2("id", "id"))
DataFrame[ATAN2(id, id): double]
```
Note that,
- This PR causes a slight behaviour changes for math functions. For instance, numbers as strings (e.g., `"1"`) were supported as arguments of binary math functions before. After this PR, it recognises it as column names.
- I also intentionally didn't document this behaviour changes since we're going ahead for Spark 3.0 and I don't think numbers as strings make much sense in math functions.
- There is another exception `when`, which takes string as literal values as below. This PR doeesn't fix this ambiguity.
```python
>>> spark.range(1).select(when(lit(True), col("id"))).show()
```
```
+--------------------------+
|CASE WHEN true THEN id END|
+--------------------------+
| 0|
+--------------------------+
```
```python
>>> spark.range(1).select(when(lit(True), "id")).show()
```
```
+--------------------------+
|CASE WHEN true THEN id END|
+--------------------------+
| id|
+--------------------------+
```
This PR also fixes as below:
https://github.com/apache/spark/pull/23882 fixed it to:
- Rename `_create_function` to `_create_name_function`
- Define new `_create_function` to take strings as column names.
This PR, I proposes to:
- Revert `_create_name_function` name to `_create_function`.
- Define new `_create_function_over_column` to take strings as column names.
## How was this patch tested?
Some unit tests were added for binary math / string functions.
Closes #24121 from HyukjinKwon/SPARK-26979.
Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-19 19:06:10 -04:00
|
|
|
# Note to developers: all of PySpark functions here take string as column names whenever possible.
|
|
|
|
# Namely, if columns are referred as arguments, they can be always both Column or string,
|
|
|
|
# even though there might be few exceptions for legacy or inevitable reasons.
|
|
|
|
# If you are fixing other language APIs together, also please note that Scala side is not the case
|
|
|
|
# since it requires to make every single overridden definition.
|
2015-02-14 02:03:22 -05:00
|
|
|
|
[SPARK-26979][PYTHON][FOLLOW-UP] Make binary math/string functions take string as columns as well
## What changes were proposed in this pull request?
This is a followup of https://github.com/apache/spark/pull/23882 to handle binary math/string functions. For instance, see the cases below:
**Before:**
```python
>>> from pyspark.sql.functions import lit, ascii
>>> spark.range(1).select(lit('a').alias("value")).select(ascii("value"))
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../spark/python/pyspark/sql/functions.py", line 51, in _
jc = getattr(sc._jvm.functions, name)(col._jc if isinstance(col, Column) else col)
File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1286, in __call__
File "/.../spark/python/pyspark/sql/utils.py", line 63, in deco
return f(*a, **kw)
File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/protocol.py", line 332, in get_return_value
py4j.protocol.Py4JError: An error occurred while calling z:org.apache.spark.sql.functions.ascii. Trace:
py4j.Py4JException: Method ascii([class java.lang.String]) does not exist
at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:318)
at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:339)
at py4j.Gateway.invoke(Gateway.java:276)
at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
at py4j.commands.CallCommand.execute(CallCommand.java:79)
at py4j.GatewayConnection.run(GatewayConnection.java:238)
at java.lang.Thread.run(Thread.java:748)
```
```python
>>> from pyspark.sql.functions import atan2
>>> spark.range(1).select(atan2("id", "id"))
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../spark/python/pyspark/sql/functions.py", line 78, in _
jc = getattr(sc._jvm.functions, name)(col1._jc if isinstance(col1, Column) else float(col1),
ValueError: could not convert string to float: id
```
**After:**
```python
>>> from pyspark.sql.functions import lit, ascii
>>> spark.range(1).select(lit('a').alias("value")).select(ascii("value"))
DataFrame[ascii(value): int]
```
```python
>>> from pyspark.sql.functions import atan2
>>> spark.range(1).select(atan2("id", "id"))
DataFrame[ATAN2(id, id): double]
```
Note that,
- This PR causes a slight behaviour changes for math functions. For instance, numbers as strings (e.g., `"1"`) were supported as arguments of binary math functions before. After this PR, it recognises it as column names.
- I also intentionally didn't document this behaviour changes since we're going ahead for Spark 3.0 and I don't think numbers as strings make much sense in math functions.
- There is another exception `when`, which takes string as literal values as below. This PR doeesn't fix this ambiguity.
```python
>>> spark.range(1).select(when(lit(True), col("id"))).show()
```
```
+--------------------------+
|CASE WHEN true THEN id END|
+--------------------------+
| 0|
+--------------------------+
```
```python
>>> spark.range(1).select(when(lit(True), "id")).show()
```
```
+--------------------------+
|CASE WHEN true THEN id END|
+--------------------------+
| id|
+--------------------------+
```
This PR also fixes as below:
https://github.com/apache/spark/pull/23882 fixed it to:
- Rename `_create_function` to `_create_name_function`
- Define new `_create_function` to take strings as column names.
This PR, I proposes to:
- Revert `_create_name_function` name to `_create_function`.
- Define new `_create_function_over_column` to take strings as column names.
## How was this patch tested?
Some unit tests were added for binary math / string functions.
Closes #24121 from HyukjinKwon/SPARK-26979.
Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-19 19:06:10 -04:00
|
|
|
|
2020-10-26 22:05:53 -04:00
|
|
|
def _get_get_jvm_function(name, sc):
|
|
|
|
"""
|
|
|
|
Retrieves JVM function identified by name from
|
|
|
|
Java gateway associated with sc.
|
|
|
|
"""
|
|
|
|
return getattr(sc._jvm.functions, name)
|
2015-02-14 02:03:22 -05:00
|
|
|
|
|
|
|
|
2020-10-26 22:05:53 -04:00
|
|
|
def _invoke_function(name, *args):
|
|
|
|
"""
|
|
|
|
Invokes JVM function identified by name with args
|
2021-02-01 19:30:50 -05:00
|
|
|
and wraps the result with :class:`~pyspark.sql.Column`.
|
[SPARK-26979][PYTHON][FOLLOW-UP] Make binary math/string functions take string as columns as well
## What changes were proposed in this pull request?
This is a followup of https://github.com/apache/spark/pull/23882 to handle binary math/string functions. For instance, see the cases below:
**Before:**
```python
>>> from pyspark.sql.functions import lit, ascii
>>> spark.range(1).select(lit('a').alias("value")).select(ascii("value"))
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../spark/python/pyspark/sql/functions.py", line 51, in _
jc = getattr(sc._jvm.functions, name)(col._jc if isinstance(col, Column) else col)
File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1286, in __call__
File "/.../spark/python/pyspark/sql/utils.py", line 63, in deco
return f(*a, **kw)
File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/protocol.py", line 332, in get_return_value
py4j.protocol.Py4JError: An error occurred while calling z:org.apache.spark.sql.functions.ascii. Trace:
py4j.Py4JException: Method ascii([class java.lang.String]) does not exist
at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:318)
at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:339)
at py4j.Gateway.invoke(Gateway.java:276)
at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
at py4j.commands.CallCommand.execute(CallCommand.java:79)
at py4j.GatewayConnection.run(GatewayConnection.java:238)
at java.lang.Thread.run(Thread.java:748)
```
```python
>>> from pyspark.sql.functions import atan2
>>> spark.range(1).select(atan2("id", "id"))
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../spark/python/pyspark/sql/functions.py", line 78, in _
jc = getattr(sc._jvm.functions, name)(col1._jc if isinstance(col1, Column) else float(col1),
ValueError: could not convert string to float: id
```
**After:**
```python
>>> from pyspark.sql.functions import lit, ascii
>>> spark.range(1).select(lit('a').alias("value")).select(ascii("value"))
DataFrame[ascii(value): int]
```
```python
>>> from pyspark.sql.functions import atan2
>>> spark.range(1).select(atan2("id", "id"))
DataFrame[ATAN2(id, id): double]
```
Note that,
- This PR causes a slight behaviour changes for math functions. For instance, numbers as strings (e.g., `"1"`) were supported as arguments of binary math functions before. After this PR, it recognises it as column names.
- I also intentionally didn't document this behaviour changes since we're going ahead for Spark 3.0 and I don't think numbers as strings make much sense in math functions.
- There is another exception `when`, which takes string as literal values as below. This PR doeesn't fix this ambiguity.
```python
>>> spark.range(1).select(when(lit(True), col("id"))).show()
```
```
+--------------------------+
|CASE WHEN true THEN id END|
+--------------------------+
| 0|
+--------------------------+
```
```python
>>> spark.range(1).select(when(lit(True), "id")).show()
```
```
+--------------------------+
|CASE WHEN true THEN id END|
+--------------------------+
| id|
+--------------------------+
```
This PR also fixes as below:
https://github.com/apache/spark/pull/23882 fixed it to:
- Rename `_create_function` to `_create_name_function`
- Define new `_create_function` to take strings as column names.
This PR, I proposes to:
- Revert `_create_name_function` name to `_create_function`.
- Define new `_create_function_over_column` to take strings as column names.
## How was this patch tested?
Some unit tests were added for binary math / string functions.
Closes #24121 from HyukjinKwon/SPARK-26979.
Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-19 19:06:10 -04:00
|
|
|
"""
|
2020-10-26 22:05:53 -04:00
|
|
|
jf = _get_get_jvm_function(name, SparkContext._active_spark_context)
|
|
|
|
return Column(jf(*args))
|
[SPARK-26979][PYTHON] Add missing string column name support for some SQL functions
## What changes were proposed in this pull request?
Most SQL functions defined in `spark.sql.functions` have two calling patterns, one with a Column object as input, and another with a string representing a column name, which is then converted into a Column object internally.
There are, however, a few notable exceptions:
- lower()
- upper()
- abs()
- bitwiseNOT()
- ltrim()
- rtrim()
- trim()
- ascii()
- base64()
- unbase64()
While this doesn't break anything, as you can easily create a Column object yourself prior to passing it to one of these functions, it has two undesirable consequences:
1. It is surprising - it breaks coder's expectations when they are first starting with Spark. Every API should be as consistent as possible, so as to make the learning curve smoother and to reduce causes for human error;
2. It gets in the way of stylistic conventions. Most of the time it makes Python code more readable to use literal names, and the API provides ample support for that, but these few exceptions prevent this pattern from being universally applicable.
This patch is meant to fix the aforementioned problem.
### Effect
This patch **enables** support for passing column names as input to those functions mentioned above.
### Side effects
This PR also **fixes** an issue with some functions being defined multiple times by using `_create_function()`.
### How it works
`_create_function()` was redefined to always convert the argument to a Column object. The old implementation has been kept under `_create_name_function()`, and is still being used to generate the following special functions:
- lit()
- col()
- column()
- asc()
- desc()
- asc_nulls_first()
- asc_nulls_last()
- desc_nulls_first()
- desc_nulls_last()
This is because these functions can only take a column name as their argument. This is not a problem, as their semantics require so.
## How was this patch tested?
Ran ./dev/run-tests and tested it manually.
Closes #23882 from asmello/col-name-support-pyspark.
Authored-by: André Sá de Mello <amello@palantir.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-17 13:58:16 -04:00
|
|
|
|
|
|
|
|
2020-10-26 22:05:53 -04:00
|
|
|
def _invoke_function_over_column(name, col):
|
|
|
|
"""
|
|
|
|
Invokes unary JVM function identified by name
|
2021-02-01 19:30:50 -05:00
|
|
|
and wraps the result with :class:`~pyspark.sql.Column`.
|
2020-10-26 22:05:53 -04:00
|
|
|
"""
|
|
|
|
return _invoke_function(name, _to_java_column(col))
|
[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
|
|
|
|
|
|
|
|
2020-10-26 22:05:53 -04:00
|
|
|
def _invoke_binary_math_function(name, col1, col2):
|
|
|
|
"""
|
|
|
|
Invokes binary JVM math function identified by name
|
2021-02-01 19:30:50 -05:00
|
|
|
and wraps the result with :class:`~pyspark.sql.Column`.
|
2020-10-26 22:05:53 -04:00
|
|
|
"""
|
|
|
|
return _invoke_function(
|
|
|
|
name,
|
[SPARK-26979][PYTHON][FOLLOW-UP] Make binary math/string functions take string as columns as well
## What changes were proposed in this pull request?
This is a followup of https://github.com/apache/spark/pull/23882 to handle binary math/string functions. For instance, see the cases below:
**Before:**
```python
>>> from pyspark.sql.functions import lit, ascii
>>> spark.range(1).select(lit('a').alias("value")).select(ascii("value"))
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../spark/python/pyspark/sql/functions.py", line 51, in _
jc = getattr(sc._jvm.functions, name)(col._jc if isinstance(col, Column) else col)
File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1286, in __call__
File "/.../spark/python/pyspark/sql/utils.py", line 63, in deco
return f(*a, **kw)
File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/protocol.py", line 332, in get_return_value
py4j.protocol.Py4JError: An error occurred while calling z:org.apache.spark.sql.functions.ascii. Trace:
py4j.Py4JException: Method ascii([class java.lang.String]) does not exist
at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:318)
at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:339)
at py4j.Gateway.invoke(Gateway.java:276)
at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
at py4j.commands.CallCommand.execute(CallCommand.java:79)
at py4j.GatewayConnection.run(GatewayConnection.java:238)
at java.lang.Thread.run(Thread.java:748)
```
```python
>>> from pyspark.sql.functions import atan2
>>> spark.range(1).select(atan2("id", "id"))
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../spark/python/pyspark/sql/functions.py", line 78, in _
jc = getattr(sc._jvm.functions, name)(col1._jc if isinstance(col1, Column) else float(col1),
ValueError: could not convert string to float: id
```
**After:**
```python
>>> from pyspark.sql.functions import lit, ascii
>>> spark.range(1).select(lit('a').alias("value")).select(ascii("value"))
DataFrame[ascii(value): int]
```
```python
>>> from pyspark.sql.functions import atan2
>>> spark.range(1).select(atan2("id", "id"))
DataFrame[ATAN2(id, id): double]
```
Note that,
- This PR causes a slight behaviour changes for math functions. For instance, numbers as strings (e.g., `"1"`) were supported as arguments of binary math functions before. After this PR, it recognises it as column names.
- I also intentionally didn't document this behaviour changes since we're going ahead for Spark 3.0 and I don't think numbers as strings make much sense in math functions.
- There is another exception `when`, which takes string as literal values as below. This PR doeesn't fix this ambiguity.
```python
>>> spark.range(1).select(when(lit(True), col("id"))).show()
```
```
+--------------------------+
|CASE WHEN true THEN id END|
+--------------------------+
| 0|
+--------------------------+
```
```python
>>> spark.range(1).select(when(lit(True), "id")).show()
```
```
+--------------------------+
|CASE WHEN true THEN id END|
+--------------------------+
| id|
+--------------------------+
```
This PR also fixes as below:
https://github.com/apache/spark/pull/23882 fixed it to:
- Rename `_create_function` to `_create_name_function`
- Define new `_create_function` to take strings as column names.
This PR, I proposes to:
- Revert `_create_name_function` name to `_create_function`.
- Define new `_create_function_over_column` to take strings as column names.
## How was this patch tested?
Some unit tests were added for binary math / string functions.
Closes #24121 from HyukjinKwon/SPARK-26979.
Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-19 19:06:10 -04:00
|
|
|
# For legacy reasons, the arguments here can be implicitly converted into floats,
|
|
|
|
# if they are not columns or strings.
|
2020-10-26 22:05:53 -04:00
|
|
|
_to_java_column(col1) if isinstance(col1, (str, Column)) else float(col1),
|
|
|
|
_to_java_column(col2) if isinstance(col2, (str, Column)) else float(col2)
|
|
|
|
)
|
2015-05-23 11:30:05 -04:00
|
|
|
|
2019-07-18 00:37:03 -04:00
|
|
|
|
[SPARK-32320][PYSPARK] Remove mutable default arguments
This is bad practice, and might lead to unexpected behaviour:
https://florimond.dev/blog/articles/2018/08/python-mutable-defaults-are-the-source-of-all-evil/
```
fokkodriesprongFan spark % grep -R "={}" python | grep def
python/pyspark/resource/profile.py: def __init__(self, _java_resource_profile=None, _exec_req={}, _task_req={}):
python/pyspark/sql/functions.py:def from_json(col, schema, options={}):
python/pyspark/sql/functions.py:def to_json(col, options={}):
python/pyspark/sql/functions.py:def schema_of_json(json, options={}):
python/pyspark/sql/functions.py:def schema_of_csv(csv, options={}):
python/pyspark/sql/functions.py:def to_csv(col, options={}):
python/pyspark/sql/functions.py:def from_csv(col, schema, options={}):
python/pyspark/sql/avro/functions.py:def from_avro(data, jsonFormatSchema, options={}):
```
```
fokkodriesprongFan spark % grep -R "=\[\]" python | grep def
python/pyspark/ml/tuning.py: def __init__(self, bestModel, avgMetrics=[], subModels=None):
python/pyspark/ml/tuning.py: def __init__(self, bestModel, validationMetrics=[], subModels=None):
```
### What changes were proposed in this pull request?
Removing the mutable default arguments.
### Why are the changes needed?
Removing the mutable default arguments, and changing the signature to `Optional[...]`.
### Does this PR introduce _any_ user-facing change?
No 👍
### How was this patch tested?
Using the Flake8 bugbear code analysis plugin.
Closes #29122 from Fokko/SPARK-32320.
Authored-by: Fokko Driesprong <fokko@apache.org>
Signed-off-by: Ruifeng Zheng <ruifengz@foxmail.com>
2020-12-07 20:35:36 -05:00
|
|
|
def _options_to_str(options=None):
|
|
|
|
if options:
|
|
|
|
return {key: to_str(value) for (key, value) in options.items()}
|
|
|
|
return {}
|
2019-07-18 00:37:03 -04:00
|
|
|
|
2020-10-26 22:05:53 -04:00
|
|
|
|
|
|
|
def lit(col):
|
|
|
|
"""
|
2021-02-01 19:30:50 -05:00
|
|
|
Creates a :class:`~pyspark.sql.Column` of literal value.
|
2015-05-23 11:30:05 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.3.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2017-07-08 02:59:34 -04:00
|
|
|
>>> df.select(lit(5).alias('height')).withColumn('spark_user', lit(True)).take(1)
|
|
|
|
[Row(height=5, spark_user=True)]
|
|
|
|
"""
|
2020-10-26 22:05:53 -04:00
|
|
|
return col if isinstance(col, Column) else _invoke_function("lit", col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.3)
|
|
|
|
def col(col):
|
|
|
|
"""
|
2021-02-01 19:30:50 -05:00
|
|
|
Returns a :class:`~pyspark.sql.Column` based on the given column name.'
|
2020-10-26 22:05:53 -04:00
|
|
|
"""
|
|
|
|
return _invoke_function("col", col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.3)
|
|
|
|
def column(col):
|
|
|
|
"""
|
2021-02-01 19:30:50 -05:00
|
|
|
Returns a :class:`~pyspark.sql.Column` based on the given column name.'
|
2020-10-26 22:05:53 -04:00
|
|
|
"""
|
|
|
|
return col(col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.3)
|
|
|
|
def asc(col):
|
|
|
|
"""
|
|
|
|
Returns a sort expression based on the ascending order of the given column name.
|
|
|
|
"""
|
2020-11-03 08:50:59 -05:00
|
|
|
return (
|
|
|
|
col.asc() if isinstance(col, Column)
|
|
|
|
else _invoke_function("asc", col)
|
|
|
|
)
|
2020-10-26 22:05:53 -04:00
|
|
|
|
|
|
|
|
|
|
|
@since(1.3)
|
|
|
|
def desc(col):
|
|
|
|
"""
|
|
|
|
Returns a sort expression based on the descending order of the given column name.
|
|
|
|
"""
|
2020-11-03 08:50:59 -05:00
|
|
|
return (
|
|
|
|
col.desc() if isinstance(col, Column)
|
|
|
|
else _invoke_function("desc", col)
|
|
|
|
)
|
2020-10-26 22:05:53 -04:00
|
|
|
|
|
|
|
|
|
|
|
@since(1.3)
|
|
|
|
def sqrt(col):
|
|
|
|
"""
|
|
|
|
Computes the square root of the specified float value.
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("sqrt", col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.3)
|
|
|
|
def abs(col):
|
|
|
|
"""
|
|
|
|
Computes the absolute value.
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("abs", col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.3)
|
|
|
|
def max(col):
|
|
|
|
"""
|
|
|
|
Aggregate function: returns the maximum value of the expression in a group.
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("max", col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.3)
|
|
|
|
def min(col):
|
|
|
|
"""
|
|
|
|
Aggregate function: returns the minimum value of the expression in a group.
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("min", col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.3)
|
|
|
|
def count(col):
|
|
|
|
"""
|
|
|
|
Aggregate function: returns the number of items in a group.
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("count", col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.3)
|
|
|
|
def sum(col):
|
|
|
|
"""
|
|
|
|
Aggregate function: returns the sum of all values in the expression.
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("sum", col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.3)
|
|
|
|
def avg(col):
|
|
|
|
"""
|
|
|
|
Aggregate function: returns the average of the values in a group.
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("avg", col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.3)
|
|
|
|
def mean(col):
|
|
|
|
"""
|
|
|
|
Aggregate function: returns the average of the values in a group.
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("mean", col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.3)
|
|
|
|
def sumDistinct(col):
|
|
|
|
"""
|
|
|
|
Aggregate function: returns the sum of distinct values in the expression.
|
2021-02-01 19:29:40 -05:00
|
|
|
|
|
|
|
.. deprecated:: 3.2.0
|
|
|
|
Use :func:`sum_distinct` instead.
|
2020-10-26 22:05:53 -04:00
|
|
|
"""
|
2021-02-01 19:29:40 -05:00
|
|
|
warnings.warn("Deprecated in 3.2, use sum_distinct instead.", FutureWarning)
|
|
|
|
return sum_distinct(col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(3.2)
|
|
|
|
def sum_distinct(col):
|
|
|
|
"""
|
|
|
|
Aggregate function: returns the sum of distinct values in the expression.
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("sum_distinct", col)
|
2020-10-26 22:05:53 -04:00
|
|
|
|
|
|
|
|
[SPARK-33678][SQL] Product aggregation function
### Why is this change being proposed?
This patch adds support for a new "product" aggregation function in `sql.functions` which multiplies-together all values in an aggregation group.
This is likely to be useful in statistical applications which involve combining probabilities, or financial applications that involve combining cumulative interest rates, but is also a versatile mathematical operation of similar status to `sum` or `stddev`. Other users [have noted](https://stackoverflow.com/questions/52991640/cumulative-product-in-spark) the absence of such a function in current releases of Spark.
This function is both much more concise than an expression of the form `exp(sum(log(...)))`, and avoids awkward edge-cases associated with some values being zero or negative, as well as being less computationally costly.
### Does this PR introduce _any_ user-facing change?
No - only adds new function.
### How was this patch tested?
Built-in tests have been added for the new `catalyst.expressions.aggregate.Product` class and its invocation via the (scala) `sql.functions.product` function. The latter, and the PySpark wrapper have also been manually tested in spark-shell and pyspark sessions. The SparkR wrapper is currently untested, and may need separate validation (I'm not an "R" user myself).
An illustration of the new functionality, within PySpark is as follows:
```
import pyspark.sql.functions as pf, pyspark.sql.window as pw
df = sqlContext.range(1, 17).toDF("x")
win = pw.Window.partitionBy(pf.lit(1)).orderBy(pf.col("x"))
df.withColumn("factorial", pf.product("x").over(win)).show(20, False)
+---+---------------+
|x |factorial |
+---+---------------+
|1 |1.0 |
|2 |2.0 |
|3 |6.0 |
|4 |24.0 |
|5 |120.0 |
|6 |720.0 |
|7 |5040.0 |
|8 |40320.0 |
|9 |362880.0 |
|10 |3628800.0 |
|11 |3.99168E7 |
|12 |4.790016E8 |
|13 |6.2270208E9 |
|14 |8.71782912E10 |
|15 |1.307674368E12 |
|16 |2.0922789888E13|
+---+---------------+
```
Closes #30745 from rwpenney/feature/agg-product.
Lead-authored-by: Richard Penney <rwp@rwpenney.uk>
Co-authored-by: Richard Penney <rwpenney@users.noreply.github.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-02 02:51:07 -05:00
|
|
|
def product(col):
|
|
|
|
"""
|
|
|
|
Aggregate function: returns the product of the values in a group.
|
|
|
|
|
|
|
|
.. versionadded:: 3.2.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
col : str, :class:`Column`
|
|
|
|
column containing values to be multiplied together
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
|
|
|
>>> df = spark.range(1, 10).toDF('x').withColumn('mod3', col('x') % 3)
|
|
|
|
>>> prods = df.groupBy('mod3').agg(product('x').alias('product'))
|
|
|
|
>>> prods.orderBy('mod3').show()
|
|
|
|
+----+-------+
|
|
|
|
|mod3|product|
|
|
|
|
+----+-------+
|
|
|
|
| 0| 162.0|
|
|
|
|
| 1| 28.0|
|
|
|
|
| 2| 80.0|
|
|
|
|
+----+-------+
|
|
|
|
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("product", col)
|
|
|
|
|
|
|
|
|
2020-10-26 22:05:53 -04:00
|
|
|
def acos(col):
|
|
|
|
"""
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
|
|
|
|
|
|
|
Returns
|
|
|
|
-------
|
2021-02-01 19:30:50 -05:00
|
|
|
:class:`~pyspark.sql.Column`
|
2020-11-02 20:00:49 -05:00
|
|
|
inverse cosine of `col`, as if computed by `java.lang.Math.acos()`
|
2020-10-26 22:05:53 -04:00
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("acos", col)
|
|
|
|
|
|
|
|
|
2020-11-26 21:00:09 -05:00
|
|
|
def acosh(col):
|
|
|
|
"""
|
|
|
|
Computes inverse hyperbolic cosine of the input column.
|
|
|
|
|
|
|
|
.. versionadded:: 3.1.0
|
|
|
|
|
|
|
|
Returns
|
|
|
|
-------
|
2021-02-01 19:30:50 -05:00
|
|
|
:class:`~pyspark.sql.Column`
|
2020-11-26 21:00:09 -05:00
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("acosh", col)
|
|
|
|
|
|
|
|
|
2020-10-26 22:05:53 -04:00
|
|
|
def asin(col):
|
|
|
|
"""
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.3.0
|
|
|
|
|
|
|
|
|
|
|
|
Returns
|
|
|
|
-------
|
2021-02-01 19:30:50 -05:00
|
|
|
:class:`~pyspark.sql.Column`
|
2020-11-02 20:00:49 -05:00
|
|
|
inverse sine of `col`, as if computed by `java.lang.Math.asin()`
|
2020-10-26 22:05:53 -04:00
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("asin", col)
|
|
|
|
|
|
|
|
|
2020-11-26 21:00:09 -05:00
|
|
|
def asinh(col):
|
|
|
|
"""
|
|
|
|
Computes inverse hyperbolic sine of the input column.
|
|
|
|
|
|
|
|
.. versionadded:: 3.1.0
|
|
|
|
|
|
|
|
Returns
|
|
|
|
-------
|
2021-02-01 19:30:50 -05:00
|
|
|
:class:`~pyspark.sql.Column`
|
2020-11-26 21:00:09 -05:00
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("asinh", col)
|
|
|
|
|
|
|
|
|
2020-10-26 22:05:53 -04:00
|
|
|
def atan(col):
|
|
|
|
"""
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
|
|
|
|
|
|
|
Returns
|
|
|
|
-------
|
2021-02-01 19:30:50 -05:00
|
|
|
:class:`~pyspark.sql.Column`
|
2020-11-02 20:00:49 -05:00
|
|
|
inverse tangent of `col`, as if computed by `java.lang.Math.atan()`
|
2020-10-26 22:05:53 -04:00
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("atan", col)
|
|
|
|
|
|
|
|
|
2020-11-26 21:00:09 -05:00
|
|
|
def atanh(col):
|
|
|
|
"""
|
|
|
|
Computes inverse hyperbolic tangent of the input column.
|
|
|
|
|
|
|
|
.. versionadded:: 3.1.0
|
|
|
|
|
|
|
|
Returns
|
|
|
|
-------
|
2021-02-01 19:30:50 -05:00
|
|
|
:class:`~pyspark.sql.Column`
|
2020-11-26 21:00:09 -05:00
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("atanh", col)
|
|
|
|
|
|
|
|
|
2020-10-26 22:05:53 -04:00
|
|
|
@since(1.4)
|
|
|
|
def cbrt(col):
|
|
|
|
"""
|
|
|
|
Computes the cube-root of the given value.
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("cbrt", col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.4)
|
|
|
|
def ceil(col):
|
|
|
|
"""
|
|
|
|
Computes the ceiling of the given value.
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("ceil", col)
|
|
|
|
|
|
|
|
|
|
|
|
def cos(col):
|
|
|
|
"""
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
angle in radians
|
|
|
|
|
|
|
|
Returns
|
|
|
|
-------
|
2021-02-01 19:30:50 -05:00
|
|
|
:class:`~pyspark.sql.Column`
|
2020-11-02 20:00:49 -05:00
|
|
|
cosine of the angle, as if computed by `java.lang.Math.cos()`.
|
2020-10-26 22:05:53 -04:00
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("cos", col)
|
|
|
|
|
|
|
|
|
|
|
|
def cosh(col):
|
|
|
|
"""
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
hyperbolic angle
|
|
|
|
|
|
|
|
Returns
|
|
|
|
-------
|
2021-02-01 19:30:50 -05:00
|
|
|
:class:`~pyspark.sql.Column`
|
2020-11-02 20:00:49 -05:00
|
|
|
hyperbolic cosine of the angle, as if computed by `java.lang.Math.cosh()`
|
2020-10-26 22:05:53 -04:00
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("cosh", col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.4)
|
|
|
|
def exp(col):
|
|
|
|
"""
|
|
|
|
Computes the exponential of the given value.
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("exp", col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.4)
|
|
|
|
def expm1(col):
|
|
|
|
"""
|
|
|
|
Computes the exponential of the given value minus one.
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("expm1", col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.4)
|
|
|
|
def floor(col):
|
|
|
|
"""
|
|
|
|
Computes the floor of the given value.
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("floor", col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.4)
|
|
|
|
def log(col):
|
|
|
|
"""
|
|
|
|
Computes the natural logarithm of the given value.
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("log", col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.4)
|
|
|
|
def log10(col):
|
|
|
|
"""
|
|
|
|
Computes the logarithm of the given value in Base 10.
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("log10", col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.4)
|
|
|
|
def log1p(col):
|
|
|
|
"""
|
|
|
|
Computes the natural logarithm of the given value plus one.
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("log1p", col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.4)
|
|
|
|
def rint(col):
|
|
|
|
"""
|
|
|
|
Returns the double value that is closest in value to the argument and
|
|
|
|
is equal to a mathematical integer.
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("rint", col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.4)
|
|
|
|
def signum(col):
|
|
|
|
"""
|
|
|
|
Computes the signum of the given value.
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("signum", col)
|
|
|
|
|
|
|
|
|
|
|
|
def sin(col):
|
|
|
|
"""
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
|
|
|
|
Returns
|
|
|
|
-------
|
2021-02-01 19:30:50 -05:00
|
|
|
:class:`~pyspark.sql.Column`
|
2020-11-02 20:00:49 -05:00
|
|
|
sine of the angle, as if computed by `java.lang.Math.sin()`
|
2020-10-26 22:05:53 -04:00
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("sin", col)
|
|
|
|
|
|
|
|
|
|
|
|
def sinh(col):
|
|
|
|
"""
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
hyperbolic angle
|
|
|
|
|
|
|
|
Returns
|
|
|
|
-------
|
2021-02-01 19:30:50 -05:00
|
|
|
:class:`~pyspark.sql.Column`
|
2020-11-02 20:00:49 -05:00
|
|
|
hyperbolic sine of the given value,
|
|
|
|
as if computed by `java.lang.Math.sinh()`
|
2020-10-26 22:05:53 -04:00
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("sinh", col)
|
|
|
|
|
|
|
|
|
|
|
|
def tan(col):
|
|
|
|
"""
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
angle in radians
|
|
|
|
|
|
|
|
Returns
|
|
|
|
-------
|
2021-02-01 19:30:50 -05:00
|
|
|
:class:`~pyspark.sql.Column`
|
2020-11-02 20:00:49 -05:00
|
|
|
tangent of the given value, as if computed by `java.lang.Math.tan()`
|
2020-10-26 22:05:53 -04:00
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("tan", col)
|
|
|
|
|
|
|
|
|
|
|
|
def tanh(col):
|
|
|
|
"""
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
hyperbolic angle
|
|
|
|
|
|
|
|
Returns
|
|
|
|
-------
|
2021-02-01 19:30:50 -05:00
|
|
|
:class:`~pyspark.sql.Column`
|
2020-11-02 20:00:49 -05:00
|
|
|
hyperbolic tangent of the given value
|
|
|
|
as if computed by `java.lang.Math.tanh()`
|
2020-10-26 22:05:53 -04:00
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("tanh", col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.4)
|
|
|
|
def toDegrees(col):
|
|
|
|
"""
|
2020-11-02 20:00:49 -05:00
|
|
|
.. deprecated:: 2.1.0
|
|
|
|
Use :func:`degrees` instead.
|
2020-10-26 22:05:53 -04:00
|
|
|
"""
|
2021-01-17 19:32:55 -05:00
|
|
|
warnings.warn("Deprecated in 2.1, use degrees instead.", FutureWarning)
|
2020-10-26 22:05:53 -04:00
|
|
|
return degrees(col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.4)
|
|
|
|
def toRadians(col):
|
|
|
|
"""
|
2020-11-02 20:00:49 -05:00
|
|
|
.. deprecated:: 2.1.0
|
|
|
|
Use :func:`radians` instead.
|
2020-10-26 22:05:53 -04:00
|
|
|
"""
|
2021-01-17 19:32:55 -05:00
|
|
|
warnings.warn("Deprecated in 2.1, use radians instead.", FutureWarning)
|
2020-10-26 22:05:53 -04:00
|
|
|
return radians(col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.4)
|
|
|
|
def bitwiseNOT(col):
|
|
|
|
"""
|
|
|
|
Computes bitwise not.
|
2021-02-01 19:29:40 -05:00
|
|
|
|
|
|
|
.. deprecated:: 3.2.0
|
|
|
|
Use :func:`bitwise_not` instead.
|
|
|
|
"""
|
|
|
|
warnings.warn("Deprecated in 3.2, use bitwise_not instead.", FutureWarning)
|
|
|
|
return bitwise_not(col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(3.2)
|
|
|
|
def bitwise_not(col):
|
|
|
|
"""
|
|
|
|
Computes bitwise not.
|
2020-10-26 22:05:53 -04:00
|
|
|
"""
|
2021-02-01 19:29:40 -05:00
|
|
|
return _invoke_function_over_column("bitwise_not", col)
|
2020-10-26 22:05:53 -04:00
|
|
|
|
|
|
|
|
|
|
|
@since(2.4)
|
|
|
|
def asc_nulls_first(col):
|
|
|
|
"""
|
|
|
|
Returns a sort expression based on the ascending order of the given
|
|
|
|
column name, and null values return before non-null values.
|
|
|
|
"""
|
2020-11-03 08:50:59 -05:00
|
|
|
return (
|
|
|
|
col.asc_nulls_first() if isinstance(col, Column)
|
|
|
|
else _invoke_function("asc_nulls_first", col)
|
|
|
|
)
|
2020-10-26 22:05:53 -04:00
|
|
|
|
|
|
|
|
|
|
|
@since(2.4)
|
|
|
|
def asc_nulls_last(col):
|
|
|
|
"""
|
|
|
|
Returns a sort expression based on the ascending order of the given
|
|
|
|
column name, and null values appear after non-null values.
|
|
|
|
"""
|
2020-11-03 08:50:59 -05:00
|
|
|
return (
|
|
|
|
col.asc_nulls_last() if isinstance(col, Column)
|
|
|
|
else _invoke_function("asc_nulls_last", col)
|
|
|
|
)
|
2020-10-26 22:05:53 -04:00
|
|
|
|
|
|
|
|
|
|
|
@since(2.4)
|
|
|
|
def desc_nulls_first(col):
|
|
|
|
"""
|
|
|
|
Returns a sort expression based on the descending order of the given
|
|
|
|
column name, and null values appear before non-null values.
|
|
|
|
"""
|
2020-11-03 08:50:59 -05:00
|
|
|
return (
|
|
|
|
col.desc_nulls_first() if isinstance(col, Column)
|
|
|
|
else _invoke_function("desc_nulls_first", col)
|
|
|
|
)
|
2020-10-26 22:05:53 -04:00
|
|
|
|
|
|
|
|
|
|
|
@since(2.4)
|
|
|
|
def desc_nulls_last(col):
|
|
|
|
"""
|
|
|
|
Returns a sort expression based on the descending order of the given
|
|
|
|
column name, and null values appear after non-null values.
|
|
|
|
"""
|
2020-11-03 08:50:59 -05:00
|
|
|
return (
|
|
|
|
col.desc_nulls_last() if isinstance(col, Column)
|
|
|
|
else _invoke_function("desc_nulls_last", col)
|
|
|
|
)
|
2020-10-26 22:05:53 -04:00
|
|
|
|
|
|
|
|
|
|
|
@since(1.6)
|
|
|
|
def stddev(col):
|
|
|
|
"""
|
|
|
|
Aggregate function: alias for stddev_samp.
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("stddev", col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.6)
|
|
|
|
def stddev_samp(col):
|
|
|
|
"""
|
|
|
|
Aggregate function: returns the unbiased sample standard deviation of
|
|
|
|
the expression in a group.
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("stddev_samp", col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.6)
|
|
|
|
def stddev_pop(col):
|
|
|
|
"""
|
|
|
|
Aggregate function: returns population standard deviation of
|
|
|
|
the expression in a group.
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("stddev_pop", col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.6)
|
|
|
|
def variance(col):
|
|
|
|
"""
|
|
|
|
Aggregate function: alias for var_samp
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("variance", col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.6)
|
|
|
|
def var_samp(col):
|
|
|
|
"""
|
|
|
|
Aggregate function: returns the unbiased sample variance of
|
|
|
|
the values in a group.
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("var_samp", col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.6)
|
|
|
|
def var_pop(col):
|
|
|
|
"""
|
|
|
|
Aggregate function: returns the population variance of the values in a group.
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("var_pop", col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.6)
|
|
|
|
def skewness(col):
|
|
|
|
"""
|
|
|
|
Aggregate function: returns the skewness of the values in a group.
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("skewness", col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.6)
|
|
|
|
def kurtosis(col):
|
|
|
|
"""
|
|
|
|
Aggregate function: returns the kurtosis of the values in a group.
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("kurtosis", col)
|
|
|
|
|
|
|
|
|
|
|
|
def collect_list(col):
|
|
|
|
"""
|
2017-07-08 02:59:34 -04:00
|
|
|
Aggregate function: returns a list of objects with duplicates.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.6.0
|
2018-05-10 12:44:49 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
The function is non-deterministic because the order of collected results depends
|
|
|
|
on the order of the rows which may be non-deterministic after a shuffle.
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2017-07-08 02:59:34 -04:00
|
|
|
>>> df2 = spark.createDataFrame([(2,), (5,), (5,)], ('age',))
|
|
|
|
>>> df2.agg(collect_list('age')).collect()
|
|
|
|
[Row(collect_list(age)=[2, 5, 5])]
|
|
|
|
"""
|
2020-10-26 22:05:53 -04:00
|
|
|
return _invoke_function_over_column("collect_list", col)
|
|
|
|
|
|
|
|
|
|
|
|
def collect_set(col):
|
|
|
|
"""
|
2017-07-08 02:59:34 -04:00
|
|
|
Aggregate function: returns a set of objects with duplicate elements eliminated.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.6.0
|
|
|
|
|
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
The function is non-deterministic because the order of collected results depends
|
|
|
|
on the order of the rows which may be non-deterministic after a shuffle.
|
2018-05-10 12:44:49 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2017-07-08 02:59:34 -04:00
|
|
|
>>> df2 = spark.createDataFrame([(2,), (5,), (5,)], ('age',))
|
|
|
|
>>> df2.agg(collect_set('age')).collect()
|
|
|
|
[Row(collect_set(age)=[5, 2])]
|
|
|
|
"""
|
2020-10-26 22:05:53 -04:00
|
|
|
return _invoke_function_over_column("collect_set", col)
|
|
|
|
|
|
|
|
|
|
|
|
def degrees(col):
|
|
|
|
"""
|
|
|
|
Converts an angle measured in radians to an approximately equivalent angle
|
|
|
|
measured in degrees.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.1.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
angle in radians
|
|
|
|
|
|
|
|
Returns
|
|
|
|
-------
|
2021-02-01 19:30:50 -05:00
|
|
|
:class:`~pyspark.sql.Column`
|
2020-11-02 20:00:49 -05:00
|
|
|
angle in degrees, as if computed by `java.lang.Math.toDegrees()`
|
2020-10-26 22:05:53 -04:00
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("degrees", col)
|
|
|
|
|
|
|
|
|
|
|
|
def radians(col):
|
|
|
|
"""
|
|
|
|
Converts an angle measured in degrees to an approximately equivalent angle
|
|
|
|
measured in radians.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.1.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
angle in degrees
|
|
|
|
|
|
|
|
Returns
|
|
|
|
-------
|
2021-02-01 19:30:50 -05:00
|
|
|
:class:`~pyspark.sql.Column`
|
2020-11-02 20:00:49 -05:00
|
|
|
angle in radians, as if computed by `java.lang.Math.toRadians()`
|
2020-10-26 22:05:53 -04:00
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("radians", col)
|
|
|
|
|
|
|
|
|
|
|
|
def atan2(col1, col2):
|
|
|
|
"""
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col1 : str, :class:`~pyspark.sql.Column` or float
|
2020-11-02 20:00:49 -05:00
|
|
|
coordinate on y-axis
|
2021-02-01 19:30:50 -05:00
|
|
|
col2 : str, :class:`~pyspark.sql.Column` or float
|
2020-11-02 20:00:49 -05:00
|
|
|
coordinate on x-axis
|
|
|
|
|
|
|
|
Returns
|
|
|
|
-------
|
2021-02-01 19:30:50 -05:00
|
|
|
:class:`~pyspark.sql.Column`
|
2020-11-02 20:00:49 -05:00
|
|
|
the `theta` component of the point
|
|
|
|
(`r`, `theta`)
|
|
|
|
in polar coordinates that corresponds to the point
|
|
|
|
(`x`, `y`) in Cartesian coordinates,
|
|
|
|
as if computed by `java.lang.Math.atan2()`
|
2020-10-26 22:05:53 -04:00
|
|
|
"""
|
|
|
|
return _invoke_binary_math_function("atan2", col1, col2)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.4)
|
|
|
|
def hypot(col1, col2):
|
|
|
|
"""
|
|
|
|
Computes ``sqrt(a^2 + b^2)`` without intermediate overflow or underflow.
|
|
|
|
"""
|
|
|
|
return _invoke_binary_math_function("hypot", col1, col2)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.4)
|
|
|
|
def pow(col1, col2):
|
|
|
|
"""
|
|
|
|
Returns the value of the first argument raised to the power of the second argument.
|
|
|
|
"""
|
|
|
|
return _invoke_binary_math_function("pow", col1, col2)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.6)
|
|
|
|
def row_number():
|
|
|
|
"""
|
|
|
|
Window function: returns a sequential number starting at 1 within a window partition.
|
|
|
|
"""
|
|
|
|
return _invoke_function("row_number")
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.6)
|
|
|
|
def dense_rank():
|
|
|
|
"""
|
|
|
|
Window function: returns the rank of rows within a window partition, without any gaps.
|
|
|
|
|
|
|
|
The difference between rank and dense_rank is that dense_rank leaves no gaps in ranking
|
|
|
|
sequence when there are ties. That is, if you were ranking a competition using dense_rank
|
|
|
|
and had three people tie for second place, you would say that all three were in second
|
|
|
|
place and that the next person came in third. Rank would give me sequential numbers, making
|
|
|
|
the person that came in third place (after the ties) would register as coming in fifth.
|
|
|
|
|
|
|
|
This is equivalent to the DENSE_RANK function in SQL.
|
|
|
|
"""
|
|
|
|
return _invoke_function("dense_rank")
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.6)
|
|
|
|
def rank():
|
|
|
|
"""
|
|
|
|
Window function: returns the rank of rows within a window partition.
|
|
|
|
|
|
|
|
The difference between rank and dense_rank is that dense_rank leaves no gaps in ranking
|
|
|
|
sequence when there are ties. That is, if you were ranking a competition using dense_rank
|
|
|
|
and had three people tie for second place, you would say that all three were in second
|
|
|
|
place and that the next person came in third. Rank would give me sequential numbers, making
|
|
|
|
the person that came in third place (after the ties) would register as coming in fifth.
|
|
|
|
|
|
|
|
This is equivalent to the RANK function in SQL.
|
|
|
|
"""
|
|
|
|
return _invoke_function("rank")
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.6)
|
|
|
|
def cume_dist():
|
|
|
|
"""
|
|
|
|
Window function: returns the cumulative distribution of values within a window partition,
|
|
|
|
i.e. the fraction of rows that are below the current row.
|
|
|
|
"""
|
|
|
|
return _invoke_function("cume_dist")
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.6)
|
|
|
|
def percent_rank():
|
|
|
|
"""
|
|
|
|
Window function: returns the relative rank (i.e. percentile) of rows within a window partition.
|
|
|
|
"""
|
|
|
|
return _invoke_function("percent_rank")
|
2015-05-01 00:56:03 -04:00
|
|
|
|
|
|
|
|
2020-03-29 01:05:16 -04:00
|
|
|
@since(1.3)
|
|
|
|
def approxCountDistinct(col, rsd=None):
|
|
|
|
"""
|
2020-11-02 20:00:49 -05:00
|
|
|
.. deprecated:: 2.1.0
|
|
|
|
Use :func:`approx_count_distinct` instead.
|
2020-03-29 01:05:16 -04:00
|
|
|
"""
|
2021-01-17 19:32:55 -05:00
|
|
|
warnings.warn("Deprecated in 2.1, use approx_count_distinct instead.", FutureWarning)
|
2020-03-29 01:05:16 -04:00
|
|
|
return approx_count_distinct(col, rsd)
|
|
|
|
|
|
|
|
|
2016-10-07 06:49:34 -04:00
|
|
|
def approx_count_distinct(col, rsd=None):
|
2021-02-01 19:30:50 -05:00
|
|
|
"""Aggregate function: returns a new :class:`~pyspark.sql.Column` for approximate distinct count
|
|
|
|
of column `col`.
|
2015-04-26 14:46:58 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.1.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
rsd : float, optional
|
|
|
|
maximum relative standard deviation allowed (default = 0.05).
|
2021-02-01 19:29:40 -05:00
|
|
|
For rsd < 0.01, it is more efficient to use :func:`count_distinct`
|
2017-07-08 02:59:34 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2017-07-08 02:59:34 -04:00
|
|
|
>>> df.agg(approx_count_distinct(df.age).alias('distinct_ages')).collect()
|
|
|
|
[Row(distinct_ages=2)]
|
2015-04-26 14:46:58 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
if rsd is None:
|
2016-10-07 06:49:34 -04:00
|
|
|
jc = sc._jvm.functions.approx_count_distinct(_to_java_column(col))
|
2015-04-26 14:46:58 -04:00
|
|
|
else:
|
2016-10-07 06:49:34 -04:00
|
|
|
jc = sc._jvm.functions.approx_count_distinct(_to_java_column(col), rsd)
|
2015-04-26 14:46:58 -04:00
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
2015-09-22 02:36:41 -04:00
|
|
|
@since(1.6)
|
|
|
|
def broadcast(df):
|
|
|
|
"""Marks a DataFrame as small enough for use in broadcast joins."""
|
|
|
|
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return DataFrame(sc._jvm.functions.broadcast(df._jdf), df.sql_ctx)
|
|
|
|
|
|
|
|
|
2015-05-07 13:58:35 -04:00
|
|
|
def coalesce(*cols):
|
|
|
|
"""Returns the first column that is not null.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> cDf = spark.createDataFrame([(None, None), (1, None), (None, 2)], ("a", "b"))
|
2015-05-07 13:58:35 -04:00
|
|
|
>>> cDf.show()
|
|
|
|
+----+----+
|
|
|
|
| a| b|
|
|
|
|
+----+----+
|
|
|
|
|null|null|
|
|
|
|
| 1|null|
|
|
|
|
|null| 2|
|
|
|
|
+----+----+
|
|
|
|
|
|
|
|
>>> cDf.select(coalesce(cDf["a"], cDf["b"])).show()
|
2016-02-21 09:53:15 -05:00
|
|
|
+--------------+
|
|
|
|
|coalesce(a, b)|
|
|
|
|
+--------------+
|
|
|
|
| null|
|
|
|
|
| 1|
|
|
|
|
| 2|
|
|
|
|
+--------------+
|
2015-05-07 13:58:35 -04:00
|
|
|
|
|
|
|
>>> cDf.select('*', coalesce(cDf["a"], lit(0.0))).show()
|
2016-02-21 09:53:15 -05:00
|
|
|
+----+----+----------------+
|
|
|
|
| a| b|coalesce(a, 0.0)|
|
|
|
|
+----+----+----------------+
|
|
|
|
|null|null| 0.0|
|
|
|
|
| 1|null| 1.0|
|
|
|
|
|null| 2| 0.0|
|
|
|
|
+----+----+----------------+
|
2015-05-07 13:58:35 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
jc = sc._jvm.functions.coalesce(_to_seq(sc, cols, _to_java_column))
|
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
2015-11-10 18:47:10 -05:00
|
|
|
def corr(col1, col2):
|
2021-02-01 19:30:50 -05:00
|
|
|
"""Returns a new :class:`~pyspark.sql.Column` for the Pearson Correlation Coefficient for
|
|
|
|
``col1`` and ``col2``.
|
2015-11-10 18:47:10 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.6.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-02-12 15:43:13 -05:00
|
|
|
>>> a = range(20)
|
|
|
|
>>> b = [2 * x for x in range(20)]
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> df = spark.createDataFrame(zip(a, b), ["a", "b"])
|
2016-02-12 15:43:13 -05:00
|
|
|
>>> df.agg(corr("a", "b").alias('c')).collect()
|
|
|
|
[Row(c=1.0)]
|
2015-11-10 18:47:10 -05:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.corr(_to_java_column(col1), _to_java_column(col2)))
|
|
|
|
|
|
|
|
|
2016-02-12 15:43:13 -05:00
|
|
|
def covar_pop(col1, col2):
|
2021-02-01 19:30:50 -05:00
|
|
|
"""Returns a new :class:`~pyspark.sql.Column` for the population covariance of ``col1`` and
|
|
|
|
``col2``.
|
2016-02-12 15:43:13 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-02-12 15:43:13 -05:00
|
|
|
>>> a = [1] * 10
|
|
|
|
>>> b = [1] * 10
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> df = spark.createDataFrame(zip(a, b), ["a", "b"])
|
2016-02-12 15:43:13 -05:00
|
|
|
>>> df.agg(covar_pop("a", "b").alias('c')).collect()
|
|
|
|
[Row(c=0.0)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.covar_pop(_to_java_column(col1), _to_java_column(col2)))
|
|
|
|
|
|
|
|
|
|
|
|
def covar_samp(col1, col2):
|
2021-02-01 19:30:50 -05:00
|
|
|
"""Returns a new :class:`~pyspark.sql.Column` for the sample covariance of ``col1`` and
|
|
|
|
``col2``.
|
2016-02-12 15:43:13 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-02-12 15:43:13 -05:00
|
|
|
>>> a = [1] * 10
|
|
|
|
>>> b = [1] * 10
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> df = spark.createDataFrame(zip(a, b), ["a", "b"])
|
2016-02-12 15:43:13 -05:00
|
|
|
>>> df.agg(covar_samp("a", "b").alias('c')).collect()
|
|
|
|
[Row(c=0.0)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.covar_samp(_to_java_column(col1), _to_java_column(col2)))
|
|
|
|
|
|
|
|
|
2015-02-14 02:03:22 -05:00
|
|
|
def countDistinct(col, *cols):
|
2021-02-01 19:30:50 -05:00
|
|
|
"""Returns a new :class:`~pyspark.sql.Column` for distinct count of ``col`` or ``cols``.
|
2015-02-14 02:03:22 -05:00
|
|
|
|
2021-02-01 19:29:40 -05:00
|
|
|
An alias of :func:`count_distinct`, and it is encouraged to use :func:`count_distinct`
|
|
|
|
directly.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.3.0
|
2021-02-01 19:29:40 -05:00
|
|
|
"""
|
|
|
|
return count_distinct(col, *cols)
|
|
|
|
|
|
|
|
|
|
|
|
def count_distinct(col, *cols):
|
|
|
|
"""Returns a new :class:`Column` for distinct count of ``col`` or ``cols``.
|
|
|
|
|
|
|
|
.. versionadded:: 3.2.0
|
2020-11-02 20:00:49 -05:00
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2021-02-01 19:29:40 -05:00
|
|
|
>>> df.agg(count_distinct(df.age, df.name).alias('c')).collect()
|
2015-02-14 02:03:22 -05:00
|
|
|
[Row(c=2)]
|
|
|
|
|
2021-02-01 19:29:40 -05:00
|
|
|
>>> df.agg(count_distinct("age", "name").alias('c')).collect()
|
2015-02-14 02:03:22 -05:00
|
|
|
[Row(c=2)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
2021-02-01 19:29:40 -05:00
|
|
|
jc = sc._jvm.functions.count_distinct(_to_java_column(col), _to_seq(sc, cols, _to_java_column))
|
2015-02-14 02:03:22 -05:00
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
2016-01-31 16:56:13 -05:00
|
|
|
def first(col, ignorenulls=False):
|
|
|
|
"""Aggregate function: returns the first value in a group.
|
|
|
|
|
|
|
|
The function by default returns the first values it sees. It will return the first non-null
|
|
|
|
value it sees when ignoreNulls is set to true. If all values are null, then null is returned.
|
2018-05-10 12:44:49 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.3.0
|
|
|
|
|
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
The function is non-deterministic because its results depends on the order of the
|
|
|
|
rows which may be non-deterministic after a shuffle.
|
2016-01-31 16:56:13 -05:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
jc = sc._jvm.functions.first(_to_java_column(col), ignorenulls)
|
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
2016-02-10 23:13:38 -05:00
|
|
|
def grouping(col):
|
|
|
|
"""
|
|
|
|
Aggregate function: indicates whether a specified column in a GROUP BY list is aggregated
|
|
|
|
or not, returns 1 for aggregated or 0 for not aggregated in the result set.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-02-10 23:13:38 -05:00
|
|
|
>>> df.cube("name").agg(grouping("name"), sum("age")).orderBy("name").show()
|
|
|
|
+-----+--------------+--------+
|
|
|
|
| name|grouping(name)|sum(age)|
|
|
|
|
+-----+--------------+--------+
|
|
|
|
| null| 1| 7|
|
|
|
|
|Alice| 0| 2|
|
|
|
|
| Bob| 0| 5|
|
|
|
|
+-----+--------------+--------+
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
jc = sc._jvm.functions.grouping(_to_java_column(col))
|
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
|
|
|
def grouping_id(*cols):
|
|
|
|
"""
|
|
|
|
Aggregate function: returns the level of grouping, equals to
|
|
|
|
|
|
|
|
(grouping(c1) << (n-1)) + (grouping(c2) << (n-2)) + ... + grouping(cn)
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
2016-02-10 23:13:38 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
The list of columns should match with grouping columns exactly, or empty (means all
|
|
|
|
the grouping columns).
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-02-10 23:13:38 -05:00
|
|
|
>>> df.cube("name").agg(grouping_id(), sum("age")).orderBy("name").show()
|
[SPARK-12720][SQL] SQL Generation Support for Cube, Rollup, and Grouping Sets
#### What changes were proposed in this pull request?
This PR is for supporting SQL generation for cube, rollup and grouping sets.
For example, a query using rollup:
```SQL
SELECT count(*) as cnt, key % 5, grouping_id() FROM t1 GROUP BY key % 5 WITH ROLLUP
```
Original logical plan:
```
Aggregate [(key#17L % cast(5 as bigint))#47L,grouping__id#46],
[(count(1),mode=Complete,isDistinct=false) AS cnt#43L,
(key#17L % cast(5 as bigint))#47L AS _c1#45L,
grouping__id#46 AS _c2#44]
+- Expand [List(key#17L, value#18, (key#17L % cast(5 as bigint))#47L, 0),
List(key#17L, value#18, null, 1)],
[key#17L,value#18,(key#17L % cast(5 as bigint))#47L,grouping__id#46]
+- Project [key#17L,
value#18,
(key#17L % cast(5 as bigint)) AS (key#17L % cast(5 as bigint))#47L]
+- Subquery t1
+- Relation[key#17L,value#18] ParquetRelation
```
Converted SQL:
```SQL
SELECT count( 1) AS `cnt`,
(`t1`.`key` % CAST(5 AS BIGINT)),
grouping_id() AS `_c2`
FROM `default`.`t1`
GROUP BY (`t1`.`key` % CAST(5 AS BIGINT))
GROUPING SETS (((`t1`.`key` % CAST(5 AS BIGINT))), ())
```
#### How was the this patch tested?
Added eight test cases in `LogicalPlanToSQLSuite`.
Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>
Closes #11283 from gatorsmile/groupingSetsToSQL.
2016-03-05 06:25:03 -05:00
|
|
|
+-----+-------------+--------+
|
|
|
|
| name|grouping_id()|sum(age)|
|
|
|
|
+-----+-------------+--------+
|
|
|
|
| null| 1| 7|
|
|
|
|
|Alice| 0| 2|
|
|
|
|
| Bob| 0| 5|
|
|
|
|
+-----+-------------+--------+
|
2016-02-10 23:13:38 -05:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
jc = sc._jvm.functions.grouping_id(_to_seq(sc, cols, _to_java_column))
|
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
2015-11-25 00:30:53 -05:00
|
|
|
@since(1.6)
|
|
|
|
def input_file_name():
|
|
|
|
"""Creates a string column for the file name of the current Spark task.
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.input_file_name())
|
|
|
|
|
|
|
|
|
|
|
|
def isnan(col):
|
|
|
|
"""An expression that returns true iff the column is NaN.
|
2015-11-26 02:24:33 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.6.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> df = spark.createDataFrame([(1.0, float('nan')), (float('nan'), 2.0)], ("a", "b"))
|
2015-11-26 02:24:33 -05:00
|
|
|
>>> df.select(isnan("a").alias("r1"), isnan(df.a).alias("r2")).collect()
|
|
|
|
[Row(r1=False, r2=False), Row(r1=True, r2=True)]
|
2015-11-25 00:30:53 -05:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.isnan(_to_java_column(col)))
|
|
|
|
|
|
|
|
|
|
|
|
def isnull(col):
|
|
|
|
"""An expression that returns true iff the column is null.
|
2015-11-26 02:24:33 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.6.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> df = spark.createDataFrame([(1, None), (None, 2)], ("a", "b"))
|
2015-11-26 02:24:33 -05:00
|
|
|
>>> df.select(isnull("a").alias("r1"), isnull(df.a).alias("r2")).collect()
|
|
|
|
[Row(r1=False, r2=False), Row(r1=True, r2=True)]
|
2015-11-25 00:30:53 -05:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.isnull(_to_java_column(col)))
|
|
|
|
|
|
|
|
|
2016-01-31 16:56:13 -05:00
|
|
|
def last(col, ignorenulls=False):
|
|
|
|
"""Aggregate function: returns the last value in a group.
|
|
|
|
|
|
|
|
The function by default returns the last values it sees. It will return the last non-null
|
|
|
|
value it sees when ignoreNulls is set to true. If all values are null, then null is returned.
|
2018-05-10 12:44:49 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.3.0
|
|
|
|
|
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
The function is non-deterministic because its results depends on the order of the
|
|
|
|
rows which may be non-deterministic after a shuffle.
|
2016-01-31 16:56:13 -05:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
jc = sc._jvm.functions.last(_to_java_column(col), ignorenulls)
|
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
2015-11-25 00:30:53 -05:00
|
|
|
def monotonically_increasing_id():
|
2015-04-28 03:39:08 -04:00
|
|
|
"""A column that generates monotonically increasing 64-bit integers.
|
|
|
|
|
|
|
|
The generated ID is guaranteed to be monotonically increasing and unique, but not consecutive.
|
|
|
|
The current implementation puts the partition ID in the upper 31 bits, and the record number
|
|
|
|
within each partition in the lower 33 bits. The assumption is that the data frame has
|
|
|
|
less than 1 billion partitions, and each partition has less than 8 billion records.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.6.0
|
|
|
|
|
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
The function is non-deterministic because its result depends on partition IDs.
|
2018-05-10 12:44:49 -04:00
|
|
|
|
2015-05-23 11:30:05 -04:00
|
|
|
As an example, consider a :class:`DataFrame` with two partitions, each with 3 records.
|
2015-04-28 03:39:08 -04:00
|
|
|
This expression would return the following IDs:
|
|
|
|
0, 1, 2, 8589934592 (1L << 33), 8589934593, 8589934594.
|
|
|
|
|
|
|
|
>>> df0 = sc.parallelize(range(2), 2).mapPartitions(lambda x: [(1,), (2,), (3,)]).toDF(['col1'])
|
2015-11-25 00:30:53 -05:00
|
|
|
>>> df0.select(monotonically_increasing_id().alias('id')).collect()
|
2015-04-28 03:39:08 -04:00
|
|
|
[Row(id=0), Row(id=1), Row(id=2), Row(id=8589934592), Row(id=8589934593), Row(id=8589934594)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
2015-11-25 00:30:53 -05:00
|
|
|
return Column(sc._jvm.functions.monotonically_increasing_id())
|
|
|
|
|
|
|
|
|
|
|
|
def nanvl(col1, col2):
|
|
|
|
"""Returns col1 if it is not NaN, or col2 if col1 is NaN.
|
|
|
|
|
2017-07-08 02:59:34 -04:00
|
|
|
Both inputs should be floating point columns (:class:`DoubleType` or :class:`FloatType`).
|
2015-11-26 02:24:33 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.6.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> df = spark.createDataFrame([(1.0, float('nan')), (float('nan'), 2.0)], ("a", "b"))
|
2015-11-26 02:24:33 -05:00
|
|
|
>>> df.select(nanvl("a", "b").alias("r1"), nanvl(df.a, df.b).alias("r2")).collect()
|
|
|
|
[Row(r1=1.0, r2=1.0), Row(r1=2.0, r2=2.0)]
|
2015-11-25 00:30:53 -05:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.nanvl(_to_java_column(col1), _to_java_column(col2)))
|
2015-04-28 03:39:08 -04:00
|
|
|
|
|
|
|
|
[SPARK-30569][SQL][PYSPARK][SPARKR] Add percentile_approx DSL functions
### What changes were proposed in this pull request?
- Adds following overloaded variants to Scala `o.a.s.sql.functions`:
- `percentile_approx(e: Column, percentage: Array[Double], accuracy: Long): Column`
- `percentile_approx(columnName: String, percentage: Array[Double], accuracy: Long): Column`
- `percentile_approx(e: Column, percentage: Double, accuracy: Long): Column`
- `percentile_approx(columnName: String, percentage: Double, accuracy: Long): Column`
- `percentile_approx(e: Column, percentage: Seq[Double], accuracy: Long): Column` (primarily for
Python interop).
- `percentile_approx(columnName: String, percentage: Seq[Double], accuracy: Long): Column`
- Adds `percentile_approx` to `pyspark.sql.functions`.
- Adds `percentile_approx` function to SparkR.
### Why are the changes needed?
Currently we support `percentile_approx` only in SQL expression. It is inconvenient and makes this function relatively unknown.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
New unit tests for SparkR an PySpark.
As for now there are no additional tests in Scala API ‒ `ApproximatePercentile` is well tested and Python (including docstrings) and R tests provide additional tests, so it seems unnecessary.
Closes #27278 from zero323/SPARK-30569.
Lead-authored-by: zero323 <mszymkiewicz@gmail.com>
Co-authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-16 21:44:21 -04:00
|
|
|
def percentile_approx(col, percentage, accuracy=10000):
|
2020-09-22 15:45:19 -04:00
|
|
|
"""Returns the approximate `percentile` of the numeric column `col` which is the smallest value
|
|
|
|
in the ordered `col` values (sorted from least to greatest) such that no more than `percentage`
|
|
|
|
of `col` values is less than the value or equal to that value.
|
[SPARK-30569][SQL][PYSPARK][SPARKR] Add percentile_approx DSL functions
### What changes were proposed in this pull request?
- Adds following overloaded variants to Scala `o.a.s.sql.functions`:
- `percentile_approx(e: Column, percentage: Array[Double], accuracy: Long): Column`
- `percentile_approx(columnName: String, percentage: Array[Double], accuracy: Long): Column`
- `percentile_approx(e: Column, percentage: Double, accuracy: Long): Column`
- `percentile_approx(columnName: String, percentage: Double, accuracy: Long): Column`
- `percentile_approx(e: Column, percentage: Seq[Double], accuracy: Long): Column` (primarily for
Python interop).
- `percentile_approx(columnName: String, percentage: Seq[Double], accuracy: Long): Column`
- Adds `percentile_approx` to `pyspark.sql.functions`.
- Adds `percentile_approx` function to SparkR.
### Why are the changes needed?
Currently we support `percentile_approx` only in SQL expression. It is inconvenient and makes this function relatively unknown.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
New unit tests for SparkR an PySpark.
As for now there are no additional tests in Scala API ‒ `ApproximatePercentile` is well tested and Python (including docstrings) and R tests provide additional tests, so it seems unnecessary.
Closes #27278 from zero323/SPARK-30569.
Lead-authored-by: zero323 <mszymkiewicz@gmail.com>
Co-authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-16 21:44:21 -04:00
|
|
|
The value of percentage must be between 0.0 and 1.0.
|
|
|
|
|
|
|
|
The accuracy parameter (default: 10000)
|
|
|
|
is a positive numeric literal which controls approximation accuracy at the cost of memory.
|
|
|
|
Higher value of accuracy yields better accuracy, 1.0/accuracy is the relative error
|
|
|
|
of the approximation.
|
|
|
|
|
|
|
|
When percentage is an array, each value of the percentage array must be between 0.0 and 1.0.
|
|
|
|
In this case, returns the approximate percentile array of column col
|
|
|
|
at the given percentage array.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 3.1.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
[SPARK-30569][SQL][PYSPARK][SPARKR] Add percentile_approx DSL functions
### What changes were proposed in this pull request?
- Adds following overloaded variants to Scala `o.a.s.sql.functions`:
- `percentile_approx(e: Column, percentage: Array[Double], accuracy: Long): Column`
- `percentile_approx(columnName: String, percentage: Array[Double], accuracy: Long): Column`
- `percentile_approx(e: Column, percentage: Double, accuracy: Long): Column`
- `percentile_approx(columnName: String, percentage: Double, accuracy: Long): Column`
- `percentile_approx(e: Column, percentage: Seq[Double], accuracy: Long): Column` (primarily for
Python interop).
- `percentile_approx(columnName: String, percentage: Seq[Double], accuracy: Long): Column`
- Adds `percentile_approx` to `pyspark.sql.functions`.
- Adds `percentile_approx` function to SparkR.
### Why are the changes needed?
Currently we support `percentile_approx` only in SQL expression. It is inconvenient and makes this function relatively unknown.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
New unit tests for SparkR an PySpark.
As for now there are no additional tests in Scala API ‒ `ApproximatePercentile` is well tested and Python (including docstrings) and R tests provide additional tests, so it seems unnecessary.
Closes #27278 from zero323/SPARK-30569.
Lead-authored-by: zero323 <mszymkiewicz@gmail.com>
Co-authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-16 21:44:21 -04:00
|
|
|
>>> key = (col("id") % 3).alias("key")
|
|
|
|
>>> value = (randn(42) + key * 10).alias("value")
|
|
|
|
>>> df = spark.range(0, 1000, 1, 1).select(key, value)
|
|
|
|
>>> df.select(
|
|
|
|
... percentile_approx("value", [0.25, 0.5, 0.75], 1000000).alias("quantiles")
|
|
|
|
... ).printSchema()
|
|
|
|
root
|
|
|
|
|-- quantiles: array (nullable = true)
|
|
|
|
| |-- element: double (containsNull = false)
|
|
|
|
|
|
|
|
>>> df.groupBy("key").agg(
|
|
|
|
... percentile_approx("value", 0.5, lit(1000000)).alias("median")
|
|
|
|
... ).printSchema()
|
|
|
|
root
|
|
|
|
|-- key: long (nullable = true)
|
|
|
|
|-- median: double (nullable = true)
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
|
|
|
|
if isinstance(percentage, (list, tuple)):
|
|
|
|
# A local list
|
|
|
|
percentage = sc._jvm.functions.array(_to_seq(sc, [
|
|
|
|
_create_column_from_literal(x) for x in percentage
|
|
|
|
]))
|
|
|
|
elif isinstance(percentage, Column):
|
|
|
|
# Already a Column
|
|
|
|
percentage = _to_java_column(percentage)
|
|
|
|
else:
|
|
|
|
# Probably scalar
|
|
|
|
percentage = _create_column_from_literal(percentage)
|
|
|
|
|
|
|
|
accuracy = (
|
|
|
|
_to_java_column(accuracy) if isinstance(accuracy, Column)
|
|
|
|
else _create_column_from_literal(accuracy)
|
|
|
|
)
|
|
|
|
|
|
|
|
return Column(sc._jvm.functions.percentile_approx(_to_java_column(col), percentage, accuracy))
|
|
|
|
|
|
|
|
|
2015-05-01 15:49:02 -04:00
|
|
|
def rand(seed=None):
|
2016-11-06 00:47:33 -04:00
|
|
|
"""Generates a random column with independent and identically distributed (i.i.d.) samples
|
2020-03-31 02:16:17 -04:00
|
|
|
uniformly distributed in [0.0, 1.0).
|
2017-07-08 02:59:34 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
2018-05-10 12:44:49 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
The function is non-deterministic in general case.
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2017-07-08 02:59:34 -04:00
|
|
|
>>> df.withColumn('rand', rand(seed=42) * 3).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(age=2, name='Alice', rand=2.4052597283576684),
|
|
|
|
Row(age=5, name='Bob', rand=2.3913904055683974)]
|
2015-05-01 15:49:02 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
2015-08-06 20:03:14 -04:00
|
|
|
if seed is not None:
|
2015-05-01 15:49:02 -04:00
|
|
|
jc = sc._jvm.functions.rand(seed)
|
|
|
|
else:
|
|
|
|
jc = sc._jvm.functions.rand()
|
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
|
|
|
def randn(seed=None):
|
2016-11-06 00:47:33 -04:00
|
|
|
"""Generates a column with independent and identically distributed (i.i.d.) samples from
|
|
|
|
the standard normal distribution.
|
2017-07-08 02:59:34 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
2018-05-10 12:44:49 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
The function is non-deterministic in general case.
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2017-07-08 02:59:34 -04:00
|
|
|
>>> df.withColumn('randn', randn(seed=42)).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(age=2, name='Alice', randn=1.1027054481455365),
|
|
|
|
Row(age=5, name='Bob', randn=0.7400395449950132)]
|
2015-05-01 15:49:02 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
2015-08-06 20:03:14 -04:00
|
|
|
if seed is not None:
|
2015-05-01 15:49:02 -04:00
|
|
|
jc = sc._jvm.functions.randn(seed)
|
|
|
|
else:
|
|
|
|
jc = sc._jvm.functions.randn()
|
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
2015-08-04 22:25:24 -04:00
|
|
|
def round(col, scale=0):
|
2015-06-30 19:59:44 -04:00
|
|
|
"""
|
[SPARK-14639] [PYTHON] [R] Add `bround` function in Python/R.
## What changes were proposed in this pull request?
This issue aims to expose Scala `bround` function in Python/R API.
`bround` function is implemented in SPARK-14614 by extending current `round` function.
We used the following semantics from Hive.
```java
public static double bround(double input, int scale) {
if (Double.isNaN(input) || Double.isInfinite(input)) {
return input;
}
return BigDecimal.valueOf(input).setScale(scale, RoundingMode.HALF_EVEN).doubleValue();
}
```
After this PR, `pyspark` and `sparkR` also support `bround` function.
**PySpark**
```python
>>> from pyspark.sql.functions import bround
>>> sqlContext.createDataFrame([(2.5,)], ['a']).select(bround('a', 0).alias('r')).collect()
[Row(r=2.0)]
```
**SparkR**
```r
> df = createDataFrame(sqlContext, data.frame(x = c(2.5, 3.5)))
> head(collect(select(df, bround(df$x, 0))))
bround(x, 0)
1 2
2 4
```
## How was this patch tested?
Pass the Jenkins tests (including new testcases).
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes #12509 from dongjoon-hyun/SPARK-14639.
2016-04-20 01:28:11 -04:00
|
|
|
Round the given value to `scale` decimal places using HALF_UP rounding mode if `scale` >= 0
|
2015-08-04 22:25:24 -04:00
|
|
|
or at integral part when `scale` < 0.
|
2015-06-26 01:07:37 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> spark.createDataFrame([(2.5,)], ['a']).select(round('a', 0).alias('r')).collect()
|
[SPARK-14639] [PYTHON] [R] Add `bround` function in Python/R.
## What changes were proposed in this pull request?
This issue aims to expose Scala `bround` function in Python/R API.
`bround` function is implemented in SPARK-14614 by extending current `round` function.
We used the following semantics from Hive.
```java
public static double bround(double input, int scale) {
if (Double.isNaN(input) || Double.isInfinite(input)) {
return input;
}
return BigDecimal.valueOf(input).setScale(scale, RoundingMode.HALF_EVEN).doubleValue();
}
```
After this PR, `pyspark` and `sparkR` also support `bround` function.
**PySpark**
```python
>>> from pyspark.sql.functions import bround
>>> sqlContext.createDataFrame([(2.5,)], ['a']).select(bround('a', 0).alias('r')).collect()
[Row(r=2.0)]
```
**SparkR**
```r
> df = createDataFrame(sqlContext, data.frame(x = c(2.5, 3.5)))
> head(collect(select(df, bround(df$x, 0))))
bround(x, 0)
1 2
2 4
```
## How was this patch tested?
Pass the Jenkins tests (including new testcases).
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes #12509 from dongjoon-hyun/SPARK-14639.
2016-04-20 01:28:11 -04:00
|
|
|
[Row(r=3.0)]
|
2015-06-26 01:07:37 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
2015-08-04 22:25:24 -04:00
|
|
|
return Column(sc._jvm.functions.round(_to_java_column(col), scale))
|
2015-06-26 01:07:37 -04:00
|
|
|
|
|
|
|
|
[SPARK-14639] [PYTHON] [R] Add `bround` function in Python/R.
## What changes were proposed in this pull request?
This issue aims to expose Scala `bround` function in Python/R API.
`bround` function is implemented in SPARK-14614 by extending current `round` function.
We used the following semantics from Hive.
```java
public static double bround(double input, int scale) {
if (Double.isNaN(input) || Double.isInfinite(input)) {
return input;
}
return BigDecimal.valueOf(input).setScale(scale, RoundingMode.HALF_EVEN).doubleValue();
}
```
After this PR, `pyspark` and `sparkR` also support `bround` function.
**PySpark**
```python
>>> from pyspark.sql.functions import bround
>>> sqlContext.createDataFrame([(2.5,)], ['a']).select(bround('a', 0).alias('r')).collect()
[Row(r=2.0)]
```
**SparkR**
```r
> df = createDataFrame(sqlContext, data.frame(x = c(2.5, 3.5)))
> head(collect(select(df, bround(df$x, 0))))
bround(x, 0)
1 2
2 4
```
## How was this patch tested?
Pass the Jenkins tests (including new testcases).
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes #12509 from dongjoon-hyun/SPARK-14639.
2016-04-20 01:28:11 -04:00
|
|
|
def bround(col, scale=0):
|
|
|
|
"""
|
|
|
|
Round the given value to `scale` decimal places using HALF_EVEN rounding mode if `scale` >= 0
|
|
|
|
or at integral part when `scale` < 0.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> spark.createDataFrame([(2.5,)], ['a']).select(bround('a', 0).alias('r')).collect()
|
[SPARK-14639] [PYTHON] [R] Add `bround` function in Python/R.
## What changes were proposed in this pull request?
This issue aims to expose Scala `bround` function in Python/R API.
`bround` function is implemented in SPARK-14614 by extending current `round` function.
We used the following semantics from Hive.
```java
public static double bround(double input, int scale) {
if (Double.isNaN(input) || Double.isInfinite(input)) {
return input;
}
return BigDecimal.valueOf(input).setScale(scale, RoundingMode.HALF_EVEN).doubleValue();
}
```
After this PR, `pyspark` and `sparkR` also support `bround` function.
**PySpark**
```python
>>> from pyspark.sql.functions import bround
>>> sqlContext.createDataFrame([(2.5,)], ['a']).select(bround('a', 0).alias('r')).collect()
[Row(r=2.0)]
```
**SparkR**
```r
> df = createDataFrame(sqlContext, data.frame(x = c(2.5, 3.5)))
> head(collect(select(df, bround(df$x, 0))))
bround(x, 0)
1 2
2 4
```
## How was this patch tested?
Pass the Jenkins tests (including new testcases).
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes #12509 from dongjoon-hyun/SPARK-14639.
2016-04-20 01:28:11 -04:00
|
|
|
[Row(r=2.0)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.bround(_to_java_column(col), scale))
|
|
|
|
|
|
|
|
|
[SPARK-8223] [SPARK-8224] [SQL] shift left and shift right
Jira:
https://issues.apache.org/jira/browse/SPARK-8223
https://issues.apache.org/jira/browse/SPARK-8224
~~I am aware of #7174 and will update this pr, if it's merged.~~ Done
I don't know if #7034 can simplify this, but we can have a look on it, if it gets merged
rxin In the Jira ticket the function as no second argument. I added a `numBits` argument that allows to specify the number of bits. I guess this improves the usability. I wanted to add `shiftleft(value)` as well, but the `selectExpr` dataframe tests crashes, if I have both. I order to do this, I added the following to the functions.scala `def shiftRight(e: Column): Column = ShiftRight(e.expr, lit(1).expr)`, but as I mentioned this doesn't pass tests like `df.selectExpr("shiftRight(a)", ...` (not enough arguments exception).
If we need the bitwise shift in order to be hive compatible, I suggest to add `shiftLeft` and something like `shiftLeftX`
Author: Tarek Auel <tarek.auel@googlemail.com>
Closes #7178 from tarekauel/8223 and squashes the following commits:
8023bb5 [Tarek Auel] [SPARK-8223][SPARK-8224] fixed test
f3f64e6 [Tarek Auel] [SPARK-8223][SPARK-8224] Integer -> Int
f628706 [Tarek Auel] [SPARK-8223][SPARK-8224] removed toString; updated function description
3b56f2a [Tarek Auel] Merge remote-tracking branch 'origin/master' into 8223
5189690 [Tarek Auel] [SPARK-8223][SPARK-8224] minor fix and style fix
9434a28 [Tarek Auel] Merge remote-tracking branch 'origin/master' into 8223
44ee324 [Tarek Auel] [SPARK-8223][SPARK-8224] docu fix
ac7fe9d [Tarek Auel] [SPARK-8223][SPARK-8224] right and left bit shift
2015-07-02 13:02:19 -04:00
|
|
|
def shiftLeft(col, numBits):
|
2016-02-22 04:52:07 -05:00
|
|
|
"""Shift the given value numBits left.
|
[SPARK-8223] [SPARK-8224] [SQL] shift left and shift right
Jira:
https://issues.apache.org/jira/browse/SPARK-8223
https://issues.apache.org/jira/browse/SPARK-8224
~~I am aware of #7174 and will update this pr, if it's merged.~~ Done
I don't know if #7034 can simplify this, but we can have a look on it, if it gets merged
rxin In the Jira ticket the function as no second argument. I added a `numBits` argument that allows to specify the number of bits. I guess this improves the usability. I wanted to add `shiftleft(value)` as well, but the `selectExpr` dataframe tests crashes, if I have both. I order to do this, I added the following to the functions.scala `def shiftRight(e: Column): Column = ShiftRight(e.expr, lit(1).expr)`, but as I mentioned this doesn't pass tests like `df.selectExpr("shiftRight(a)", ...` (not enough arguments exception).
If we need the bitwise shift in order to be hive compatible, I suggest to add `shiftLeft` and something like `shiftLeftX`
Author: Tarek Auel <tarek.auel@googlemail.com>
Closes #7178 from tarekauel/8223 and squashes the following commits:
8023bb5 [Tarek Auel] [SPARK-8223][SPARK-8224] fixed test
f3f64e6 [Tarek Auel] [SPARK-8223][SPARK-8224] Integer -> Int
f628706 [Tarek Auel] [SPARK-8223][SPARK-8224] removed toString; updated function description
3b56f2a [Tarek Auel] Merge remote-tracking branch 'origin/master' into 8223
5189690 [Tarek Auel] [SPARK-8223][SPARK-8224] minor fix and style fix
9434a28 [Tarek Auel] Merge remote-tracking branch 'origin/master' into 8223
44ee324 [Tarek Auel] [SPARK-8223][SPARK-8224] docu fix
ac7fe9d [Tarek Auel] [SPARK-8223][SPARK-8224] right and left bit shift
2015-07-02 13:02:19 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
2021-02-01 19:29:40 -05:00
|
|
|
.. deprecated:: 3.2.0
|
|
|
|
Use :func:`shiftleft` instead.
|
|
|
|
"""
|
|
|
|
warnings.warn("Deprecated in 3.2, use shiftleft instead.", FutureWarning)
|
|
|
|
return shiftleft(col, numBits)
|
|
|
|
|
|
|
|
|
|
|
|
def shiftleft(col, numBits):
|
|
|
|
"""Shift the given value numBits left.
|
|
|
|
|
|
|
|
.. versionadded:: 3.2.0
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2021-02-01 19:29:40 -05:00
|
|
|
>>> spark.createDataFrame([(21,)], ['a']).select(shiftleft('a', 1).alias('r')).collect()
|
[SPARK-8223] [SPARK-8224] [SQL] shift left and shift right
Jira:
https://issues.apache.org/jira/browse/SPARK-8223
https://issues.apache.org/jira/browse/SPARK-8224
~~I am aware of #7174 and will update this pr, if it's merged.~~ Done
I don't know if #7034 can simplify this, but we can have a look on it, if it gets merged
rxin In the Jira ticket the function as no second argument. I added a `numBits` argument that allows to specify the number of bits. I guess this improves the usability. I wanted to add `shiftleft(value)` as well, but the `selectExpr` dataframe tests crashes, if I have both. I order to do this, I added the following to the functions.scala `def shiftRight(e: Column): Column = ShiftRight(e.expr, lit(1).expr)`, but as I mentioned this doesn't pass tests like `df.selectExpr("shiftRight(a)", ...` (not enough arguments exception).
If we need the bitwise shift in order to be hive compatible, I suggest to add `shiftLeft` and something like `shiftLeftX`
Author: Tarek Auel <tarek.auel@googlemail.com>
Closes #7178 from tarekauel/8223 and squashes the following commits:
8023bb5 [Tarek Auel] [SPARK-8223][SPARK-8224] fixed test
f3f64e6 [Tarek Auel] [SPARK-8223][SPARK-8224] Integer -> Int
f628706 [Tarek Auel] [SPARK-8223][SPARK-8224] removed toString; updated function description
3b56f2a [Tarek Auel] Merge remote-tracking branch 'origin/master' into 8223
5189690 [Tarek Auel] [SPARK-8223][SPARK-8224] minor fix and style fix
9434a28 [Tarek Auel] Merge remote-tracking branch 'origin/master' into 8223
44ee324 [Tarek Auel] [SPARK-8223][SPARK-8224] docu fix
ac7fe9d [Tarek Auel] [SPARK-8223][SPARK-8224] right and left bit shift
2015-07-02 13:02:19 -04:00
|
|
|
[Row(r=42)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
2021-02-01 19:29:40 -05:00
|
|
|
return Column(sc._jvm.functions.shiftleft(_to_java_column(col), numBits))
|
[SPARK-8223] [SPARK-8224] [SQL] shift left and shift right
Jira:
https://issues.apache.org/jira/browse/SPARK-8223
https://issues.apache.org/jira/browse/SPARK-8224
~~I am aware of #7174 and will update this pr, if it's merged.~~ Done
I don't know if #7034 can simplify this, but we can have a look on it, if it gets merged
rxin In the Jira ticket the function as no second argument. I added a `numBits` argument that allows to specify the number of bits. I guess this improves the usability. I wanted to add `shiftleft(value)` as well, but the `selectExpr` dataframe tests crashes, if I have both. I order to do this, I added the following to the functions.scala `def shiftRight(e: Column): Column = ShiftRight(e.expr, lit(1).expr)`, but as I mentioned this doesn't pass tests like `df.selectExpr("shiftRight(a)", ...` (not enough arguments exception).
If we need the bitwise shift in order to be hive compatible, I suggest to add `shiftLeft` and something like `shiftLeftX`
Author: Tarek Auel <tarek.auel@googlemail.com>
Closes #7178 from tarekauel/8223 and squashes the following commits:
8023bb5 [Tarek Auel] [SPARK-8223][SPARK-8224] fixed test
f3f64e6 [Tarek Auel] [SPARK-8223][SPARK-8224] Integer -> Int
f628706 [Tarek Auel] [SPARK-8223][SPARK-8224] removed toString; updated function description
3b56f2a [Tarek Auel] Merge remote-tracking branch 'origin/master' into 8223
5189690 [Tarek Auel] [SPARK-8223][SPARK-8224] minor fix and style fix
9434a28 [Tarek Auel] Merge remote-tracking branch 'origin/master' into 8223
44ee324 [Tarek Auel] [SPARK-8223][SPARK-8224] docu fix
ac7fe9d [Tarek Auel] [SPARK-8223][SPARK-8224] right and left bit shift
2015-07-02 13:02:19 -04:00
|
|
|
|
|
|
|
|
|
|
|
def shiftRight(col, numBits):
|
2016-11-06 00:47:33 -04:00
|
|
|
"""(Signed) shift the given value numBits right.
|
[SPARK-8223] [SPARK-8224] [SQL] shift left and shift right
Jira:
https://issues.apache.org/jira/browse/SPARK-8223
https://issues.apache.org/jira/browse/SPARK-8224
~~I am aware of #7174 and will update this pr, if it's merged.~~ Done
I don't know if #7034 can simplify this, but we can have a look on it, if it gets merged
rxin In the Jira ticket the function as no second argument. I added a `numBits` argument that allows to specify the number of bits. I guess this improves the usability. I wanted to add `shiftleft(value)` as well, but the `selectExpr` dataframe tests crashes, if I have both. I order to do this, I added the following to the functions.scala `def shiftRight(e: Column): Column = ShiftRight(e.expr, lit(1).expr)`, but as I mentioned this doesn't pass tests like `df.selectExpr("shiftRight(a)", ...` (not enough arguments exception).
If we need the bitwise shift in order to be hive compatible, I suggest to add `shiftLeft` and something like `shiftLeftX`
Author: Tarek Auel <tarek.auel@googlemail.com>
Closes #7178 from tarekauel/8223 and squashes the following commits:
8023bb5 [Tarek Auel] [SPARK-8223][SPARK-8224] fixed test
f3f64e6 [Tarek Auel] [SPARK-8223][SPARK-8224] Integer -> Int
f628706 [Tarek Auel] [SPARK-8223][SPARK-8224] removed toString; updated function description
3b56f2a [Tarek Auel] Merge remote-tracking branch 'origin/master' into 8223
5189690 [Tarek Auel] [SPARK-8223][SPARK-8224] minor fix and style fix
9434a28 [Tarek Auel] Merge remote-tracking branch 'origin/master' into 8223
44ee324 [Tarek Auel] [SPARK-8223][SPARK-8224] docu fix
ac7fe9d [Tarek Auel] [SPARK-8223][SPARK-8224] right and left bit shift
2015-07-02 13:02:19 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
2021-02-01 19:29:40 -05:00
|
|
|
.. deprecated:: 3.2.0
|
|
|
|
Use :func:`shiftright` instead.
|
|
|
|
"""
|
|
|
|
warnings.warn("Deprecated in 3.2, use shiftright instead.", FutureWarning)
|
|
|
|
return shiftright(col, numBits)
|
|
|
|
|
|
|
|
|
|
|
|
def shiftright(col, numBits):
|
|
|
|
"""(Signed) shift the given value numBits right.
|
|
|
|
|
|
|
|
.. versionadded:: 3.2.0
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2021-02-01 19:29:40 -05:00
|
|
|
>>> spark.createDataFrame([(42,)], ['a']).select(shiftright('a', 1).alias('r')).collect()
|
[SPARK-8223] [SPARK-8224] [SQL] shift left and shift right
Jira:
https://issues.apache.org/jira/browse/SPARK-8223
https://issues.apache.org/jira/browse/SPARK-8224
~~I am aware of #7174 and will update this pr, if it's merged.~~ Done
I don't know if #7034 can simplify this, but we can have a look on it, if it gets merged
rxin In the Jira ticket the function as no second argument. I added a `numBits` argument that allows to specify the number of bits. I guess this improves the usability. I wanted to add `shiftleft(value)` as well, but the `selectExpr` dataframe tests crashes, if I have both. I order to do this, I added the following to the functions.scala `def shiftRight(e: Column): Column = ShiftRight(e.expr, lit(1).expr)`, but as I mentioned this doesn't pass tests like `df.selectExpr("shiftRight(a)", ...` (not enough arguments exception).
If we need the bitwise shift in order to be hive compatible, I suggest to add `shiftLeft` and something like `shiftLeftX`
Author: Tarek Auel <tarek.auel@googlemail.com>
Closes #7178 from tarekauel/8223 and squashes the following commits:
8023bb5 [Tarek Auel] [SPARK-8223][SPARK-8224] fixed test
f3f64e6 [Tarek Auel] [SPARK-8223][SPARK-8224] Integer -> Int
f628706 [Tarek Auel] [SPARK-8223][SPARK-8224] removed toString; updated function description
3b56f2a [Tarek Auel] Merge remote-tracking branch 'origin/master' into 8223
5189690 [Tarek Auel] [SPARK-8223][SPARK-8224] minor fix and style fix
9434a28 [Tarek Auel] Merge remote-tracking branch 'origin/master' into 8223
44ee324 [Tarek Auel] [SPARK-8223][SPARK-8224] docu fix
ac7fe9d [Tarek Auel] [SPARK-8223][SPARK-8224] right and left bit shift
2015-07-02 13:02:19 -04:00
|
|
|
[Row(r=21)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
jc = sc._jvm.functions.shiftRight(_to_java_column(col), numBits)
|
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
2015-07-03 18:39:16 -04:00
|
|
|
def shiftRightUnsigned(col, numBits):
|
2016-02-22 04:52:07 -05:00
|
|
|
"""Unsigned shift the given value numBits right.
|
2015-07-03 18:39:16 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
2021-02-01 19:29:40 -05:00
|
|
|
.. deprecated:: 3.2.0
|
|
|
|
Use :func:`shiftrightunsigned` instead.
|
|
|
|
"""
|
|
|
|
warnings.warn("Deprecated in 3.2, use shiftrightunsigned instead.", FutureWarning)
|
|
|
|
return shiftrightunsigned(col, numBits)
|
|
|
|
|
|
|
|
|
|
|
|
def shiftrightunsigned(col, numBits):
|
|
|
|
"""Unsigned shift the given value numBits right.
|
|
|
|
|
|
|
|
.. versionadded:: 3.2.0
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> df = spark.createDataFrame([(-42,)], ['a'])
|
2021-02-01 19:29:40 -05:00
|
|
|
>>> df.select(shiftrightunsigned('a', 1).alias('r')).collect()
|
2015-07-03 18:39:16 -04:00
|
|
|
[Row(r=9223372036854775787)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
jc = sc._jvm.functions.shiftRightUnsigned(_to_java_column(col), numBits)
|
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
2015-11-25 00:30:53 -05:00
|
|
|
def spark_partition_id():
|
2016-11-04 01:27:35 -04:00
|
|
|
"""A column for partition ID.
|
2015-02-14 02:03:22 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.6.0
|
2015-04-26 14:46:58 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Notes
|
|
|
|
-----
|
2020-11-27 11:22:45 -05:00
|
|
|
This is non deterministic because it depends on data partitioning and task scheduling.
|
2020-11-02 20:00:49 -05:00
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2015-11-25 00:30:53 -05:00
|
|
|
>>> df.repartition(1).select(spark_partition_id().alias("pid")).collect()
|
2015-04-26 14:46:58 -04:00
|
|
|
[Row(pid=0), Row(pid=0)]
|
2015-02-14 02:03:22 -05:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
2015-11-25 00:30:53 -05:00
|
|
|
return Column(sc._jvm.functions.spark_partition_id())
|
2015-02-14 02:03:22 -05:00
|
|
|
|
|
|
|
|
2015-07-25 03:34:59 -04:00
|
|
|
def expr(str):
|
|
|
|
"""Parses the expression string into the column that it represents
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2015-07-25 03:34:59 -04:00
|
|
|
>>> df.select(expr("length(name)")).collect()
|
2015-11-10 14:06:29 -05:00
|
|
|
[Row(length(name)=5), Row(length(name)=3)]
|
2015-07-25 03:34:59 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.expr(str))
|
|
|
|
|
|
|
|
|
2015-05-01 15:49:02 -04:00
|
|
|
def struct(*cols):
|
|
|
|
"""Creates a new struct column.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
cols : list, set, str or :class:`~pyspark.sql.Column`
|
|
|
|
column names or :class:`~pyspark.sql.Column`\\s to contain in the output struct.
|
2015-05-01 15:49:02 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2015-05-01 15:49:02 -04:00
|
|
|
>>> df.select(struct('age', 'name').alias("struct")).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(struct=Row(age=2, name='Alice')), Row(struct=Row(age=5, name='Bob'))]
|
2015-05-01 15:49:02 -04:00
|
|
|
>>> df.select(struct([df.age, df.name]).alias("struct")).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(struct=Row(age=2, name='Alice')), Row(struct=Row(age=5, name='Bob'))]
|
2015-05-01 15:49:02 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
if len(cols) == 1 and isinstance(cols[0], (list, set)):
|
|
|
|
cols = cols[0]
|
|
|
|
jc = sc._jvm.functions.struct(_to_seq(sc, cols, _to_java_column))
|
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
2015-08-04 22:25:24 -04:00
|
|
|
def greatest(*cols):
|
|
|
|
"""
|
|
|
|
Returns the greatest value of the list of column names, skipping null values.
|
|
|
|
This function takes at least 2 parameters. It will return null iff all parameters are null.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> df = spark.createDataFrame([(1, 4, 3)], ['a', 'b', 'c'])
|
2015-08-04 22:25:24 -04:00
|
|
|
>>> df.select(greatest(df.a, df.b, df.c).alias("greatest")).collect()
|
|
|
|
[Row(greatest=4)]
|
|
|
|
"""
|
|
|
|
if len(cols) < 2:
|
|
|
|
raise ValueError("greatest should take at least two columns")
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.greatest(_to_seq(sc, cols, _to_java_column)))
|
|
|
|
|
|
|
|
|
|
|
|
def least(*cols):
|
|
|
|
"""
|
|
|
|
Returns the least value of the list of column names, skipping null values.
|
|
|
|
This function takes at least 2 parameters. It will return null iff all parameters are null.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> df = spark.createDataFrame([(1, 4, 3)], ['a', 'b', 'c'])
|
2015-08-04 22:25:24 -04:00
|
|
|
>>> df.select(least(df.a, df.b, df.c).alias("least")).collect()
|
|
|
|
[Row(least=1)]
|
|
|
|
"""
|
|
|
|
if len(cols) < 2:
|
|
|
|
raise ValueError("least should take at least two columns")
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.least(_to_seq(sc, cols, _to_java_column)))
|
|
|
|
|
|
|
|
|
2015-05-13 00:43:34 -04:00
|
|
|
def when(condition, value):
|
|
|
|
"""Evaluates a list of conditions and returns one of multiple possible result expressions.
|
2021-02-01 19:30:50 -05:00
|
|
|
If :func:`pyspark.sql.Column.otherwise` is not invoked, None is returned for unmatched
|
|
|
|
conditions.
|
2015-05-13 00:43:34 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
condition : :class:`~pyspark.sql.Column`
|
|
|
|
a boolean :class:`~pyspark.sql.Column` expression.
|
2020-11-02 20:00:49 -05:00
|
|
|
value :
|
2021-02-01 19:30:50 -05:00
|
|
|
a literal value, or a :class:`~pyspark.sql.Column` expression.
|
2015-05-13 00:43:34 -04:00
|
|
|
|
|
|
|
>>> df.select(when(df['age'] == 2, 3).otherwise(4).alias("age")).collect()
|
|
|
|
[Row(age=3), Row(age=4)]
|
|
|
|
|
|
|
|
>>> df.select(when(df.age == 2, df.age + 1).alias("age")).collect()
|
|
|
|
[Row(age=3), Row(age=None)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
if not isinstance(condition, Column):
|
|
|
|
raise TypeError("condition should be a Column")
|
|
|
|
v = value._jc if isinstance(value, Column) else value
|
|
|
|
jc = sc._jvm.functions.when(condition._jc, v)
|
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
2015-06-18 21:41:15 -04:00
|
|
|
def log(arg1, arg2=None):
|
2015-06-18 02:31:30 -04:00
|
|
|
"""Returns the first argument-based logarithm of the second argument.
|
|
|
|
|
2015-06-18 21:41:15 -04:00
|
|
|
If there is only one argument, then this takes the natural logarithm of the argument.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-03-02 18:26:34 -05:00
|
|
|
>>> df.select(log(10.0, df.age).alias('ten')).rdd.map(lambda l: str(l.ten)[:7]).collect()
|
2015-06-18 02:31:30 -04:00
|
|
|
['0.30102', '0.69897']
|
|
|
|
|
2016-03-02 18:26:34 -05:00
|
|
|
>>> df.select(log(df.age).alias('e')).rdd.map(lambda l: str(l.e)[:7]).collect()
|
2015-06-18 02:31:30 -04:00
|
|
|
['0.69314', '1.60943']
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
2015-06-18 21:41:15 -04:00
|
|
|
if arg2 is None:
|
|
|
|
jc = sc._jvm.functions.log(_to_java_column(arg1))
|
|
|
|
else:
|
|
|
|
jc = sc._jvm.functions.log(arg1, _to_java_column(arg2))
|
2015-06-18 02:31:30 -04:00
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
2015-06-30 19:59:44 -04:00
|
|
|
def log2(col):
|
|
|
|
"""Returns the base-2 logarithm of the argument.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> spark.createDataFrame([(4,)], ['a']).select(log2('a').alias('log2')).collect()
|
2015-06-30 19:59:44 -04:00
|
|
|
[Row(log2=2.0)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.log2(_to_java_column(col)))
|
|
|
|
|
|
|
|
|
2015-08-04 22:25:24 -04:00
|
|
|
def conv(col, fromBase, toBase):
|
|
|
|
"""
|
|
|
|
Convert a number in a string column from one base to another.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> df = spark.createDataFrame([("010101",)], ['n'])
|
2015-08-04 22:25:24 -04:00
|
|
|
>>> df.select(conv(df.n, 2, 16).alias('hex')).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(hex='15')]
|
2015-08-04 22:25:24 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.conv(_to_java_column(col), fromBase, toBase))
|
|
|
|
|
|
|
|
|
|
|
|
def factorial(col):
|
|
|
|
"""
|
|
|
|
Computes the factorial of the given value.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> df = spark.createDataFrame([(5,)], ['n'])
|
2015-08-04 22:25:24 -04:00
|
|
|
>>> df.select(factorial(df.n).alias('f')).collect()
|
|
|
|
[Row(f=120)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.factorial(_to_java_column(col)))
|
|
|
|
|
|
|
|
|
|
|
|
# --------------- Window functions ------------------------
|
|
|
|
|
2018-12-27 11:02:41 -05:00
|
|
|
def lag(col, offset=1, default=None):
|
2015-05-23 11:30:05 -04:00
|
|
|
"""
|
|
|
|
Window function: returns the value that is `offset` rows before the current row, and
|
2021-02-01 19:30:50 -05:00
|
|
|
`default` if there is less than `offset` rows before the current row. For example,
|
2015-05-23 11:30:05 -04:00
|
|
|
an `offset` of one will return the previous row at any given point in the window partition.
|
|
|
|
|
|
|
|
This is equivalent to the LAG function in SQL.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column or expression
|
|
|
|
offset : int, optional
|
|
|
|
number of row to extend
|
|
|
|
default : optional
|
|
|
|
default value
|
2015-05-23 11:30:05 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
2018-12-27 11:02:41 -05:00
|
|
|
return Column(sc._jvm.functions.lag(_to_java_column(col), offset, default))
|
2015-05-23 11:30:05 -04:00
|
|
|
|
|
|
|
|
2018-12-27 11:02:41 -05:00
|
|
|
def lead(col, offset=1, default=None):
|
2015-05-23 11:30:05 -04:00
|
|
|
"""
|
|
|
|
Window function: returns the value that is `offset` rows after the current row, and
|
2021-02-01 19:30:50 -05:00
|
|
|
`default` if there is less than `offset` rows after the current row. For example,
|
2015-05-23 11:30:05 -04:00
|
|
|
an `offset` of one will return the next row at any given point in the window partition.
|
|
|
|
|
|
|
|
This is equivalent to the LEAD function in SQL.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column or expression
|
|
|
|
offset : int, optional
|
|
|
|
number of row to extend
|
|
|
|
default : optional
|
|
|
|
default value
|
2015-05-23 11:30:05 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
2018-12-27 11:02:41 -05:00
|
|
|
return Column(sc._jvm.functions.lead(_to_java_column(col), offset, default))
|
2015-05-23 11:30:05 -04:00
|
|
|
|
|
|
|
|
2020-09-29 01:14:28 -04:00
|
|
|
def nth_value(col, offset, ignoreNulls=False):
|
|
|
|
"""
|
|
|
|
Window function: returns the value that is the `offset`\\th row of the window frame
|
|
|
|
(counting from 1), and `null` if the size of window frame is less than `offset` rows.
|
|
|
|
|
|
|
|
It will return the `offset`\\th non-null value it sees when `ignoreNulls` is set to
|
|
|
|
true. If all values are null, then null is returned.
|
|
|
|
|
|
|
|
This is equivalent to the nth_value function in SQL.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 3.1.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column or expression
|
|
|
|
offset : int, optional
|
|
|
|
number of row to use as the value
|
|
|
|
ignoreNulls : bool, optional
|
|
|
|
indicates the Nth value should skip null in the
|
2020-09-29 01:14:28 -04:00
|
|
|
determination of which row to use
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.nth_value(_to_java_column(col), offset, ignoreNulls))
|
|
|
|
|
|
|
|
|
2015-05-23 11:30:05 -04:00
|
|
|
def ntile(n):
|
|
|
|
"""
|
2015-08-14 16:55:29 -04:00
|
|
|
Window function: returns the ntile group id (from 1 to `n` inclusive)
|
2015-08-19 04:42:41 -04:00
|
|
|
in an ordered window partition. For example, if `n` is 4, the first
|
2015-08-14 16:55:29 -04:00
|
|
|
quarter of the rows will get value 1, the second quarter will get 2,
|
|
|
|
the third quarter will get 3, and the last quarter will get 4.
|
2015-05-23 11:30:05 -04:00
|
|
|
|
|
|
|
This is equivalent to the NTILE function in SQL.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
n : int
|
|
|
|
an integer
|
2015-05-23 11:30:05 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.ntile(int(n)))
|
|
|
|
|
|
|
|
|
2015-08-04 22:25:24 -04:00
|
|
|
# ---------------------- Date/Timestamp functions ------------------------------
|
|
|
|
|
|
|
|
@since(1.5)
|
|
|
|
def current_date():
|
|
|
|
"""
|
2020-09-29 01:20:12 -04:00
|
|
|
Returns the current date at the start of query evaluation as a :class:`DateType` column.
|
|
|
|
All calls of current_date within the same query return the same value.
|
2015-08-04 22:25:24 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.current_date())
|
|
|
|
|
|
|
|
|
|
|
|
def current_timestamp():
|
|
|
|
"""
|
2020-09-29 01:20:12 -04:00
|
|
|
Returns the current timestamp at the start of query evaluation as a :class:`TimestampType`
|
|
|
|
column. All calls of current_timestamp within the same query return the same value.
|
2015-08-04 22:25:24 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.current_timestamp())
|
|
|
|
|
|
|
|
|
|
|
|
def date_format(date, format):
|
2015-07-19 01:48:05 -04:00
|
|
|
"""
|
|
|
|
Converts a date/timestamp/string to a value of string in the format specified by the date
|
|
|
|
format given by the second argument.
|
|
|
|
|
|
|
|
A pattern could be for instance `dd.MM.yyyy` and could return a string like '18.03.1993'. All
|
2020-03-20 09:59:26 -04:00
|
|
|
pattern letters of `datetime pattern`_. can be used.
|
2015-07-19 01:48:05 -04:00
|
|
|
|
2020-03-20 09:59:26 -04:00
|
|
|
.. _datetime pattern: https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
|
2015-07-19 01:48:05 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
Whenever possible, use specialized functions like `year`.
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2017-07-08 02:59:34 -04:00
|
|
|
>>> df = spark.createDataFrame([('2015-04-08',)], ['dt'])
|
|
|
|
>>> df.select(date_format('dt', 'MM/dd/yyy').alias('date')).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(date='04/08/2015')]
|
2015-07-19 01:48:05 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
2015-08-04 22:25:24 -04:00
|
|
|
return Column(sc._jvm.functions.date_format(_to_java_column(date), format))
|
2015-07-19 01:48:05 -04:00
|
|
|
|
|
|
|
|
|
|
|
def year(col):
|
|
|
|
"""
|
|
|
|
Extract the year of a given date as integer.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2017-07-08 02:59:34 -04:00
|
|
|
>>> df = spark.createDataFrame([('2015-04-08',)], ['dt'])
|
|
|
|
>>> df.select(year('dt').alias('year')).collect()
|
2015-07-19 01:48:05 -04:00
|
|
|
[Row(year=2015)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
[SPARK-8186] [SPARK-8187] [SPARK-8194] [SPARK-8198] [SPARK-9133] [SPARK-9290] [SQL] functions: date_add, date_sub, add_months, months_between, time-interval calculation
This PR is based on #7589 , thanks to adrian-wang
Added SQL function date_add, date_sub, add_months, month_between, also add a rule for
add/subtract of date/timestamp and interval.
Closes #7589
cc rxin
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Author: Davies Liu <davies@databricks.com>
Closes #7754 from davies/date_add and squashes the following commits:
e8c633a [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
9e8e085 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
6224ce4 [Davies Liu] fix conclict
bd18cd4 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
e47ff2c [Davies Liu] add python api, fix date functions
01943d0 [Davies Liu] Merge branch 'master' into date_add
522e91a [Daoyuan Wang] fix
e8a639a [Daoyuan Wang] fix
42df486 [Daoyuan Wang] fix style
87c4b77 [Daoyuan Wang] function add_months, months_between and some fixes
1a68e03 [Daoyuan Wang] poc of time interval calculation
c506661 [Daoyuan Wang] function date_add , date_sub
2015-07-30 16:21:46 -04:00
|
|
|
return Column(sc._jvm.functions.year(_to_java_column(col)))
|
2015-07-19 01:48:05 -04:00
|
|
|
|
|
|
|
|
|
|
|
def quarter(col):
|
|
|
|
"""
|
|
|
|
Extract the quarter of a given date as integer.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2017-07-08 02:59:34 -04:00
|
|
|
>>> df = spark.createDataFrame([('2015-04-08',)], ['dt'])
|
|
|
|
>>> df.select(quarter('dt').alias('quarter')).collect()
|
2015-07-19 01:48:05 -04:00
|
|
|
[Row(quarter=2)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
[SPARK-8186] [SPARK-8187] [SPARK-8194] [SPARK-8198] [SPARK-9133] [SPARK-9290] [SQL] functions: date_add, date_sub, add_months, months_between, time-interval calculation
This PR is based on #7589 , thanks to adrian-wang
Added SQL function date_add, date_sub, add_months, month_between, also add a rule for
add/subtract of date/timestamp and interval.
Closes #7589
cc rxin
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Author: Davies Liu <davies@databricks.com>
Closes #7754 from davies/date_add and squashes the following commits:
e8c633a [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
9e8e085 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
6224ce4 [Davies Liu] fix conclict
bd18cd4 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
e47ff2c [Davies Liu] add python api, fix date functions
01943d0 [Davies Liu] Merge branch 'master' into date_add
522e91a [Daoyuan Wang] fix
e8a639a [Daoyuan Wang] fix
42df486 [Daoyuan Wang] fix style
87c4b77 [Daoyuan Wang] function add_months, months_between and some fixes
1a68e03 [Daoyuan Wang] poc of time interval calculation
c506661 [Daoyuan Wang] function date_add , date_sub
2015-07-30 16:21:46 -04:00
|
|
|
return Column(sc._jvm.functions.quarter(_to_java_column(col)))
|
2015-07-19 01:48:05 -04:00
|
|
|
|
|
|
|
|
|
|
|
def month(col):
|
|
|
|
"""
|
|
|
|
Extract the month of a given date as integer.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2017-07-08 02:59:34 -04:00
|
|
|
>>> df = spark.createDataFrame([('2015-04-08',)], ['dt'])
|
|
|
|
>>> df.select(month('dt').alias('month')).collect()
|
2015-07-19 01:48:05 -04:00
|
|
|
[Row(month=4)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
[SPARK-8186] [SPARK-8187] [SPARK-8194] [SPARK-8198] [SPARK-9133] [SPARK-9290] [SQL] functions: date_add, date_sub, add_months, months_between, time-interval calculation
This PR is based on #7589 , thanks to adrian-wang
Added SQL function date_add, date_sub, add_months, month_between, also add a rule for
add/subtract of date/timestamp and interval.
Closes #7589
cc rxin
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Author: Davies Liu <davies@databricks.com>
Closes #7754 from davies/date_add and squashes the following commits:
e8c633a [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
9e8e085 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
6224ce4 [Davies Liu] fix conclict
bd18cd4 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
e47ff2c [Davies Liu] add python api, fix date functions
01943d0 [Davies Liu] Merge branch 'master' into date_add
522e91a [Daoyuan Wang] fix
e8a639a [Daoyuan Wang] fix
42df486 [Daoyuan Wang] fix style
87c4b77 [Daoyuan Wang] function add_months, months_between and some fixes
1a68e03 [Daoyuan Wang] poc of time interval calculation
c506661 [Daoyuan Wang] function date_add , date_sub
2015-07-30 16:21:46 -04:00
|
|
|
return Column(sc._jvm.functions.month(_to_java_column(col)))
|
2015-07-19 01:48:05 -04:00
|
|
|
|
|
|
|
|
2017-11-09 00:44:39 -05:00
|
|
|
def dayofweek(col):
|
|
|
|
"""
|
|
|
|
Extract the day of the week of a given date as integer.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.3.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2017-11-09 00:44:39 -05:00
|
|
|
>>> df = spark.createDataFrame([('2015-04-08',)], ['dt'])
|
|
|
|
>>> df.select(dayofweek('dt').alias('day')).collect()
|
|
|
|
[Row(day=4)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.dayofweek(_to_java_column(col)))
|
|
|
|
|
|
|
|
|
2015-07-19 04:17:22 -04:00
|
|
|
def dayofmonth(col):
|
2015-07-19 01:48:05 -04:00
|
|
|
"""
|
|
|
|
Extract the day of the month of a given date as integer.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2017-07-08 02:59:34 -04:00
|
|
|
>>> df = spark.createDataFrame([('2015-04-08',)], ['dt'])
|
|
|
|
>>> df.select(dayofmonth('dt').alias('day')).collect()
|
2015-07-19 01:48:05 -04:00
|
|
|
[Row(day=8)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
[SPARK-8186] [SPARK-8187] [SPARK-8194] [SPARK-8198] [SPARK-9133] [SPARK-9290] [SQL] functions: date_add, date_sub, add_months, months_between, time-interval calculation
This PR is based on #7589 , thanks to adrian-wang
Added SQL function date_add, date_sub, add_months, month_between, also add a rule for
add/subtract of date/timestamp and interval.
Closes #7589
cc rxin
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Author: Davies Liu <davies@databricks.com>
Closes #7754 from davies/date_add and squashes the following commits:
e8c633a [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
9e8e085 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
6224ce4 [Davies Liu] fix conclict
bd18cd4 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
e47ff2c [Davies Liu] add python api, fix date functions
01943d0 [Davies Liu] Merge branch 'master' into date_add
522e91a [Daoyuan Wang] fix
e8a639a [Daoyuan Wang] fix
42df486 [Daoyuan Wang] fix style
87c4b77 [Daoyuan Wang] function add_months, months_between and some fixes
1a68e03 [Daoyuan Wang] poc of time interval calculation
c506661 [Daoyuan Wang] function date_add , date_sub
2015-07-30 16:21:46 -04:00
|
|
|
return Column(sc._jvm.functions.dayofmonth(_to_java_column(col)))
|
2015-07-19 01:48:05 -04:00
|
|
|
|
|
|
|
|
2015-07-19 04:17:22 -04:00
|
|
|
def dayofyear(col):
|
2015-07-19 01:48:05 -04:00
|
|
|
"""
|
|
|
|
Extract the day of the year of a given date as integer.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2017-07-08 02:59:34 -04:00
|
|
|
>>> df = spark.createDataFrame([('2015-04-08',)], ['dt'])
|
|
|
|
>>> df.select(dayofyear('dt').alias('day')).collect()
|
2015-07-19 01:48:05 -04:00
|
|
|
[Row(day=98)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
[SPARK-8186] [SPARK-8187] [SPARK-8194] [SPARK-8198] [SPARK-9133] [SPARK-9290] [SQL] functions: date_add, date_sub, add_months, months_between, time-interval calculation
This PR is based on #7589 , thanks to adrian-wang
Added SQL function date_add, date_sub, add_months, month_between, also add a rule for
add/subtract of date/timestamp and interval.
Closes #7589
cc rxin
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Author: Davies Liu <davies@databricks.com>
Closes #7754 from davies/date_add and squashes the following commits:
e8c633a [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
9e8e085 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
6224ce4 [Davies Liu] fix conclict
bd18cd4 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
e47ff2c [Davies Liu] add python api, fix date functions
01943d0 [Davies Liu] Merge branch 'master' into date_add
522e91a [Daoyuan Wang] fix
e8a639a [Daoyuan Wang] fix
42df486 [Daoyuan Wang] fix style
87c4b77 [Daoyuan Wang] function add_months, months_between and some fixes
1a68e03 [Daoyuan Wang] poc of time interval calculation
c506661 [Daoyuan Wang] function date_add , date_sub
2015-07-30 16:21:46 -04:00
|
|
|
return Column(sc._jvm.functions.dayofyear(_to_java_column(col)))
|
2015-07-19 01:48:05 -04:00
|
|
|
|
|
|
|
|
|
|
|
def hour(col):
|
|
|
|
"""
|
|
|
|
Extract the hours of a given date as integer.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2017-07-08 02:59:34 -04:00
|
|
|
>>> df = spark.createDataFrame([('2015-04-08 13:08:15',)], ['ts'])
|
|
|
|
>>> df.select(hour('ts').alias('hour')).collect()
|
2015-07-19 01:48:05 -04:00
|
|
|
[Row(hour=13)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
[SPARK-8186] [SPARK-8187] [SPARK-8194] [SPARK-8198] [SPARK-9133] [SPARK-9290] [SQL] functions: date_add, date_sub, add_months, months_between, time-interval calculation
This PR is based on #7589 , thanks to adrian-wang
Added SQL function date_add, date_sub, add_months, month_between, also add a rule for
add/subtract of date/timestamp and interval.
Closes #7589
cc rxin
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Author: Davies Liu <davies@databricks.com>
Closes #7754 from davies/date_add and squashes the following commits:
e8c633a [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
9e8e085 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
6224ce4 [Davies Liu] fix conclict
bd18cd4 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
e47ff2c [Davies Liu] add python api, fix date functions
01943d0 [Davies Liu] Merge branch 'master' into date_add
522e91a [Daoyuan Wang] fix
e8a639a [Daoyuan Wang] fix
42df486 [Daoyuan Wang] fix style
87c4b77 [Daoyuan Wang] function add_months, months_between and some fixes
1a68e03 [Daoyuan Wang] poc of time interval calculation
c506661 [Daoyuan Wang] function date_add , date_sub
2015-07-30 16:21:46 -04:00
|
|
|
return Column(sc._jvm.functions.hour(_to_java_column(col)))
|
2015-07-19 01:48:05 -04:00
|
|
|
|
|
|
|
|
|
|
|
def minute(col):
|
|
|
|
"""
|
|
|
|
Extract the minutes of a given date as integer.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2017-07-08 02:59:34 -04:00
|
|
|
>>> df = spark.createDataFrame([('2015-04-08 13:08:15',)], ['ts'])
|
|
|
|
>>> df.select(minute('ts').alias('minute')).collect()
|
2015-07-19 01:48:05 -04:00
|
|
|
[Row(minute=8)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
[SPARK-8186] [SPARK-8187] [SPARK-8194] [SPARK-8198] [SPARK-9133] [SPARK-9290] [SQL] functions: date_add, date_sub, add_months, months_between, time-interval calculation
This PR is based on #7589 , thanks to adrian-wang
Added SQL function date_add, date_sub, add_months, month_between, also add a rule for
add/subtract of date/timestamp and interval.
Closes #7589
cc rxin
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Author: Davies Liu <davies@databricks.com>
Closes #7754 from davies/date_add and squashes the following commits:
e8c633a [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
9e8e085 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
6224ce4 [Davies Liu] fix conclict
bd18cd4 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
e47ff2c [Davies Liu] add python api, fix date functions
01943d0 [Davies Liu] Merge branch 'master' into date_add
522e91a [Daoyuan Wang] fix
e8a639a [Daoyuan Wang] fix
42df486 [Daoyuan Wang] fix style
87c4b77 [Daoyuan Wang] function add_months, months_between and some fixes
1a68e03 [Daoyuan Wang] poc of time interval calculation
c506661 [Daoyuan Wang] function date_add , date_sub
2015-07-30 16:21:46 -04:00
|
|
|
return Column(sc._jvm.functions.minute(_to_java_column(col)))
|
2015-07-19 01:48:05 -04:00
|
|
|
|
|
|
|
|
|
|
|
def second(col):
|
|
|
|
"""
|
|
|
|
Extract the seconds of a given date as integer.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2017-07-08 02:59:34 -04:00
|
|
|
>>> df = spark.createDataFrame([('2015-04-08 13:08:15',)], ['ts'])
|
|
|
|
>>> df.select(second('ts').alias('second')).collect()
|
2015-07-19 01:48:05 -04:00
|
|
|
[Row(second=15)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
[SPARK-8186] [SPARK-8187] [SPARK-8194] [SPARK-8198] [SPARK-9133] [SPARK-9290] [SQL] functions: date_add, date_sub, add_months, months_between, time-interval calculation
This PR is based on #7589 , thanks to adrian-wang
Added SQL function date_add, date_sub, add_months, month_between, also add a rule for
add/subtract of date/timestamp and interval.
Closes #7589
cc rxin
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Author: Davies Liu <davies@databricks.com>
Closes #7754 from davies/date_add and squashes the following commits:
e8c633a [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
9e8e085 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
6224ce4 [Davies Liu] fix conclict
bd18cd4 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
e47ff2c [Davies Liu] add python api, fix date functions
01943d0 [Davies Liu] Merge branch 'master' into date_add
522e91a [Daoyuan Wang] fix
e8a639a [Daoyuan Wang] fix
42df486 [Daoyuan Wang] fix style
87c4b77 [Daoyuan Wang] function add_months, months_between and some fixes
1a68e03 [Daoyuan Wang] poc of time interval calculation
c506661 [Daoyuan Wang] function date_add , date_sub
2015-07-30 16:21:46 -04:00
|
|
|
return Column(sc._jvm.functions.second(_to_java_column(col)))
|
2015-07-19 01:48:05 -04:00
|
|
|
|
|
|
|
|
2015-07-19 04:17:22 -04:00
|
|
|
def weekofyear(col):
|
2015-07-19 01:48:05 -04:00
|
|
|
"""
|
|
|
|
Extract the week number of a given date as integer.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2017-07-08 02:59:34 -04:00
|
|
|
>>> df = spark.createDataFrame([('2015-04-08',)], ['dt'])
|
|
|
|
>>> df.select(weekofyear(df.dt).alias('week')).collect()
|
2015-07-19 01:48:05 -04:00
|
|
|
[Row(week=15)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
[SPARK-8186] [SPARK-8187] [SPARK-8194] [SPARK-8198] [SPARK-9133] [SPARK-9290] [SQL] functions: date_add, date_sub, add_months, months_between, time-interval calculation
This PR is based on #7589 , thanks to adrian-wang
Added SQL function date_add, date_sub, add_months, month_between, also add a rule for
add/subtract of date/timestamp and interval.
Closes #7589
cc rxin
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Author: Davies Liu <davies@databricks.com>
Closes #7754 from davies/date_add and squashes the following commits:
e8c633a [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
9e8e085 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
6224ce4 [Davies Liu] fix conclict
bd18cd4 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
e47ff2c [Davies Liu] add python api, fix date functions
01943d0 [Davies Liu] Merge branch 'master' into date_add
522e91a [Daoyuan Wang] fix
e8a639a [Daoyuan Wang] fix
42df486 [Daoyuan Wang] fix style
87c4b77 [Daoyuan Wang] function add_months, months_between and some fixes
1a68e03 [Daoyuan Wang] poc of time interval calculation
c506661 [Daoyuan Wang] function date_add , date_sub
2015-07-30 16:21:46 -04:00
|
|
|
return Column(sc._jvm.functions.weekofyear(_to_java_column(col)))
|
|
|
|
|
|
|
|
|
|
|
|
def date_add(start, days):
|
|
|
|
"""
|
|
|
|
Returns the date that is `days` days after `start`
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2017-07-08 02:59:34 -04:00
|
|
|
>>> df = spark.createDataFrame([('2015-04-08',)], ['dt'])
|
|
|
|
>>> df.select(date_add(df.dt, 1).alias('next_date')).collect()
|
|
|
|
[Row(next_date=datetime.date(2015, 4, 9))]
|
[SPARK-8186] [SPARK-8187] [SPARK-8194] [SPARK-8198] [SPARK-9133] [SPARK-9290] [SQL] functions: date_add, date_sub, add_months, months_between, time-interval calculation
This PR is based on #7589 , thanks to adrian-wang
Added SQL function date_add, date_sub, add_months, month_between, also add a rule for
add/subtract of date/timestamp and interval.
Closes #7589
cc rxin
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Author: Davies Liu <davies@databricks.com>
Closes #7754 from davies/date_add and squashes the following commits:
e8c633a [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
9e8e085 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
6224ce4 [Davies Liu] fix conclict
bd18cd4 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
e47ff2c [Davies Liu] add python api, fix date functions
01943d0 [Davies Liu] Merge branch 'master' into date_add
522e91a [Daoyuan Wang] fix
e8a639a [Daoyuan Wang] fix
42df486 [Daoyuan Wang] fix style
87c4b77 [Daoyuan Wang] function add_months, months_between and some fixes
1a68e03 [Daoyuan Wang] poc of time interval calculation
c506661 [Daoyuan Wang] function date_add , date_sub
2015-07-30 16:21:46 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.date_add(_to_java_column(start), days))
|
|
|
|
|
|
|
|
|
|
|
|
def date_sub(start, days):
|
|
|
|
"""
|
|
|
|
Returns the date that is `days` days before `start`
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2017-07-08 02:59:34 -04:00
|
|
|
>>> df = spark.createDataFrame([('2015-04-08',)], ['dt'])
|
|
|
|
>>> df.select(date_sub(df.dt, 1).alias('prev_date')).collect()
|
|
|
|
[Row(prev_date=datetime.date(2015, 4, 7))]
|
[SPARK-8186] [SPARK-8187] [SPARK-8194] [SPARK-8198] [SPARK-9133] [SPARK-9290] [SQL] functions: date_add, date_sub, add_months, months_between, time-interval calculation
This PR is based on #7589 , thanks to adrian-wang
Added SQL function date_add, date_sub, add_months, month_between, also add a rule for
add/subtract of date/timestamp and interval.
Closes #7589
cc rxin
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Author: Davies Liu <davies@databricks.com>
Closes #7754 from davies/date_add and squashes the following commits:
e8c633a [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
9e8e085 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
6224ce4 [Davies Liu] fix conclict
bd18cd4 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
e47ff2c [Davies Liu] add python api, fix date functions
01943d0 [Davies Liu] Merge branch 'master' into date_add
522e91a [Daoyuan Wang] fix
e8a639a [Daoyuan Wang] fix
42df486 [Daoyuan Wang] fix style
87c4b77 [Daoyuan Wang] function add_months, months_between and some fixes
1a68e03 [Daoyuan Wang] poc of time interval calculation
c506661 [Daoyuan Wang] function date_add , date_sub
2015-07-30 16:21:46 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.date_sub(_to_java_column(start), days))
|
|
|
|
|
|
|
|
|
2015-08-04 22:25:24 -04:00
|
|
|
def datediff(end, start):
|
|
|
|
"""
|
|
|
|
Returns the number of days from `start` to `end`.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> df = spark.createDataFrame([('2015-04-08','2015-05-10')], ['d1', 'd2'])
|
2015-08-04 22:25:24 -04:00
|
|
|
>>> df.select(datediff(df.d2, df.d1).alias('diff')).collect()
|
|
|
|
[Row(diff=32)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.datediff(_to_java_column(end), _to_java_column(start)))
|
|
|
|
|
|
|
|
|
[SPARK-8186] [SPARK-8187] [SPARK-8194] [SPARK-8198] [SPARK-9133] [SPARK-9290] [SQL] functions: date_add, date_sub, add_months, months_between, time-interval calculation
This PR is based on #7589 , thanks to adrian-wang
Added SQL function date_add, date_sub, add_months, month_between, also add a rule for
add/subtract of date/timestamp and interval.
Closes #7589
cc rxin
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Author: Davies Liu <davies@databricks.com>
Closes #7754 from davies/date_add and squashes the following commits:
e8c633a [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
9e8e085 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
6224ce4 [Davies Liu] fix conclict
bd18cd4 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
e47ff2c [Davies Liu] add python api, fix date functions
01943d0 [Davies Liu] Merge branch 'master' into date_add
522e91a [Daoyuan Wang] fix
e8a639a [Daoyuan Wang] fix
42df486 [Daoyuan Wang] fix style
87c4b77 [Daoyuan Wang] function add_months, months_between and some fixes
1a68e03 [Daoyuan Wang] poc of time interval calculation
c506661 [Daoyuan Wang] function date_add , date_sub
2015-07-30 16:21:46 -04:00
|
|
|
def add_months(start, months):
|
|
|
|
"""
|
|
|
|
Returns the date that is `months` months after `start`
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2017-07-08 02:59:34 -04:00
|
|
|
>>> df = spark.createDataFrame([('2015-04-08',)], ['dt'])
|
|
|
|
>>> df.select(add_months(df.dt, 1).alias('next_month')).collect()
|
|
|
|
[Row(next_month=datetime.date(2015, 5, 8))]
|
[SPARK-8186] [SPARK-8187] [SPARK-8194] [SPARK-8198] [SPARK-9133] [SPARK-9290] [SQL] functions: date_add, date_sub, add_months, months_between, time-interval calculation
This PR is based on #7589 , thanks to adrian-wang
Added SQL function date_add, date_sub, add_months, month_between, also add a rule for
add/subtract of date/timestamp and interval.
Closes #7589
cc rxin
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Author: Davies Liu <davies@databricks.com>
Closes #7754 from davies/date_add and squashes the following commits:
e8c633a [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
9e8e085 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
6224ce4 [Davies Liu] fix conclict
bd18cd4 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
e47ff2c [Davies Liu] add python api, fix date functions
01943d0 [Davies Liu] Merge branch 'master' into date_add
522e91a [Daoyuan Wang] fix
e8a639a [Daoyuan Wang] fix
42df486 [Daoyuan Wang] fix style
87c4b77 [Daoyuan Wang] function add_months, months_between and some fixes
1a68e03 [Daoyuan Wang] poc of time interval calculation
c506661 [Daoyuan Wang] function date_add , date_sub
2015-07-30 16:21:46 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.add_months(_to_java_column(start), months))
|
|
|
|
|
|
|
|
|
2018-04-25 23:19:20 -04:00
|
|
|
def months_between(date1, date2, roundOff=True):
|
[SPARK-8186] [SPARK-8187] [SPARK-8194] [SPARK-8198] [SPARK-9133] [SPARK-9290] [SQL] functions: date_add, date_sub, add_months, months_between, time-interval calculation
This PR is based on #7589 , thanks to adrian-wang
Added SQL function date_add, date_sub, add_months, month_between, also add a rule for
add/subtract of date/timestamp and interval.
Closes #7589
cc rxin
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Author: Davies Liu <davies@databricks.com>
Closes #7754 from davies/date_add and squashes the following commits:
e8c633a [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
9e8e085 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
6224ce4 [Davies Liu] fix conclict
bd18cd4 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
e47ff2c [Davies Liu] add python api, fix date functions
01943d0 [Davies Liu] Merge branch 'master' into date_add
522e91a [Daoyuan Wang] fix
e8a639a [Daoyuan Wang] fix
42df486 [Daoyuan Wang] fix style
87c4b77 [Daoyuan Wang] function add_months, months_between and some fixes
1a68e03 [Daoyuan Wang] poc of time interval calculation
c506661 [Daoyuan Wang] function date_add , date_sub
2015-07-30 16:21:46 -04:00
|
|
|
"""
|
2018-05-11 15:42:23 -04:00
|
|
|
Returns number of months between dates date1 and date2.
|
|
|
|
If date1 is later than date2, then the result is positive.
|
|
|
|
If date1 and date2 are on the same day of month, or both are the last day of month,
|
|
|
|
returns an integer (time of day will be ignored).
|
|
|
|
The result is rounded off to 8 digits unless `roundOff` is set to `False`.
|
[SPARK-8186] [SPARK-8187] [SPARK-8194] [SPARK-8198] [SPARK-9133] [SPARK-9290] [SQL] functions: date_add, date_sub, add_months, months_between, time-interval calculation
This PR is based on #7589 , thanks to adrian-wang
Added SQL function date_add, date_sub, add_months, month_between, also add a rule for
add/subtract of date/timestamp and interval.
Closes #7589
cc rxin
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Author: Davies Liu <davies@databricks.com>
Closes #7754 from davies/date_add and squashes the following commits:
e8c633a [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
9e8e085 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
6224ce4 [Davies Liu] fix conclict
bd18cd4 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
e47ff2c [Davies Liu] add python api, fix date functions
01943d0 [Davies Liu] Merge branch 'master' into date_add
522e91a [Daoyuan Wang] fix
e8a639a [Daoyuan Wang] fix
42df486 [Daoyuan Wang] fix style
87c4b77 [Daoyuan Wang] function add_months, months_between and some fixes
1a68e03 [Daoyuan Wang] poc of time interval calculation
c506661 [Daoyuan Wang] function date_add , date_sub
2015-07-30 16:21:46 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2017-07-08 02:59:34 -04:00
|
|
|
>>> df = spark.createDataFrame([('1997-02-28 10:30:00', '1996-10-30')], ['date1', 'date2'])
|
|
|
|
>>> df.select(months_between(df.date1, df.date2).alias('months')).collect()
|
2018-04-25 23:19:20 -04:00
|
|
|
[Row(months=3.94959677)]
|
|
|
|
>>> df.select(months_between(df.date1, df.date2, False).alias('months')).collect()
|
|
|
|
[Row(months=3.9495967741935485)]
|
[SPARK-8186] [SPARK-8187] [SPARK-8194] [SPARK-8198] [SPARK-9133] [SPARK-9290] [SQL] functions: date_add, date_sub, add_months, months_between, time-interval calculation
This PR is based on #7589 , thanks to adrian-wang
Added SQL function date_add, date_sub, add_months, month_between, also add a rule for
add/subtract of date/timestamp and interval.
Closes #7589
cc rxin
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Author: Davies Liu <davies@databricks.com>
Closes #7754 from davies/date_add and squashes the following commits:
e8c633a [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
9e8e085 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
6224ce4 [Davies Liu] fix conclict
bd18cd4 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
e47ff2c [Davies Liu] add python api, fix date functions
01943d0 [Davies Liu] Merge branch 'master' into date_add
522e91a [Daoyuan Wang] fix
e8a639a [Daoyuan Wang] fix
42df486 [Daoyuan Wang] fix style
87c4b77 [Daoyuan Wang] function add_months, months_between and some fixes
1a68e03 [Daoyuan Wang] poc of time interval calculation
c506661 [Daoyuan Wang] function date_add , date_sub
2015-07-30 16:21:46 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
2018-04-25 23:19:20 -04:00
|
|
|
return Column(sc._jvm.functions.months_between(
|
|
|
|
_to_java_column(date1), _to_java_column(date2), roundOff))
|
2015-07-19 01:48:05 -04:00
|
|
|
|
|
|
|
|
2017-02-07 09:50:30 -05:00
|
|
|
def to_date(col, format=None):
|
2021-02-01 19:30:50 -05:00
|
|
|
"""Converts a :class:`~pyspark.sql.Column` into :class:`pyspark.sql.types.DateType`
|
2020-03-20 09:59:26 -04:00
|
|
|
using the optionally specified format. Specify formats according to `datetime pattern`_.
|
[SPARK-20639][SQL] Add single argument support for to_timestamp in SQL with documentation improvement
## What changes were proposed in this pull request?
This PR proposes three things as below:
- Use casting rules to a timestamp in `to_timestamp` by default (it was `yyyy-MM-dd HH:mm:ss`).
- Support single argument for `to_timestamp` similarly with APIs in other languages.
For example, the one below works
```
import org.apache.spark.sql.functions._
Seq("2016-12-31 00:12:00.00").toDF("a").select(to_timestamp(col("a"))).show()
```
prints
```
+----------------------------------------+
|to_timestamp(`a`, 'yyyy-MM-dd HH:mm:ss')|
+----------------------------------------+
| 2016-12-31 00:12:00|
+----------------------------------------+
```
whereas this does not work in SQL.
**Before**
```
spark-sql> SELECT to_timestamp('2016-12-31 00:12:00');
Error in query: Invalid number of arguments for function to_timestamp; line 1 pos 7
```
**After**
```
spark-sql> SELECT to_timestamp('2016-12-31 00:12:00');
2016-12-31 00:12:00
```
- Related document improvement for SQL function descriptions and other API descriptions accordingly.
**Before**
```
spark-sql> DESCRIBE FUNCTION extended to_date;
...
Usage: to_date(date_str, fmt) - Parses the `left` expression with the `fmt` expression. Returns null with invalid input.
Extended Usage:
Examples:
> SELECT to_date('2016-12-31', 'yyyy-MM-dd');
2016-12-31
```
```
spark-sql> DESCRIBE FUNCTION extended to_timestamp;
...
Usage: to_timestamp(timestamp, fmt) - Parses the `left` expression with the `format` expression to a timestamp. Returns null with invalid input.
Extended Usage:
Examples:
> SELECT to_timestamp('2016-12-31', 'yyyy-MM-dd');
2016-12-31 00:00:00.0
```
**After**
```
spark-sql> DESCRIBE FUNCTION extended to_date;
...
Usage:
to_date(date_str[, fmt]) - Parses the `date_str` expression with the `fmt` expression to
a date. Returns null with invalid input. By default, it follows casting rules to a date if
the `fmt` is omitted.
Extended Usage:
Examples:
> SELECT to_date('2009-07-30 04:17:52');
2009-07-30
> SELECT to_date('2016-12-31', 'yyyy-MM-dd');
2016-12-31
```
```
spark-sql> DESCRIBE FUNCTION extended to_timestamp;
...
Usage:
to_timestamp(timestamp[, fmt]) - Parses the `timestamp` expression with the `fmt` expression to
a timestamp. Returns null with invalid input. By default, it follows casting rules to
a timestamp if the `fmt` is omitted.
Extended Usage:
Examples:
> SELECT to_timestamp('2016-12-31 00:12:00');
2016-12-31 00:12:00
> SELECT to_timestamp('2016-12-31', 'yyyy-MM-dd');
2016-12-31 00:00:00
```
## How was this patch tested?
Added tests in `datetime.sql`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #17901 from HyukjinKwon/to_timestamp_arg.
2017-05-12 04:42:58 -04:00
|
|
|
By default, it follows casting rules to :class:`pyspark.sql.types.DateType` if the format
|
2020-01-23 23:10:09 -05:00
|
|
|
is omitted. Equivalent to ``col.cast("date")``.
|
2015-07-30 22:22:38 -04:00
|
|
|
|
2020-07-27 04:49:21 -04:00
|
|
|
.. _datetime pattern: https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.2.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> df = spark.createDataFrame([('1997-02-28 10:30:00',)], ['t'])
|
2015-07-30 22:22:38 -04:00
|
|
|
>>> df.select(to_date(df.t).alias('date')).collect()
|
|
|
|
[Row(date=datetime.date(1997, 2, 28))]
|
2017-02-07 09:50:30 -05:00
|
|
|
|
|
|
|
>>> df = spark.createDataFrame([('1997-02-28 10:30:00',)], ['t'])
|
|
|
|
>>> df.select(to_date(df.t, 'yyyy-MM-dd HH:mm:ss').alias('date')).collect()
|
|
|
|
[Row(date=datetime.date(1997, 2, 28))]
|
2015-07-30 22:22:38 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
2017-02-07 09:50:30 -05:00
|
|
|
if format is None:
|
|
|
|
jc = sc._jvm.functions.to_date(_to_java_column(col))
|
|
|
|
else:
|
|
|
|
jc = sc._jvm.functions.to_date(_to_java_column(col), format)
|
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
|
|
|
def to_timestamp(col, format=None):
|
2021-02-01 19:30:50 -05:00
|
|
|
"""Converts a :class:`~pyspark.sql.Column` into :class:`pyspark.sql.types.TimestampType`
|
2020-03-20 09:59:26 -04:00
|
|
|
using the optionally specified format. Specify formats according to `datetime pattern`_.
|
[SPARK-20639][SQL] Add single argument support for to_timestamp in SQL with documentation improvement
## What changes were proposed in this pull request?
This PR proposes three things as below:
- Use casting rules to a timestamp in `to_timestamp` by default (it was `yyyy-MM-dd HH:mm:ss`).
- Support single argument for `to_timestamp` similarly with APIs in other languages.
For example, the one below works
```
import org.apache.spark.sql.functions._
Seq("2016-12-31 00:12:00.00").toDF("a").select(to_timestamp(col("a"))).show()
```
prints
```
+----------------------------------------+
|to_timestamp(`a`, 'yyyy-MM-dd HH:mm:ss')|
+----------------------------------------+
| 2016-12-31 00:12:00|
+----------------------------------------+
```
whereas this does not work in SQL.
**Before**
```
spark-sql> SELECT to_timestamp('2016-12-31 00:12:00');
Error in query: Invalid number of arguments for function to_timestamp; line 1 pos 7
```
**After**
```
spark-sql> SELECT to_timestamp('2016-12-31 00:12:00');
2016-12-31 00:12:00
```
- Related document improvement for SQL function descriptions and other API descriptions accordingly.
**Before**
```
spark-sql> DESCRIBE FUNCTION extended to_date;
...
Usage: to_date(date_str, fmt) - Parses the `left` expression with the `fmt` expression. Returns null with invalid input.
Extended Usage:
Examples:
> SELECT to_date('2016-12-31', 'yyyy-MM-dd');
2016-12-31
```
```
spark-sql> DESCRIBE FUNCTION extended to_timestamp;
...
Usage: to_timestamp(timestamp, fmt) - Parses the `left` expression with the `format` expression to a timestamp. Returns null with invalid input.
Extended Usage:
Examples:
> SELECT to_timestamp('2016-12-31', 'yyyy-MM-dd');
2016-12-31 00:00:00.0
```
**After**
```
spark-sql> DESCRIBE FUNCTION extended to_date;
...
Usage:
to_date(date_str[, fmt]) - Parses the `date_str` expression with the `fmt` expression to
a date. Returns null with invalid input. By default, it follows casting rules to a date if
the `fmt` is omitted.
Extended Usage:
Examples:
> SELECT to_date('2009-07-30 04:17:52');
2009-07-30
> SELECT to_date('2016-12-31', 'yyyy-MM-dd');
2016-12-31
```
```
spark-sql> DESCRIBE FUNCTION extended to_timestamp;
...
Usage:
to_timestamp(timestamp[, fmt]) - Parses the `timestamp` expression with the `fmt` expression to
a timestamp. Returns null with invalid input. By default, it follows casting rules to
a timestamp if the `fmt` is omitted.
Extended Usage:
Examples:
> SELECT to_timestamp('2016-12-31 00:12:00');
2016-12-31 00:12:00
> SELECT to_timestamp('2016-12-31', 'yyyy-MM-dd');
2016-12-31 00:00:00
```
## How was this patch tested?
Added tests in `datetime.sql`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #17901 from HyukjinKwon/to_timestamp_arg.
2017-05-12 04:42:58 -04:00
|
|
|
By default, it follows casting rules to :class:`pyspark.sql.types.TimestampType` if the format
|
2020-01-23 23:10:09 -05:00
|
|
|
is omitted. Equivalent to ``col.cast("timestamp")``.
|
2017-02-07 09:50:30 -05:00
|
|
|
|
2020-07-27 04:49:21 -04:00
|
|
|
.. _datetime pattern: https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.2.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2017-02-07 09:50:30 -05:00
|
|
|
>>> df = spark.createDataFrame([('1997-02-28 10:30:00',)], ['t'])
|
|
|
|
>>> df.select(to_timestamp(df.t).alias('dt')).collect()
|
|
|
|
[Row(dt=datetime.datetime(1997, 2, 28, 10, 30))]
|
|
|
|
|
|
|
|
>>> df = spark.createDataFrame([('1997-02-28 10:30:00',)], ['t'])
|
|
|
|
>>> df.select(to_timestamp(df.t, 'yyyy-MM-dd HH:mm:ss').alias('dt')).collect()
|
|
|
|
[Row(dt=datetime.datetime(1997, 2, 28, 10, 30))]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
if format is None:
|
|
|
|
jc = sc._jvm.functions.to_timestamp(_to_java_column(col))
|
|
|
|
else:
|
|
|
|
jc = sc._jvm.functions.to_timestamp(_to_java_column(col), format)
|
|
|
|
return Column(jc)
|
2015-07-30 22:22:38 -04:00
|
|
|
|
|
|
|
|
|
|
|
def trunc(date, format):
|
|
|
|
"""
|
|
|
|
Returns date truncated to the unit specified by the format.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
date : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
format : str
|
|
|
|
'year', 'yyyy', 'yy' or 'month', 'mon', 'mm'
|
2015-07-30 22:22:38 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> df = spark.createDataFrame([('1997-02-28',)], ['d'])
|
2015-07-30 22:22:38 -04:00
|
|
|
>>> df.select(trunc(df.d, 'year').alias('year')).collect()
|
|
|
|
[Row(year=datetime.date(1997, 1, 1))]
|
|
|
|
>>> df.select(trunc(df.d, 'mon').alias('month')).collect()
|
|
|
|
[Row(month=datetime.date(1997, 2, 1))]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.trunc(_to_java_column(date), format))
|
|
|
|
|
|
|
|
|
2017-12-19 23:22:33 -05:00
|
|
|
def date_trunc(format, timestamp):
|
|
|
|
"""
|
|
|
|
Returns timestamp truncated to the unit specified by the format.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.3.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
format : str
|
|
|
|
'year', 'yyyy', 'yy', 'month', 'mon', 'mm',
|
2017-12-19 23:22:33 -05:00
|
|
|
'day', 'dd', 'hour', 'minute', 'second', 'week', 'quarter'
|
2021-02-01 19:30:50 -05:00
|
|
|
timestamp : :class:`~pyspark.sql.Column` or str
|
2017-12-19 23:22:33 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2017-12-19 23:22:33 -05:00
|
|
|
>>> df = spark.createDataFrame([('1997-02-28 05:02:11',)], ['t'])
|
|
|
|
>>> df.select(date_trunc('year', df.t).alias('year')).collect()
|
|
|
|
[Row(year=datetime.datetime(1997, 1, 1, 0, 0))]
|
|
|
|
>>> df.select(date_trunc('mon', df.t).alias('month')).collect()
|
|
|
|
[Row(month=datetime.datetime(1997, 2, 1, 0, 0))]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.date_trunc(format, _to_java_column(timestamp)))
|
|
|
|
|
|
|
|
|
2015-08-04 22:25:24 -04:00
|
|
|
def next_day(date, dayOfWeek):
|
2015-08-01 11:48:46 -04:00
|
|
|
"""
|
2015-08-04 22:25:24 -04:00
|
|
|
Returns the first date which is later than the value of the date column.
|
2015-08-01 11:48:46 -04:00
|
|
|
|
2015-08-04 22:25:24 -04:00
|
|
|
Day of the week parameter is case insensitive, and accepts:
|
|
|
|
"Mon", "Tue", "Wed", "Thu", "Fri", "Sat", "Sun".
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> df = spark.createDataFrame([('2015-07-27',)], ['d'])
|
2015-08-04 22:25:24 -04:00
|
|
|
>>> df.select(next_day(df.d, 'Sun').alias('date')).collect()
|
|
|
|
[Row(date=datetime.date(2015, 8, 2))]
|
2015-08-01 11:48:46 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
2015-08-04 22:25:24 -04:00
|
|
|
return Column(sc._jvm.functions.next_day(_to_java_column(date), dayOfWeek))
|
2015-08-01 11:48:46 -04:00
|
|
|
|
|
|
|
|
2015-08-04 22:25:24 -04:00
|
|
|
def last_day(date):
|
2015-08-01 00:18:01 -04:00
|
|
|
"""
|
2015-08-04 22:25:24 -04:00
|
|
|
Returns the last day of the month which the given date belongs to.
|
2015-08-01 00:18:01 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> df = spark.createDataFrame([('1997-02-10',)], ['d'])
|
2015-08-04 22:25:24 -04:00
|
|
|
>>> df.select(last_day(df.d).alias('date')).collect()
|
|
|
|
[Row(date=datetime.date(1997, 2, 28))]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.last_day(_to_java_column(date)))
|
|
|
|
|
|
|
|
|
2020-03-11 02:11:13 -04:00
|
|
|
def from_unixtime(timestamp, format="yyyy-MM-dd HH:mm:ss"):
|
2015-08-04 22:25:24 -04:00
|
|
|
"""
|
|
|
|
Converts the number of seconds from unix epoch (1970-01-01 00:00:00 UTC) to a string
|
|
|
|
representing the timestamp of that moment in the current system time zone in the given
|
|
|
|
format.
|
2017-07-08 02:59:34 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2017-07-11 02:23:03 -04:00
|
|
|
>>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles")
|
2017-07-08 02:59:34 -04:00
|
|
|
>>> time_df = spark.createDataFrame([(1428476400,)], ['unix_time'])
|
|
|
|
>>> time_df.select(from_unixtime('unix_time').alias('ts')).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(ts='2015-04-08 00:00:00')]
|
2017-07-11 02:23:03 -04:00
|
|
|
>>> spark.conf.unset("spark.sql.session.timeZone")
|
2015-08-04 22:25:24 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.from_unixtime(_to_java_column(timestamp), format))
|
|
|
|
|
|
|
|
|
2020-03-11 02:11:13 -04:00
|
|
|
def unix_timestamp(timestamp=None, format='yyyy-MM-dd HH:mm:ss'):
|
2015-08-04 22:25:24 -04:00
|
|
|
"""
|
2020-03-11 02:11:13 -04:00
|
|
|
Convert time string with given pattern ('yyyy-MM-dd HH:mm:ss', by default)
|
2015-08-04 22:25:24 -04:00
|
|
|
to Unix time stamp (in seconds), using the default timezone and the default
|
|
|
|
locale, return null if fail.
|
|
|
|
|
|
|
|
if `timestamp` is None, then it returns current timestamp.
|
2017-07-08 02:59:34 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2017-07-11 02:23:03 -04:00
|
|
|
>>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles")
|
2017-07-08 02:59:34 -04:00
|
|
|
>>> time_df = spark.createDataFrame([('2015-04-08',)], ['dt'])
|
|
|
|
>>> time_df.select(unix_timestamp('dt', 'yyyy-MM-dd').alias('unix_time')).collect()
|
|
|
|
[Row(unix_time=1428476400)]
|
2017-07-11 02:23:03 -04:00
|
|
|
>>> spark.conf.unset("spark.sql.session.timeZone")
|
2015-08-04 22:25:24 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
if timestamp is None:
|
|
|
|
return Column(sc._jvm.functions.unix_timestamp())
|
|
|
|
return Column(sc._jvm.functions.unix_timestamp(_to_java_column(timestamp), format))
|
|
|
|
|
|
|
|
|
|
|
|
def from_utc_timestamp(timestamp, tz):
|
|
|
|
"""
|
2018-09-27 03:02:20 -04:00
|
|
|
This is a common function for databases supporting TIMESTAMP WITHOUT TIMEZONE. This function
|
|
|
|
takes a timestamp which is timezone-agnostic, and interprets it as a timestamp in UTC, and
|
|
|
|
renders that timestamp as a timestamp in the given time zone.
|
|
|
|
|
|
|
|
However, timestamp in Spark represents number of microseconds from the Unix epoch, which is not
|
|
|
|
timezone-agnostic. So in Spark this function just shift the timestamp value from UTC timezone to
|
|
|
|
the given timezone.
|
|
|
|
|
|
|
|
This function may return confusing result if the input is a string with timezone, e.g.
|
|
|
|
'2018-03-13T06:18:23+00:00'. The reason is that, Spark firstly cast the string to timestamp
|
|
|
|
according to the timezone in the string, and finally display the result by converting the
|
|
|
|
timestamp to string according to the session local timezone.
|
2015-08-04 22:25:24 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
timestamp : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
the column that contains timestamps
|
2021-02-01 19:30:50 -05:00
|
|
|
tz : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
A string detailing the time zone ID that the input should be adjusted to. It should
|
|
|
|
be in the format of either region-based zone IDs or zone offsets. Region IDs must
|
|
|
|
have the form 'area/city', such as 'America/Los_Angeles'. Zone offsets must be in
|
|
|
|
the format '(+|-)HH:mm', for example '-08:00' or '+01:00'. Also 'UTC' and 'Z' are
|
|
|
|
supported as aliases of '+00:00'. Other short names are not recommended to use
|
|
|
|
because they can be ambiguous.
|
|
|
|
|
|
|
|
.. versionchanged:: 2.4
|
2021-02-01 19:30:50 -05:00
|
|
|
`tz` can take a :class:`~pyspark.sql.Column` containing timezone ID strings.
|
2020-11-02 20:00:49 -05:00
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2018-07-06 06:28:54 -04:00
|
|
|
>>> df = spark.createDataFrame([('1997-02-28 10:30:00', 'JST')], ['ts', 'tz'])
|
|
|
|
>>> df.select(from_utc_timestamp(df.ts, "PST").alias('local_time')).collect()
|
2017-07-08 02:59:34 -04:00
|
|
|
[Row(local_time=datetime.datetime(1997, 2, 28, 2, 30))]
|
2018-07-06 06:28:54 -04:00
|
|
|
>>> df.select(from_utc_timestamp(df.ts, df.tz).alias('local_time')).collect()
|
|
|
|
[Row(local_time=datetime.datetime(1997, 2, 28, 19, 30))]
|
2015-08-04 22:25:24 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
2018-07-06 06:28:54 -04:00
|
|
|
if isinstance(tz, Column):
|
|
|
|
tz = _to_java_column(tz)
|
2015-08-04 22:25:24 -04:00
|
|
|
return Column(sc._jvm.functions.from_utc_timestamp(_to_java_column(timestamp), tz))
|
|
|
|
|
|
|
|
|
|
|
|
def to_utc_timestamp(timestamp, tz):
|
|
|
|
"""
|
2018-09-27 03:02:20 -04:00
|
|
|
This is a common function for databases supporting TIMESTAMP WITHOUT TIMEZONE. This function
|
|
|
|
takes a timestamp which is timezone-agnostic, and interprets it as a timestamp in the given
|
|
|
|
timezone, and renders that timestamp as a timestamp in UTC.
|
|
|
|
|
|
|
|
However, timestamp in Spark represents number of microseconds from the Unix epoch, which is not
|
|
|
|
timezone-agnostic. So in Spark this function just shift the timestamp value from the given
|
|
|
|
timezone to UTC timezone.
|
|
|
|
|
|
|
|
This function may return confusing result if the input is a string with timezone, e.g.
|
|
|
|
'2018-03-13T06:18:23+00:00'. The reason is that, Spark firstly cast the string to timestamp
|
|
|
|
according to the timezone in the string, and finally display the result by converting the
|
|
|
|
timestamp to string according to the session local timezone.
|
2015-08-04 22:25:24 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
timestamp : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
the column that contains timestamps
|
2021-02-01 19:30:50 -05:00
|
|
|
tz : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
A string detailing the time zone ID that the input should be adjusted to. It should
|
|
|
|
be in the format of either region-based zone IDs or zone offsets. Region IDs must
|
|
|
|
have the form 'area/city', such as 'America/Los_Angeles'. Zone offsets must be in
|
|
|
|
the format '(+|-)HH:mm', for example '-08:00' or '+01:00'. Also 'UTC' and 'Z' are
|
|
|
|
upported as aliases of '+00:00'. Other short names are not recommended to use
|
|
|
|
because they can be ambiguous.
|
|
|
|
|
|
|
|
.. versionchanged:: 2.4.0
|
2021-02-01 19:30:50 -05:00
|
|
|
`tz` can take a :class:`~pyspark.sql.Column` containing timezone ID strings.
|
2020-11-02 20:00:49 -05:00
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2018-07-06 06:28:54 -04:00
|
|
|
>>> df = spark.createDataFrame([('1997-02-28 10:30:00', 'JST')], ['ts', 'tz'])
|
2017-07-08 02:59:34 -04:00
|
|
|
>>> df.select(to_utc_timestamp(df.ts, "PST").alias('utc_time')).collect()
|
|
|
|
[Row(utc_time=datetime.datetime(1997, 2, 28, 18, 30))]
|
2018-07-06 06:28:54 -04:00
|
|
|
>>> df.select(to_utc_timestamp(df.ts, df.tz).alias('utc_time')).collect()
|
|
|
|
[Row(utc_time=datetime.datetime(1997, 2, 28, 1, 30))]
|
2015-08-04 22:25:24 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
2018-07-06 06:28:54 -04:00
|
|
|
if isinstance(tz, Column):
|
|
|
|
tz = _to_java_column(tz)
|
2015-08-04 22:25:24 -04:00
|
|
|
return Column(sc._jvm.functions.to_utc_timestamp(_to_java_column(timestamp), tz))
|
|
|
|
|
|
|
|
|
2020-06-16 04:35:35 -04:00
|
|
|
def timestamp_seconds(col):
|
|
|
|
"""
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 3.1.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2020-06-16 04:35:35 -04:00
|
|
|
>>> from pyspark.sql.functions import timestamp_seconds
|
[SPARK-32088][PYTHON] Pin the timezone in timestamp_seconds doctest
### What changes were proposed in this pull request?
Add American timezone during timestamp_seconds doctest
### Why are the changes needed?
`timestamp_seconds` doctest in `functions.py` used default timezone to get expected result
For example:
```python
>>> time_df = spark.createDataFrame([(1230219000,)], ['unix_time'])
>>> time_df.select(timestamp_seconds(time_df.unix_time).alias('ts')).collect()
[Row(ts=datetime.datetime(2008, 12, 25, 7, 30))]
```
But when we have a non-american timezone, the test case will get different test result.
For example, when we set current timezone as `Asia/Shanghai`, the test result will be
```
[Row(ts=datetime.datetime(2008, 12, 25, 23, 30))]
```
So no matter where we run the test case ,we will always get the expected permanent result if we set the timezone on one specific area.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Unit test
Closes #28932 from GuoPhilipse/SPARK-32088-fix-timezone-issue.
Lead-authored-by: GuoPhilipse <46367746+GuoPhilipse@users.noreply.github.com>
Co-authored-by: GuoPhilipse <guofei_ok@126.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-26 22:06:31 -04:00
|
|
|
>>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles")
|
2020-06-16 04:35:35 -04:00
|
|
|
>>> time_df = spark.createDataFrame([(1230219000,)], ['unix_time'])
|
2020-07-01 00:17:49 -04:00
|
|
|
>>> time_df.select(timestamp_seconds(time_df.unix_time).alias('ts')).show()
|
|
|
|
+-------------------+
|
|
|
|
| ts|
|
|
|
|
+-------------------+
|
|
|
|
|2008-12-25 07:30:00|
|
|
|
|
+-------------------+
|
[SPARK-32088][PYTHON] Pin the timezone in timestamp_seconds doctest
### What changes were proposed in this pull request?
Add American timezone during timestamp_seconds doctest
### Why are the changes needed?
`timestamp_seconds` doctest in `functions.py` used default timezone to get expected result
For example:
```python
>>> time_df = spark.createDataFrame([(1230219000,)], ['unix_time'])
>>> time_df.select(timestamp_seconds(time_df.unix_time).alias('ts')).collect()
[Row(ts=datetime.datetime(2008, 12, 25, 7, 30))]
```
But when we have a non-american timezone, the test case will get different test result.
For example, when we set current timezone as `Asia/Shanghai`, the test result will be
```
[Row(ts=datetime.datetime(2008, 12, 25, 23, 30))]
```
So no matter where we run the test case ,we will always get the expected permanent result if we set the timezone on one specific area.
### Does this PR introduce _any_ user-facing change?
No
### How was this patch tested?
Unit test
Closes #28932 from GuoPhilipse/SPARK-32088-fix-timezone-issue.
Lead-authored-by: GuoPhilipse <46367746+GuoPhilipse@users.noreply.github.com>
Co-authored-by: GuoPhilipse <guofei_ok@126.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-26 22:06:31 -04:00
|
|
|
>>> spark.conf.unset("spark.sql.session.timeZone")
|
2020-06-16 04:35:35 -04:00
|
|
|
"""
|
|
|
|
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.timestamp_seconds(_to_java_column(col)))
|
|
|
|
|
|
|
|
|
[SPARK-14353] Dataset Time Window `window` API for Python, and SQL
## What changes were proposed in this pull request?
The `window` function was added to Dataset with [this PR](https://github.com/apache/spark/pull/12008).
This PR adds the Python, and SQL, API for this function.
With this PR, SQL, Java, and Scala will share the same APIs as in users can use:
- `window(timeColumn, windowDuration)`
- `window(timeColumn, windowDuration, slideDuration)`
- `window(timeColumn, windowDuration, slideDuration, startTime)`
In Python, users can access all APIs above, but in addition they can do
- In Python:
`window(timeColumn, windowDuration, startTime=...)`
that is, they can provide the startTime without providing the `slideDuration`. In this case, we will generate tumbling windows.
## How was this patch tested?
Unit tests + manual tests
Author: Burak Yavuz <brkyvz@gmail.com>
Closes #12136 from brkyvz/python-windows.
2016-04-05 16:18:39 -04:00
|
|
|
def window(timeColumn, windowDuration, slideDuration=None, startTime=None):
|
|
|
|
"""Bucketize rows into one or more time windows given a timestamp specifying column. Window
|
|
|
|
starts are inclusive but the window ends are exclusive, e.g. 12:05 will be in the window
|
|
|
|
[12:05,12:10) but not in [12:00,12:05). Windows can support microsecond precision. Windows in
|
|
|
|
the order of months are not supported.
|
|
|
|
|
2016-07-28 17:57:15 -04:00
|
|
|
The time column must be of :class:`pyspark.sql.types.TimestampType`.
|
[SPARK-14353] Dataset Time Window `window` API for Python, and SQL
## What changes were proposed in this pull request?
The `window` function was added to Dataset with [this PR](https://github.com/apache/spark/pull/12008).
This PR adds the Python, and SQL, API for this function.
With this PR, SQL, Java, and Scala will share the same APIs as in users can use:
- `window(timeColumn, windowDuration)`
- `window(timeColumn, windowDuration, slideDuration)`
- `window(timeColumn, windowDuration, slideDuration, startTime)`
In Python, users can access all APIs above, but in addition they can do
- In Python:
`window(timeColumn, windowDuration, startTime=...)`
that is, they can provide the startTime without providing the `slideDuration`. In this case, we will generate tumbling windows.
## How was this patch tested?
Unit tests + manual tests
Author: Burak Yavuz <brkyvz@gmail.com>
Closes #12136 from brkyvz/python-windows.
2016-04-05 16:18:39 -04:00
|
|
|
|
|
|
|
Durations are provided as strings, e.g. '1 second', '1 day 12 hours', '2 minutes'. Valid
|
|
|
|
interval strings are 'week', 'day', 'hour', 'minute', 'second', 'millisecond', 'microsecond'.
|
2016-07-28 17:57:15 -04:00
|
|
|
If the ``slideDuration`` is not provided, the windows will be tumbling windows.
|
[SPARK-14353] Dataset Time Window `window` API for Python, and SQL
## What changes were proposed in this pull request?
The `window` function was added to Dataset with [this PR](https://github.com/apache/spark/pull/12008).
This PR adds the Python, and SQL, API for this function.
With this PR, SQL, Java, and Scala will share the same APIs as in users can use:
- `window(timeColumn, windowDuration)`
- `window(timeColumn, windowDuration, slideDuration)`
- `window(timeColumn, windowDuration, slideDuration, startTime)`
In Python, users can access all APIs above, but in addition they can do
- In Python:
`window(timeColumn, windowDuration, startTime=...)`
that is, they can provide the startTime without providing the `slideDuration`. In this case, we will generate tumbling windows.
## How was this patch tested?
Unit tests + manual tests
Author: Burak Yavuz <brkyvz@gmail.com>
Closes #12136 from brkyvz/python-windows.
2016-04-05 16:18:39 -04:00
|
|
|
|
|
|
|
The startTime is the offset with respect to 1970-01-01 00:00:00 UTC with which to start
|
|
|
|
window intervals. For example, in order to have hourly tumbling windows that start 15 minutes
|
|
|
|
past the hour, e.g. 12:15-13:15, 13:15-14:15... provide `startTime` as `15 minutes`.
|
|
|
|
|
|
|
|
The output column will be a struct called 'window' by default with the nested columns 'start'
|
2016-07-28 17:57:15 -04:00
|
|
|
and 'end', where 'start' and 'end' will be of :class:`pyspark.sql.types.TimestampType`.
|
[SPARK-14353] Dataset Time Window `window` API for Python, and SQL
## What changes were proposed in this pull request?
The `window` function was added to Dataset with [this PR](https://github.com/apache/spark/pull/12008).
This PR adds the Python, and SQL, API for this function.
With this PR, SQL, Java, and Scala will share the same APIs as in users can use:
- `window(timeColumn, windowDuration)`
- `window(timeColumn, windowDuration, slideDuration)`
- `window(timeColumn, windowDuration, slideDuration, startTime)`
In Python, users can access all APIs above, but in addition they can do
- In Python:
`window(timeColumn, windowDuration, startTime=...)`
that is, they can provide the startTime without providing the `slideDuration`. In this case, we will generate tumbling windows.
## How was this patch tested?
Unit tests + manual tests
Author: Burak Yavuz <brkyvz@gmail.com>
Closes #12136 from brkyvz/python-windows.
2016-04-05 16:18:39 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> df = spark.createDataFrame([("2016-03-11 09:00:07", 1)]).toDF("date", "val")
|
[SPARK-14353] Dataset Time Window `window` API for Python, and SQL
## What changes were proposed in this pull request?
The `window` function was added to Dataset with [this PR](https://github.com/apache/spark/pull/12008).
This PR adds the Python, and SQL, API for this function.
With this PR, SQL, Java, and Scala will share the same APIs as in users can use:
- `window(timeColumn, windowDuration)`
- `window(timeColumn, windowDuration, slideDuration)`
- `window(timeColumn, windowDuration, slideDuration, startTime)`
In Python, users can access all APIs above, but in addition they can do
- In Python:
`window(timeColumn, windowDuration, startTime=...)`
that is, they can provide the startTime without providing the `slideDuration`. In this case, we will generate tumbling windows.
## How was this patch tested?
Unit tests + manual tests
Author: Burak Yavuz <brkyvz@gmail.com>
Closes #12136 from brkyvz/python-windows.
2016-04-05 16:18:39 -04:00
|
|
|
>>> w = df.groupBy(window("date", "5 seconds")).agg(sum("val").alias("sum"))
|
|
|
|
>>> w.select(w.window.start.cast("string").alias("start"),
|
|
|
|
... w.window.end.cast("string").alias("end"), "sum").collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(start='2016-03-11 09:00:05', end='2016-03-11 09:00:10', sum=1)]
|
[SPARK-14353] Dataset Time Window `window` API for Python, and SQL
## What changes were proposed in this pull request?
The `window` function was added to Dataset with [this PR](https://github.com/apache/spark/pull/12008).
This PR adds the Python, and SQL, API for this function.
With this PR, SQL, Java, and Scala will share the same APIs as in users can use:
- `window(timeColumn, windowDuration)`
- `window(timeColumn, windowDuration, slideDuration)`
- `window(timeColumn, windowDuration, slideDuration, startTime)`
In Python, users can access all APIs above, but in addition they can do
- In Python:
`window(timeColumn, windowDuration, startTime=...)`
that is, they can provide the startTime without providing the `slideDuration`. In this case, we will generate tumbling windows.
## How was this patch tested?
Unit tests + manual tests
Author: Burak Yavuz <brkyvz@gmail.com>
Closes #12136 from brkyvz/python-windows.
2016-04-05 16:18:39 -04:00
|
|
|
"""
|
|
|
|
def check_string_field(field, fieldName):
|
|
|
|
if not field or type(field) is not str:
|
|
|
|
raise TypeError("%s should be provided as a string" % fieldName)
|
|
|
|
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
time_col = _to_java_column(timeColumn)
|
|
|
|
check_string_field(windowDuration, "windowDuration")
|
|
|
|
if slideDuration and startTime:
|
|
|
|
check_string_field(slideDuration, "slideDuration")
|
|
|
|
check_string_field(startTime, "startTime")
|
|
|
|
res = sc._jvm.functions.window(time_col, windowDuration, slideDuration, startTime)
|
|
|
|
elif slideDuration:
|
|
|
|
check_string_field(slideDuration, "slideDuration")
|
|
|
|
res = sc._jvm.functions.window(time_col, windowDuration, slideDuration)
|
|
|
|
elif startTime:
|
|
|
|
check_string_field(startTime, "startTime")
|
|
|
|
res = sc._jvm.functions.window(time_col, windowDuration, windowDuration, startTime)
|
|
|
|
else:
|
|
|
|
res = sc._jvm.functions.window(time_col, windowDuration)
|
|
|
|
return Column(res)
|
|
|
|
|
|
|
|
|
2015-08-04 22:25:24 -04:00
|
|
|
# ---------------------------- misc functions ----------------------------------
|
|
|
|
|
|
|
|
def crc32(col):
|
|
|
|
"""
|
|
|
|
Calculates the cyclic redundancy check value (CRC32) of a binary column and
|
|
|
|
returns the value as a bigint.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> spark.createDataFrame([('ABC',)], ['a']).select(crc32('a').alias('crc32')).collect()
|
2015-08-12 18:27:52 -04:00
|
|
|
[Row(crc32=2743272264)]
|
2015-08-04 22:25:24 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
2015-08-12 18:27:52 -04:00
|
|
|
return Column(sc._jvm.functions.crc32(_to_java_column(col)))
|
2015-08-04 22:25:24 -04:00
|
|
|
|
|
|
|
|
|
|
|
def md5(col):
|
|
|
|
"""Calculates the MD5 digest and returns the value as a 32 character hex string.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> spark.createDataFrame([('ABC',)], ['a']).select(md5('a').alias('hash')).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(hash='902fbdd2b1df0c4f70b4a5d23525e932')]
|
2015-08-04 22:25:24 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
jc = sc._jvm.functions.md5(_to_java_column(col))
|
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
|
|
|
def sha1(col):
|
|
|
|
"""Returns the hex string result of SHA-1.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> spark.createDataFrame([('ABC',)], ['a']).select(sha1('a').alias('hash')).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(hash='3c01bdbb26f358bab27f267924aa2c9a03fcfdb8')]
|
2015-08-04 22:25:24 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
jc = sc._jvm.functions.sha1(_to_java_column(col))
|
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
|
|
|
def sha2(col, numBits):
|
|
|
|
"""Returns the hex string result of SHA-2 family of hash functions (SHA-224, SHA-256, SHA-384,
|
|
|
|
and SHA-512). The numBits indicates the desired bit length of the result, which must have a
|
|
|
|
value of 224, 256, 384, 512, or 0 (which is equivalent to 256).
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2015-08-04 22:25:24 -04:00
|
|
|
>>> digests = df.select(sha2(df.name, 256).alias('s')).collect()
|
|
|
|
>>> digests[0]
|
2020-07-13 22:22:44 -04:00
|
|
|
Row(s='3bc51062973c458d5a6f2d8d64a023246354ad7e064b1e4e009ec8a0699a3043')
|
2015-08-04 22:25:24 -04:00
|
|
|
>>> digests[1]
|
2020-07-13 22:22:44 -04:00
|
|
|
Row(s='cd9fb1e148ccd8442e5aa74904cc73bf6fb54d1d54d333bd596aa9bb4bb4e961')
|
2015-08-04 22:25:24 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
jc = sc._jvm.functions.sha2(_to_java_column(col), numBits)
|
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
2016-01-05 13:23:36 -05:00
|
|
|
def hash(*cols):
|
2016-05-27 01:39:14 -04:00
|
|
|
"""Calculates the hash code of given columns, and returns the result as an int column.
|
2016-01-05 13:23:36 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> spark.createDataFrame([('ABC',)], ['a']).select(hash('a').alias('hash')).collect()
|
2016-01-13 15:29:02 -05:00
|
|
|
[Row(hash=-757602832)]
|
2016-01-05 13:23:36 -05:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
jc = sc._jvm.functions.hash(_to_seq(sc, cols, _to_java_column))
|
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
2019-03-20 04:34:34 -04:00
|
|
|
def xxhash64(*cols):
|
|
|
|
"""Calculates the hash code of given columns using the 64-bit variant of the xxHash algorithm,
|
|
|
|
and returns the result as a long column.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 3.0.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2019-03-20 04:34:34 -04:00
|
|
|
>>> spark.createDataFrame([('ABC',)], ['a']).select(xxhash64('a').alias('hash')).collect()
|
|
|
|
[Row(hash=4105715581806190027)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
jc = sc._jvm.functions.xxhash64(_to_seq(sc, cols, _to_java_column))
|
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
[SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true
## What changes were proposed in this pull request?
Adds a SQL function `raise_error` which underlies the refactored `assert_true` function. `assert_true` now also (optionally) accepts a custom error message field.
`raise_error` is exposed in SQL, Python, Scala, and R.
`assert_true` was previously only exposed in SQL; it is now also exposed in Python, Scala, and R.
### Why are the changes needed?
Improves usability of `assert_true` by clarifying error messaging, and adds the useful helper function `raise_error`.
### Does this PR introduce _any_ user-facing change?
Yes:
- Adds `raise_error` function to the SQL, Python, Scala, and R APIs.
- Adds `assert_true` function to the SQL, Python and R APIs.
### How was this patch tested?
Adds unit tests in SQL, Python, Scala, and R for `assert_true` and `raise_error`.
Closes #29947 from karenfeng/spark-32793.
Lead-authored-by: Karen Feng <karen.feng@databricks.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-10-07 23:05:39 -04:00
|
|
|
def assert_true(col, errMsg=None):
|
|
|
|
"""
|
|
|
|
Returns null if the input column is true; throws an exception with the provided error message
|
|
|
|
otherwise.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 3.1.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
[SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true
## What changes were proposed in this pull request?
Adds a SQL function `raise_error` which underlies the refactored `assert_true` function. `assert_true` now also (optionally) accepts a custom error message field.
`raise_error` is exposed in SQL, Python, Scala, and R.
`assert_true` was previously only exposed in SQL; it is now also exposed in Python, Scala, and R.
### Why are the changes needed?
Improves usability of `assert_true` by clarifying error messaging, and adds the useful helper function `raise_error`.
### Does this PR introduce _any_ user-facing change?
Yes:
- Adds `raise_error` function to the SQL, Python, Scala, and R APIs.
- Adds `assert_true` function to the SQL, Python and R APIs.
### How was this patch tested?
Adds unit tests in SQL, Python, Scala, and R for `assert_true` and `raise_error`.
Closes #29947 from karenfeng/spark-32793.
Lead-authored-by: Karen Feng <karen.feng@databricks.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-10-07 23:05:39 -04:00
|
|
|
>>> df = spark.createDataFrame([(0,1)], ['a', 'b'])
|
|
|
|
>>> df.select(assert_true(df.a < df.b).alias('r')).collect()
|
|
|
|
[Row(r=None)]
|
|
|
|
>>> df = spark.createDataFrame([(0,1)], ['a', 'b'])
|
|
|
|
>>> df.select(assert_true(df.a < df.b, df.a).alias('r')).collect()
|
|
|
|
[Row(r=None)]
|
|
|
|
>>> df = spark.createDataFrame([(0,1)], ['a', 'b'])
|
|
|
|
>>> df.select(assert_true(df.a < df.b, 'error').alias('r')).collect()
|
|
|
|
[Row(r=None)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
if errMsg is None:
|
|
|
|
return Column(sc._jvm.functions.assert_true(_to_java_column(col)))
|
|
|
|
if not isinstance(errMsg, (str, Column)):
|
|
|
|
raise TypeError(
|
|
|
|
"errMsg should be a Column or a str, got {}".format(type(errMsg))
|
|
|
|
)
|
|
|
|
|
|
|
|
errMsg = (
|
|
|
|
_create_column_from_literal(errMsg)
|
|
|
|
if isinstance(errMsg, str)
|
|
|
|
else _to_java_column(errMsg)
|
|
|
|
)
|
|
|
|
return Column(sc._jvm.functions.assert_true(_to_java_column(col), errMsg))
|
|
|
|
|
|
|
|
|
|
|
|
@since(3.1)
|
|
|
|
def raise_error(errMsg):
|
|
|
|
"""
|
|
|
|
Throws an exception with the provided error message.
|
|
|
|
"""
|
|
|
|
if not isinstance(errMsg, (str, Column)):
|
|
|
|
raise TypeError(
|
|
|
|
"errMsg should be a Column or a str, got {}".format(type(errMsg))
|
|
|
|
)
|
|
|
|
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
errMsg = (
|
|
|
|
_create_column_from_literal(errMsg)
|
|
|
|
if isinstance(errMsg, str)
|
|
|
|
else _to_java_column(errMsg)
|
|
|
|
)
|
|
|
|
return Column(sc._jvm.functions.raise_error(errMsg))
|
|
|
|
|
|
|
|
|
2015-08-04 22:25:24 -04:00
|
|
|
# ---------------------- String/Binary functions ------------------------------
|
|
|
|
|
2020-10-26 22:05:53 -04:00
|
|
|
@since(1.5)
|
|
|
|
def upper(col):
|
|
|
|
"""
|
|
|
|
Converts a string expression to upper case.
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("upper", col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.5)
|
|
|
|
def lower(col):
|
|
|
|
"""
|
|
|
|
Converts a string expression to lower case.
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("lower", col)
|
|
|
|
|
2015-08-04 22:25:24 -04:00
|
|
|
|
2020-10-26 22:05:53 -04:00
|
|
|
@since(1.5)
|
|
|
|
def ascii(col):
|
|
|
|
"""
|
|
|
|
Computes the numeric value of the first character of the string column.
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("ascii", col)
|
2015-08-04 22:25:24 -04:00
|
|
|
|
2020-10-26 22:05:53 -04:00
|
|
|
|
|
|
|
@since(1.5)
|
|
|
|
def base64(col):
|
|
|
|
"""
|
|
|
|
Computes the BASE64 encoding of a binary column and returns it as a string column.
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("base64", col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.5)
|
|
|
|
def unbase64(col):
|
|
|
|
"""
|
|
|
|
Decodes a BASE64 encoded string column and returns it as a binary column.
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("unbase64", col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.5)
|
|
|
|
def ltrim(col):
|
|
|
|
"""
|
|
|
|
Trim the spaces from left end for the specified string value.
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("ltrim", col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.5)
|
|
|
|
def rtrim(col):
|
|
|
|
"""
|
|
|
|
Trim the spaces from right end for the specified string value.
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("rtrim", col)
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.5)
|
|
|
|
def trim(col):
|
|
|
|
"""
|
|
|
|
Trim the spaces from both ends for the specified string column.
|
|
|
|
"""
|
|
|
|
return _invoke_function_over_column("trim", col)
|
2015-08-04 22:25:24 -04:00
|
|
|
|
|
|
|
|
|
|
|
def concat_ws(sep, *cols):
|
|
|
|
"""
|
|
|
|
Concatenates multiple input string columns together into a single string column,
|
|
|
|
using the given separator.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> df = spark.createDataFrame([('abcd','123')], ['s', 'd'])
|
2015-08-04 22:25:24 -04:00
|
|
|
>>> df.select(concat_ws('-', df.s, df.d).alias('s')).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(s='abcd-123')]
|
2015-08-04 22:25:24 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.concat_ws(sep, _to_seq(sc, cols, _to_java_column)))
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.5)
|
|
|
|
def decode(col, charset):
|
|
|
|
"""
|
|
|
|
Computes the first argument into a string from a binary using the provided character set
|
|
|
|
(one of 'US-ASCII', 'ISO-8859-1', 'UTF-8', 'UTF-16BE', 'UTF-16LE', 'UTF-16').
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.decode(_to_java_column(col), charset))
|
|
|
|
|
|
|
|
|
|
|
|
@since(1.5)
|
|
|
|
def encode(col, charset):
|
|
|
|
"""
|
|
|
|
Computes the first argument into a binary from a string using the provided character set
|
|
|
|
(one of 'US-ASCII', 'ISO-8859-1', 'UTF-8', 'UTF-16BE', 'UTF-16LE', 'UTF-16').
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.encode(_to_java_column(col), charset))
|
|
|
|
|
|
|
|
|
|
|
|
def format_number(col, d):
|
|
|
|
"""
|
2017-03-26 21:40:00 -04:00
|
|
|
Formats the number X to a format like '#,--#,--#.--', rounded to d decimal places
|
|
|
|
with HALF_EVEN round mode, and returns the result as a string.
|
2015-08-04 22:25:24 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
the column name of the numeric value to be formatted
|
|
|
|
d : int
|
|
|
|
the N decimal places
|
2015-08-04 22:25:24 -04:00
|
|
|
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> spark.createDataFrame([(5,)], ['a']).select(format_number('a', 4).alias('v')).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(v='5.0000')]
|
2015-08-04 22:25:24 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.format_number(_to_java_column(col), d))
|
|
|
|
|
|
|
|
|
|
|
|
def format_string(format, *cols):
|
|
|
|
"""
|
|
|
|
Formats the arguments in printf-style and returns the result as a string column.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
format : str
|
|
|
|
string that can contain embedded format tags and used as result column's value
|
2021-02-01 19:30:50 -05:00
|
|
|
cols : :class:`~pyspark.sql.Column` or str
|
|
|
|
column names or :class:`~pyspark.sql.Column`\\s to be used in formatting
|
2015-08-04 22:25:24 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> df = spark.createDataFrame([(5, "hello")], ['a', 'b'])
|
2015-08-04 22:25:24 -04:00
|
|
|
>>> df.select(format_string('%d %s', df.a, df.b).alias('v')).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(v='5 hello')]
|
2015-08-04 22:25:24 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.format_string(format, _to_seq(sc, cols, _to_java_column)))
|
|
|
|
|
|
|
|
|
|
|
|
def instr(str, substr):
|
|
|
|
"""
|
|
|
|
Locate the position of the first occurrence of substr column in the given string.
|
|
|
|
Returns null if either of the arguments are null.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
The position is not zero based, but 1 based index. Returns 0 if substr
|
|
|
|
could not be found in str.
|
2015-08-04 22:25:24 -04:00
|
|
|
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> df = spark.createDataFrame([('abcd',)], ['s',])
|
2015-08-04 22:25:24 -04:00
|
|
|
>>> df.select(instr(df.s, 'b').alias('s')).collect()
|
|
|
|
[Row(s=2)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.instr(_to_java_column(str), substr))
|
|
|
|
|
|
|
|
|
2020-01-23 02:16:47 -05:00
|
|
|
def overlay(src, replace, pos, len=-1):
|
|
|
|
"""
|
|
|
|
Overlay the specified portion of `src` with `replace`,
|
|
|
|
starting from byte position `pos` of `src` and proceeding for `len` bytes.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 3.0.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2020-01-23 02:16:47 -05:00
|
|
|
>>> df = spark.createDataFrame([("SPARK_SQL", "CORE")], ("x", "y"))
|
|
|
|
>>> df.select(overlay("x", "y", 7).alias("overlayed")).show()
|
|
|
|
+----------+
|
|
|
|
| overlayed|
|
|
|
|
+----------+
|
|
|
|
|SPARK_CORE|
|
|
|
|
+----------+
|
|
|
|
"""
|
|
|
|
if not isinstance(pos, (int, str, Column)):
|
|
|
|
raise TypeError(
|
|
|
|
"pos should be an integer or a Column / column name, got {}".format(type(pos)))
|
|
|
|
if len is not None and not isinstance(len, (int, str, Column)):
|
|
|
|
raise TypeError(
|
|
|
|
"len should be an integer or a Column / column name, got {}".format(type(len)))
|
|
|
|
|
|
|
|
pos = _create_column_from_literal(pos) if isinstance(pos, int) else _to_java_column(pos)
|
|
|
|
len = _create_column_from_literal(len) if isinstance(len, int) else _to_java_column(len)
|
|
|
|
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
|
|
|
|
return Column(sc._jvm.functions.overlay(
|
|
|
|
_to_java_column(src),
|
|
|
|
_to_java_column(replace),
|
|
|
|
pos,
|
|
|
|
len
|
|
|
|
))
|
|
|
|
|
|
|
|
|
2015-08-04 22:25:24 -04:00
|
|
|
def substring(str, pos, len):
|
|
|
|
"""
|
|
|
|
Substring starts at `pos` and is of length `len` when str is String type or
|
|
|
|
returns the slice of byte array that starts at `pos` in byte and is of length `len`
|
2017-08-07 12:16:03 -04:00
|
|
|
when str is Binary type.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
2015-08-04 22:25:24 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
The position is not zero based, but 1 based index.
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> df = spark.createDataFrame([('abcd',)], ['s',])
|
2015-08-04 22:25:24 -04:00
|
|
|
>>> df.select(substring(df.s, 1, 2).alias('s')).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(s='ab')]
|
2015-08-04 22:25:24 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.substring(_to_java_column(str), pos, len))
|
|
|
|
|
|
|
|
|
|
|
|
def substring_index(str, delim, count):
|
|
|
|
"""
|
|
|
|
Returns the substring from string str before count occurrences of the delimiter delim.
|
|
|
|
If count is positive, everything the left of the final delimiter (counting from left) is
|
|
|
|
returned. If count is negative, every to the right of the final delimiter (counting from the
|
|
|
|
right) is returned. substring_index performs a case-sensitive match when searching for delim.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> df = spark.createDataFrame([('a.b.c.d',)], ['s'])
|
2015-08-04 22:25:24 -04:00
|
|
|
>>> df.select(substring_index(df.s, '.', 2).alias('s')).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(s='a.b')]
|
2015-08-01 00:18:01 -04:00
|
|
|
>>> df.select(substring_index(df.s, '.', -3).alias('s')).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(s='b.c.d')]
|
2015-08-01 00:18:01 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.substring_index(_to_java_column(str), delim, count))
|
|
|
|
|
|
|
|
|
2015-08-04 22:25:24 -04:00
|
|
|
def levenshtein(left, right):
|
|
|
|
"""Computes the Levenshtein distance of the two given strings.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> df0 = spark.createDataFrame([('kitten', 'sitting',)], ['l', 'r'])
|
2015-08-04 22:25:24 -04:00
|
|
|
>>> df0.select(levenshtein('l', 'r').alias('d')).collect()
|
|
|
|
[Row(d=3)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
jc = sc._jvm.functions.levenshtein(_to_java_column(left), _to_java_column(right))
|
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
[SPARK-15397][SQL] fix string udf locate as hive
## What changes were proposed in this pull request?
in hive, `locate("aa", "aaa", 0)` would yield 0, `locate("aa", "aaa", 1)` would yield 1 and `locate("aa", "aaa", 2)` would yield 2, while in Spark, `locate("aa", "aaa", 0)` would yield 1, `locate("aa", "aaa", 1)` would yield 2 and `locate("aa", "aaa", 2)` would yield 0. This results from the different understanding of the third parameter in udf `locate`. It means the starting index and starts from 1, so when we use 0, the return would always be 0.
## How was this patch tested?
tested with modified `StringExpressionsSuite` and `StringFunctionsSuite`
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Closes #13186 from adrian-wang/locate.
2016-05-24 02:29:15 -04:00
|
|
|
def locate(substr, str, pos=1):
|
2015-08-04 22:25:24 -04:00
|
|
|
"""
|
|
|
|
Locate the position of the first occurrence of substr in a string column, after position pos.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
substr : str
|
|
|
|
a string
|
2021-02-01 19:30:50 -05:00
|
|
|
str : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
a Column of :class:`pyspark.sql.types.StringType`
|
|
|
|
pos : int, optional
|
|
|
|
start position (zero based)
|
2015-08-04 22:25:24 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
The position is not zero based, but 1 based index. Returns 0 if substr
|
|
|
|
could not be found in str.
|
2015-08-04 22:25:24 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> df = spark.createDataFrame([('abcd',)], ['s',])
|
2015-08-04 22:25:24 -04:00
|
|
|
>>> df.select(locate('b', df.s, 1).alias('s')).collect()
|
|
|
|
[Row(s=2)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.locate(substr, _to_java_column(str), pos))
|
|
|
|
|
|
|
|
|
|
|
|
def lpad(col, len, pad):
|
|
|
|
"""
|
|
|
|
Left-pad the string column to width `len` with `pad`.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> df = spark.createDataFrame([('abcd',)], ['s',])
|
2015-08-04 22:25:24 -04:00
|
|
|
>>> df.select(lpad(df.s, 6, '#').alias('s')).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(s='##abcd')]
|
2015-08-04 22:25:24 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.lpad(_to_java_column(col), len, pad))
|
|
|
|
|
|
|
|
|
|
|
|
def rpad(col, len, pad):
|
|
|
|
"""
|
|
|
|
Right-pad the string column to width `len` with `pad`.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> df = spark.createDataFrame([('abcd',)], ['s',])
|
2015-08-04 22:25:24 -04:00
|
|
|
>>> df.select(rpad(df.s, 6, '#').alias('s')).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(s='abcd##')]
|
2015-08-04 22:25:24 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.rpad(_to_java_column(col), len, pad))
|
|
|
|
|
|
|
|
|
|
|
|
def repeat(col, n):
|
|
|
|
"""
|
|
|
|
Repeats a string column n times, and returns it as a new string column.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> df = spark.createDataFrame([('ab',)], ['s',])
|
2015-08-04 22:25:24 -04:00
|
|
|
>>> df.select(repeat(df.s, 3).alias('s')).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(s='ababab')]
|
2015-08-04 22:25:24 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.repeat(_to_java_column(col), n))
|
|
|
|
|
|
|
|
|
2018-10-06 02:30:43 -04:00
|
|
|
def split(str, pattern, limit=-1):
|
2015-08-04 22:25:24 -04:00
|
|
|
"""
|
2018-10-06 02:30:43 -04:00
|
|
|
Splits str around matches of the given pattern.
|
2015-08-04 22:25:24 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
str : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
a string expression to split
|
|
|
|
pattern : str
|
|
|
|
a string representing a regular expression. The regex string should be
|
2018-10-06 02:30:43 -04:00
|
|
|
a Java regular expression.
|
2020-11-02 20:00:49 -05:00
|
|
|
limit : int, optional
|
|
|
|
an integer which controls the number of times `pattern` is applied.
|
2015-08-04 22:25:24 -04:00
|
|
|
|
2018-10-06 02:30:43 -04:00
|
|
|
* ``limit > 0``: The resulting array's length will not be more than `limit`, and the
|
|
|
|
resulting array's last entry will contain all input beyond the last
|
|
|
|
matched pattern.
|
|
|
|
* ``limit <= 0``: `pattern` will be applied as many times as possible, and the resulting
|
|
|
|
array can be of any size.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionchanged:: 3.0
|
|
|
|
`split` now takes an optional `limit` field. If not provided, default limit value is -1.
|
2018-10-06 02:30:43 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2018-10-06 02:30:43 -04:00
|
|
|
>>> df = spark.createDataFrame([('oneAtwoBthreeC',)], ['s',])
|
|
|
|
>>> df.select(split(df.s, '[ABC]', 2).alias('s')).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(s=['one', 'twoBthreeC'])]
|
2018-10-06 02:30:43 -04:00
|
|
|
>>> df.select(split(df.s, '[ABC]', -1).alias('s')).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(s=['one', 'two', 'three', ''])]
|
2015-08-04 22:25:24 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
2018-10-06 02:30:43 -04:00
|
|
|
return Column(sc._jvm.functions.split(_to_java_column(str), pattern, limit))
|
2015-08-04 22:25:24 -04:00
|
|
|
|
|
|
|
|
|
|
|
def regexp_extract(str, pattern, idx):
|
2018-09-12 23:19:43 -04:00
|
|
|
r"""Extract a specific group matched by a Java regex, from the specified string column.
|
2016-08-10 05:14:43 -04:00
|
|
|
If the regex did not match, or the specified group did not match, an empty string is returned.
|
2015-08-04 22:25:24 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> df = spark.createDataFrame([('100-200',)], ['str'])
|
2018-09-12 23:19:43 -04:00
|
|
|
>>> df.select(regexp_extract('str', r'(\d+)-(\d+)', 1).alias('d')).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(d='100')]
|
2016-08-10 05:14:43 -04:00
|
|
|
>>> df = spark.createDataFrame([('foo',)], ['str'])
|
2018-09-12 23:19:43 -04:00
|
|
|
>>> df.select(regexp_extract('str', r'(\d+)', 1).alias('d')).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(d='')]
|
2016-08-07 07:20:07 -04:00
|
|
|
>>> df = spark.createDataFrame([('aaaac',)], ['str'])
|
|
|
|
>>> df.select(regexp_extract('str', '(a+)(b)?(c)', 2).alias('d')).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(d='')]
|
2015-08-04 22:25:24 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
jc = sc._jvm.functions.regexp_extract(_to_java_column(str), pattern, idx)
|
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
|
|
|
def regexp_replace(str, pattern, replacement):
|
2018-09-14 21:13:07 -04:00
|
|
|
r"""Replace all substrings of the specified string value that match regexp with rep.
|
2015-08-04 22:25:24 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> df = spark.createDataFrame([('100-200',)], ['str'])
|
2018-09-12 23:19:43 -04:00
|
|
|
>>> df.select(regexp_replace('str', r'(\d+)', '--').alias('d')).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(d='-----')]
|
2015-08-04 22:25:24 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
jc = sc._jvm.functions.regexp_replace(_to_java_column(str), pattern, replacement)
|
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
2015-08-02 00:44:57 -04:00
|
|
|
def initcap(col):
|
|
|
|
"""Translate the first letter of each word to upper case in the sentence.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> spark.createDataFrame([('ab cd',)], ['a']).select(initcap("a").alias('v')).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(v='Ab Cd')]
|
2015-08-02 00:44:57 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.initcap(_to_java_column(col)))
|
|
|
|
|
|
|
|
|
2015-08-04 22:25:24 -04:00
|
|
|
def soundex(col):
|
|
|
|
"""
|
|
|
|
Returns the SoundEx encoding for a string
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> df = spark.createDataFrame([("Peters",),("Uhrbach",)], ['name'])
|
2015-08-04 22:25:24 -04:00
|
|
|
>>> df.select(soundex(df.name).alias("soundex")).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(soundex='P362'), Row(soundex='U612')]
|
2015-08-04 22:25:24 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.soundex(_to_java_column(col)))
|
|
|
|
|
|
|
|
|
|
|
|
def bin(col):
|
|
|
|
"""Returns the string representation of the binary value of the given column.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2015-08-04 22:25:24 -04:00
|
|
|
>>> df.select(bin(df.age).alias('c')).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(c='10'), Row(c='101')]
|
2015-08-04 22:25:24 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
jc = sc._jvm.functions.bin(_to_java_column(col))
|
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
|
|
|
def hex(col):
|
2016-07-28 17:57:15 -04:00
|
|
|
"""Computes hex value of the given column, which could be :class:`pyspark.sql.types.StringType`,
|
|
|
|
:class:`pyspark.sql.types.BinaryType`, :class:`pyspark.sql.types.IntegerType` or
|
|
|
|
:class:`pyspark.sql.types.LongType`.
|
2015-08-04 22:25:24 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> spark.createDataFrame([('ABC', 3)], ['a', 'b']).select(hex('a'), hex('b')).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(hex(a)='414243', hex(b)='3')]
|
2015-08-04 22:25:24 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
jc = sc._jvm.functions.hex(_to_java_column(col))
|
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
|
|
|
def unhex(col):
|
|
|
|
"""Inverse of hex. Interprets each pair of characters as a hexadecimal number
|
|
|
|
and converts to the byte representation of number.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> spark.createDataFrame([('414243',)], ['a']).select(unhex('a')).collect()
|
2015-08-04 22:25:24 -04:00
|
|
|
[Row(unhex(a)=bytearray(b'ABC'))]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.unhex(_to_java_column(col)))
|
|
|
|
|
|
|
|
|
|
|
|
def length(col):
|
2018-02-06 19:46:43 -05:00
|
|
|
"""Computes the character length of string data or number of bytes of binary data.
|
|
|
|
The length of character data includes the trailing spaces. The length of binary data
|
|
|
|
includes binary zeros.
|
2015-08-04 22:25:24 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2018-02-06 19:46:43 -05:00
|
|
|
>>> spark.createDataFrame([('ABC ',)], ['a']).select(length('a').alias('length')).collect()
|
|
|
|
[Row(length=4)]
|
2015-08-04 22:25:24 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.length(_to_java_column(col)))
|
|
|
|
|
|
|
|
|
2015-08-06 12:02:30 -04:00
|
|
|
def translate(srcCol, matching, replace):
|
|
|
|
"""A function translate any character in the `srcCol` by a character in `matching`.
|
|
|
|
The characters in `replace` is corresponding to the characters in `matching`.
|
|
|
|
The translate will happen when any character in the string matching with the character
|
|
|
|
in the `matching`.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-07-06 13:45:51 -04:00
|
|
|
>>> spark.createDataFrame([('translate',)], ['a']).select(translate('a', "rnlt", "123") \\
|
|
|
|
... .alias('r')).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(r='1a2s3ae')]
|
2015-08-06 12:02:30 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.translate(_to_java_column(srcCol), matching, replace))
|
|
|
|
|
|
|
|
|
2015-08-04 22:25:24 -04:00
|
|
|
# ---------------------- Collection functions ------------------------------
|
|
|
|
|
2016-03-25 12:50:06 -04:00
|
|
|
def create_map(*cols):
|
|
|
|
"""Creates a new map column.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
cols : :class:`~pyspark.sql.Column` or str
|
|
|
|
column names or :class:`~pyspark.sql.Column`\\s that are
|
2018-01-30 07:55:55 -05:00
|
|
|
grouped as key-value pairs, e.g. (key1, value1, key2, value2, ...).
|
2016-03-25 12:50:06 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2016-03-25 12:50:06 -04:00
|
|
|
>>> df.select(create_map('name', 'age').alias("map")).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(map={'Alice': 2}), Row(map={'Bob': 5})]
|
2016-03-25 12:50:06 -04:00
|
|
|
>>> df.select(create_map([df.name, df.age]).alias("map")).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(map={'Alice': 2}), Row(map={'Bob': 5})]
|
2016-03-25 12:50:06 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
if len(cols) == 1 and isinstance(cols[0], (list, set)):
|
|
|
|
cols = cols[0]
|
|
|
|
jc = sc._jvm.functions.map(_to_seq(sc, cols, _to_java_column))
|
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
2018-06-12 15:31:22 -04:00
|
|
|
def map_from_arrays(col1, col2):
|
|
|
|
"""Creates a new map from two arrays.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col1 : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column containing a set of keys. All elements should not be null
|
2021-02-01 19:30:50 -05:00
|
|
|
col2 : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column containing a set of values
|
2018-06-12 15:31:22 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2018-06-12 15:31:22 -04:00
|
|
|
>>> df = spark.createDataFrame([([2, 5], ['a', 'b'])], ['k', 'v'])
|
|
|
|
>>> df.select(map_from_arrays(df.k, df.v).alias("map")).show()
|
|
|
|
+----------------+
|
|
|
|
| map|
|
|
|
|
+----------------+
|
2020-08-04 10:57:09 -04:00
|
|
|
|{2 -> a, 5 -> b}|
|
2018-06-12 15:31:22 -04:00
|
|
|
+----------------+
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.map_from_arrays(_to_java_column(col1), _to_java_column(col2)))
|
|
|
|
|
|
|
|
|
2015-08-04 22:25:24 -04:00
|
|
|
def array(*cols):
|
|
|
|
"""Creates a new array column.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
cols : :class:`~pyspark.sql.Column` or str
|
|
|
|
column names or :class:`~pyspark.sql.Column`\\s that have
|
2015-08-04 22:25:24 -04:00
|
|
|
the same data type.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2015-08-04 22:25:24 -04:00
|
|
|
>>> df.select(array('age', 'age').alias("arr")).collect()
|
|
|
|
[Row(arr=[2, 2]), Row(arr=[5, 5])]
|
|
|
|
>>> df.select(array([df.age, df.age]).alias("arr")).collect()
|
|
|
|
[Row(arr=[2, 2]), Row(arr=[5, 5])]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
if len(cols) == 1 and isinstance(cols[0], (list, set)):
|
|
|
|
cols = cols[0]
|
|
|
|
jc = sc._jvm.functions.array(_to_seq(sc, cols, _to_java_column))
|
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
2015-08-05 01:32:21 -04:00
|
|
|
def array_contains(col, value):
|
|
|
|
"""
|
2017-03-26 21:40:00 -04:00
|
|
|
Collection function: returns null if the array is null, true if the array contains the
|
|
|
|
given value, and false otherwise.
|
2015-08-05 01:32:21 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column containing array
|
|
|
|
value :
|
|
|
|
value or column to check for in array
|
2015-08-05 01:32:21 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> df = spark.createDataFrame([(["a", "b", "c"],), ([],)], ['data'])
|
2015-08-05 01:32:21 -04:00
|
|
|
>>> df.select(array_contains(df.data, "a")).collect()
|
2016-02-21 09:53:15 -05:00
|
|
|
[Row(array_contains(data, a)=True), Row(array_contains(data, a)=False)]
|
[SPARK-29627][PYTHON][SQL] Allow array_contains to take column instances
### What changes were proposed in this pull request?
This PR proposes to allow `array_contains` to take column instances.
### Why are the changes needed?
For consistent support in Scala and Python APIs. Scala allows column instances at `array_contains`
Scala:
```scala
import org.apache.spark.sql.functions._
val df = Seq(Array("a", "b", "c"), Array.empty[String]).toDF("data")
df.select(array_contains($"data", lit("a"))).show()
```
Python:
```python
from pyspark.sql.functions import array_contains, lit
df = spark.createDataFrame([(["a", "b", "c"],), ([],)], ['data'])
df.select(array_contains(df.data, lit("a"))).show()
```
However, PySpark sides does not allow.
### Does this PR introduce any user-facing change?
Yes.
```python
from pyspark.sql.functions import array_contains, lit
df = spark.createDataFrame([(["a", "b", "c"],), ([],)], ['data'])
df.select(array_contains(df.data, lit("a"))).show()
```
**Before:**
```
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../spark/python/pyspark/sql/functions.py", line 1950, in array_contains
return Column(sc._jvm.functions.array_contains(_to_java_column(col), value))
File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1277, in __call__
File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1241, in _build_args
File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1228, in _get_args
File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_collections.py", line 500, in convert
File "/.../spark/python/pyspark/sql/column.py", line 344, in __iter__
raise TypeError("Column is not iterable")
TypeError: Column is not iterable
```
**After:**
```
+-----------------------+
|array_contains(data, a)|
+-----------------------+
| true|
| false|
+-----------------------+
```
### How was this patch tested?
Manually tested and added a doctest.
Closes #26288 from HyukjinKwon/SPARK-29627.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-10-29 20:45:19 -04:00
|
|
|
>>> df.select(array_contains(df.data, lit("a"))).collect()
|
|
|
|
[Row(array_contains(data, a)=True), Row(array_contains(data, a)=False)]
|
2015-08-05 01:32:21 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
[SPARK-29627][PYTHON][SQL] Allow array_contains to take column instances
### What changes were proposed in this pull request?
This PR proposes to allow `array_contains` to take column instances.
### Why are the changes needed?
For consistent support in Scala and Python APIs. Scala allows column instances at `array_contains`
Scala:
```scala
import org.apache.spark.sql.functions._
val df = Seq(Array("a", "b", "c"), Array.empty[String]).toDF("data")
df.select(array_contains($"data", lit("a"))).show()
```
Python:
```python
from pyspark.sql.functions import array_contains, lit
df = spark.createDataFrame([(["a", "b", "c"],), ([],)], ['data'])
df.select(array_contains(df.data, lit("a"))).show()
```
However, PySpark sides does not allow.
### Does this PR introduce any user-facing change?
Yes.
```python
from pyspark.sql.functions import array_contains, lit
df = spark.createDataFrame([(["a", "b", "c"],), ([],)], ['data'])
df.select(array_contains(df.data, lit("a"))).show()
```
**Before:**
```
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../spark/python/pyspark/sql/functions.py", line 1950, in array_contains
return Column(sc._jvm.functions.array_contains(_to_java_column(col), value))
File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1277, in __call__
File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1241, in _build_args
File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1228, in _get_args
File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_collections.py", line 500, in convert
File "/.../spark/python/pyspark/sql/column.py", line 344, in __iter__
raise TypeError("Column is not iterable")
TypeError: Column is not iterable
```
**After:**
```
+-----------------------+
|array_contains(data, a)|
+-----------------------+
| true|
| false|
+-----------------------+
```
### How was this patch tested?
Manually tested and added a doctest.
Closes #26288 from HyukjinKwon/SPARK-29627.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-10-29 20:45:19 -04:00
|
|
|
value = value._jc if isinstance(value, Column) else value
|
2015-08-05 01:32:21 -04:00
|
|
|
return Column(sc._jvm.functions.array_contains(_to_java_column(col), value))
|
|
|
|
|
|
|
|
|
2018-05-17 08:45:32 -04:00
|
|
|
def arrays_overlap(a1, a2):
|
|
|
|
"""
|
|
|
|
Collection function: returns true if the arrays contain any common non-null element; if not,
|
|
|
|
returns null if both the arrays are non-empty and any of them contains a null element; returns
|
|
|
|
false otherwise.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.4.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2018-05-17 08:45:32 -04:00
|
|
|
>>> df = spark.createDataFrame([(["a", "b"], ["b", "c"]), (["a"], ["b", "c"])], ['x', 'y'])
|
|
|
|
>>> df.select(arrays_overlap(df.x, df.y).alias("overlap")).collect()
|
|
|
|
[Row(overlap=True), Row(overlap=False)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.arrays_overlap(_to_java_column(a1), _to_java_column(a2)))
|
|
|
|
|
|
|
|
|
2018-05-07 03:57:37 -04:00
|
|
|
def slice(x, start, length):
|
|
|
|
"""
|
|
|
|
Collection function: returns an array containing all the elements in `x` from index `start`
|
2019-09-24 05:57:54 -04:00
|
|
|
(array indices start at 1, or from the end if `start` is negative) with the specified `length`.
|
2019-12-16 02:29:09 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
x : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
the array to be sliced
|
2021-02-01 19:30:50 -05:00
|
|
|
start : :class:`~pyspark.sql.Column` or int
|
2020-11-02 20:00:49 -05:00
|
|
|
the starting index
|
2021-02-01 19:30:50 -05:00
|
|
|
length : :class:`~pyspark.sql.Column` or int
|
2020-11-02 20:00:49 -05:00
|
|
|
the length of the slice
|
2019-12-16 02:29:09 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2018-05-07 03:57:37 -04:00
|
|
|
>>> df = spark.createDataFrame([([1, 2, 3],), ([4, 5],)], ['x'])
|
|
|
|
>>> df.select(slice(df.x, 2, 2).alias("sliced")).collect()
|
|
|
|
[Row(sliced=[2, 3]), Row(sliced=[5])]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
2020-07-23 00:53:50 -04:00
|
|
|
return Column(sc._jvm.functions.slice(
|
|
|
|
_to_java_column(x),
|
|
|
|
start._jc if isinstance(start, Column) else start,
|
|
|
|
length._jc if isinstance(length, Column) else length
|
|
|
|
))
|
2018-05-07 03:57:37 -04:00
|
|
|
|
|
|
|
|
2018-04-26 00:37:13 -04:00
|
|
|
def array_join(col, delimiter, null_replacement=None):
|
|
|
|
"""
|
|
|
|
Concatenates the elements of `column` using the `delimiter`. Null values are replaced with
|
|
|
|
`null_replacement` if set, otherwise they are ignored.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.4.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2018-04-26 00:37:13 -04:00
|
|
|
>>> df = spark.createDataFrame([(["a", "b", "c"],), (["a", None],)], ['data'])
|
|
|
|
>>> df.select(array_join(df.data, ",").alias("joined")).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(joined='a,b,c'), Row(joined='a')]
|
2018-04-26 00:37:13 -04:00
|
|
|
>>> df.select(array_join(df.data, ",", "NULL").alias("joined")).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(joined='a,b,c'), Row(joined='a,NULL')]
|
2018-04-26 00:37:13 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
if null_replacement is None:
|
|
|
|
return Column(sc._jvm.functions.array_join(_to_java_column(col), delimiter))
|
|
|
|
else:
|
|
|
|
return Column(sc._jvm.functions.array_join(
|
|
|
|
_to_java_column(col), delimiter, null_replacement))
|
|
|
|
|
|
|
|
|
[SPARK-23736][SQL] Extending the concat function to support array columns
## What changes were proposed in this pull request?
The PR adds a logic for easy concatenation of multiple array columns and covers:
- Concat expression has been extended to support array columns
- A Python wrapper
## How was this patch tested?
New tests added into:
- CollectionExpressionsSuite
- DataFrameFunctionsSuite
- typeCoercion/native/concat.sql
## Codegen examples
### Primitive-type elements
```
val df = Seq(
(Seq(1 ,2), Seq(3, 4)),
(Seq(1, 2, 3), null)
).toDF("a", "b")
df.filter('a.isNotNull).select(concat('a, 'b)).debugCodegen()
```
Result:
```
/* 033 */ boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 034 */ ArrayData inputadapter_value = inputadapter_isNull ?
/* 035 */ null : (inputadapter_row.getArray(0));
/* 036 */
/* 037 */ if (!(!inputadapter_isNull)) continue;
/* 038 */
/* 039 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 040 */
/* 041 */ ArrayData[] project_args = new ArrayData[2];
/* 042 */
/* 043 */ if (!false) {
/* 044 */ project_args[0] = inputadapter_value;
/* 045 */ }
/* 046 */
/* 047 */ boolean inputadapter_isNull1 = inputadapter_row.isNullAt(1);
/* 048 */ ArrayData inputadapter_value1 = inputadapter_isNull1 ?
/* 049 */ null : (inputadapter_row.getArray(1));
/* 050 */ if (!inputadapter_isNull1) {
/* 051 */ project_args[1] = inputadapter_value1;
/* 052 */ }
/* 053 */
/* 054 */ ArrayData project_value = new Object() {
/* 055 */ public ArrayData concat(ArrayData[] args) {
/* 056 */ for (int z = 0; z < 2; z++) {
/* 057 */ if (args[z] == null) return null;
/* 058 */ }
/* 059 */
/* 060 */ long project_numElements = 0L;
/* 061 */ for (int z = 0; z < 2; z++) {
/* 062 */ project_numElements += args[z].numElements();
/* 063 */ }
/* 064 */ if (project_numElements > 2147483632) {
/* 065 */ throw new RuntimeException("Unsuccessful try to concat arrays with " + project_numElements +
/* 066 */ " elements due to exceeding the array size limit 2147483632.");
/* 067 */ }
/* 068 */
/* 069 */ long project_size = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(
/* 070 */ project_numElements,
/* 071 */ 4);
/* 072 */ if (project_size > 2147483632) {
/* 073 */ throw new RuntimeException("Unsuccessful try to concat arrays with " + project_size +
/* 074 */ " bytes of data due to exceeding the limit 2147483632 bytes" +
/* 075 */ " for UnsafeArrayData.");
/* 076 */ }
/* 077 */
/* 078 */ byte[] project_array = new byte[(int)project_size];
/* 079 */ UnsafeArrayData project_arrayData = new UnsafeArrayData();
/* 080 */ Platform.putLong(project_array, 16, project_numElements);
/* 081 */ project_arrayData.pointTo(project_array, 16, (int)project_size);
/* 082 */ int project_counter = 0;
/* 083 */ for (int y = 0; y < 2; y++) {
/* 084 */ for (int z = 0; z < args[y].numElements(); z++) {
/* 085 */ if (args[y].isNullAt(z)) {
/* 086 */ project_arrayData.setNullAt(project_counter);
/* 087 */ } else {
/* 088 */ project_arrayData.setInt(
/* 089 */ project_counter,
/* 090 */ args[y].getInt(z)
/* 091 */ );
/* 092 */ }
/* 093 */ project_counter++;
/* 094 */ }
/* 095 */ }
/* 096 */ return project_arrayData;
/* 097 */ }
/* 098 */ }.concat(project_args);
/* 099 */ boolean project_isNull = project_value == null;
```
### Non-primitive-type elements
```
val df = Seq(
(Seq("aa" ,"bb"), Seq("ccc", "ddd")),
(Seq("x", "y"), null)
).toDF("a", "b")
df.filter('a.isNotNull).select(concat('a, 'b)).debugCodegen()
```
Result:
```
/* 033 */ boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 034 */ ArrayData inputadapter_value = inputadapter_isNull ?
/* 035 */ null : (inputadapter_row.getArray(0));
/* 036 */
/* 037 */ if (!(!inputadapter_isNull)) continue;
/* 038 */
/* 039 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 040 */
/* 041 */ ArrayData[] project_args = new ArrayData[2];
/* 042 */
/* 043 */ if (!false) {
/* 044 */ project_args[0] = inputadapter_value;
/* 045 */ }
/* 046 */
/* 047 */ boolean inputadapter_isNull1 = inputadapter_row.isNullAt(1);
/* 048 */ ArrayData inputadapter_value1 = inputadapter_isNull1 ?
/* 049 */ null : (inputadapter_row.getArray(1));
/* 050 */ if (!inputadapter_isNull1) {
/* 051 */ project_args[1] = inputadapter_value1;
/* 052 */ }
/* 053 */
/* 054 */ ArrayData project_value = new Object() {
/* 055 */ public ArrayData concat(ArrayData[] args) {
/* 056 */ for (int z = 0; z < 2; z++) {
/* 057 */ if (args[z] == null) return null;
/* 058 */ }
/* 059 */
/* 060 */ long project_numElements = 0L;
/* 061 */ for (int z = 0; z < 2; z++) {
/* 062 */ project_numElements += args[z].numElements();
/* 063 */ }
/* 064 */ if (project_numElements > 2147483632) {
/* 065 */ throw new RuntimeException("Unsuccessful try to concat arrays with " + project_numElements +
/* 066 */ " elements due to exceeding the array size limit 2147483632.");
/* 067 */ }
/* 068 */
/* 069 */ Object[] project_arrayObjects = new Object[(int)project_numElements];
/* 070 */ int project_counter = 0;
/* 071 */ for (int y = 0; y < 2; y++) {
/* 072 */ for (int z = 0; z < args[y].numElements(); z++) {
/* 073 */ project_arrayObjects[project_counter] = args[y].getUTF8String(z);
/* 074 */ project_counter++;
/* 075 */ }
/* 076 */ }
/* 077 */ return new org.apache.spark.sql.catalyst.util.GenericArrayData(project_arrayObjects);
/* 078 */ }
/* 079 */ }.concat(project_args);
/* 080 */ boolean project_isNull = project_value == null;
```
Author: mn-mikke <mrkAha12346github>
Closes #20858 from mn-mikke/feature/array-api-concat_arrays-to-master.
2018-04-20 01:58:11 -04:00
|
|
|
def concat(*cols):
|
|
|
|
"""
|
|
|
|
Concatenates multiple input columns together into a single column.
|
|
|
|
The function works with strings, binary and compatible array columns.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
[SPARK-23736][SQL] Extending the concat function to support array columns
## What changes were proposed in this pull request?
The PR adds a logic for easy concatenation of multiple array columns and covers:
- Concat expression has been extended to support array columns
- A Python wrapper
## How was this patch tested?
New tests added into:
- CollectionExpressionsSuite
- DataFrameFunctionsSuite
- typeCoercion/native/concat.sql
## Codegen examples
### Primitive-type elements
```
val df = Seq(
(Seq(1 ,2), Seq(3, 4)),
(Seq(1, 2, 3), null)
).toDF("a", "b")
df.filter('a.isNotNull).select(concat('a, 'b)).debugCodegen()
```
Result:
```
/* 033 */ boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 034 */ ArrayData inputadapter_value = inputadapter_isNull ?
/* 035 */ null : (inputadapter_row.getArray(0));
/* 036 */
/* 037 */ if (!(!inputadapter_isNull)) continue;
/* 038 */
/* 039 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 040 */
/* 041 */ ArrayData[] project_args = new ArrayData[2];
/* 042 */
/* 043 */ if (!false) {
/* 044 */ project_args[0] = inputadapter_value;
/* 045 */ }
/* 046 */
/* 047 */ boolean inputadapter_isNull1 = inputadapter_row.isNullAt(1);
/* 048 */ ArrayData inputadapter_value1 = inputadapter_isNull1 ?
/* 049 */ null : (inputadapter_row.getArray(1));
/* 050 */ if (!inputadapter_isNull1) {
/* 051 */ project_args[1] = inputadapter_value1;
/* 052 */ }
/* 053 */
/* 054 */ ArrayData project_value = new Object() {
/* 055 */ public ArrayData concat(ArrayData[] args) {
/* 056 */ for (int z = 0; z < 2; z++) {
/* 057 */ if (args[z] == null) return null;
/* 058 */ }
/* 059 */
/* 060 */ long project_numElements = 0L;
/* 061 */ for (int z = 0; z < 2; z++) {
/* 062 */ project_numElements += args[z].numElements();
/* 063 */ }
/* 064 */ if (project_numElements > 2147483632) {
/* 065 */ throw new RuntimeException("Unsuccessful try to concat arrays with " + project_numElements +
/* 066 */ " elements due to exceeding the array size limit 2147483632.");
/* 067 */ }
/* 068 */
/* 069 */ long project_size = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(
/* 070 */ project_numElements,
/* 071 */ 4);
/* 072 */ if (project_size > 2147483632) {
/* 073 */ throw new RuntimeException("Unsuccessful try to concat arrays with " + project_size +
/* 074 */ " bytes of data due to exceeding the limit 2147483632 bytes" +
/* 075 */ " for UnsafeArrayData.");
/* 076 */ }
/* 077 */
/* 078 */ byte[] project_array = new byte[(int)project_size];
/* 079 */ UnsafeArrayData project_arrayData = new UnsafeArrayData();
/* 080 */ Platform.putLong(project_array, 16, project_numElements);
/* 081 */ project_arrayData.pointTo(project_array, 16, (int)project_size);
/* 082 */ int project_counter = 0;
/* 083 */ for (int y = 0; y < 2; y++) {
/* 084 */ for (int z = 0; z < args[y].numElements(); z++) {
/* 085 */ if (args[y].isNullAt(z)) {
/* 086 */ project_arrayData.setNullAt(project_counter);
/* 087 */ } else {
/* 088 */ project_arrayData.setInt(
/* 089 */ project_counter,
/* 090 */ args[y].getInt(z)
/* 091 */ );
/* 092 */ }
/* 093 */ project_counter++;
/* 094 */ }
/* 095 */ }
/* 096 */ return project_arrayData;
/* 097 */ }
/* 098 */ }.concat(project_args);
/* 099 */ boolean project_isNull = project_value == null;
```
### Non-primitive-type elements
```
val df = Seq(
(Seq("aa" ,"bb"), Seq("ccc", "ddd")),
(Seq("x", "y"), null)
).toDF("a", "b")
df.filter('a.isNotNull).select(concat('a, 'b)).debugCodegen()
```
Result:
```
/* 033 */ boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 034 */ ArrayData inputadapter_value = inputadapter_isNull ?
/* 035 */ null : (inputadapter_row.getArray(0));
/* 036 */
/* 037 */ if (!(!inputadapter_isNull)) continue;
/* 038 */
/* 039 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 040 */
/* 041 */ ArrayData[] project_args = new ArrayData[2];
/* 042 */
/* 043 */ if (!false) {
/* 044 */ project_args[0] = inputadapter_value;
/* 045 */ }
/* 046 */
/* 047 */ boolean inputadapter_isNull1 = inputadapter_row.isNullAt(1);
/* 048 */ ArrayData inputadapter_value1 = inputadapter_isNull1 ?
/* 049 */ null : (inputadapter_row.getArray(1));
/* 050 */ if (!inputadapter_isNull1) {
/* 051 */ project_args[1] = inputadapter_value1;
/* 052 */ }
/* 053 */
/* 054 */ ArrayData project_value = new Object() {
/* 055 */ public ArrayData concat(ArrayData[] args) {
/* 056 */ for (int z = 0; z < 2; z++) {
/* 057 */ if (args[z] == null) return null;
/* 058 */ }
/* 059 */
/* 060 */ long project_numElements = 0L;
/* 061 */ for (int z = 0; z < 2; z++) {
/* 062 */ project_numElements += args[z].numElements();
/* 063 */ }
/* 064 */ if (project_numElements > 2147483632) {
/* 065 */ throw new RuntimeException("Unsuccessful try to concat arrays with " + project_numElements +
/* 066 */ " elements due to exceeding the array size limit 2147483632.");
/* 067 */ }
/* 068 */
/* 069 */ Object[] project_arrayObjects = new Object[(int)project_numElements];
/* 070 */ int project_counter = 0;
/* 071 */ for (int y = 0; y < 2; y++) {
/* 072 */ for (int z = 0; z < args[y].numElements(); z++) {
/* 073 */ project_arrayObjects[project_counter] = args[y].getUTF8String(z);
/* 074 */ project_counter++;
/* 075 */ }
/* 076 */ }
/* 077 */ return new org.apache.spark.sql.catalyst.util.GenericArrayData(project_arrayObjects);
/* 078 */ }
/* 079 */ }.concat(project_args);
/* 080 */ boolean project_isNull = project_value == null;
```
Author: mn-mikke <mrkAha12346github>
Closes #20858 from mn-mikke/feature/array-api-concat_arrays-to-master.
2018-04-20 01:58:11 -04:00
|
|
|
>>> df = spark.createDataFrame([('abcd','123')], ['s', 'd'])
|
|
|
|
>>> df.select(concat(df.s, df.d).alias('s')).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(s='abcd123')]
|
[SPARK-23736][SQL] Extending the concat function to support array columns
## What changes were proposed in this pull request?
The PR adds a logic for easy concatenation of multiple array columns and covers:
- Concat expression has been extended to support array columns
- A Python wrapper
## How was this patch tested?
New tests added into:
- CollectionExpressionsSuite
- DataFrameFunctionsSuite
- typeCoercion/native/concat.sql
## Codegen examples
### Primitive-type elements
```
val df = Seq(
(Seq(1 ,2), Seq(3, 4)),
(Seq(1, 2, 3), null)
).toDF("a", "b")
df.filter('a.isNotNull).select(concat('a, 'b)).debugCodegen()
```
Result:
```
/* 033 */ boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 034 */ ArrayData inputadapter_value = inputadapter_isNull ?
/* 035 */ null : (inputadapter_row.getArray(0));
/* 036 */
/* 037 */ if (!(!inputadapter_isNull)) continue;
/* 038 */
/* 039 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 040 */
/* 041 */ ArrayData[] project_args = new ArrayData[2];
/* 042 */
/* 043 */ if (!false) {
/* 044 */ project_args[0] = inputadapter_value;
/* 045 */ }
/* 046 */
/* 047 */ boolean inputadapter_isNull1 = inputadapter_row.isNullAt(1);
/* 048 */ ArrayData inputadapter_value1 = inputadapter_isNull1 ?
/* 049 */ null : (inputadapter_row.getArray(1));
/* 050 */ if (!inputadapter_isNull1) {
/* 051 */ project_args[1] = inputadapter_value1;
/* 052 */ }
/* 053 */
/* 054 */ ArrayData project_value = new Object() {
/* 055 */ public ArrayData concat(ArrayData[] args) {
/* 056 */ for (int z = 0; z < 2; z++) {
/* 057 */ if (args[z] == null) return null;
/* 058 */ }
/* 059 */
/* 060 */ long project_numElements = 0L;
/* 061 */ for (int z = 0; z < 2; z++) {
/* 062 */ project_numElements += args[z].numElements();
/* 063 */ }
/* 064 */ if (project_numElements > 2147483632) {
/* 065 */ throw new RuntimeException("Unsuccessful try to concat arrays with " + project_numElements +
/* 066 */ " elements due to exceeding the array size limit 2147483632.");
/* 067 */ }
/* 068 */
/* 069 */ long project_size = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(
/* 070 */ project_numElements,
/* 071 */ 4);
/* 072 */ if (project_size > 2147483632) {
/* 073 */ throw new RuntimeException("Unsuccessful try to concat arrays with " + project_size +
/* 074 */ " bytes of data due to exceeding the limit 2147483632 bytes" +
/* 075 */ " for UnsafeArrayData.");
/* 076 */ }
/* 077 */
/* 078 */ byte[] project_array = new byte[(int)project_size];
/* 079 */ UnsafeArrayData project_arrayData = new UnsafeArrayData();
/* 080 */ Platform.putLong(project_array, 16, project_numElements);
/* 081 */ project_arrayData.pointTo(project_array, 16, (int)project_size);
/* 082 */ int project_counter = 0;
/* 083 */ for (int y = 0; y < 2; y++) {
/* 084 */ for (int z = 0; z < args[y].numElements(); z++) {
/* 085 */ if (args[y].isNullAt(z)) {
/* 086 */ project_arrayData.setNullAt(project_counter);
/* 087 */ } else {
/* 088 */ project_arrayData.setInt(
/* 089 */ project_counter,
/* 090 */ args[y].getInt(z)
/* 091 */ );
/* 092 */ }
/* 093 */ project_counter++;
/* 094 */ }
/* 095 */ }
/* 096 */ return project_arrayData;
/* 097 */ }
/* 098 */ }.concat(project_args);
/* 099 */ boolean project_isNull = project_value == null;
```
### Non-primitive-type elements
```
val df = Seq(
(Seq("aa" ,"bb"), Seq("ccc", "ddd")),
(Seq("x", "y"), null)
).toDF("a", "b")
df.filter('a.isNotNull).select(concat('a, 'b)).debugCodegen()
```
Result:
```
/* 033 */ boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 034 */ ArrayData inputadapter_value = inputadapter_isNull ?
/* 035 */ null : (inputadapter_row.getArray(0));
/* 036 */
/* 037 */ if (!(!inputadapter_isNull)) continue;
/* 038 */
/* 039 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 040 */
/* 041 */ ArrayData[] project_args = new ArrayData[2];
/* 042 */
/* 043 */ if (!false) {
/* 044 */ project_args[0] = inputadapter_value;
/* 045 */ }
/* 046 */
/* 047 */ boolean inputadapter_isNull1 = inputadapter_row.isNullAt(1);
/* 048 */ ArrayData inputadapter_value1 = inputadapter_isNull1 ?
/* 049 */ null : (inputadapter_row.getArray(1));
/* 050 */ if (!inputadapter_isNull1) {
/* 051 */ project_args[1] = inputadapter_value1;
/* 052 */ }
/* 053 */
/* 054 */ ArrayData project_value = new Object() {
/* 055 */ public ArrayData concat(ArrayData[] args) {
/* 056 */ for (int z = 0; z < 2; z++) {
/* 057 */ if (args[z] == null) return null;
/* 058 */ }
/* 059 */
/* 060 */ long project_numElements = 0L;
/* 061 */ for (int z = 0; z < 2; z++) {
/* 062 */ project_numElements += args[z].numElements();
/* 063 */ }
/* 064 */ if (project_numElements > 2147483632) {
/* 065 */ throw new RuntimeException("Unsuccessful try to concat arrays with " + project_numElements +
/* 066 */ " elements due to exceeding the array size limit 2147483632.");
/* 067 */ }
/* 068 */
/* 069 */ Object[] project_arrayObjects = new Object[(int)project_numElements];
/* 070 */ int project_counter = 0;
/* 071 */ for (int y = 0; y < 2; y++) {
/* 072 */ for (int z = 0; z < args[y].numElements(); z++) {
/* 073 */ project_arrayObjects[project_counter] = args[y].getUTF8String(z);
/* 074 */ project_counter++;
/* 075 */ }
/* 076 */ }
/* 077 */ return new org.apache.spark.sql.catalyst.util.GenericArrayData(project_arrayObjects);
/* 078 */ }
/* 079 */ }.concat(project_args);
/* 080 */ boolean project_isNull = project_value == null;
```
Author: mn-mikke <mrkAha12346github>
Closes #20858 from mn-mikke/feature/array-api-concat_arrays-to-master.
2018-04-20 01:58:11 -04:00
|
|
|
|
|
|
|
>>> df = spark.createDataFrame([([1, 2], [3, 4], [5]), ([1, 2], None, [3])], ['a', 'b', 'c'])
|
|
|
|
>>> df.select(concat(df.a, df.b, df.c).alias("arr")).collect()
|
|
|
|
[Row(arr=[1, 2, 3, 4, 5]), Row(arr=None)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.concat(_to_seq(sc, cols, _to_java_column)))
|
|
|
|
|
|
|
|
|
2018-04-18 22:59:17 -04:00
|
|
|
def array_position(col, value):
|
|
|
|
"""
|
|
|
|
Collection function: Locates the position of the first occurrence of the given value
|
|
|
|
in the given array. Returns null if either of the arguments are null.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.4.0
|
2018-04-18 22:59:17 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
The position is not zero based, but 1 based index. Returns 0 if the given
|
|
|
|
value could not be found in the array.
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2018-04-18 22:59:17 -04:00
|
|
|
>>> df = spark.createDataFrame([(["c", "b", "a"],), ([],)], ['data'])
|
|
|
|
>>> df.select(array_position(df.data, "a")).collect()
|
|
|
|
[Row(array_position(data, a)=3), Row(array_position(data, a)=0)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.array_position(_to_java_column(col), value))
|
|
|
|
|
|
|
|
|
2018-04-19 08:00:10 -04:00
|
|
|
def element_at(col, extraction):
|
|
|
|
"""
|
|
|
|
Collection function: Returns element of array at given index in extraction if col is array.
|
|
|
|
Returns value for the given key in extraction if col is map.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column containing array or map
|
|
|
|
extraction :
|
|
|
|
index to check for in array or key to check for in map
|
2018-04-19 08:00:10 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
The position is not zero based, but 1 based index.
|
2018-04-19 08:00:10 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2018-04-19 08:00:10 -04:00
|
|
|
>>> df = spark.createDataFrame([(["a", "b", "c"],), ([],)], ['data'])
|
|
|
|
>>> df.select(element_at(df.data, 1)).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(element_at(data, 1)='a'), Row(element_at(data, 1)=None)]
|
2018-04-19 08:00:10 -04:00
|
|
|
|
|
|
|
>>> df = spark.createDataFrame([({"a": 1.0, "b": 2.0},), ({},)], ['data'])
|
[SPARK-29240][PYTHON] Pass Py4J column instance to support PySpark column in element_at function
### What changes were proposed in this pull request?
This PR makes `element_at` in PySpark able to take PySpark `Column` instances.
### Why are the changes needed?
To match with Scala side. Seems it was intended but not working correctly as a bug.
### Does this PR introduce any user-facing change?
Yes. See below:
```python
from pyspark.sql import functions as F
x = spark.createDataFrame([([1,2,3],1),([4,5,6],2),([7,8,9],3)],['list','num'])
x.withColumn('aa',F.element_at('list',x.num.cast('int'))).show()
```
Before:
```
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../spark/python/pyspark/sql/functions.py", line 2059, in element_at
return Column(sc._jvm.functions.element_at(_to_java_column(col), extraction))
File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1277, in __call__
File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1241, in _build_args
File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1228, in _get_args
File "/.../forked/spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_collections.py", line 500, in convert
File "/.../spark/python/pyspark/sql/column.py", line 344, in __iter__
raise TypeError("Column is not iterable")
TypeError: Column is not iterable
```
After:
```
+---------+---+---+
| list|num| aa|
+---------+---+---+
|[1, 2, 3]| 1| 1|
|[4, 5, 6]| 2| 5|
|[7, 8, 9]| 3| 9|
+---------+---+---+
```
### How was this patch tested?
Manually tested against literal, Python native types, and PySpark column.
Closes #25950 from HyukjinKwon/SPARK-29240.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-27 14:04:55 -04:00
|
|
|
>>> df.select(element_at(df.data, lit("a"))).collect()
|
2018-04-19 08:00:10 -04:00
|
|
|
[Row(element_at(data, a)=1.0), Row(element_at(data, a)=None)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
[SPARK-29240][PYTHON] Pass Py4J column instance to support PySpark column in element_at function
### What changes were proposed in this pull request?
This PR makes `element_at` in PySpark able to take PySpark `Column` instances.
### Why are the changes needed?
To match with Scala side. Seems it was intended but not working correctly as a bug.
### Does this PR introduce any user-facing change?
Yes. See below:
```python
from pyspark.sql import functions as F
x = spark.createDataFrame([([1,2,3],1),([4,5,6],2),([7,8,9],3)],['list','num'])
x.withColumn('aa',F.element_at('list',x.num.cast('int'))).show()
```
Before:
```
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/.../spark/python/pyspark/sql/functions.py", line 2059, in element_at
return Column(sc._jvm.functions.element_at(_to_java_column(col), extraction))
File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1277, in __call__
File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1241, in _build_args
File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1228, in _get_args
File "/.../forked/spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_collections.py", line 500, in convert
File "/.../spark/python/pyspark/sql/column.py", line 344, in __iter__
raise TypeError("Column is not iterable")
TypeError: Column is not iterable
```
After:
```
+---------+---+---+
| list|num| aa|
+---------+---+---+
|[1, 2, 3]| 1| 1|
|[4, 5, 6]| 2| 5|
|[7, 8, 9]| 3| 9|
+---------+---+---+
```
### How was this patch tested?
Manually tested against literal, Python native types, and PySpark column.
Closes #25950 from HyukjinKwon/SPARK-29240.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-27 14:04:55 -04:00
|
|
|
return Column(sc._jvm.functions.element_at(
|
2020-10-26 22:05:53 -04:00
|
|
|
_to_java_column(col), lit(extraction)._jc))
|
2018-04-19 08:00:10 -04:00
|
|
|
|
|
|
|
|
2018-06-01 01:04:26 -04:00
|
|
|
def array_remove(col, element):
|
|
|
|
"""
|
|
|
|
Collection function: Remove all elements that equal to element from the given array.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column containing array
|
|
|
|
element :
|
|
|
|
element to be removed from the array
|
2018-06-01 01:04:26 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2018-06-01 01:04:26 -04:00
|
|
|
>>> df = spark.createDataFrame([([1, 2, 3, 1, 1],), ([],)], ['data'])
|
|
|
|
>>> df.select(array_remove(df.data, 1)).collect()
|
|
|
|
[Row(array_remove(data, 1)=[2, 3]), Row(array_remove(data, 1)=[])]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.array_remove(_to_java_column(col), element))
|
|
|
|
|
|
|
|
|
2018-06-20 23:24:53 -04:00
|
|
|
def array_distinct(col):
|
|
|
|
"""
|
|
|
|
Collection function: removes duplicate values from the array.
|
2020-02-18 02:46:45 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.4.0
|
2018-06-20 23:24:53 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column or expression
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2018-06-20 23:24:53 -04:00
|
|
|
>>> df = spark.createDataFrame([([1, 2, 3, 2],), ([4, 5, 5, 4],)], ['data'])
|
|
|
|
>>> df.select(array_distinct(df.data)).collect()
|
|
|
|
[Row(array_distinct(data)=[1, 2, 3]), Row(array_distinct(data)=[4, 5])]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.array_distinct(_to_java_column(col)))
|
|
|
|
|
|
|
|
|
2018-08-06 10:27:57 -04:00
|
|
|
def array_intersect(col1, col2):
|
|
|
|
"""
|
|
|
|
Collection function: returns an array of the elements in the intersection of col1 and col2,
|
|
|
|
without duplicates.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col1 : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column containing array
|
2021-02-01 19:30:50 -05:00
|
|
|
col2 : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column containing array
|
2018-08-06 10:27:57 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2018-08-06 10:27:57 -04:00
|
|
|
>>> from pyspark.sql import Row
|
|
|
|
>>> df = spark.createDataFrame([Row(c1=["b", "a", "c"], c2=["c", "d", "a", "f"])])
|
|
|
|
>>> df.select(array_intersect(df.c1, df.c2)).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(array_intersect(c1, c2)=['a', 'c'])]
|
2018-08-06 10:27:57 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.array_intersect(_to_java_column(col1), _to_java_column(col2)))
|
|
|
|
|
|
|
|
|
2018-07-12 04:42:29 -04:00
|
|
|
def array_union(col1, col2):
|
|
|
|
"""
|
|
|
|
Collection function: returns an array of the elements in the union of col1 and col2,
|
|
|
|
without duplicates.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col1 : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column containing array
|
2021-02-01 19:30:50 -05:00
|
|
|
col2 : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column containing array
|
2018-07-12 04:42:29 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2018-07-12 04:42:29 -04:00
|
|
|
>>> from pyspark.sql import Row
|
|
|
|
>>> df = spark.createDataFrame([Row(c1=["b", "a", "c"], c2=["c", "d", "a", "f"])])
|
|
|
|
>>> df.select(array_union(df.c1, df.c2)).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(array_union(c1, c2)=['b', 'a', 'c', 'd', 'f'])]
|
2018-07-12 04:42:29 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.array_union(_to_java_column(col1), _to_java_column(col2)))
|
|
|
|
|
|
|
|
|
2018-08-01 14:52:30 -04:00
|
|
|
def array_except(col1, col2):
|
|
|
|
"""
|
|
|
|
Collection function: returns an array of the elements in col1 but not in col2,
|
|
|
|
without duplicates.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col1 : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column containing array
|
2021-02-01 19:30:50 -05:00
|
|
|
col2 : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column containing array
|
2018-08-01 14:52:30 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2018-08-01 14:52:30 -04:00
|
|
|
>>> from pyspark.sql import Row
|
|
|
|
>>> df = spark.createDataFrame([Row(c1=["b", "a", "c"], c2=["c", "d", "a", "f"])])
|
|
|
|
>>> df.select(array_except(df.c1, df.c2)).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(array_except(c1, c2)=['b'])]
|
2018-08-01 14:52:30 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.array_except(_to_java_column(col1), _to_java_column(col2)))
|
|
|
|
|
|
|
|
|
2015-08-04 22:25:24 -04:00
|
|
|
def explode(col):
|
2019-04-26 21:30:12 -04:00
|
|
|
"""
|
|
|
|
Returns a new row for each element in the given array or map.
|
|
|
|
Uses the default column name `col` for elements in the array and
|
|
|
|
`key` and `value` for elements in the map unless specified otherwise.
|
2015-08-04 22:25:24 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.4.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2015-08-04 22:25:24 -04:00
|
|
|
>>> from pyspark.sql import Row
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> eDF = spark.createDataFrame([Row(a=1, intlist=[1,2,3], mapfield={"a": "b"})])
|
2015-08-04 22:25:24 -04:00
|
|
|
>>> eDF.select(explode(eDF.intlist).alias("anInt")).collect()
|
|
|
|
[Row(anInt=1), Row(anInt=2), Row(anInt=3)]
|
|
|
|
|
|
|
|
>>> eDF.select(explode(eDF.mapfield).alias("key", "value")).show()
|
|
|
|
+---+-----+
|
|
|
|
|key|value|
|
|
|
|
+---+-----+
|
|
|
|
| a| b|
|
|
|
|
+---+-----+
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
jc = sc._jvm.functions.explode(_to_java_column(col))
|
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
[SPARK-16289][SQL] Implement posexplode table generating function
## What changes were proposed in this pull request?
This PR implements `posexplode` table generating function. Currently, master branch raises the following exception for `map` argument. It's different from Hive.
**Before**
```scala
scala> sql("select posexplode(map('a', 1, 'b', 2))").show
org.apache.spark.sql.AnalysisException: No handler for Hive UDF ... posexplode() takes an array as a parameter; line 1 pos 7
```
**After**
```scala
scala> sql("select posexplode(map('a', 1, 'b', 2))").show
+---+---+-----+
|pos|key|value|
+---+---+-----+
| 0| a| 1|
| 1| b| 2|
+---+---+-----+
```
For `array` argument, `after` is the same with `before`.
```
scala> sql("select posexplode(array(1, 2, 3))").show
+---+---+
|pos|col|
+---+---+
| 0| 1|
| 1| 2|
| 2| 3|
+---+---+
```
## How was this patch tested?
Pass the Jenkins tests with newly added testcases.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes #13971 from dongjoon-hyun/SPARK-16289.
2016-06-30 15:03:54 -04:00
|
|
|
def posexplode(col):
|
2019-04-26 21:30:12 -04:00
|
|
|
"""
|
|
|
|
Returns a new row for each element with position in the given array or map.
|
|
|
|
Uses the default column name `pos` for position, and `col` for elements in the
|
|
|
|
array and `key` and `value` for elements in the map unless specified otherwise.
|
[SPARK-16289][SQL] Implement posexplode table generating function
## What changes were proposed in this pull request?
This PR implements `posexplode` table generating function. Currently, master branch raises the following exception for `map` argument. It's different from Hive.
**Before**
```scala
scala> sql("select posexplode(map('a', 1, 'b', 2))").show
org.apache.spark.sql.AnalysisException: No handler for Hive UDF ... posexplode() takes an array as a parameter; line 1 pos 7
```
**After**
```scala
scala> sql("select posexplode(map('a', 1, 'b', 2))").show
+---+---+-----+
|pos|key|value|
+---+---+-----+
| 0| a| 1|
| 1| b| 2|
+---+---+-----+
```
For `array` argument, `after` is the same with `before`.
```
scala> sql("select posexplode(array(1, 2, 3))").show
+---+---+
|pos|col|
+---+---+
| 0| 1|
| 1| 2|
| 2| 3|
+---+---+
```
## How was this patch tested?
Pass the Jenkins tests with newly added testcases.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes #13971 from dongjoon-hyun/SPARK-16289.
2016-06-30 15:03:54 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.1.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
[SPARK-16289][SQL] Implement posexplode table generating function
## What changes were proposed in this pull request?
This PR implements `posexplode` table generating function. Currently, master branch raises the following exception for `map` argument. It's different from Hive.
**Before**
```scala
scala> sql("select posexplode(map('a', 1, 'b', 2))").show
org.apache.spark.sql.AnalysisException: No handler for Hive UDF ... posexplode() takes an array as a parameter; line 1 pos 7
```
**After**
```scala
scala> sql("select posexplode(map('a', 1, 'b', 2))").show
+---+---+-----+
|pos|key|value|
+---+---+-----+
| 0| a| 1|
| 1| b| 2|
+---+---+-----+
```
For `array` argument, `after` is the same with `before`.
```
scala> sql("select posexplode(array(1, 2, 3))").show
+---+---+
|pos|col|
+---+---+
| 0| 1|
| 1| 2|
| 2| 3|
+---+---+
```
## How was this patch tested?
Pass the Jenkins tests with newly added testcases.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes #13971 from dongjoon-hyun/SPARK-16289.
2016-06-30 15:03:54 -04:00
|
|
|
>>> from pyspark.sql import Row
|
|
|
|
>>> eDF = spark.createDataFrame([Row(a=1, intlist=[1,2,3], mapfield={"a": "b"})])
|
|
|
|
>>> eDF.select(posexplode(eDF.intlist)).collect()
|
|
|
|
[Row(pos=0, col=1), Row(pos=1, col=2), Row(pos=2, col=3)]
|
|
|
|
|
|
|
|
>>> eDF.select(posexplode(eDF.mapfield)).show()
|
|
|
|
+---+---+-----+
|
|
|
|
|pos|key|value|
|
|
|
|
+---+---+-----+
|
|
|
|
| 0| a| b|
|
|
|
|
+---+---+-----+
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
jc = sc._jvm.functions.posexplode(_to_java_column(col))
|
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
2017-06-21 17:59:52 -04:00
|
|
|
def explode_outer(col):
|
2019-04-26 21:30:12 -04:00
|
|
|
"""
|
|
|
|
Returns a new row for each element in the given array or map.
|
2017-06-21 17:59:52 -04:00
|
|
|
Unlike explode, if the array/map is null or empty then null is produced.
|
2019-04-26 21:30:12 -04:00
|
|
|
Uses the default column name `col` for elements in the array and
|
|
|
|
`key` and `value` for elements in the map unless specified otherwise.
|
2017-06-21 17:59:52 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.3.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2017-06-21 17:59:52 -04:00
|
|
|
>>> df = spark.createDataFrame(
|
|
|
|
... [(1, ["foo", "bar"], {"x": 1.0}), (2, [], {}), (3, None, None)],
|
|
|
|
... ("id", "an_array", "a_map")
|
|
|
|
... )
|
|
|
|
>>> df.select("id", "an_array", explode_outer("a_map")).show()
|
|
|
|
+---+----------+----+-----+
|
|
|
|
| id| an_array| key|value|
|
|
|
|
+---+----------+----+-----+
|
|
|
|
| 1|[foo, bar]| x| 1.0|
|
|
|
|
| 2| []|null| null|
|
|
|
|
| 3| null|null| null|
|
|
|
|
+---+----------+----+-----+
|
|
|
|
|
|
|
|
>>> df.select("id", "a_map", explode_outer("an_array")).show()
|
[SPARK-23023][SQL] Cast field data to strings in showString
## What changes were proposed in this pull request?
The current `Datset.showString` prints rows thru `RowEncoder` deserializers like;
```
scala> Seq(Seq(Seq(1, 2), Seq(3), Seq(4, 5, 6))).toDF("a").show(false)
+------------------------------------------------------------+
|a |
+------------------------------------------------------------+
|[WrappedArray(1, 2), WrappedArray(3), WrappedArray(4, 5, 6)]|
+------------------------------------------------------------+
```
This result is incorrect because the correct one is;
```
scala> Seq(Seq(Seq(1, 2), Seq(3), Seq(4, 5, 6))).toDF("a").show(false)
+------------------------+
|a |
+------------------------+
|[[1, 2], [3], [4, 5, 6]]|
+------------------------+
```
So, this pr fixed code in `showString` to cast field data to strings before printing.
## How was this patch tested?
Added tests in `DataFrameSuite`.
Author: Takeshi Yamamuro <yamamuro@apache.org>
Closes #20214 from maropu/SPARK-23023.
2018-01-15 03:26:52 -05:00
|
|
|
+---+----------+----+
|
|
|
|
| id| a_map| col|
|
|
|
|
+---+----------+----+
|
2020-08-04 10:57:09 -04:00
|
|
|
| 1|{x -> 1.0}| foo|
|
|
|
|
| 1|{x -> 1.0}| bar|
|
|
|
|
| 2| {}|null|
|
[SPARK-23023][SQL] Cast field data to strings in showString
## What changes were proposed in this pull request?
The current `Datset.showString` prints rows thru `RowEncoder` deserializers like;
```
scala> Seq(Seq(Seq(1, 2), Seq(3), Seq(4, 5, 6))).toDF("a").show(false)
+------------------------------------------------------------+
|a |
+------------------------------------------------------------+
|[WrappedArray(1, 2), WrappedArray(3), WrappedArray(4, 5, 6)]|
+------------------------------------------------------------+
```
This result is incorrect because the correct one is;
```
scala> Seq(Seq(Seq(1, 2), Seq(3), Seq(4, 5, 6))).toDF("a").show(false)
+------------------------+
|a |
+------------------------+
|[[1, 2], [3], [4, 5, 6]]|
+------------------------+
```
So, this pr fixed code in `showString` to cast field data to strings before printing.
## How was this patch tested?
Added tests in `DataFrameSuite`.
Author: Takeshi Yamamuro <yamamuro@apache.org>
Closes #20214 from maropu/SPARK-23023.
2018-01-15 03:26:52 -05:00
|
|
|
| 3| null|null|
|
|
|
|
+---+----------+----+
|
2017-06-21 17:59:52 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
jc = sc._jvm.functions.explode_outer(_to_java_column(col))
|
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
|
|
|
def posexplode_outer(col):
|
2019-04-26 21:30:12 -04:00
|
|
|
"""
|
|
|
|
Returns a new row for each element with position in the given array or map.
|
2017-06-21 17:59:52 -04:00
|
|
|
Unlike posexplode, if the array/map is null or empty then the row (null, null) is produced.
|
2019-04-26 21:30:12 -04:00
|
|
|
Uses the default column name `pos` for position, and `col` for elements in the
|
|
|
|
array and `key` and `value` for elements in the map unless specified otherwise.
|
2017-06-21 17:59:52 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.3.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2017-06-21 17:59:52 -04:00
|
|
|
>>> df = spark.createDataFrame(
|
|
|
|
... [(1, ["foo", "bar"], {"x": 1.0}), (2, [], {}), (3, None, None)],
|
|
|
|
... ("id", "an_array", "a_map")
|
|
|
|
... )
|
|
|
|
>>> df.select("id", "an_array", posexplode_outer("a_map")).show()
|
|
|
|
+---+----------+----+----+-----+
|
|
|
|
| id| an_array| pos| key|value|
|
|
|
|
+---+----------+----+----+-----+
|
|
|
|
| 1|[foo, bar]| 0| x| 1.0|
|
|
|
|
| 2| []|null|null| null|
|
|
|
|
| 3| null|null|null| null|
|
|
|
|
+---+----------+----+----+-----+
|
|
|
|
>>> df.select("id", "a_map", posexplode_outer("an_array")).show()
|
[SPARK-23023][SQL] Cast field data to strings in showString
## What changes were proposed in this pull request?
The current `Datset.showString` prints rows thru `RowEncoder` deserializers like;
```
scala> Seq(Seq(Seq(1, 2), Seq(3), Seq(4, 5, 6))).toDF("a").show(false)
+------------------------------------------------------------+
|a |
+------------------------------------------------------------+
|[WrappedArray(1, 2), WrappedArray(3), WrappedArray(4, 5, 6)]|
+------------------------------------------------------------+
```
This result is incorrect because the correct one is;
```
scala> Seq(Seq(Seq(1, 2), Seq(3), Seq(4, 5, 6))).toDF("a").show(false)
+------------------------+
|a |
+------------------------+
|[[1, 2], [3], [4, 5, 6]]|
+------------------------+
```
So, this pr fixed code in `showString` to cast field data to strings before printing.
## How was this patch tested?
Added tests in `DataFrameSuite`.
Author: Takeshi Yamamuro <yamamuro@apache.org>
Closes #20214 from maropu/SPARK-23023.
2018-01-15 03:26:52 -05:00
|
|
|
+---+----------+----+----+
|
|
|
|
| id| a_map| pos| col|
|
|
|
|
+---+----------+----+----+
|
2020-08-04 10:57:09 -04:00
|
|
|
| 1|{x -> 1.0}| 0| foo|
|
|
|
|
| 1|{x -> 1.0}| 1| bar|
|
|
|
|
| 2| {}|null|null|
|
[SPARK-23023][SQL] Cast field data to strings in showString
## What changes were proposed in this pull request?
The current `Datset.showString` prints rows thru `RowEncoder` deserializers like;
```
scala> Seq(Seq(Seq(1, 2), Seq(3), Seq(4, 5, 6))).toDF("a").show(false)
+------------------------------------------------------------+
|a |
+------------------------------------------------------------+
|[WrappedArray(1, 2), WrappedArray(3), WrappedArray(4, 5, 6)]|
+------------------------------------------------------------+
```
This result is incorrect because the correct one is;
```
scala> Seq(Seq(Seq(1, 2), Seq(3), Seq(4, 5, 6))).toDF("a").show(false)
+------------------------+
|a |
+------------------------+
|[[1, 2], [3], [4, 5, 6]]|
+------------------------+
```
So, this pr fixed code in `showString` to cast field data to strings before printing.
## How was this patch tested?
Added tests in `DataFrameSuite`.
Author: Takeshi Yamamuro <yamamuro@apache.org>
Closes #20214 from maropu/SPARK-23023.
2018-01-15 03:26:52 -05:00
|
|
|
| 3| null|null|null|
|
|
|
|
+---+----------+----+----+
|
2017-06-21 17:59:52 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
jc = sc._jvm.functions.posexplode_outer(_to_java_column(col))
|
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
2015-11-25 00:30:53 -05:00
|
|
|
def get_json_object(col, path):
|
|
|
|
"""
|
|
|
|
Extracts json object from a json string based on json path specified, and returns json string
|
|
|
|
of the extracted json object. It will return null if the input json string is invalid.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.6.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
string column in json format
|
|
|
|
path : str
|
|
|
|
path to the json object to extract
|
2015-11-26 02:24:33 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2015-11-26 02:24:33 -05:00
|
|
|
>>> data = [("1", '''{"f1": "value1", "f2": "value2"}'''), ("2", '''{"f1": "value12"}''')]
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> df = spark.createDataFrame(data, ("key", "jstring"))
|
2016-07-06 13:45:51 -04:00
|
|
|
>>> df.select(df.key, get_json_object(df.jstring, '$.f1').alias("c0"), \\
|
|
|
|
... get_json_object(df.jstring, '$.f2').alias("c1") ).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(key='1', c0='value1', c1='value2'), Row(key='2', c0='value12', c1=None)]
|
2015-11-25 00:30:53 -05:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
jc = sc._jvm.functions.get_json_object(_to_java_column(col), path)
|
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
2015-11-26 02:24:33 -05:00
|
|
|
def json_tuple(col, *fields):
|
2015-11-25 00:30:53 -05:00
|
|
|
"""Creates a new row for a json column according to the given field names.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.6.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
string column in json format
|
|
|
|
fields : str
|
|
|
|
fields to extract
|
2015-11-25 00:30:53 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2015-11-26 02:24:33 -05:00
|
|
|
>>> data = [("1", '''{"f1": "value1", "f2": "value2"}'''), ("2", '''{"f1": "value12"}''')]
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> df = spark.createDataFrame(data, ("key", "jstring"))
|
2015-11-26 02:24:33 -05:00
|
|
|
>>> df.select(df.key, json_tuple(df.jstring, 'f1', 'f2')).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(key='1', c0='value1', c1='value2'), Row(key='2', c0='value12', c1=None)]
|
2015-11-25 00:30:53 -05:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
2015-11-26 02:24:33 -05:00
|
|
|
jc = sc._jvm.functions.json_tuple(_to_java_column(col), _to_seq(sc, fields))
|
2015-11-25 00:30:53 -05:00
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
[SPARK-32320][PYSPARK] Remove mutable default arguments
This is bad practice, and might lead to unexpected behaviour:
https://florimond.dev/blog/articles/2018/08/python-mutable-defaults-are-the-source-of-all-evil/
```
fokkodriesprongFan spark % grep -R "={}" python | grep def
python/pyspark/resource/profile.py: def __init__(self, _java_resource_profile=None, _exec_req={}, _task_req={}):
python/pyspark/sql/functions.py:def from_json(col, schema, options={}):
python/pyspark/sql/functions.py:def to_json(col, options={}):
python/pyspark/sql/functions.py:def schema_of_json(json, options={}):
python/pyspark/sql/functions.py:def schema_of_csv(csv, options={}):
python/pyspark/sql/functions.py:def to_csv(col, options={}):
python/pyspark/sql/functions.py:def from_csv(col, schema, options={}):
python/pyspark/sql/avro/functions.py:def from_avro(data, jsonFormatSchema, options={}):
```
```
fokkodriesprongFan spark % grep -R "=\[\]" python | grep def
python/pyspark/ml/tuning.py: def __init__(self, bestModel, avgMetrics=[], subModels=None):
python/pyspark/ml/tuning.py: def __init__(self, bestModel, validationMetrics=[], subModels=None):
```
### What changes were proposed in this pull request?
Removing the mutable default arguments.
### Why are the changes needed?
Removing the mutable default arguments, and changing the signature to `Optional[...]`.
### Does this PR introduce _any_ user-facing change?
No 👍
### How was this patch tested?
Using the Flake8 bugbear code analysis plugin.
Closes #29122 from Fokko/SPARK-32320.
Authored-by: Fokko Driesprong <fokko@apache.org>
Signed-off-by: Ruifeng Zheng <ruifengz@foxmail.com>
2020-12-07 20:35:36 -05:00
|
|
|
def from_json(col, schema, options=None):
|
2016-09-29 16:01:10 -04:00
|
|
|
"""
|
[SPARK-24027][SQL] Support MapType with StringType for keys as the root type by from_json
## What changes were proposed in this pull request?
Currently, the from_json function support StructType or ArrayType as the root type. The PR allows to specify MapType(StringType, DataType) as the root type additionally to mentioned types. For example:
```scala
import org.apache.spark.sql.types._
val schema = MapType(StringType, IntegerType)
val in = Seq("""{"a": 1, "b": 2, "c": 3}""").toDS()
in.select(from_json($"value", schema, Map[String, String]())).collect()
```
```
res1: Array[org.apache.spark.sql.Row] = Array([Map(a -> 1, b -> 2, c -> 3)])
```
## How was this patch tested?
It was checked by new tests for the map type with integer type and struct type as value types. Also roundtrip tests like from_json(to_json) and to_json(from_json) for MapType are added.
Author: Maxim Gekk <maxim.gekk@databricks.com>
Author: Maxim Gekk <max.gekk@gmail.com>
Closes #21108 from MaxGekk/from_json-map-type.
2018-05-14 17:05:42 -04:00
|
|
|
Parses a column containing a JSON string into a :class:`MapType` with :class:`StringType`
|
2018-08-13 08:13:09 -04:00
|
|
|
as keys type, :class:`StructType` or :class:`ArrayType` with
|
[SPARK-24027][SQL] Support MapType with StringType for keys as the root type by from_json
## What changes were proposed in this pull request?
Currently, the from_json function support StructType or ArrayType as the root type. The PR allows to specify MapType(StringType, DataType) as the root type additionally to mentioned types. For example:
```scala
import org.apache.spark.sql.types._
val schema = MapType(StringType, IntegerType)
val in = Seq("""{"a": 1, "b": 2, "c": 3}""").toDS()
in.select(from_json($"value", schema, Map[String, String]())).collect()
```
```
res1: Array[org.apache.spark.sql.Row] = Array([Map(a -> 1, b -> 2, c -> 3)])
```
## How was this patch tested?
It was checked by new tests for the map type with integer type and struct type as value types. Also roundtrip tests like from_json(to_json) and to_json(from_json) for MapType are added.
Author: Maxim Gekk <maxim.gekk@databricks.com>
Author: Maxim Gekk <max.gekk@gmail.com>
Closes #21108 from MaxGekk/from_json-map-type.
2018-05-14 17:05:42 -04:00
|
|
|
the specified schema. Returns `null`, in the case of an unparseable string.
|
2016-09-29 16:01:10 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.1.0
|
2016-09-29 16:01:10 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
string column in json format
|
|
|
|
schema : :class:`DataType` or str
|
|
|
|
a StructType or ArrayType of StructType to use when parsing the json column.
|
[SPARK-21266][R][PYTHON] Support schema a DDL-formatted string in dapply/gapply/from_json
## What changes were proposed in this pull request?
This PR supports schema in a DDL formatted string for `from_json` in R/Python and `dapply` and `gapply` in R, which are commonly used and/or consistent with Scala APIs.
Additionally, this PR exposes `structType` in R to allow working around in other possible corner cases.
**Python**
`from_json`
```python
from pyspark.sql.functions import from_json
data = [(1, '''{"a": 1}''')]
df = spark.createDataFrame(data, ("key", "value"))
df.select(from_json(df.value, "a INT").alias("json")).show()
```
**R**
`from_json`
```R
df <- sql("SELECT named_struct('name', 'Bob') as people")
df <- mutate(df, people_json = to_json(df$people))
head(select(df, from_json(df$people_json, "name STRING")))
```
`structType.character`
```R
structType("a STRING, b INT")
```
`dapply`
```R
dapply(createDataFrame(list(list(1.0)), "a"), function(x) {x}, "a DOUBLE")
```
`gapply`
```R
gapply(createDataFrame(list(list(1.0)), "a"), "a", function(key, x) { x }, "a DOUBLE")
```
## How was this patch tested?
Doc tests for `from_json` in Python and unit tests `test_sparkSQL.R` in R.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #18498 from HyukjinKwon/SPARK-21266.
2017-07-10 13:40:03 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionchanged:: 2.3
|
|
|
|
the DDL-formatted string is also supported for ``schema``.
|
|
|
|
options : dict, optional
|
|
|
|
options to control parsing. accepts the same options as the json datasource
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-09-29 16:01:10 -04:00
|
|
|
>>> from pyspark.sql.types import *
|
|
|
|
>>> data = [(1, '''{"a": 1}''')]
|
|
|
|
>>> schema = StructType([StructField("a", IntegerType())])
|
|
|
|
>>> df = spark.createDataFrame(data, ("key", "value"))
|
|
|
|
>>> df.select(from_json(df.value, schema).alias("json")).collect()
|
|
|
|
[Row(json=Row(a=1))]
|
[SPARK-21266][R][PYTHON] Support schema a DDL-formatted string in dapply/gapply/from_json
## What changes were proposed in this pull request?
This PR supports schema in a DDL formatted string for `from_json` in R/Python and `dapply` and `gapply` in R, which are commonly used and/or consistent with Scala APIs.
Additionally, this PR exposes `structType` in R to allow working around in other possible corner cases.
**Python**
`from_json`
```python
from pyspark.sql.functions import from_json
data = [(1, '''{"a": 1}''')]
df = spark.createDataFrame(data, ("key", "value"))
df.select(from_json(df.value, "a INT").alias("json")).show()
```
**R**
`from_json`
```R
df <- sql("SELECT named_struct('name', 'Bob') as people")
df <- mutate(df, people_json = to_json(df$people))
head(select(df, from_json(df$people_json, "name STRING")))
```
`structType.character`
```R
structType("a STRING, b INT")
```
`dapply`
```R
dapply(createDataFrame(list(list(1.0)), "a"), function(x) {x}, "a DOUBLE")
```
`gapply`
```R
gapply(createDataFrame(list(list(1.0)), "a"), "a", function(key, x) { x }, "a DOUBLE")
```
## How was this patch tested?
Doc tests for `from_json` in Python and unit tests `test_sparkSQL.R` in R.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #18498 from HyukjinKwon/SPARK-21266.
2017-07-10 13:40:03 -04:00
|
|
|
>>> df.select(from_json(df.value, "a INT").alias("json")).collect()
|
|
|
|
[Row(json=Row(a=1))]
|
[SPARK-24543][SQL] Support any type as DDL string for from_json's schema
## What changes were proposed in this pull request?
In the PR, I propose to support any DataType represented as DDL string for the from_json function. After the changes, it will be possible to specify `MapType` in SQL like:
```sql
select from_json('{"a":1, "b":2}', 'map<string, int>')
```
and in Scala (similar in other languages)
```scala
val in = Seq("""{"a": {"b": 1}}""").toDS()
val schema = "map<string, map<string, int>>"
val out = in.select(from_json($"value", schema, Map.empty[String, String]))
```
## How was this patch tested?
Added a couple sql tests and modified existing tests for Python and Scala. The former tests were modified because it is not imported for them in which format schema for `from_json` is provided.
Author: Maxim Gekk <maxim.gekk@databricks.com>
Closes #21550 from MaxGekk/from_json-ddl-schema.
2018-06-14 16:27:27 -04:00
|
|
|
>>> df.select(from_json(df.value, "MAP<STRING,INT>").alias("json")).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(json={'a': 1})]
|
[SPARK-19595][SQL] Support json array in from_json
## What changes were proposed in this pull request?
This PR proposes to both,
**Do not allow json arrays with multiple elements and return null in `from_json` with `StructType` as the schema.**
Currently, it only reads the single row when the input is a json array. So, the codes below:
```scala
import org.apache.spark.sql.functions._
import org.apache.spark.sql.types._
val schema = StructType(StructField("a", IntegerType) :: Nil)
Seq(("""[{"a": 1}, {"a": 2}]""")).toDF("struct").select(from_json(col("struct"), schema)).show()
```
prints
```
+--------------------+
|jsontostruct(struct)|
+--------------------+
| [1]|
+--------------------+
```
This PR simply suggests to print this as `null` if the schema is `StructType` and input is json array.with multiple elements
```
+--------------------+
|jsontostruct(struct)|
+--------------------+
| null|
+--------------------+
```
**Support json arrays in `from_json` with `ArrayType` as the schema.**
```scala
import org.apache.spark.sql.functions._
import org.apache.spark.sql.types._
val schema = ArrayType(StructType(StructField("a", IntegerType) :: Nil))
Seq(("""[{"a": 1}, {"a": 2}]""")).toDF("array").select(from_json(col("array"), schema)).show()
```
prints
```
+-------------------+
|jsontostruct(array)|
+-------------------+
| [[1], [2]]|
+-------------------+
```
## How was this patch tested?
Unit test in `JsonExpressionsSuite`, `JsonFunctionsSuite`, Python doctests and manual test.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #16929 from HyukjinKwon/disallow-array.
2017-03-05 17:35:06 -05:00
|
|
|
>>> data = [(1, '''[{"a": 1}]''')]
|
|
|
|
>>> schema = ArrayType(StructType([StructField("a", IntegerType())]))
|
|
|
|
>>> df = spark.createDataFrame(data, ("key", "value"))
|
|
|
|
>>> df.select(from_json(df.value, schema).alias("json")).collect()
|
|
|
|
[Row(json=[Row(a=1)])]
|
2018-07-03 21:38:18 -04:00
|
|
|
>>> schema = schema_of_json(lit('''{"a": 0}'''))
|
|
|
|
>>> df.select(from_json(df.value, schema).alias("json")).collect()
|
2018-10-24 07:09:15 -04:00
|
|
|
[Row(json=Row(a=None))]
|
2018-08-13 08:13:09 -04:00
|
|
|
>>> data = [(1, '''[1, 2, 3]''')]
|
|
|
|
>>> schema = ArrayType(IntegerType())
|
|
|
|
>>> df = spark.createDataFrame(data, ("key", "value"))
|
|
|
|
>>> df.select(from_json(df.value, schema).alias("json")).collect()
|
|
|
|
[Row(json=[1, 2, 3])]
|
2016-09-29 16:01:10 -04:00
|
|
|
"""
|
|
|
|
|
|
|
|
sc = SparkContext._active_spark_context
|
[SPARK-21266][R][PYTHON] Support schema a DDL-formatted string in dapply/gapply/from_json
## What changes were proposed in this pull request?
This PR supports schema in a DDL formatted string for `from_json` in R/Python and `dapply` and `gapply` in R, which are commonly used and/or consistent with Scala APIs.
Additionally, this PR exposes `structType` in R to allow working around in other possible corner cases.
**Python**
`from_json`
```python
from pyspark.sql.functions import from_json
data = [(1, '''{"a": 1}''')]
df = spark.createDataFrame(data, ("key", "value"))
df.select(from_json(df.value, "a INT").alias("json")).show()
```
**R**
`from_json`
```R
df <- sql("SELECT named_struct('name', 'Bob') as people")
df <- mutate(df, people_json = to_json(df$people))
head(select(df, from_json(df$people_json, "name STRING")))
```
`structType.character`
```R
structType("a STRING, b INT")
```
`dapply`
```R
dapply(createDataFrame(list(list(1.0)), "a"), function(x) {x}, "a DOUBLE")
```
`gapply`
```R
gapply(createDataFrame(list(list(1.0)), "a"), "a", function(key, x) { x }, "a DOUBLE")
```
## How was this patch tested?
Doc tests for `from_json` in Python and unit tests `test_sparkSQL.R` in R.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #18498 from HyukjinKwon/SPARK-21266.
2017-07-10 13:40:03 -04:00
|
|
|
if isinstance(schema, DataType):
|
|
|
|
schema = schema.json()
|
2018-07-03 21:38:18 -04:00
|
|
|
elif isinstance(schema, Column):
|
|
|
|
schema = _to_java_column(schema)
|
2019-07-18 00:37:03 -04:00
|
|
|
jc = sc._jvm.functions.from_json(_to_java_column(col), schema, _options_to_str(options))
|
2016-09-29 16:01:10 -04:00
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
[SPARK-32320][PYSPARK] Remove mutable default arguments
This is bad practice, and might lead to unexpected behaviour:
https://florimond.dev/blog/articles/2018/08/python-mutable-defaults-are-the-source-of-all-evil/
```
fokkodriesprongFan spark % grep -R "={}" python | grep def
python/pyspark/resource/profile.py: def __init__(self, _java_resource_profile=None, _exec_req={}, _task_req={}):
python/pyspark/sql/functions.py:def from_json(col, schema, options={}):
python/pyspark/sql/functions.py:def to_json(col, options={}):
python/pyspark/sql/functions.py:def schema_of_json(json, options={}):
python/pyspark/sql/functions.py:def schema_of_csv(csv, options={}):
python/pyspark/sql/functions.py:def to_csv(col, options={}):
python/pyspark/sql/functions.py:def from_csv(col, schema, options={}):
python/pyspark/sql/avro/functions.py:def from_avro(data, jsonFormatSchema, options={}):
```
```
fokkodriesprongFan spark % grep -R "=\[\]" python | grep def
python/pyspark/ml/tuning.py: def __init__(self, bestModel, avgMetrics=[], subModels=None):
python/pyspark/ml/tuning.py: def __init__(self, bestModel, validationMetrics=[], subModels=None):
```
### What changes were proposed in this pull request?
Removing the mutable default arguments.
### Why are the changes needed?
Removing the mutable default arguments, and changing the signature to `Optional[...]`.
### Does this PR introduce _any_ user-facing change?
No 👍
### How was this patch tested?
Using the Flake8 bugbear code analysis plugin.
Closes #29122 from Fokko/SPARK-32320.
Authored-by: Fokko Driesprong <fokko@apache.org>
Signed-off-by: Ruifeng Zheng <ruifengz@foxmail.com>
2020-12-07 20:35:36 -05:00
|
|
|
def to_json(col, options=None):
|
2016-11-01 15:46:41 -04:00
|
|
|
"""
|
[SPARK-25252][SQL] Support arrays of any types by to_json
## What changes were proposed in this pull request?
In the PR, I propose to extended `to_json` and support any types as element types of input arrays. It should allow converting arrays of primitive types and arrays of arrays. For example:
```
select to_json(array('1','2','3'))
> ["1","2","3"]
select to_json(array(array(1,2,3),array(4)))
> [[1,2,3],[4]]
```
## How was this patch tested?
Added a couple sql tests for arrays of primitive type and of arrays. Also I added round trip test `from_json` -> `to_json`.
Closes #22226 from MaxGekk/to_json-array.
Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-09-06 00:35:59 -04:00
|
|
|
Converts a column containing a :class:`StructType`, :class:`ArrayType` or a :class:`MapType`
|
[SPARK-21513][SQL][FOLLOWUP] Allow UDF to_json support converting MapType to json for PySpark and SparkR
## What changes were proposed in this pull request?
In previous work SPARK-21513, we has allowed `MapType` and `ArrayType` of `MapType`s convert to a json string but only for Scala API. In this follow-up PR, we will make SparkSQL support it for PySpark and SparkR, too. We also fix some little bugs and comments of the previous work in this follow-up PR.
### For PySpark
```
>>> data = [(1, {"name": "Alice"})]
>>> df = spark.createDataFrame(data, ("key", "value"))
>>> df.select(to_json(df.value).alias("json")).collect()
[Row(json=u'{"name":"Alice")']
>>> data = [(1, [{"name": "Alice"}, {"name": "Bob"}])]
>>> df = spark.createDataFrame(data, ("key", "value"))
>>> df.select(to_json(df.value).alias("json")).collect()
[Row(json=u'[{"name":"Alice"},{"name":"Bob"}]')]
```
### For SparkR
```
# Converts a map into a JSON object
df2 <- sql("SELECT map('name', 'Bob')) as people")
df2 <- mutate(df2, people_json = to_json(df2$people))
# Converts an array of maps into a JSON array
df2 <- sql("SELECT array(map('name', 'Bob'), map('name', 'Alice')) as people")
df2 <- mutate(df2, people_json = to_json(df2$people))
```
## How was this patch tested?
Add unit test cases.
cc viirya HyukjinKwon
Author: goldmedal <liugs963@gmail.com>
Closes #19223 from goldmedal/SPARK-21513-fp-PySaprkAndSparkR.
2017-09-14 22:53:10 -04:00
|
|
|
into a JSON string. Throws an exception, in the case of an unsupported type.
|
2016-11-01 15:46:41 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.1.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column containing a struct, an array or a map.
|
|
|
|
options : dict, optional
|
|
|
|
options to control converting. accepts the same options as the JSON datasource.
|
|
|
|
Additionally the function supports the `pretty` option which enables
|
|
|
|
pretty JSON generation.
|
2016-11-01 15:46:41 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2016-11-01 15:46:41 -04:00
|
|
|
>>> from pyspark.sql import Row
|
|
|
|
>>> from pyspark.sql.types import *
|
2020-07-13 22:22:44 -04:00
|
|
|
>>> data = [(1, Row(age=2, name='Alice'))]
|
2016-11-01 15:46:41 -04:00
|
|
|
>>> df = spark.createDataFrame(data, ("key", "value"))
|
|
|
|
>>> df.select(to_json(df.value).alias("json")).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(json='{"age":2,"name":"Alice"}')]
|
|
|
|
>>> data = [(1, [Row(age=2, name='Alice'), Row(age=3, name='Bob')])]
|
2017-03-20 01:33:01 -04:00
|
|
|
>>> df = spark.createDataFrame(data, ("key", "value"))
|
|
|
|
>>> df.select(to_json(df.value).alias("json")).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(json='[{"age":2,"name":"Alice"},{"age":3,"name":"Bob"}]')]
|
[SPARK-21513][SQL][FOLLOWUP] Allow UDF to_json support converting MapType to json for PySpark and SparkR
## What changes were proposed in this pull request?
In previous work SPARK-21513, we has allowed `MapType` and `ArrayType` of `MapType`s convert to a json string but only for Scala API. In this follow-up PR, we will make SparkSQL support it for PySpark and SparkR, too. We also fix some little bugs and comments of the previous work in this follow-up PR.
### For PySpark
```
>>> data = [(1, {"name": "Alice"})]
>>> df = spark.createDataFrame(data, ("key", "value"))
>>> df.select(to_json(df.value).alias("json")).collect()
[Row(json=u'{"name":"Alice")']
>>> data = [(1, [{"name": "Alice"}, {"name": "Bob"}])]
>>> df = spark.createDataFrame(data, ("key", "value"))
>>> df.select(to_json(df.value).alias("json")).collect()
[Row(json=u'[{"name":"Alice"},{"name":"Bob"}]')]
```
### For SparkR
```
# Converts a map into a JSON object
df2 <- sql("SELECT map('name', 'Bob')) as people")
df2 <- mutate(df2, people_json = to_json(df2$people))
# Converts an array of maps into a JSON array
df2 <- sql("SELECT array(map('name', 'Bob'), map('name', 'Alice')) as people")
df2 <- mutate(df2, people_json = to_json(df2$people))
```
## How was this patch tested?
Add unit test cases.
cc viirya HyukjinKwon
Author: goldmedal <liugs963@gmail.com>
Closes #19223 from goldmedal/SPARK-21513-fp-PySaprkAndSparkR.
2017-09-14 22:53:10 -04:00
|
|
|
>>> data = [(1, {"name": "Alice"})]
|
|
|
|
>>> df = spark.createDataFrame(data, ("key", "value"))
|
|
|
|
>>> df.select(to_json(df.value).alias("json")).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(json='{"name":"Alice"}')]
|
[SPARK-21513][SQL][FOLLOWUP] Allow UDF to_json support converting MapType to json for PySpark and SparkR
## What changes were proposed in this pull request?
In previous work SPARK-21513, we has allowed `MapType` and `ArrayType` of `MapType`s convert to a json string but only for Scala API. In this follow-up PR, we will make SparkSQL support it for PySpark and SparkR, too. We also fix some little bugs and comments of the previous work in this follow-up PR.
### For PySpark
```
>>> data = [(1, {"name": "Alice"})]
>>> df = spark.createDataFrame(data, ("key", "value"))
>>> df.select(to_json(df.value).alias("json")).collect()
[Row(json=u'{"name":"Alice")']
>>> data = [(1, [{"name": "Alice"}, {"name": "Bob"}])]
>>> df = spark.createDataFrame(data, ("key", "value"))
>>> df.select(to_json(df.value).alias("json")).collect()
[Row(json=u'[{"name":"Alice"},{"name":"Bob"}]')]
```
### For SparkR
```
# Converts a map into a JSON object
df2 <- sql("SELECT map('name', 'Bob')) as people")
df2 <- mutate(df2, people_json = to_json(df2$people))
# Converts an array of maps into a JSON array
df2 <- sql("SELECT array(map('name', 'Bob'), map('name', 'Alice')) as people")
df2 <- mutate(df2, people_json = to_json(df2$people))
```
## How was this patch tested?
Add unit test cases.
cc viirya HyukjinKwon
Author: goldmedal <liugs963@gmail.com>
Closes #19223 from goldmedal/SPARK-21513-fp-PySaprkAndSparkR.
2017-09-14 22:53:10 -04:00
|
|
|
>>> data = [(1, [{"name": "Alice"}, {"name": "Bob"}])]
|
|
|
|
>>> df = spark.createDataFrame(data, ("key", "value"))
|
|
|
|
>>> df.select(to_json(df.value).alias("json")).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(json='[{"name":"Alice"},{"name":"Bob"}]')]
|
[SPARK-25252][SQL] Support arrays of any types by to_json
## What changes were proposed in this pull request?
In the PR, I propose to extended `to_json` and support any types as element types of input arrays. It should allow converting arrays of primitive types and arrays of arrays. For example:
```
select to_json(array('1','2','3'))
> ["1","2","3"]
select to_json(array(array(1,2,3),array(4)))
> [[1,2,3],[4]]
```
## How was this patch tested?
Added a couple sql tests for arrays of primitive type and of arrays. Also I added round trip test `from_json` -> `to_json`.
Closes #22226 from MaxGekk/to_json-array.
Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-09-06 00:35:59 -04:00
|
|
|
>>> data = [(1, ["Alice", "Bob"])]
|
|
|
|
>>> df = spark.createDataFrame(data, ("key", "value"))
|
|
|
|
>>> df.select(to_json(df.value).alias("json")).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(json='["Alice","Bob"]')]
|
2016-11-01 15:46:41 -04:00
|
|
|
"""
|
|
|
|
|
|
|
|
sc = SparkContext._active_spark_context
|
2019-07-18 00:37:03 -04:00
|
|
|
jc = sc._jvm.functions.to_json(_to_java_column(col), _options_to_str(options))
|
2016-11-01 15:46:41 -04:00
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
[SPARK-32320][PYSPARK] Remove mutable default arguments
This is bad practice, and might lead to unexpected behaviour:
https://florimond.dev/blog/articles/2018/08/python-mutable-defaults-are-the-source-of-all-evil/
```
fokkodriesprongFan spark % grep -R "={}" python | grep def
python/pyspark/resource/profile.py: def __init__(self, _java_resource_profile=None, _exec_req={}, _task_req={}):
python/pyspark/sql/functions.py:def from_json(col, schema, options={}):
python/pyspark/sql/functions.py:def to_json(col, options={}):
python/pyspark/sql/functions.py:def schema_of_json(json, options={}):
python/pyspark/sql/functions.py:def schema_of_csv(csv, options={}):
python/pyspark/sql/functions.py:def to_csv(col, options={}):
python/pyspark/sql/functions.py:def from_csv(col, schema, options={}):
python/pyspark/sql/avro/functions.py:def from_avro(data, jsonFormatSchema, options={}):
```
```
fokkodriesprongFan spark % grep -R "=\[\]" python | grep def
python/pyspark/ml/tuning.py: def __init__(self, bestModel, avgMetrics=[], subModels=None):
python/pyspark/ml/tuning.py: def __init__(self, bestModel, validationMetrics=[], subModels=None):
```
### What changes were proposed in this pull request?
Removing the mutable default arguments.
### Why are the changes needed?
Removing the mutable default arguments, and changing the signature to `Optional[...]`.
### Does this PR introduce _any_ user-facing change?
No 👍
### How was this patch tested?
Using the Flake8 bugbear code analysis plugin.
Closes #29122 from Fokko/SPARK-32320.
Authored-by: Fokko Driesprong <fokko@apache.org>
Signed-off-by: Ruifeng Zheng <ruifengz@foxmail.com>
2020-12-07 20:35:36 -05:00
|
|
|
def schema_of_json(json, options=None):
|
2018-07-03 21:38:18 -04:00
|
|
|
"""
|
2018-10-26 10:14:43 -04:00
|
|
|
Parses a JSON string and infers its schema in DDL format.
|
2018-07-03 21:38:18 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.4.0
|
2018-09-29 05:53:30 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
json : :class:`~pyspark.sql.Column` or str
|
2020-11-17 21:32:27 -05:00
|
|
|
a JSON string or a foldable string column containing a JSON string.
|
2020-11-02 20:00:49 -05:00
|
|
|
options : dict, optional
|
|
|
|
options to control parsing. accepts the same options as the JSON datasource
|
2018-07-03 21:38:18 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionchanged:: 3.0
|
|
|
|
It accepts `options` parameter to control schema inferring.
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2018-10-26 10:14:43 -04:00
|
|
|
>>> df = spark.range(1)
|
2018-07-03 21:38:18 -04:00
|
|
|
>>> df.select(schema_of_json(lit('{"a": 0}')).alias("json")).collect()
|
2020-10-28 21:30:41 -04:00
|
|
|
[Row(json='STRUCT<`a`: BIGINT>')]
|
2018-10-26 10:14:43 -04:00
|
|
|
>>> schema = schema_of_json('{a: 1}', {'allowUnquotedFieldNames':'true'})
|
2018-09-29 05:53:30 -04:00
|
|
|
>>> df.select(schema.alias("json")).collect()
|
2020-10-28 21:30:41 -04:00
|
|
|
[Row(json='STRUCT<`a`: BIGINT>')]
|
2018-07-03 21:38:18 -04:00
|
|
|
"""
|
2020-07-13 22:22:44 -04:00
|
|
|
if isinstance(json, str):
|
2018-10-26 10:14:43 -04:00
|
|
|
col = _create_column_from_literal(json)
|
|
|
|
elif isinstance(json, Column):
|
|
|
|
col = _to_java_column(json)
|
|
|
|
else:
|
|
|
|
raise TypeError("schema argument should be a column or string")
|
2018-07-03 21:38:18 -04:00
|
|
|
|
|
|
|
sc = SparkContext._active_spark_context
|
2019-07-18 00:37:03 -04:00
|
|
|
jc = sc._jvm.functions.schema_of_json(col, _options_to_str(options))
|
2018-07-03 21:38:18 -04:00
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
[SPARK-32320][PYSPARK] Remove mutable default arguments
This is bad practice, and might lead to unexpected behaviour:
https://florimond.dev/blog/articles/2018/08/python-mutable-defaults-are-the-source-of-all-evil/
```
fokkodriesprongFan spark % grep -R "={}" python | grep def
python/pyspark/resource/profile.py: def __init__(self, _java_resource_profile=None, _exec_req={}, _task_req={}):
python/pyspark/sql/functions.py:def from_json(col, schema, options={}):
python/pyspark/sql/functions.py:def to_json(col, options={}):
python/pyspark/sql/functions.py:def schema_of_json(json, options={}):
python/pyspark/sql/functions.py:def schema_of_csv(csv, options={}):
python/pyspark/sql/functions.py:def to_csv(col, options={}):
python/pyspark/sql/functions.py:def from_csv(col, schema, options={}):
python/pyspark/sql/avro/functions.py:def from_avro(data, jsonFormatSchema, options={}):
```
```
fokkodriesprongFan spark % grep -R "=\[\]" python | grep def
python/pyspark/ml/tuning.py: def __init__(self, bestModel, avgMetrics=[], subModels=None):
python/pyspark/ml/tuning.py: def __init__(self, bestModel, validationMetrics=[], subModels=None):
```
### What changes were proposed in this pull request?
Removing the mutable default arguments.
### Why are the changes needed?
Removing the mutable default arguments, and changing the signature to `Optional[...]`.
### Does this PR introduce _any_ user-facing change?
No 👍
### How was this patch tested?
Using the Flake8 bugbear code analysis plugin.
Closes #29122 from Fokko/SPARK-32320.
Authored-by: Fokko Driesprong <fokko@apache.org>
Signed-off-by: Ruifeng Zheng <ruifengz@foxmail.com>
2020-12-07 20:35:36 -05:00
|
|
|
def schema_of_csv(csv, options=None):
|
2018-10-31 21:14:16 -04:00
|
|
|
"""
|
|
|
|
Parses a CSV string and infers its schema in DDL format.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 3.0.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
csv : :class:`~pyspark.sql.Column` or str
|
2020-11-17 21:32:27 -05:00
|
|
|
a CSV string or a foldable string column containing a CSV string.
|
2020-11-02 20:00:49 -05:00
|
|
|
options : dict, optional
|
|
|
|
options to control parsing. accepts the same options as the CSV datasource
|
2018-10-31 21:14:16 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2018-10-31 21:14:16 -04:00
|
|
|
>>> df = spark.range(1)
|
|
|
|
>>> df.select(schema_of_csv(lit('1|a'), {'sep':'|'}).alias("csv")).collect()
|
2020-10-29 08:02:10 -04:00
|
|
|
[Row(csv='STRUCT<`_c0`: INT, `_c1`: STRING>')]
|
2018-10-31 21:14:16 -04:00
|
|
|
>>> df.select(schema_of_csv('1|a', {'sep':'|'}).alias("csv")).collect()
|
2020-10-29 08:02:10 -04:00
|
|
|
[Row(csv='STRUCT<`_c0`: INT, `_c1`: STRING>')]
|
2018-10-31 21:14:16 -04:00
|
|
|
"""
|
2020-07-13 22:22:44 -04:00
|
|
|
if isinstance(csv, str):
|
2018-10-31 21:14:16 -04:00
|
|
|
col = _create_column_from_literal(csv)
|
|
|
|
elif isinstance(csv, Column):
|
|
|
|
col = _to_java_column(csv)
|
|
|
|
else:
|
|
|
|
raise TypeError("schema argument should be a column or string")
|
|
|
|
|
|
|
|
sc = SparkContext._active_spark_context
|
2019-07-18 00:37:03 -04:00
|
|
|
jc = sc._jvm.functions.schema_of_csv(col, _options_to_str(options))
|
2018-10-31 21:14:16 -04:00
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
[SPARK-32320][PYSPARK] Remove mutable default arguments
This is bad practice, and might lead to unexpected behaviour:
https://florimond.dev/blog/articles/2018/08/python-mutable-defaults-are-the-source-of-all-evil/
```
fokkodriesprongFan spark % grep -R "={}" python | grep def
python/pyspark/resource/profile.py: def __init__(self, _java_resource_profile=None, _exec_req={}, _task_req={}):
python/pyspark/sql/functions.py:def from_json(col, schema, options={}):
python/pyspark/sql/functions.py:def to_json(col, options={}):
python/pyspark/sql/functions.py:def schema_of_json(json, options={}):
python/pyspark/sql/functions.py:def schema_of_csv(csv, options={}):
python/pyspark/sql/functions.py:def to_csv(col, options={}):
python/pyspark/sql/functions.py:def from_csv(col, schema, options={}):
python/pyspark/sql/avro/functions.py:def from_avro(data, jsonFormatSchema, options={}):
```
```
fokkodriesprongFan spark % grep -R "=\[\]" python | grep def
python/pyspark/ml/tuning.py: def __init__(self, bestModel, avgMetrics=[], subModels=None):
python/pyspark/ml/tuning.py: def __init__(self, bestModel, validationMetrics=[], subModels=None):
```
### What changes were proposed in this pull request?
Removing the mutable default arguments.
### Why are the changes needed?
Removing the mutable default arguments, and changing the signature to `Optional[...]`.
### Does this PR introduce _any_ user-facing change?
No 👍
### How was this patch tested?
Using the Flake8 bugbear code analysis plugin.
Closes #29122 from Fokko/SPARK-32320.
Authored-by: Fokko Driesprong <fokko@apache.org>
Signed-off-by: Ruifeng Zheng <ruifengz@foxmail.com>
2020-12-07 20:35:36 -05:00
|
|
|
def to_csv(col, options=None):
|
2018-11-04 01:57:38 -05:00
|
|
|
"""
|
|
|
|
Converts a column containing a :class:`StructType` into a CSV string.
|
|
|
|
Throws an exception, in the case of an unsupported type.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 3.0.0
|
2018-11-04 01:57:38 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column containing a struct.
|
|
|
|
options: dict, optional
|
|
|
|
options to control converting. accepts the same options as the CSV datasource.
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2018-11-04 01:57:38 -05:00
|
|
|
>>> from pyspark.sql import Row
|
2020-07-13 22:22:44 -04:00
|
|
|
>>> data = [(1, Row(age=2, name='Alice'))]
|
2018-11-04 01:57:38 -05:00
|
|
|
>>> df = spark.createDataFrame(data, ("key", "value"))
|
|
|
|
>>> df.select(to_csv(df.value).alias("csv")).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(csv='2,Alice')]
|
2018-11-04 01:57:38 -05:00
|
|
|
"""
|
|
|
|
|
|
|
|
sc = SparkContext._active_spark_context
|
2019-07-18 00:37:03 -04:00
|
|
|
jc = sc._jvm.functions.to_csv(_to_java_column(col), _options_to_str(options))
|
2018-11-04 01:57:38 -05:00
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
2015-07-21 03:53:20 -04:00
|
|
|
def size(col):
|
|
|
|
"""
|
|
|
|
Collection function: returns the length of the array or map stored in the column.
|
2015-07-31 19:05:26 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column or expression
|
2015-07-21 03:53:20 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2016-05-23 21:14:48 -04:00
|
|
|
>>> df = spark.createDataFrame([([1, 2, 3],),([1],),([],)], ['data'])
|
2015-07-21 03:53:20 -04:00
|
|
|
>>> df.select(size(df.data)).collect()
|
|
|
|
[Row(size(data)=3), Row(size(data)=1), Row(size(data)=0)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.size(_to_java_column(col)))
|
|
|
|
|
|
|
|
|
2018-04-17 04:55:35 -04:00
|
|
|
def array_min(col):
|
|
|
|
"""
|
|
|
|
Collection function: returns the minimum value of the array.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column or expression
|
2018-04-17 04:55:35 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2018-04-17 04:55:35 -04:00
|
|
|
>>> df = spark.createDataFrame([([2, 1, 3],), ([None, 10, -1],)], ['data'])
|
|
|
|
>>> df.select(array_min(df.data).alias('min')).collect()
|
|
|
|
[Row(min=1), Row(min=-1)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.array_min(_to_java_column(col)))
|
|
|
|
|
|
|
|
|
2018-04-16 00:45:55 -04:00
|
|
|
def array_max(col):
|
|
|
|
"""
|
|
|
|
Collection function: returns the maximum value of the array.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column or expression
|
2018-04-16 00:45:55 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2018-04-16 00:45:55 -04:00
|
|
|
>>> df = spark.createDataFrame([([2, 1, 3],), ([None, 10, -1],)], ['data'])
|
|
|
|
>>> df.select(array_max(df.data).alias('max')).collect()
|
|
|
|
[Row(max=3), Row(max=10)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.array_max(_to_java_column(col)))
|
|
|
|
|
|
|
|
|
2015-08-01 11:32:29 -04:00
|
|
|
def sort_array(col, asc=True):
|
|
|
|
"""
|
2016-11-06 00:47:33 -04:00
|
|
|
Collection function: sorts the input array in ascending or descending order according
|
2018-05-07 02:22:23 -04:00
|
|
|
to the natural ordering of the array elements. Null elements will be placed at the beginning
|
|
|
|
of the returned array in ascending order or at the end of the returned array in descending
|
|
|
|
order.
|
2015-08-01 11:32:29 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
2015-08-01 11:32:29 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column or expression
|
|
|
|
asc : bool, optional
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2018-05-07 02:22:23 -04:00
|
|
|
>>> df = spark.createDataFrame([([2, 1, None, 3],),([1],),([],)], ['data'])
|
2015-08-01 11:32:29 -04:00
|
|
|
>>> df.select(sort_array(df.data).alias('r')).collect()
|
2018-05-07 02:22:23 -04:00
|
|
|
[Row(r=[None, 1, 2, 3]), Row(r=[1]), Row(r=[])]
|
2015-08-01 11:32:29 -04:00
|
|
|
>>> df.select(sort_array(df.data, asc=False).alias('r')).collect()
|
2018-05-07 02:22:23 -04:00
|
|
|
[Row(r=[3, 2, 1, None]), Row(r=[1]), Row(r=[])]
|
2018-04-17 04:55:35 -04:00
|
|
|
"""
|
2015-08-01 11:32:29 -04:00
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.sort_array(_to_java_column(col), asc))
|
|
|
|
|
|
|
|
|
2018-05-07 02:22:23 -04:00
|
|
|
def array_sort(col):
|
|
|
|
"""
|
|
|
|
Collection function: sorts the input array in ascending order. The elements of the input array
|
|
|
|
must be orderable. Null elements will be placed at the end of the returned array.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column or expression
|
2018-05-07 02:22:23 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2018-05-07 02:22:23 -04:00
|
|
|
>>> df = spark.createDataFrame([([2, 1, None, 3],),([1],),([],)], ['data'])
|
|
|
|
>>> df.select(array_sort(df.data).alias('r')).collect()
|
|
|
|
[Row(r=[1, 2, 3, None]), Row(r=[1]), Row(r=[])]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.array_sort(_to_java_column(col)))
|
|
|
|
|
|
|
|
|
2018-07-27 10:02:48 -04:00
|
|
|
def shuffle(col):
|
|
|
|
"""
|
|
|
|
Collection function: Generates a random permutation of the given array.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column or expression
|
2018-07-27 10:02:48 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
The function is non-deterministic.
|
2018-07-27 10:02:48 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2018-07-27 10:02:48 -04:00
|
|
|
>>> df = spark.createDataFrame([([1, 20, 3, 5],), ([1, 20, None, 3],)], ['data'])
|
|
|
|
>>> df.select(shuffle(df.data).alias('s')).collect() # doctest: +SKIP
|
|
|
|
[Row(s=[3, 1, 5, 20]), Row(s=[20, None, 3, 1])]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.shuffle(_to_java_column(col)))
|
|
|
|
|
|
|
|
|
[SPARK-23926][SQL] Extending reverse function to support ArrayType arguments
## What changes were proposed in this pull request?
This PR extends `reverse` functions to be able to operate over array columns and covers:
- Introduction of `Reverse` expression that represents logic for reversing arrays and also strings
- Removal of `StringReverse` expression
- A wrapper for PySpark
## How was this patch tested?
New tests added into:
- CollectionExpressionsSuite
- DataFrameFunctionsSuite
## Codegen examples
### Primitive type
```
val df = Seq(
Seq(1, 3, 4, 2),
null
).toDF("i")
df.filter($"i".isNotNull || $"i".isNull).select(reverse($"i")).debugCodegen
```
Result:
```
/* 032 */ boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 033 */ ArrayData inputadapter_value = inputadapter_isNull ?
/* 034 */ null : (inputadapter_row.getArray(0));
/* 035 */
/* 036 */ boolean filter_value = true;
/* 037 */
/* 038 */ if (!(!inputadapter_isNull)) {
/* 039 */ filter_value = inputadapter_isNull;
/* 040 */ }
/* 041 */ if (!filter_value) continue;
/* 042 */
/* 043 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 044 */
/* 045 */ boolean project_isNull = inputadapter_isNull;
/* 046 */ ArrayData project_value = null;
/* 047 */
/* 048 */ if (!inputadapter_isNull) {
/* 049 */ final int project_length = inputadapter_value.numElements();
/* 050 */ project_value = inputadapter_value.copy();
/* 051 */ for(int k = 0; k < project_length / 2; k++) {
/* 052 */ int l = project_length - k - 1;
/* 053 */ boolean isNullAtK = project_value.isNullAt(k);
/* 054 */ boolean isNullAtL = project_value.isNullAt(l);
/* 055 */ if(!isNullAtK) {
/* 056 */ int el = project_value.getInt(k);
/* 057 */ if(!isNullAtL) {
/* 058 */ project_value.setInt(k, project_value.getInt(l));
/* 059 */ } else {
/* 060 */ project_value.setNullAt(k);
/* 061 */ }
/* 062 */ project_value.setInt(l, el);
/* 063 */ } else if (!isNullAtL) {
/* 064 */ project_value.setInt(k, project_value.getInt(l));
/* 065 */ project_value.setNullAt(l);
/* 066 */ }
/* 067 */ }
/* 068 */
/* 069 */ }
```
### Non-primitive type
```
val df = Seq(
Seq("a", "c", "d", "b"),
null
).toDF("s")
df.filter($"s".isNotNull || $"s".isNull).select(reverse($"s")).debugCodegen
```
Result:
```
/* 032 */ boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 033 */ ArrayData inputadapter_value = inputadapter_isNull ?
/* 034 */ null : (inputadapter_row.getArray(0));
/* 035 */
/* 036 */ boolean filter_value = true;
/* 037 */
/* 038 */ if (!(!inputadapter_isNull)) {
/* 039 */ filter_value = inputadapter_isNull;
/* 040 */ }
/* 041 */ if (!filter_value) continue;
/* 042 */
/* 043 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 044 */
/* 045 */ boolean project_isNull = inputadapter_isNull;
/* 046 */ ArrayData project_value = null;
/* 047 */
/* 048 */ if (!inputadapter_isNull) {
/* 049 */ final int project_length = inputadapter_value.numElements();
/* 050 */ project_value = new org.apache.spark.sql.catalyst.util.GenericArrayData(new Object[project_length]);
/* 051 */ for(int k = 0; k < project_length; k++) {
/* 052 */ int l = project_length - k - 1;
/* 053 */ project_value.update(k, inputadapter_value.getUTF8String(l));
/* 054 */ }
/* 055 */
/* 056 */ }
```
Author: mn-mikke <mrkAha12346github>
Closes #21034 from mn-mikke/feature/array-api-reverse-to-master.
2018-04-18 05:41:55 -04:00
|
|
|
def reverse(col):
|
|
|
|
"""
|
|
|
|
Collection function: returns a reversed string or an array with reverse order of elements.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.5.0
|
[SPARK-23926][SQL] Extending reverse function to support ArrayType arguments
## What changes were proposed in this pull request?
This PR extends `reverse` functions to be able to operate over array columns and covers:
- Introduction of `Reverse` expression that represents logic for reversing arrays and also strings
- Removal of `StringReverse` expression
- A wrapper for PySpark
## How was this patch tested?
New tests added into:
- CollectionExpressionsSuite
- DataFrameFunctionsSuite
## Codegen examples
### Primitive type
```
val df = Seq(
Seq(1, 3, 4, 2),
null
).toDF("i")
df.filter($"i".isNotNull || $"i".isNull).select(reverse($"i")).debugCodegen
```
Result:
```
/* 032 */ boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 033 */ ArrayData inputadapter_value = inputadapter_isNull ?
/* 034 */ null : (inputadapter_row.getArray(0));
/* 035 */
/* 036 */ boolean filter_value = true;
/* 037 */
/* 038 */ if (!(!inputadapter_isNull)) {
/* 039 */ filter_value = inputadapter_isNull;
/* 040 */ }
/* 041 */ if (!filter_value) continue;
/* 042 */
/* 043 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 044 */
/* 045 */ boolean project_isNull = inputadapter_isNull;
/* 046 */ ArrayData project_value = null;
/* 047 */
/* 048 */ if (!inputadapter_isNull) {
/* 049 */ final int project_length = inputadapter_value.numElements();
/* 050 */ project_value = inputadapter_value.copy();
/* 051 */ for(int k = 0; k < project_length / 2; k++) {
/* 052 */ int l = project_length - k - 1;
/* 053 */ boolean isNullAtK = project_value.isNullAt(k);
/* 054 */ boolean isNullAtL = project_value.isNullAt(l);
/* 055 */ if(!isNullAtK) {
/* 056 */ int el = project_value.getInt(k);
/* 057 */ if(!isNullAtL) {
/* 058 */ project_value.setInt(k, project_value.getInt(l));
/* 059 */ } else {
/* 060 */ project_value.setNullAt(k);
/* 061 */ }
/* 062 */ project_value.setInt(l, el);
/* 063 */ } else if (!isNullAtL) {
/* 064 */ project_value.setInt(k, project_value.getInt(l));
/* 065 */ project_value.setNullAt(l);
/* 066 */ }
/* 067 */ }
/* 068 */
/* 069 */ }
```
### Non-primitive type
```
val df = Seq(
Seq("a", "c", "d", "b"),
null
).toDF("s")
df.filter($"s".isNotNull || $"s".isNull).select(reverse($"s")).debugCodegen
```
Result:
```
/* 032 */ boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 033 */ ArrayData inputadapter_value = inputadapter_isNull ?
/* 034 */ null : (inputadapter_row.getArray(0));
/* 035 */
/* 036 */ boolean filter_value = true;
/* 037 */
/* 038 */ if (!(!inputadapter_isNull)) {
/* 039 */ filter_value = inputadapter_isNull;
/* 040 */ }
/* 041 */ if (!filter_value) continue;
/* 042 */
/* 043 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 044 */
/* 045 */ boolean project_isNull = inputadapter_isNull;
/* 046 */ ArrayData project_value = null;
/* 047 */
/* 048 */ if (!inputadapter_isNull) {
/* 049 */ final int project_length = inputadapter_value.numElements();
/* 050 */ project_value = new org.apache.spark.sql.catalyst.util.GenericArrayData(new Object[project_length]);
/* 051 */ for(int k = 0; k < project_length; k++) {
/* 052 */ int l = project_length - k - 1;
/* 053 */ project_value.update(k, inputadapter_value.getUTF8String(l));
/* 054 */ }
/* 055 */
/* 056 */ }
```
Author: mn-mikke <mrkAha12346github>
Closes #21034 from mn-mikke/feature/array-api-reverse-to-master.
2018-04-18 05:41:55 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column or expression
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
[SPARK-23926][SQL] Extending reverse function to support ArrayType arguments
## What changes were proposed in this pull request?
This PR extends `reverse` functions to be able to operate over array columns and covers:
- Introduction of `Reverse` expression that represents logic for reversing arrays and also strings
- Removal of `StringReverse` expression
- A wrapper for PySpark
## How was this patch tested?
New tests added into:
- CollectionExpressionsSuite
- DataFrameFunctionsSuite
## Codegen examples
### Primitive type
```
val df = Seq(
Seq(1, 3, 4, 2),
null
).toDF("i")
df.filter($"i".isNotNull || $"i".isNull).select(reverse($"i")).debugCodegen
```
Result:
```
/* 032 */ boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 033 */ ArrayData inputadapter_value = inputadapter_isNull ?
/* 034 */ null : (inputadapter_row.getArray(0));
/* 035 */
/* 036 */ boolean filter_value = true;
/* 037 */
/* 038 */ if (!(!inputadapter_isNull)) {
/* 039 */ filter_value = inputadapter_isNull;
/* 040 */ }
/* 041 */ if (!filter_value) continue;
/* 042 */
/* 043 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 044 */
/* 045 */ boolean project_isNull = inputadapter_isNull;
/* 046 */ ArrayData project_value = null;
/* 047 */
/* 048 */ if (!inputadapter_isNull) {
/* 049 */ final int project_length = inputadapter_value.numElements();
/* 050 */ project_value = inputadapter_value.copy();
/* 051 */ for(int k = 0; k < project_length / 2; k++) {
/* 052 */ int l = project_length - k - 1;
/* 053 */ boolean isNullAtK = project_value.isNullAt(k);
/* 054 */ boolean isNullAtL = project_value.isNullAt(l);
/* 055 */ if(!isNullAtK) {
/* 056 */ int el = project_value.getInt(k);
/* 057 */ if(!isNullAtL) {
/* 058 */ project_value.setInt(k, project_value.getInt(l));
/* 059 */ } else {
/* 060 */ project_value.setNullAt(k);
/* 061 */ }
/* 062 */ project_value.setInt(l, el);
/* 063 */ } else if (!isNullAtL) {
/* 064 */ project_value.setInt(k, project_value.getInt(l));
/* 065 */ project_value.setNullAt(l);
/* 066 */ }
/* 067 */ }
/* 068 */
/* 069 */ }
```
### Non-primitive type
```
val df = Seq(
Seq("a", "c", "d", "b"),
null
).toDF("s")
df.filter($"s".isNotNull || $"s".isNull).select(reverse($"s")).debugCodegen
```
Result:
```
/* 032 */ boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 033 */ ArrayData inputadapter_value = inputadapter_isNull ?
/* 034 */ null : (inputadapter_row.getArray(0));
/* 035 */
/* 036 */ boolean filter_value = true;
/* 037 */
/* 038 */ if (!(!inputadapter_isNull)) {
/* 039 */ filter_value = inputadapter_isNull;
/* 040 */ }
/* 041 */ if (!filter_value) continue;
/* 042 */
/* 043 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 044 */
/* 045 */ boolean project_isNull = inputadapter_isNull;
/* 046 */ ArrayData project_value = null;
/* 047 */
/* 048 */ if (!inputadapter_isNull) {
/* 049 */ final int project_length = inputadapter_value.numElements();
/* 050 */ project_value = new org.apache.spark.sql.catalyst.util.GenericArrayData(new Object[project_length]);
/* 051 */ for(int k = 0; k < project_length; k++) {
/* 052 */ int l = project_length - k - 1;
/* 053 */ project_value.update(k, inputadapter_value.getUTF8String(l));
/* 054 */ }
/* 055 */
/* 056 */ }
```
Author: mn-mikke <mrkAha12346github>
Closes #21034 from mn-mikke/feature/array-api-reverse-to-master.
2018-04-18 05:41:55 -04:00
|
|
|
>>> df = spark.createDataFrame([('Spark SQL',)], ['data'])
|
|
|
|
>>> df.select(reverse(df.data).alias('s')).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(s='LQS krapS')]
|
[SPARK-23926][SQL] Extending reverse function to support ArrayType arguments
## What changes were proposed in this pull request?
This PR extends `reverse` functions to be able to operate over array columns and covers:
- Introduction of `Reverse` expression that represents logic for reversing arrays and also strings
- Removal of `StringReverse` expression
- A wrapper for PySpark
## How was this patch tested?
New tests added into:
- CollectionExpressionsSuite
- DataFrameFunctionsSuite
## Codegen examples
### Primitive type
```
val df = Seq(
Seq(1, 3, 4, 2),
null
).toDF("i")
df.filter($"i".isNotNull || $"i".isNull).select(reverse($"i")).debugCodegen
```
Result:
```
/* 032 */ boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 033 */ ArrayData inputadapter_value = inputadapter_isNull ?
/* 034 */ null : (inputadapter_row.getArray(0));
/* 035 */
/* 036 */ boolean filter_value = true;
/* 037 */
/* 038 */ if (!(!inputadapter_isNull)) {
/* 039 */ filter_value = inputadapter_isNull;
/* 040 */ }
/* 041 */ if (!filter_value) continue;
/* 042 */
/* 043 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 044 */
/* 045 */ boolean project_isNull = inputadapter_isNull;
/* 046 */ ArrayData project_value = null;
/* 047 */
/* 048 */ if (!inputadapter_isNull) {
/* 049 */ final int project_length = inputadapter_value.numElements();
/* 050 */ project_value = inputadapter_value.copy();
/* 051 */ for(int k = 0; k < project_length / 2; k++) {
/* 052 */ int l = project_length - k - 1;
/* 053 */ boolean isNullAtK = project_value.isNullAt(k);
/* 054 */ boolean isNullAtL = project_value.isNullAt(l);
/* 055 */ if(!isNullAtK) {
/* 056 */ int el = project_value.getInt(k);
/* 057 */ if(!isNullAtL) {
/* 058 */ project_value.setInt(k, project_value.getInt(l));
/* 059 */ } else {
/* 060 */ project_value.setNullAt(k);
/* 061 */ }
/* 062 */ project_value.setInt(l, el);
/* 063 */ } else if (!isNullAtL) {
/* 064 */ project_value.setInt(k, project_value.getInt(l));
/* 065 */ project_value.setNullAt(l);
/* 066 */ }
/* 067 */ }
/* 068 */
/* 069 */ }
```
### Non-primitive type
```
val df = Seq(
Seq("a", "c", "d", "b"),
null
).toDF("s")
df.filter($"s".isNotNull || $"s".isNull).select(reverse($"s")).debugCodegen
```
Result:
```
/* 032 */ boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 033 */ ArrayData inputadapter_value = inputadapter_isNull ?
/* 034 */ null : (inputadapter_row.getArray(0));
/* 035 */
/* 036 */ boolean filter_value = true;
/* 037 */
/* 038 */ if (!(!inputadapter_isNull)) {
/* 039 */ filter_value = inputadapter_isNull;
/* 040 */ }
/* 041 */ if (!filter_value) continue;
/* 042 */
/* 043 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 044 */
/* 045 */ boolean project_isNull = inputadapter_isNull;
/* 046 */ ArrayData project_value = null;
/* 047 */
/* 048 */ if (!inputadapter_isNull) {
/* 049 */ final int project_length = inputadapter_value.numElements();
/* 050 */ project_value = new org.apache.spark.sql.catalyst.util.GenericArrayData(new Object[project_length]);
/* 051 */ for(int k = 0; k < project_length; k++) {
/* 052 */ int l = project_length - k - 1;
/* 053 */ project_value.update(k, inputadapter_value.getUTF8String(l));
/* 054 */ }
/* 055 */
/* 056 */ }
```
Author: mn-mikke <mrkAha12346github>
Closes #21034 from mn-mikke/feature/array-api-reverse-to-master.
2018-04-18 05:41:55 -04:00
|
|
|
>>> df = spark.createDataFrame([([2, 1, 3],) ,([1],) ,([],)], ['data'])
|
|
|
|
>>> df.select(reverse(df.data).alias('r')).collect()
|
|
|
|
[Row(r=[3, 1, 2]), Row(r=[1]), Row(r=[])]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.reverse(_to_java_column(col)))
|
|
|
|
|
|
|
|
|
[SPARK-23821][SQL] Collection function: flatten
## What changes were proposed in this pull request?
This PR adds a new collection function that transforms an array of arrays into a single array. The PR comprises:
- An expression for flattening array structure
- Flatten function
- A wrapper for PySpark
## How was this patch tested?
New tests added into:
- CollectionExpressionsSuite
- DataFrameFunctionsSuite
## Codegen examples
### Primitive type
```
val df = Seq(
Seq(Seq(1, 2), Seq(4, 5)),
Seq(null, Seq(1))
).toDF("i")
df.filter($"i".isNotNull || $"i".isNull).select(flatten($"i")).debugCodegen
```
Result:
```
/* 033 */ boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 034 */ ArrayData inputadapter_value = inputadapter_isNull ?
/* 035 */ null : (inputadapter_row.getArray(0));
/* 036 */
/* 037 */ boolean filter_value = true;
/* 038 */
/* 039 */ if (!(!inputadapter_isNull)) {
/* 040 */ filter_value = inputadapter_isNull;
/* 041 */ }
/* 042 */ if (!filter_value) continue;
/* 043 */
/* 044 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 045 */
/* 046 */ boolean project_isNull = inputadapter_isNull;
/* 047 */ ArrayData project_value = null;
/* 048 */
/* 049 */ if (!inputadapter_isNull) {
/* 050 */ for (int z = 0; !project_isNull && z < inputadapter_value.numElements(); z++) {
/* 051 */ project_isNull |= inputadapter_value.isNullAt(z);
/* 052 */ }
/* 053 */ if (!project_isNull) {
/* 054 */ long project_numElements = 0;
/* 055 */ for (int z = 0; z < inputadapter_value.numElements(); z++) {
/* 056 */ project_numElements += inputadapter_value.getArray(z).numElements();
/* 057 */ }
/* 058 */ if (project_numElements > 2147483632) {
/* 059 */ throw new RuntimeException("Unsuccessful try to flatten an array of arrays with " +
/* 060 */ project_numElements + " elements due to exceeding the array size limit 2147483632.");
/* 061 */ }
/* 062 */
/* 063 */ long project_size = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(
/* 064 */ project_numElements,
/* 065 */ 4);
/* 066 */ if (project_size > 2147483632) {
/* 067 */ throw new RuntimeException("Unsuccessful try to flatten an array of arrays with " +
/* 068 */ project_size + " bytes of data due to exceeding the limit 2147483632" +
/* 069 */ " bytes for UnsafeArrayData.");
/* 070 */ }
/* 071 */
/* 072 */ byte[] project_array = new byte[(int)project_size];
/* 073 */ UnsafeArrayData project_tempArrayData = new UnsafeArrayData();
/* 074 */ Platform.putLong(project_array, 16, project_numElements);
/* 075 */ project_tempArrayData.pointTo(project_array, 16, (int)project_size);
/* 076 */ int project_counter = 0;
/* 077 */ for (int k = 0; k < inputadapter_value.numElements(); k++) {
/* 078 */ ArrayData arr = inputadapter_value.getArray(k);
/* 079 */ for (int l = 0; l < arr.numElements(); l++) {
/* 080 */ if (arr.isNullAt(l)) {
/* 081 */ project_tempArrayData.setNullAt(project_counter);
/* 082 */ } else {
/* 083 */ project_tempArrayData.setInt(
/* 084 */ project_counter,
/* 085 */ arr.getInt(l)
/* 086 */ );
/* 087 */ }
/* 088 */ project_counter++;
/* 089 */ }
/* 090 */ }
/* 091 */ project_value = project_tempArrayData;
/* 092 */
/* 093 */ }
/* 094 */
/* 095 */ }
```
### Non-primitive type
```
val df = Seq(
Seq(Seq("a", "b"), Seq(null, "d")),
Seq(null, Seq("a"))
).toDF("s")
df.filter($"s".isNotNull || $"s".isNull).select(flatten($"s")).debugCodegen
```
Result:
```
/* 033 */ boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 034 */ ArrayData inputadapter_value = inputadapter_isNull ?
/* 035 */ null : (inputadapter_row.getArray(0));
/* 036 */
/* 037 */ boolean filter_value = true;
/* 038 */
/* 039 */ if (!(!inputadapter_isNull)) {
/* 040 */ filter_value = inputadapter_isNull;
/* 041 */ }
/* 042 */ if (!filter_value) continue;
/* 043 */
/* 044 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 045 */
/* 046 */ boolean project_isNull = inputadapter_isNull;
/* 047 */ ArrayData project_value = null;
/* 048 */
/* 049 */ if (!inputadapter_isNull) {
/* 050 */ for (int z = 0; !project_isNull && z < inputadapter_value.numElements(); z++) {
/* 051 */ project_isNull |= inputadapter_value.isNullAt(z);
/* 052 */ }
/* 053 */ if (!project_isNull) {
/* 054 */ long project_numElements = 0;
/* 055 */ for (int z = 0; z < inputadapter_value.numElements(); z++) {
/* 056 */ project_numElements += inputadapter_value.getArray(z).numElements();
/* 057 */ }
/* 058 */ if (project_numElements > 2147483632) {
/* 059 */ throw new RuntimeException("Unsuccessful try to flatten an array of arrays with " +
/* 060 */ project_numElements + " elements due to exceeding the array size limit 2147483632.");
/* 061 */ }
/* 062 */
/* 063 */ Object[] project_arrayObject = new Object[(int)project_numElements];
/* 064 */ int project_counter = 0;
/* 065 */ for (int k = 0; k < inputadapter_value.numElements(); k++) {
/* 066 */ ArrayData arr = inputadapter_value.getArray(k);
/* 067 */ for (int l = 0; l < arr.numElements(); l++) {
/* 068 */ project_arrayObject[project_counter] = arr.getUTF8String(l);
/* 069 */ project_counter++;
/* 070 */ }
/* 071 */ }
/* 072 */ project_value = new org.apache.spark.sql.catalyst.util.GenericArrayData(project_arrayObject);
/* 073 */
/* 074 */ }
/* 075 */
/* 076 */ }
```
Author: mn-mikke <mrkAha12346github>
Closes #20938 from mn-mikke/feature/array-api-flatten-to-master.
2018-04-24 22:19:08 -04:00
|
|
|
def flatten(col):
|
|
|
|
"""
|
|
|
|
Collection function: creates a single array from an array of arrays.
|
|
|
|
If a structure of nested arrays is deeper than two levels,
|
|
|
|
only one level of nesting is removed.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.4.0
|
[SPARK-23821][SQL] Collection function: flatten
## What changes were proposed in this pull request?
This PR adds a new collection function that transforms an array of arrays into a single array. The PR comprises:
- An expression for flattening array structure
- Flatten function
- A wrapper for PySpark
## How was this patch tested?
New tests added into:
- CollectionExpressionsSuite
- DataFrameFunctionsSuite
## Codegen examples
### Primitive type
```
val df = Seq(
Seq(Seq(1, 2), Seq(4, 5)),
Seq(null, Seq(1))
).toDF("i")
df.filter($"i".isNotNull || $"i".isNull).select(flatten($"i")).debugCodegen
```
Result:
```
/* 033 */ boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 034 */ ArrayData inputadapter_value = inputadapter_isNull ?
/* 035 */ null : (inputadapter_row.getArray(0));
/* 036 */
/* 037 */ boolean filter_value = true;
/* 038 */
/* 039 */ if (!(!inputadapter_isNull)) {
/* 040 */ filter_value = inputadapter_isNull;
/* 041 */ }
/* 042 */ if (!filter_value) continue;
/* 043 */
/* 044 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 045 */
/* 046 */ boolean project_isNull = inputadapter_isNull;
/* 047 */ ArrayData project_value = null;
/* 048 */
/* 049 */ if (!inputadapter_isNull) {
/* 050 */ for (int z = 0; !project_isNull && z < inputadapter_value.numElements(); z++) {
/* 051 */ project_isNull |= inputadapter_value.isNullAt(z);
/* 052 */ }
/* 053 */ if (!project_isNull) {
/* 054 */ long project_numElements = 0;
/* 055 */ for (int z = 0; z < inputadapter_value.numElements(); z++) {
/* 056 */ project_numElements += inputadapter_value.getArray(z).numElements();
/* 057 */ }
/* 058 */ if (project_numElements > 2147483632) {
/* 059 */ throw new RuntimeException("Unsuccessful try to flatten an array of arrays with " +
/* 060 */ project_numElements + " elements due to exceeding the array size limit 2147483632.");
/* 061 */ }
/* 062 */
/* 063 */ long project_size = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(
/* 064 */ project_numElements,
/* 065 */ 4);
/* 066 */ if (project_size > 2147483632) {
/* 067 */ throw new RuntimeException("Unsuccessful try to flatten an array of arrays with " +
/* 068 */ project_size + " bytes of data due to exceeding the limit 2147483632" +
/* 069 */ " bytes for UnsafeArrayData.");
/* 070 */ }
/* 071 */
/* 072 */ byte[] project_array = new byte[(int)project_size];
/* 073 */ UnsafeArrayData project_tempArrayData = new UnsafeArrayData();
/* 074 */ Platform.putLong(project_array, 16, project_numElements);
/* 075 */ project_tempArrayData.pointTo(project_array, 16, (int)project_size);
/* 076 */ int project_counter = 0;
/* 077 */ for (int k = 0; k < inputadapter_value.numElements(); k++) {
/* 078 */ ArrayData arr = inputadapter_value.getArray(k);
/* 079 */ for (int l = 0; l < arr.numElements(); l++) {
/* 080 */ if (arr.isNullAt(l)) {
/* 081 */ project_tempArrayData.setNullAt(project_counter);
/* 082 */ } else {
/* 083 */ project_tempArrayData.setInt(
/* 084 */ project_counter,
/* 085 */ arr.getInt(l)
/* 086 */ );
/* 087 */ }
/* 088 */ project_counter++;
/* 089 */ }
/* 090 */ }
/* 091 */ project_value = project_tempArrayData;
/* 092 */
/* 093 */ }
/* 094 */
/* 095 */ }
```
### Non-primitive type
```
val df = Seq(
Seq(Seq("a", "b"), Seq(null, "d")),
Seq(null, Seq("a"))
).toDF("s")
df.filter($"s".isNotNull || $"s".isNull).select(flatten($"s")).debugCodegen
```
Result:
```
/* 033 */ boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 034 */ ArrayData inputadapter_value = inputadapter_isNull ?
/* 035 */ null : (inputadapter_row.getArray(0));
/* 036 */
/* 037 */ boolean filter_value = true;
/* 038 */
/* 039 */ if (!(!inputadapter_isNull)) {
/* 040 */ filter_value = inputadapter_isNull;
/* 041 */ }
/* 042 */ if (!filter_value) continue;
/* 043 */
/* 044 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 045 */
/* 046 */ boolean project_isNull = inputadapter_isNull;
/* 047 */ ArrayData project_value = null;
/* 048 */
/* 049 */ if (!inputadapter_isNull) {
/* 050 */ for (int z = 0; !project_isNull && z < inputadapter_value.numElements(); z++) {
/* 051 */ project_isNull |= inputadapter_value.isNullAt(z);
/* 052 */ }
/* 053 */ if (!project_isNull) {
/* 054 */ long project_numElements = 0;
/* 055 */ for (int z = 0; z < inputadapter_value.numElements(); z++) {
/* 056 */ project_numElements += inputadapter_value.getArray(z).numElements();
/* 057 */ }
/* 058 */ if (project_numElements > 2147483632) {
/* 059 */ throw new RuntimeException("Unsuccessful try to flatten an array of arrays with " +
/* 060 */ project_numElements + " elements due to exceeding the array size limit 2147483632.");
/* 061 */ }
/* 062 */
/* 063 */ Object[] project_arrayObject = new Object[(int)project_numElements];
/* 064 */ int project_counter = 0;
/* 065 */ for (int k = 0; k < inputadapter_value.numElements(); k++) {
/* 066 */ ArrayData arr = inputadapter_value.getArray(k);
/* 067 */ for (int l = 0; l < arr.numElements(); l++) {
/* 068 */ project_arrayObject[project_counter] = arr.getUTF8String(l);
/* 069 */ project_counter++;
/* 070 */ }
/* 071 */ }
/* 072 */ project_value = new org.apache.spark.sql.catalyst.util.GenericArrayData(project_arrayObject);
/* 073 */
/* 074 */ }
/* 075 */
/* 076 */ }
```
Author: mn-mikke <mrkAha12346github>
Closes #20938 from mn-mikke/feature/array-api-flatten-to-master.
2018-04-24 22:19:08 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column or expression
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
[SPARK-23821][SQL] Collection function: flatten
## What changes were proposed in this pull request?
This PR adds a new collection function that transforms an array of arrays into a single array. The PR comprises:
- An expression for flattening array structure
- Flatten function
- A wrapper for PySpark
## How was this patch tested?
New tests added into:
- CollectionExpressionsSuite
- DataFrameFunctionsSuite
## Codegen examples
### Primitive type
```
val df = Seq(
Seq(Seq(1, 2), Seq(4, 5)),
Seq(null, Seq(1))
).toDF("i")
df.filter($"i".isNotNull || $"i".isNull).select(flatten($"i")).debugCodegen
```
Result:
```
/* 033 */ boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 034 */ ArrayData inputadapter_value = inputadapter_isNull ?
/* 035 */ null : (inputadapter_row.getArray(0));
/* 036 */
/* 037 */ boolean filter_value = true;
/* 038 */
/* 039 */ if (!(!inputadapter_isNull)) {
/* 040 */ filter_value = inputadapter_isNull;
/* 041 */ }
/* 042 */ if (!filter_value) continue;
/* 043 */
/* 044 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 045 */
/* 046 */ boolean project_isNull = inputadapter_isNull;
/* 047 */ ArrayData project_value = null;
/* 048 */
/* 049 */ if (!inputadapter_isNull) {
/* 050 */ for (int z = 0; !project_isNull && z < inputadapter_value.numElements(); z++) {
/* 051 */ project_isNull |= inputadapter_value.isNullAt(z);
/* 052 */ }
/* 053 */ if (!project_isNull) {
/* 054 */ long project_numElements = 0;
/* 055 */ for (int z = 0; z < inputadapter_value.numElements(); z++) {
/* 056 */ project_numElements += inputadapter_value.getArray(z).numElements();
/* 057 */ }
/* 058 */ if (project_numElements > 2147483632) {
/* 059 */ throw new RuntimeException("Unsuccessful try to flatten an array of arrays with " +
/* 060 */ project_numElements + " elements due to exceeding the array size limit 2147483632.");
/* 061 */ }
/* 062 */
/* 063 */ long project_size = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(
/* 064 */ project_numElements,
/* 065 */ 4);
/* 066 */ if (project_size > 2147483632) {
/* 067 */ throw new RuntimeException("Unsuccessful try to flatten an array of arrays with " +
/* 068 */ project_size + " bytes of data due to exceeding the limit 2147483632" +
/* 069 */ " bytes for UnsafeArrayData.");
/* 070 */ }
/* 071 */
/* 072 */ byte[] project_array = new byte[(int)project_size];
/* 073 */ UnsafeArrayData project_tempArrayData = new UnsafeArrayData();
/* 074 */ Platform.putLong(project_array, 16, project_numElements);
/* 075 */ project_tempArrayData.pointTo(project_array, 16, (int)project_size);
/* 076 */ int project_counter = 0;
/* 077 */ for (int k = 0; k < inputadapter_value.numElements(); k++) {
/* 078 */ ArrayData arr = inputadapter_value.getArray(k);
/* 079 */ for (int l = 0; l < arr.numElements(); l++) {
/* 080 */ if (arr.isNullAt(l)) {
/* 081 */ project_tempArrayData.setNullAt(project_counter);
/* 082 */ } else {
/* 083 */ project_tempArrayData.setInt(
/* 084 */ project_counter,
/* 085 */ arr.getInt(l)
/* 086 */ );
/* 087 */ }
/* 088 */ project_counter++;
/* 089 */ }
/* 090 */ }
/* 091 */ project_value = project_tempArrayData;
/* 092 */
/* 093 */ }
/* 094 */
/* 095 */ }
```
### Non-primitive type
```
val df = Seq(
Seq(Seq("a", "b"), Seq(null, "d")),
Seq(null, Seq("a"))
).toDF("s")
df.filter($"s".isNotNull || $"s".isNull).select(flatten($"s")).debugCodegen
```
Result:
```
/* 033 */ boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 034 */ ArrayData inputadapter_value = inputadapter_isNull ?
/* 035 */ null : (inputadapter_row.getArray(0));
/* 036 */
/* 037 */ boolean filter_value = true;
/* 038 */
/* 039 */ if (!(!inputadapter_isNull)) {
/* 040 */ filter_value = inputadapter_isNull;
/* 041 */ }
/* 042 */ if (!filter_value) continue;
/* 043 */
/* 044 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 045 */
/* 046 */ boolean project_isNull = inputadapter_isNull;
/* 047 */ ArrayData project_value = null;
/* 048 */
/* 049 */ if (!inputadapter_isNull) {
/* 050 */ for (int z = 0; !project_isNull && z < inputadapter_value.numElements(); z++) {
/* 051 */ project_isNull |= inputadapter_value.isNullAt(z);
/* 052 */ }
/* 053 */ if (!project_isNull) {
/* 054 */ long project_numElements = 0;
/* 055 */ for (int z = 0; z < inputadapter_value.numElements(); z++) {
/* 056 */ project_numElements += inputadapter_value.getArray(z).numElements();
/* 057 */ }
/* 058 */ if (project_numElements > 2147483632) {
/* 059 */ throw new RuntimeException("Unsuccessful try to flatten an array of arrays with " +
/* 060 */ project_numElements + " elements due to exceeding the array size limit 2147483632.");
/* 061 */ }
/* 062 */
/* 063 */ Object[] project_arrayObject = new Object[(int)project_numElements];
/* 064 */ int project_counter = 0;
/* 065 */ for (int k = 0; k < inputadapter_value.numElements(); k++) {
/* 066 */ ArrayData arr = inputadapter_value.getArray(k);
/* 067 */ for (int l = 0; l < arr.numElements(); l++) {
/* 068 */ project_arrayObject[project_counter] = arr.getUTF8String(l);
/* 069 */ project_counter++;
/* 070 */ }
/* 071 */ }
/* 072 */ project_value = new org.apache.spark.sql.catalyst.util.GenericArrayData(project_arrayObject);
/* 073 */
/* 074 */ }
/* 075 */
/* 076 */ }
```
Author: mn-mikke <mrkAha12346github>
Closes #20938 from mn-mikke/feature/array-api-flatten-to-master.
2018-04-24 22:19:08 -04:00
|
|
|
>>> df = spark.createDataFrame([([[1, 2, 3], [4, 5], [6]],), ([None, [4, 5]],)], ['data'])
|
|
|
|
>>> df.select(flatten(df.data).alias('r')).collect()
|
|
|
|
[Row(r=[1, 2, 3, 4, 5, 6]), Row(r=None)]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.flatten(_to_java_column(col)))
|
|
|
|
|
|
|
|
|
2017-06-19 14:40:07 -04:00
|
|
|
def map_keys(col):
|
|
|
|
"""
|
|
|
|
Collection function: Returns an unordered array containing the keys of the map.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.3.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column or expression
|
2017-06-19 14:40:07 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2017-06-19 14:40:07 -04:00
|
|
|
>>> from pyspark.sql.functions import map_keys
|
|
|
|
>>> df = spark.sql("SELECT map(1, 'a', 2, 'b') as data")
|
|
|
|
>>> df.select(map_keys("data").alias("keys")).show()
|
|
|
|
+------+
|
|
|
|
| keys|
|
|
|
|
+------+
|
|
|
|
|[1, 2]|
|
|
|
|
+------+
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.map_keys(_to_java_column(col)))
|
|
|
|
|
|
|
|
|
|
|
|
def map_values(col):
|
|
|
|
"""
|
|
|
|
Collection function: Returns an unordered array containing the values of the map.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.3.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column or expression
|
2017-06-19 14:40:07 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2017-06-19 14:40:07 -04:00
|
|
|
>>> from pyspark.sql.functions import map_values
|
|
|
|
>>> df = spark.sql("SELECT map(1, 'a', 2, 'b') as data")
|
|
|
|
>>> df.select(map_values("data").alias("values")).show()
|
|
|
|
+------+
|
|
|
|
|values|
|
|
|
|
+------+
|
|
|
|
|[a, b]|
|
|
|
|
+------+
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.map_values(_to_java_column(col)))
|
|
|
|
|
|
|
|
|
[SPARK-23935][SQL] Adding map_entries function
## What changes were proposed in this pull request?
This PR adds `map_entries` function that returns an unordered array of all entries in the given map.
## How was this patch tested?
New tests added into:
- `CollectionExpressionSuite`
- `DataFrameFunctionsSuite`
## CodeGen examples
### Primitive types
```
val df = Seq(Map(1 -> 5, 2 -> 6)).toDF("m")
df.filter('m.isNotNull).select(map_entries('m)).debugCodegen
```
Result:
```
/* 042 */ boolean project_isNull_0 = false;
/* 043 */
/* 044 */ ArrayData project_value_0 = null;
/* 045 */
/* 046 */ final int project_numElements_0 = inputadapter_value_0.numElements();
/* 047 */ final ArrayData project_keys_0 = inputadapter_value_0.keyArray();
/* 048 */ final ArrayData project_values_0 = inputadapter_value_0.valueArray();
/* 049 */
/* 050 */ final long project_size_0 = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(
/* 051 */ project_numElements_0,
/* 052 */ 32);
/* 053 */ if (project_size_0 > 2147483632) {
/* 054 */ final Object[] project_internalRowArray_0 = new Object[project_numElements_0];
/* 055 */ for (int z = 0; z < project_numElements_0; z++) {
/* 056 */ project_internalRowArray_0[z] = new org.apache.spark.sql.catalyst.expressions.GenericInternalRow(new Object[]{project_keys_0.getInt(z), project_values_0.getInt(z)});
/* 057 */ }
/* 058 */ project_value_0 = new org.apache.spark.sql.catalyst.util.GenericArrayData(project_internalRowArray_0);
/* 059 */
/* 060 */ } else {
/* 061 */ final byte[] project_arrayBytes_0 = new byte[(int)project_size_0];
/* 062 */ UnsafeArrayData project_unsafeArrayData_0 = new UnsafeArrayData();
/* 063 */ Platform.putLong(project_arrayBytes_0, 16, project_numElements_0);
/* 064 */ project_unsafeArrayData_0.pointTo(project_arrayBytes_0, 16, (int)project_size_0);
/* 065 */
/* 066 */ final int project_structsOffset_0 = UnsafeArrayData.calculateHeaderPortionInBytes(project_numElements_0) + project_numElements_0 * 8;
/* 067 */ UnsafeRow project_unsafeRow_0 = new UnsafeRow(2);
/* 068 */ for (int z = 0; z < project_numElements_0; z++) {
/* 069 */ long offset = project_structsOffset_0 + z * 24L;
/* 070 */ project_unsafeArrayData_0.setLong(z, (offset << 32) + 24L);
/* 071 */ project_unsafeRow_0.pointTo(project_arrayBytes_0, 16 + offset, 24);
/* 072 */ project_unsafeRow_0.setInt(0, project_keys_0.getInt(z));
/* 073 */ project_unsafeRow_0.setInt(1, project_values_0.getInt(z));
/* 074 */ }
/* 075 */ project_value_0 = project_unsafeArrayData_0;
/* 076 */
/* 077 */ }
```
### Non-primitive types
```
val df = Seq(Map("a" -> "foo", "b" -> null)).toDF("m")
df.filter('m.isNotNull).select(map_entries('m)).debugCodegen
```
Result:
```
/* 042 */ boolean project_isNull_0 = false;
/* 043 */
/* 044 */ ArrayData project_value_0 = null;
/* 045 */
/* 046 */ final int project_numElements_0 = inputadapter_value_0.numElements();
/* 047 */ final ArrayData project_keys_0 = inputadapter_value_0.keyArray();
/* 048 */ final ArrayData project_values_0 = inputadapter_value_0.valueArray();
/* 049 */
/* 050 */ final Object[] project_internalRowArray_0 = new Object[project_numElements_0];
/* 051 */ for (int z = 0; z < project_numElements_0; z++) {
/* 052 */ project_internalRowArray_0[z] = new org.apache.spark.sql.catalyst.expressions.GenericInternalRow(new Object[]{project_keys_0.getUTF8String(z), project_values_0.getUTF8String(z)});
/* 053 */ }
/* 054 */ project_value_0 = new org.apache.spark.sql.catalyst.util.GenericArrayData(project_internalRowArray_0);
```
Author: Marek Novotny <mn.mikke@gmail.com>
Closes #21236 from mn-mikke/feature/array-api-map_entries-to-master.
2018-05-21 10:14:03 -04:00
|
|
|
def map_entries(col):
|
|
|
|
"""
|
|
|
|
Collection function: Returns an unordered array of all entries in the given map.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 3.0.0
|
[SPARK-23935][SQL] Adding map_entries function
## What changes were proposed in this pull request?
This PR adds `map_entries` function that returns an unordered array of all entries in the given map.
## How was this patch tested?
New tests added into:
- `CollectionExpressionSuite`
- `DataFrameFunctionsSuite`
## CodeGen examples
### Primitive types
```
val df = Seq(Map(1 -> 5, 2 -> 6)).toDF("m")
df.filter('m.isNotNull).select(map_entries('m)).debugCodegen
```
Result:
```
/* 042 */ boolean project_isNull_0 = false;
/* 043 */
/* 044 */ ArrayData project_value_0 = null;
/* 045 */
/* 046 */ final int project_numElements_0 = inputadapter_value_0.numElements();
/* 047 */ final ArrayData project_keys_0 = inputadapter_value_0.keyArray();
/* 048 */ final ArrayData project_values_0 = inputadapter_value_0.valueArray();
/* 049 */
/* 050 */ final long project_size_0 = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(
/* 051 */ project_numElements_0,
/* 052 */ 32);
/* 053 */ if (project_size_0 > 2147483632) {
/* 054 */ final Object[] project_internalRowArray_0 = new Object[project_numElements_0];
/* 055 */ for (int z = 0; z < project_numElements_0; z++) {
/* 056 */ project_internalRowArray_0[z] = new org.apache.spark.sql.catalyst.expressions.GenericInternalRow(new Object[]{project_keys_0.getInt(z), project_values_0.getInt(z)});
/* 057 */ }
/* 058 */ project_value_0 = new org.apache.spark.sql.catalyst.util.GenericArrayData(project_internalRowArray_0);
/* 059 */
/* 060 */ } else {
/* 061 */ final byte[] project_arrayBytes_0 = new byte[(int)project_size_0];
/* 062 */ UnsafeArrayData project_unsafeArrayData_0 = new UnsafeArrayData();
/* 063 */ Platform.putLong(project_arrayBytes_0, 16, project_numElements_0);
/* 064 */ project_unsafeArrayData_0.pointTo(project_arrayBytes_0, 16, (int)project_size_0);
/* 065 */
/* 066 */ final int project_structsOffset_0 = UnsafeArrayData.calculateHeaderPortionInBytes(project_numElements_0) + project_numElements_0 * 8;
/* 067 */ UnsafeRow project_unsafeRow_0 = new UnsafeRow(2);
/* 068 */ for (int z = 0; z < project_numElements_0; z++) {
/* 069 */ long offset = project_structsOffset_0 + z * 24L;
/* 070 */ project_unsafeArrayData_0.setLong(z, (offset << 32) + 24L);
/* 071 */ project_unsafeRow_0.pointTo(project_arrayBytes_0, 16 + offset, 24);
/* 072 */ project_unsafeRow_0.setInt(0, project_keys_0.getInt(z));
/* 073 */ project_unsafeRow_0.setInt(1, project_values_0.getInt(z));
/* 074 */ }
/* 075 */ project_value_0 = project_unsafeArrayData_0;
/* 076 */
/* 077 */ }
```
### Non-primitive types
```
val df = Seq(Map("a" -> "foo", "b" -> null)).toDF("m")
df.filter('m.isNotNull).select(map_entries('m)).debugCodegen
```
Result:
```
/* 042 */ boolean project_isNull_0 = false;
/* 043 */
/* 044 */ ArrayData project_value_0 = null;
/* 045 */
/* 046 */ final int project_numElements_0 = inputadapter_value_0.numElements();
/* 047 */ final ArrayData project_keys_0 = inputadapter_value_0.keyArray();
/* 048 */ final ArrayData project_values_0 = inputadapter_value_0.valueArray();
/* 049 */
/* 050 */ final Object[] project_internalRowArray_0 = new Object[project_numElements_0];
/* 051 */ for (int z = 0; z < project_numElements_0; z++) {
/* 052 */ project_internalRowArray_0[z] = new org.apache.spark.sql.catalyst.expressions.GenericInternalRow(new Object[]{project_keys_0.getUTF8String(z), project_values_0.getUTF8String(z)});
/* 053 */ }
/* 054 */ project_value_0 = new org.apache.spark.sql.catalyst.util.GenericArrayData(project_internalRowArray_0);
```
Author: Marek Novotny <mn.mikke@gmail.com>
Closes #21236 from mn-mikke/feature/array-api-map_entries-to-master.
2018-05-21 10:14:03 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column or expression
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
[SPARK-23935][SQL] Adding map_entries function
## What changes were proposed in this pull request?
This PR adds `map_entries` function that returns an unordered array of all entries in the given map.
## How was this patch tested?
New tests added into:
- `CollectionExpressionSuite`
- `DataFrameFunctionsSuite`
## CodeGen examples
### Primitive types
```
val df = Seq(Map(1 -> 5, 2 -> 6)).toDF("m")
df.filter('m.isNotNull).select(map_entries('m)).debugCodegen
```
Result:
```
/* 042 */ boolean project_isNull_0 = false;
/* 043 */
/* 044 */ ArrayData project_value_0 = null;
/* 045 */
/* 046 */ final int project_numElements_0 = inputadapter_value_0.numElements();
/* 047 */ final ArrayData project_keys_0 = inputadapter_value_0.keyArray();
/* 048 */ final ArrayData project_values_0 = inputadapter_value_0.valueArray();
/* 049 */
/* 050 */ final long project_size_0 = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(
/* 051 */ project_numElements_0,
/* 052 */ 32);
/* 053 */ if (project_size_0 > 2147483632) {
/* 054 */ final Object[] project_internalRowArray_0 = new Object[project_numElements_0];
/* 055 */ for (int z = 0; z < project_numElements_0; z++) {
/* 056 */ project_internalRowArray_0[z] = new org.apache.spark.sql.catalyst.expressions.GenericInternalRow(new Object[]{project_keys_0.getInt(z), project_values_0.getInt(z)});
/* 057 */ }
/* 058 */ project_value_0 = new org.apache.spark.sql.catalyst.util.GenericArrayData(project_internalRowArray_0);
/* 059 */
/* 060 */ } else {
/* 061 */ final byte[] project_arrayBytes_0 = new byte[(int)project_size_0];
/* 062 */ UnsafeArrayData project_unsafeArrayData_0 = new UnsafeArrayData();
/* 063 */ Platform.putLong(project_arrayBytes_0, 16, project_numElements_0);
/* 064 */ project_unsafeArrayData_0.pointTo(project_arrayBytes_0, 16, (int)project_size_0);
/* 065 */
/* 066 */ final int project_structsOffset_0 = UnsafeArrayData.calculateHeaderPortionInBytes(project_numElements_0) + project_numElements_0 * 8;
/* 067 */ UnsafeRow project_unsafeRow_0 = new UnsafeRow(2);
/* 068 */ for (int z = 0; z < project_numElements_0; z++) {
/* 069 */ long offset = project_structsOffset_0 + z * 24L;
/* 070 */ project_unsafeArrayData_0.setLong(z, (offset << 32) + 24L);
/* 071 */ project_unsafeRow_0.pointTo(project_arrayBytes_0, 16 + offset, 24);
/* 072 */ project_unsafeRow_0.setInt(0, project_keys_0.getInt(z));
/* 073 */ project_unsafeRow_0.setInt(1, project_values_0.getInt(z));
/* 074 */ }
/* 075 */ project_value_0 = project_unsafeArrayData_0;
/* 076 */
/* 077 */ }
```
### Non-primitive types
```
val df = Seq(Map("a" -> "foo", "b" -> null)).toDF("m")
df.filter('m.isNotNull).select(map_entries('m)).debugCodegen
```
Result:
```
/* 042 */ boolean project_isNull_0 = false;
/* 043 */
/* 044 */ ArrayData project_value_0 = null;
/* 045 */
/* 046 */ final int project_numElements_0 = inputadapter_value_0.numElements();
/* 047 */ final ArrayData project_keys_0 = inputadapter_value_0.keyArray();
/* 048 */ final ArrayData project_values_0 = inputadapter_value_0.valueArray();
/* 049 */
/* 050 */ final Object[] project_internalRowArray_0 = new Object[project_numElements_0];
/* 051 */ for (int z = 0; z < project_numElements_0; z++) {
/* 052 */ project_internalRowArray_0[z] = new org.apache.spark.sql.catalyst.expressions.GenericInternalRow(new Object[]{project_keys_0.getUTF8String(z), project_values_0.getUTF8String(z)});
/* 053 */ }
/* 054 */ project_value_0 = new org.apache.spark.sql.catalyst.util.GenericArrayData(project_internalRowArray_0);
```
Author: Marek Novotny <mn.mikke@gmail.com>
Closes #21236 from mn-mikke/feature/array-api-map_entries-to-master.
2018-05-21 10:14:03 -04:00
|
|
|
>>> from pyspark.sql.functions import map_entries
|
|
|
|
>>> df = spark.sql("SELECT map(1, 'a', 2, 'b') as data")
|
|
|
|
>>> df.select(map_entries("data").alias("entries")).show()
|
|
|
|
+----------------+
|
|
|
|
| entries|
|
|
|
|
+----------------+
|
2020-08-04 10:57:09 -04:00
|
|
|
|[{1, a}, {2, b}]|
|
[SPARK-23935][SQL] Adding map_entries function
## What changes were proposed in this pull request?
This PR adds `map_entries` function that returns an unordered array of all entries in the given map.
## How was this patch tested?
New tests added into:
- `CollectionExpressionSuite`
- `DataFrameFunctionsSuite`
## CodeGen examples
### Primitive types
```
val df = Seq(Map(1 -> 5, 2 -> 6)).toDF("m")
df.filter('m.isNotNull).select(map_entries('m)).debugCodegen
```
Result:
```
/* 042 */ boolean project_isNull_0 = false;
/* 043 */
/* 044 */ ArrayData project_value_0 = null;
/* 045 */
/* 046 */ final int project_numElements_0 = inputadapter_value_0.numElements();
/* 047 */ final ArrayData project_keys_0 = inputadapter_value_0.keyArray();
/* 048 */ final ArrayData project_values_0 = inputadapter_value_0.valueArray();
/* 049 */
/* 050 */ final long project_size_0 = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(
/* 051 */ project_numElements_0,
/* 052 */ 32);
/* 053 */ if (project_size_0 > 2147483632) {
/* 054 */ final Object[] project_internalRowArray_0 = new Object[project_numElements_0];
/* 055 */ for (int z = 0; z < project_numElements_0; z++) {
/* 056 */ project_internalRowArray_0[z] = new org.apache.spark.sql.catalyst.expressions.GenericInternalRow(new Object[]{project_keys_0.getInt(z), project_values_0.getInt(z)});
/* 057 */ }
/* 058 */ project_value_0 = new org.apache.spark.sql.catalyst.util.GenericArrayData(project_internalRowArray_0);
/* 059 */
/* 060 */ } else {
/* 061 */ final byte[] project_arrayBytes_0 = new byte[(int)project_size_0];
/* 062 */ UnsafeArrayData project_unsafeArrayData_0 = new UnsafeArrayData();
/* 063 */ Platform.putLong(project_arrayBytes_0, 16, project_numElements_0);
/* 064 */ project_unsafeArrayData_0.pointTo(project_arrayBytes_0, 16, (int)project_size_0);
/* 065 */
/* 066 */ final int project_structsOffset_0 = UnsafeArrayData.calculateHeaderPortionInBytes(project_numElements_0) + project_numElements_0 * 8;
/* 067 */ UnsafeRow project_unsafeRow_0 = new UnsafeRow(2);
/* 068 */ for (int z = 0; z < project_numElements_0; z++) {
/* 069 */ long offset = project_structsOffset_0 + z * 24L;
/* 070 */ project_unsafeArrayData_0.setLong(z, (offset << 32) + 24L);
/* 071 */ project_unsafeRow_0.pointTo(project_arrayBytes_0, 16 + offset, 24);
/* 072 */ project_unsafeRow_0.setInt(0, project_keys_0.getInt(z));
/* 073 */ project_unsafeRow_0.setInt(1, project_values_0.getInt(z));
/* 074 */ }
/* 075 */ project_value_0 = project_unsafeArrayData_0;
/* 076 */
/* 077 */ }
```
### Non-primitive types
```
val df = Seq(Map("a" -> "foo", "b" -> null)).toDF("m")
df.filter('m.isNotNull).select(map_entries('m)).debugCodegen
```
Result:
```
/* 042 */ boolean project_isNull_0 = false;
/* 043 */
/* 044 */ ArrayData project_value_0 = null;
/* 045 */
/* 046 */ final int project_numElements_0 = inputadapter_value_0.numElements();
/* 047 */ final ArrayData project_keys_0 = inputadapter_value_0.keyArray();
/* 048 */ final ArrayData project_values_0 = inputadapter_value_0.valueArray();
/* 049 */
/* 050 */ final Object[] project_internalRowArray_0 = new Object[project_numElements_0];
/* 051 */ for (int z = 0; z < project_numElements_0; z++) {
/* 052 */ project_internalRowArray_0[z] = new org.apache.spark.sql.catalyst.expressions.GenericInternalRow(new Object[]{project_keys_0.getUTF8String(z), project_values_0.getUTF8String(z)});
/* 053 */ }
/* 054 */ project_value_0 = new org.apache.spark.sql.catalyst.util.GenericArrayData(project_internalRowArray_0);
```
Author: Marek Novotny <mn.mikke@gmail.com>
Closes #21236 from mn-mikke/feature/array-api-map_entries-to-master.
2018-05-21 10:14:03 -04:00
|
|
|
+----------------+
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.map_entries(_to_java_column(col)))
|
|
|
|
|
|
|
|
|
[SPARK-23934][SQL] Adding map_from_entries function
## What changes were proposed in this pull request?
The PR adds the `map_from_entries` function that returns a map created from the given array of entries.
## How was this patch tested?
New tests added into:
- `CollectionExpressionSuite`
- `DataFrameFunctionSuite`
## CodeGen Examples
### Primitive-type Keys and Values
```
val idf = Seq(
Seq((1, 10), (2, 20), (3, 10)),
Seq((1, 10), null, (2, 20))
).toDF("a")
idf.filter('a.isNotNull).select(map_from_entries('a)).debugCodegen
```
Result:
```
/* 042 */ boolean project_isNull_0 = false;
/* 043 */ MapData project_value_0 = null;
/* 044 */
/* 045 */ for (int project_idx_2 = 0; !project_isNull_0 && project_idx_2 < inputadapter_value_0.numElements(); project_idx_2++) {
/* 046 */ project_isNull_0 |= inputadapter_value_0.isNullAt(project_idx_2);
/* 047 */ }
/* 048 */ if (!project_isNull_0) {
/* 049 */ final int project_numEntries_0 = inputadapter_value_0.numElements();
/* 050 */
/* 051 */ final long project_keySectionSize_0 = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(project_numEntries_0, 4);
/* 052 */ final long project_valueSectionSize_0 = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(project_numEntries_0, 4);
/* 053 */ final long project_byteArraySize_0 = 8 + project_keySectionSize_0 + project_valueSectionSize_0;
/* 054 */ if (project_byteArraySize_0 > 2147483632) {
/* 055 */ final Object[] project_keys_0 = new Object[project_numEntries_0];
/* 056 */ final Object[] project_values_0 = new Object[project_numEntries_0];
/* 057 */
/* 058 */ for (int project_idx_1 = 0; project_idx_1 < project_numEntries_0; project_idx_1++) {
/* 059 */ InternalRow project_entry_1 = inputadapter_value_0.getStruct(project_idx_1, 2);
/* 060 */
/* 061 */ project_keys_0[project_idx_1] = project_entry_1.getInt(0);
/* 062 */ project_values_0[project_idx_1] = project_entry_1.getInt(1);
/* 063 */ }
/* 064 */
/* 065 */ project_value_0 = org.apache.spark.sql.catalyst.util.ArrayBasedMapData.apply(project_keys_0, project_values_0);
/* 066 */
/* 067 */ } else {
/* 068 */ final byte[] project_byteArray_0 = new byte[(int)project_byteArraySize_0];
/* 069 */ UnsafeMapData project_unsafeMapData_0 = new UnsafeMapData();
/* 070 */ Platform.putLong(project_byteArray_0, 16, project_keySectionSize_0);
/* 071 */ Platform.putLong(project_byteArray_0, 24, project_numEntries_0);
/* 072 */ Platform.putLong(project_byteArray_0, 24 + project_keySectionSize_0, project_numEntries_0);
/* 073 */ project_unsafeMapData_0.pointTo(project_byteArray_0, 16, (int)project_byteArraySize_0);
/* 074 */ ArrayData project_keyArrayData_0 = project_unsafeMapData_0.keyArray();
/* 075 */ ArrayData project_valueArrayData_0 = project_unsafeMapData_0.valueArray();
/* 076 */
/* 077 */ for (int project_idx_0 = 0; project_idx_0 < project_numEntries_0; project_idx_0++) {
/* 078 */ InternalRow project_entry_0 = inputadapter_value_0.getStruct(project_idx_0, 2);
/* 079 */
/* 080 */ project_keyArrayData_0.setInt(project_idx_0, project_entry_0.getInt(0));
/* 081 */ project_valueArrayData_0.setInt(project_idx_0, project_entry_0.getInt(1));
/* 082 */ }
/* 083 */
/* 084 */ project_value_0 = project_unsafeMapData_0;
/* 085 */ }
/* 086 */
/* 087 */ }
```
### Non-primitive-type Keys and Values
```
val sdf = Seq(
Seq(("a", null), ("b", "bb"), ("c", "aa")),
Seq(("a", "aa"), null, (null, "bb"))
).toDF("a")
sdf.filter('a.isNotNull).select(map_from_entries('a)).debugCodegen
```
Result:
```
/* 042 */ boolean project_isNull_0 = false;
/* 043 */ MapData project_value_0 = null;
/* 044 */
/* 045 */ for (int project_idx_1 = 0; !project_isNull_0 && project_idx_1 < inputadapter_value_0.numElements(); project_idx_1++) {
/* 046 */ project_isNull_0 |= inputadapter_value_0.isNullAt(project_idx_1);
/* 047 */ }
/* 048 */ if (!project_isNull_0) {
/* 049 */ final int project_numEntries_0 = inputadapter_value_0.numElements();
/* 050 */
/* 051 */ final Object[] project_keys_0 = new Object[project_numEntries_0];
/* 052 */ final Object[] project_values_0 = new Object[project_numEntries_0];
/* 053 */
/* 054 */ for (int project_idx_0 = 0; project_idx_0 < project_numEntries_0; project_idx_0++) {
/* 055 */ InternalRow project_entry_0 = inputadapter_value_0.getStruct(project_idx_0, 2);
/* 056 */
/* 057 */ if (project_entry_0.isNullAt(0)) {
/* 058 */ throw new RuntimeException("The first field from a struct (key) can't be null.");
/* 059 */ }
/* 060 */
/* 061 */ project_keys_0[project_idx_0] = project_entry_0.getUTF8String(0);
/* 062 */ project_values_0[project_idx_0] = project_entry_0.getUTF8String(1);
/* 063 */ }
/* 064 */
/* 065 */ project_value_0 = org.apache.spark.sql.catalyst.util.ArrayBasedMapData.apply(project_keys_0, project_values_0);
/* 066 */
/* 067 */ }
```
Author: Marek Novotny <mn.mikke@gmail.com>
Closes #21282 from mn-mikke/feature/array-api-map_from_entries-to-master.
2018-06-22 03:18:22 -04:00
|
|
|
def map_from_entries(col):
|
|
|
|
"""
|
|
|
|
Collection function: Returns a map created from the given array of entries.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.4.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column or expression
|
[SPARK-23934][SQL] Adding map_from_entries function
## What changes were proposed in this pull request?
The PR adds the `map_from_entries` function that returns a map created from the given array of entries.
## How was this patch tested?
New tests added into:
- `CollectionExpressionSuite`
- `DataFrameFunctionSuite`
## CodeGen Examples
### Primitive-type Keys and Values
```
val idf = Seq(
Seq((1, 10), (2, 20), (3, 10)),
Seq((1, 10), null, (2, 20))
).toDF("a")
idf.filter('a.isNotNull).select(map_from_entries('a)).debugCodegen
```
Result:
```
/* 042 */ boolean project_isNull_0 = false;
/* 043 */ MapData project_value_0 = null;
/* 044 */
/* 045 */ for (int project_idx_2 = 0; !project_isNull_0 && project_idx_2 < inputadapter_value_0.numElements(); project_idx_2++) {
/* 046 */ project_isNull_0 |= inputadapter_value_0.isNullAt(project_idx_2);
/* 047 */ }
/* 048 */ if (!project_isNull_0) {
/* 049 */ final int project_numEntries_0 = inputadapter_value_0.numElements();
/* 050 */
/* 051 */ final long project_keySectionSize_0 = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(project_numEntries_0, 4);
/* 052 */ final long project_valueSectionSize_0 = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(project_numEntries_0, 4);
/* 053 */ final long project_byteArraySize_0 = 8 + project_keySectionSize_0 + project_valueSectionSize_0;
/* 054 */ if (project_byteArraySize_0 > 2147483632) {
/* 055 */ final Object[] project_keys_0 = new Object[project_numEntries_0];
/* 056 */ final Object[] project_values_0 = new Object[project_numEntries_0];
/* 057 */
/* 058 */ for (int project_idx_1 = 0; project_idx_1 < project_numEntries_0; project_idx_1++) {
/* 059 */ InternalRow project_entry_1 = inputadapter_value_0.getStruct(project_idx_1, 2);
/* 060 */
/* 061 */ project_keys_0[project_idx_1] = project_entry_1.getInt(0);
/* 062 */ project_values_0[project_idx_1] = project_entry_1.getInt(1);
/* 063 */ }
/* 064 */
/* 065 */ project_value_0 = org.apache.spark.sql.catalyst.util.ArrayBasedMapData.apply(project_keys_0, project_values_0);
/* 066 */
/* 067 */ } else {
/* 068 */ final byte[] project_byteArray_0 = new byte[(int)project_byteArraySize_0];
/* 069 */ UnsafeMapData project_unsafeMapData_0 = new UnsafeMapData();
/* 070 */ Platform.putLong(project_byteArray_0, 16, project_keySectionSize_0);
/* 071 */ Platform.putLong(project_byteArray_0, 24, project_numEntries_0);
/* 072 */ Platform.putLong(project_byteArray_0, 24 + project_keySectionSize_0, project_numEntries_0);
/* 073 */ project_unsafeMapData_0.pointTo(project_byteArray_0, 16, (int)project_byteArraySize_0);
/* 074 */ ArrayData project_keyArrayData_0 = project_unsafeMapData_0.keyArray();
/* 075 */ ArrayData project_valueArrayData_0 = project_unsafeMapData_0.valueArray();
/* 076 */
/* 077 */ for (int project_idx_0 = 0; project_idx_0 < project_numEntries_0; project_idx_0++) {
/* 078 */ InternalRow project_entry_0 = inputadapter_value_0.getStruct(project_idx_0, 2);
/* 079 */
/* 080 */ project_keyArrayData_0.setInt(project_idx_0, project_entry_0.getInt(0));
/* 081 */ project_valueArrayData_0.setInt(project_idx_0, project_entry_0.getInt(1));
/* 082 */ }
/* 083 */
/* 084 */ project_value_0 = project_unsafeMapData_0;
/* 085 */ }
/* 086 */
/* 087 */ }
```
### Non-primitive-type Keys and Values
```
val sdf = Seq(
Seq(("a", null), ("b", "bb"), ("c", "aa")),
Seq(("a", "aa"), null, (null, "bb"))
).toDF("a")
sdf.filter('a.isNotNull).select(map_from_entries('a)).debugCodegen
```
Result:
```
/* 042 */ boolean project_isNull_0 = false;
/* 043 */ MapData project_value_0 = null;
/* 044 */
/* 045 */ for (int project_idx_1 = 0; !project_isNull_0 && project_idx_1 < inputadapter_value_0.numElements(); project_idx_1++) {
/* 046 */ project_isNull_0 |= inputadapter_value_0.isNullAt(project_idx_1);
/* 047 */ }
/* 048 */ if (!project_isNull_0) {
/* 049 */ final int project_numEntries_0 = inputadapter_value_0.numElements();
/* 050 */
/* 051 */ final Object[] project_keys_0 = new Object[project_numEntries_0];
/* 052 */ final Object[] project_values_0 = new Object[project_numEntries_0];
/* 053 */
/* 054 */ for (int project_idx_0 = 0; project_idx_0 < project_numEntries_0; project_idx_0++) {
/* 055 */ InternalRow project_entry_0 = inputadapter_value_0.getStruct(project_idx_0, 2);
/* 056 */
/* 057 */ if (project_entry_0.isNullAt(0)) {
/* 058 */ throw new RuntimeException("The first field from a struct (key) can't be null.");
/* 059 */ }
/* 060 */
/* 061 */ project_keys_0[project_idx_0] = project_entry_0.getUTF8String(0);
/* 062 */ project_values_0[project_idx_0] = project_entry_0.getUTF8String(1);
/* 063 */ }
/* 064 */
/* 065 */ project_value_0 = org.apache.spark.sql.catalyst.util.ArrayBasedMapData.apply(project_keys_0, project_values_0);
/* 066 */
/* 067 */ }
```
Author: Marek Novotny <mn.mikke@gmail.com>
Closes #21282 from mn-mikke/feature/array-api-map_from_entries-to-master.
2018-06-22 03:18:22 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
[SPARK-23934][SQL] Adding map_from_entries function
## What changes were proposed in this pull request?
The PR adds the `map_from_entries` function that returns a map created from the given array of entries.
## How was this patch tested?
New tests added into:
- `CollectionExpressionSuite`
- `DataFrameFunctionSuite`
## CodeGen Examples
### Primitive-type Keys and Values
```
val idf = Seq(
Seq((1, 10), (2, 20), (3, 10)),
Seq((1, 10), null, (2, 20))
).toDF("a")
idf.filter('a.isNotNull).select(map_from_entries('a)).debugCodegen
```
Result:
```
/* 042 */ boolean project_isNull_0 = false;
/* 043 */ MapData project_value_0 = null;
/* 044 */
/* 045 */ for (int project_idx_2 = 0; !project_isNull_0 && project_idx_2 < inputadapter_value_0.numElements(); project_idx_2++) {
/* 046 */ project_isNull_0 |= inputadapter_value_0.isNullAt(project_idx_2);
/* 047 */ }
/* 048 */ if (!project_isNull_0) {
/* 049 */ final int project_numEntries_0 = inputadapter_value_0.numElements();
/* 050 */
/* 051 */ final long project_keySectionSize_0 = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(project_numEntries_0, 4);
/* 052 */ final long project_valueSectionSize_0 = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(project_numEntries_0, 4);
/* 053 */ final long project_byteArraySize_0 = 8 + project_keySectionSize_0 + project_valueSectionSize_0;
/* 054 */ if (project_byteArraySize_0 > 2147483632) {
/* 055 */ final Object[] project_keys_0 = new Object[project_numEntries_0];
/* 056 */ final Object[] project_values_0 = new Object[project_numEntries_0];
/* 057 */
/* 058 */ for (int project_idx_1 = 0; project_idx_1 < project_numEntries_0; project_idx_1++) {
/* 059 */ InternalRow project_entry_1 = inputadapter_value_0.getStruct(project_idx_1, 2);
/* 060 */
/* 061 */ project_keys_0[project_idx_1] = project_entry_1.getInt(0);
/* 062 */ project_values_0[project_idx_1] = project_entry_1.getInt(1);
/* 063 */ }
/* 064 */
/* 065 */ project_value_0 = org.apache.spark.sql.catalyst.util.ArrayBasedMapData.apply(project_keys_0, project_values_0);
/* 066 */
/* 067 */ } else {
/* 068 */ final byte[] project_byteArray_0 = new byte[(int)project_byteArraySize_0];
/* 069 */ UnsafeMapData project_unsafeMapData_0 = new UnsafeMapData();
/* 070 */ Platform.putLong(project_byteArray_0, 16, project_keySectionSize_0);
/* 071 */ Platform.putLong(project_byteArray_0, 24, project_numEntries_0);
/* 072 */ Platform.putLong(project_byteArray_0, 24 + project_keySectionSize_0, project_numEntries_0);
/* 073 */ project_unsafeMapData_0.pointTo(project_byteArray_0, 16, (int)project_byteArraySize_0);
/* 074 */ ArrayData project_keyArrayData_0 = project_unsafeMapData_0.keyArray();
/* 075 */ ArrayData project_valueArrayData_0 = project_unsafeMapData_0.valueArray();
/* 076 */
/* 077 */ for (int project_idx_0 = 0; project_idx_0 < project_numEntries_0; project_idx_0++) {
/* 078 */ InternalRow project_entry_0 = inputadapter_value_0.getStruct(project_idx_0, 2);
/* 079 */
/* 080 */ project_keyArrayData_0.setInt(project_idx_0, project_entry_0.getInt(0));
/* 081 */ project_valueArrayData_0.setInt(project_idx_0, project_entry_0.getInt(1));
/* 082 */ }
/* 083 */
/* 084 */ project_value_0 = project_unsafeMapData_0;
/* 085 */ }
/* 086 */
/* 087 */ }
```
### Non-primitive-type Keys and Values
```
val sdf = Seq(
Seq(("a", null), ("b", "bb"), ("c", "aa")),
Seq(("a", "aa"), null, (null, "bb"))
).toDF("a")
sdf.filter('a.isNotNull).select(map_from_entries('a)).debugCodegen
```
Result:
```
/* 042 */ boolean project_isNull_0 = false;
/* 043 */ MapData project_value_0 = null;
/* 044 */
/* 045 */ for (int project_idx_1 = 0; !project_isNull_0 && project_idx_1 < inputadapter_value_0.numElements(); project_idx_1++) {
/* 046 */ project_isNull_0 |= inputadapter_value_0.isNullAt(project_idx_1);
/* 047 */ }
/* 048 */ if (!project_isNull_0) {
/* 049 */ final int project_numEntries_0 = inputadapter_value_0.numElements();
/* 050 */
/* 051 */ final Object[] project_keys_0 = new Object[project_numEntries_0];
/* 052 */ final Object[] project_values_0 = new Object[project_numEntries_0];
/* 053 */
/* 054 */ for (int project_idx_0 = 0; project_idx_0 < project_numEntries_0; project_idx_0++) {
/* 055 */ InternalRow project_entry_0 = inputadapter_value_0.getStruct(project_idx_0, 2);
/* 056 */
/* 057 */ if (project_entry_0.isNullAt(0)) {
/* 058 */ throw new RuntimeException("The first field from a struct (key) can't be null.");
/* 059 */ }
/* 060 */
/* 061 */ project_keys_0[project_idx_0] = project_entry_0.getUTF8String(0);
/* 062 */ project_values_0[project_idx_0] = project_entry_0.getUTF8String(1);
/* 063 */ }
/* 064 */
/* 065 */ project_value_0 = org.apache.spark.sql.catalyst.util.ArrayBasedMapData.apply(project_keys_0, project_values_0);
/* 066 */
/* 067 */ }
```
Author: Marek Novotny <mn.mikke@gmail.com>
Closes #21282 from mn-mikke/feature/array-api-map_from_entries-to-master.
2018-06-22 03:18:22 -04:00
|
|
|
>>> from pyspark.sql.functions import map_from_entries
|
|
|
|
>>> df = spark.sql("SELECT array(struct(1, 'a'), struct(2, 'b')) as data")
|
|
|
|
>>> df.select(map_from_entries("data").alias("map")).show()
|
|
|
|
+----------------+
|
|
|
|
| map|
|
|
|
|
+----------------+
|
2020-08-04 10:57:09 -04:00
|
|
|
|{1 -> a, 2 -> b}|
|
[SPARK-23934][SQL] Adding map_from_entries function
## What changes were proposed in this pull request?
The PR adds the `map_from_entries` function that returns a map created from the given array of entries.
## How was this patch tested?
New tests added into:
- `CollectionExpressionSuite`
- `DataFrameFunctionSuite`
## CodeGen Examples
### Primitive-type Keys and Values
```
val idf = Seq(
Seq((1, 10), (2, 20), (3, 10)),
Seq((1, 10), null, (2, 20))
).toDF("a")
idf.filter('a.isNotNull).select(map_from_entries('a)).debugCodegen
```
Result:
```
/* 042 */ boolean project_isNull_0 = false;
/* 043 */ MapData project_value_0 = null;
/* 044 */
/* 045 */ for (int project_idx_2 = 0; !project_isNull_0 && project_idx_2 < inputadapter_value_0.numElements(); project_idx_2++) {
/* 046 */ project_isNull_0 |= inputadapter_value_0.isNullAt(project_idx_2);
/* 047 */ }
/* 048 */ if (!project_isNull_0) {
/* 049 */ final int project_numEntries_0 = inputadapter_value_0.numElements();
/* 050 */
/* 051 */ final long project_keySectionSize_0 = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(project_numEntries_0, 4);
/* 052 */ final long project_valueSectionSize_0 = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(project_numEntries_0, 4);
/* 053 */ final long project_byteArraySize_0 = 8 + project_keySectionSize_0 + project_valueSectionSize_0;
/* 054 */ if (project_byteArraySize_0 > 2147483632) {
/* 055 */ final Object[] project_keys_0 = new Object[project_numEntries_0];
/* 056 */ final Object[] project_values_0 = new Object[project_numEntries_0];
/* 057 */
/* 058 */ for (int project_idx_1 = 0; project_idx_1 < project_numEntries_0; project_idx_1++) {
/* 059 */ InternalRow project_entry_1 = inputadapter_value_0.getStruct(project_idx_1, 2);
/* 060 */
/* 061 */ project_keys_0[project_idx_1] = project_entry_1.getInt(0);
/* 062 */ project_values_0[project_idx_1] = project_entry_1.getInt(1);
/* 063 */ }
/* 064 */
/* 065 */ project_value_0 = org.apache.spark.sql.catalyst.util.ArrayBasedMapData.apply(project_keys_0, project_values_0);
/* 066 */
/* 067 */ } else {
/* 068 */ final byte[] project_byteArray_0 = new byte[(int)project_byteArraySize_0];
/* 069 */ UnsafeMapData project_unsafeMapData_0 = new UnsafeMapData();
/* 070 */ Platform.putLong(project_byteArray_0, 16, project_keySectionSize_0);
/* 071 */ Platform.putLong(project_byteArray_0, 24, project_numEntries_0);
/* 072 */ Platform.putLong(project_byteArray_0, 24 + project_keySectionSize_0, project_numEntries_0);
/* 073 */ project_unsafeMapData_0.pointTo(project_byteArray_0, 16, (int)project_byteArraySize_0);
/* 074 */ ArrayData project_keyArrayData_0 = project_unsafeMapData_0.keyArray();
/* 075 */ ArrayData project_valueArrayData_0 = project_unsafeMapData_0.valueArray();
/* 076 */
/* 077 */ for (int project_idx_0 = 0; project_idx_0 < project_numEntries_0; project_idx_0++) {
/* 078 */ InternalRow project_entry_0 = inputadapter_value_0.getStruct(project_idx_0, 2);
/* 079 */
/* 080 */ project_keyArrayData_0.setInt(project_idx_0, project_entry_0.getInt(0));
/* 081 */ project_valueArrayData_0.setInt(project_idx_0, project_entry_0.getInt(1));
/* 082 */ }
/* 083 */
/* 084 */ project_value_0 = project_unsafeMapData_0;
/* 085 */ }
/* 086 */
/* 087 */ }
```
### Non-primitive-type Keys and Values
```
val sdf = Seq(
Seq(("a", null), ("b", "bb"), ("c", "aa")),
Seq(("a", "aa"), null, (null, "bb"))
).toDF("a")
sdf.filter('a.isNotNull).select(map_from_entries('a)).debugCodegen
```
Result:
```
/* 042 */ boolean project_isNull_0 = false;
/* 043 */ MapData project_value_0 = null;
/* 044 */
/* 045 */ for (int project_idx_1 = 0; !project_isNull_0 && project_idx_1 < inputadapter_value_0.numElements(); project_idx_1++) {
/* 046 */ project_isNull_0 |= inputadapter_value_0.isNullAt(project_idx_1);
/* 047 */ }
/* 048 */ if (!project_isNull_0) {
/* 049 */ final int project_numEntries_0 = inputadapter_value_0.numElements();
/* 050 */
/* 051 */ final Object[] project_keys_0 = new Object[project_numEntries_0];
/* 052 */ final Object[] project_values_0 = new Object[project_numEntries_0];
/* 053 */
/* 054 */ for (int project_idx_0 = 0; project_idx_0 < project_numEntries_0; project_idx_0++) {
/* 055 */ InternalRow project_entry_0 = inputadapter_value_0.getStruct(project_idx_0, 2);
/* 056 */
/* 057 */ if (project_entry_0.isNullAt(0)) {
/* 058 */ throw new RuntimeException("The first field from a struct (key) can't be null.");
/* 059 */ }
/* 060 */
/* 061 */ project_keys_0[project_idx_0] = project_entry_0.getUTF8String(0);
/* 062 */ project_values_0[project_idx_0] = project_entry_0.getUTF8String(1);
/* 063 */ }
/* 064 */
/* 065 */ project_value_0 = org.apache.spark.sql.catalyst.util.ArrayBasedMapData.apply(project_keys_0, project_values_0);
/* 066 */
/* 067 */ }
```
Author: Marek Novotny <mn.mikke@gmail.com>
Closes #21282 from mn-mikke/feature/array-api-map_from_entries-to-master.
2018-06-22 03:18:22 -04:00
|
|
|
+----------------+
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.map_from_entries(_to_java_column(col)))
|
|
|
|
|
|
|
|
|
2018-05-17 00:31:14 -04:00
|
|
|
def array_repeat(col, count):
|
|
|
|
"""
|
|
|
|
Collection function: creates an array containing a column repeated count times.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.4.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2018-05-17 00:31:14 -04:00
|
|
|
>>> df = spark.createDataFrame([('ab',)], ['data'])
|
|
|
|
>>> df.select(array_repeat(df.data, 3).alias('r')).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(r=['ab', 'ab', 'ab'])]
|
2018-05-17 00:31:14 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
2019-07-18 15:58:48 -04:00
|
|
|
return Column(sc._jvm.functions.array_repeat(
|
|
|
|
_to_java_column(col),
|
|
|
|
_to_java_column(count) if isinstance(count, Column) else count
|
|
|
|
))
|
2018-05-17 00:31:14 -04:00
|
|
|
|
|
|
|
|
2018-06-12 14:57:25 -04:00
|
|
|
def arrays_zip(*cols):
|
|
|
|
"""
|
|
|
|
Collection function: Returns a merged array of structs in which the N-th struct contains all
|
|
|
|
N-th values of input arrays.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.4.0
|
2018-06-12 14:57:25 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
cols : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
columns of arrays to be merged.
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2018-06-12 14:57:25 -04:00
|
|
|
>>> from pyspark.sql.functions import arrays_zip
|
|
|
|
>>> df = spark.createDataFrame([(([1, 2, 3], [2, 3, 4]))], ['vals1', 'vals2'])
|
|
|
|
>>> df.select(arrays_zip(df.vals1, df.vals2).alias('zipped')).collect()
|
|
|
|
[Row(zipped=[Row(vals1=1, vals2=2), Row(vals1=2, vals2=3), Row(vals1=3, vals2=4)])]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.arrays_zip(_to_seq(sc, cols, _to_java_column)))
|
|
|
|
|
|
|
|
|
2018-07-09 08:21:38 -04:00
|
|
|
def map_concat(*cols):
|
|
|
|
"""Returns the union of all the given maps.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.4.0
|
2018-07-09 08:21:38 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
cols : :class:`~pyspark.sql.Column` or str
|
|
|
|
column names or :class:`~pyspark.sql.Column`\\s
|
2020-11-02 20:00:49 -05:00
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2018-07-09 08:21:38 -04:00
|
|
|
>>> from pyspark.sql.functions import map_concat
|
2020-02-17 09:06:58 -05:00
|
|
|
>>> df = spark.sql("SELECT map(1, 'a', 2, 'b') as map1, map(3, 'c') as map2")
|
2018-07-09 08:21:38 -04:00
|
|
|
>>> df.select(map_concat("map1", "map2").alias("map3")).show(truncate=False)
|
[SPARK-25829][SQL] remove duplicated map keys with last wins policy
## What changes were proposed in this pull request?
Currently duplicated map keys are not handled consistently. For example, map look up respects the duplicated key appears first, `Dataset.collect` only keeps the duplicated key appears last, `MapKeys` returns duplicated keys, etc.
This PR proposes to remove duplicated map keys with last wins policy, to follow Java/Scala and Presto. It only applies to built-in functions, as users can create map with duplicated map keys via private APIs anyway.
updated functions: `CreateMap`, `MapFromArrays`, `MapFromEntries`, `StringToMap`, `MapConcat`, `TransformKeys`.
For other places:
1. data source v1 doesn't have this problem, as users need to provide a java/scala map, which can't have duplicated keys.
2. data source v2 may have this problem. I've added a note to `ArrayBasedMapData` to ask the caller to take care of duplicated keys. In the future we should enforce it in the stable data APIs for data source v2.
3. UDF doesn't have this problem, as users need to provide a java/scala map. Same as data source v1.
4. file format. I checked all of them and only parquet does not enforce it. For backward compatibility reasons I change nothing but leave a note saying that the behavior will be undefined if users write map with duplicated keys to parquet files. Maybe we can add a config and fail by default if parquet files have map with duplicated keys. This can be done in followup.
## How was this patch tested?
updated tests and new tests
Closes #23124 from cloud-fan/map.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-28 10:42:13 -05:00
|
|
|
+------------------------+
|
|
|
|
|map3 |
|
|
|
|
+------------------------+
|
2020-08-04 10:57:09 -04:00
|
|
|
|{1 -> a, 2 -> b, 3 -> c}|
|
[SPARK-25829][SQL] remove duplicated map keys with last wins policy
## What changes were proposed in this pull request?
Currently duplicated map keys are not handled consistently. For example, map look up respects the duplicated key appears first, `Dataset.collect` only keeps the duplicated key appears last, `MapKeys` returns duplicated keys, etc.
This PR proposes to remove duplicated map keys with last wins policy, to follow Java/Scala and Presto. It only applies to built-in functions, as users can create map with duplicated map keys via private APIs anyway.
updated functions: `CreateMap`, `MapFromArrays`, `MapFromEntries`, `StringToMap`, `MapConcat`, `TransformKeys`.
For other places:
1. data source v1 doesn't have this problem, as users need to provide a java/scala map, which can't have duplicated keys.
2. data source v2 may have this problem. I've added a note to `ArrayBasedMapData` to ask the caller to take care of duplicated keys. In the future we should enforce it in the stable data APIs for data source v2.
3. UDF doesn't have this problem, as users need to provide a java/scala map. Same as data source v1.
4. file format. I checked all of them and only parquet does not enforce it. For backward compatibility reasons I change nothing but leave a note saying that the behavior will be undefined if users write map with duplicated keys to parquet files. Maybe we can add a config and fail by default if parquet files have map with duplicated keys. This can be done in followup.
## How was this patch tested?
updated tests and new tests
Closes #23124 from cloud-fan/map.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-28 10:42:13 -05:00
|
|
|
+------------------------+
|
2018-07-09 08:21:38 -04:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
if len(cols) == 1 and isinstance(cols[0], (list, set)):
|
|
|
|
cols = cols[0]
|
|
|
|
jc = sc._jvm.functions.map_concat(_to_seq(sc, cols, _to_java_column))
|
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
2018-07-20 05:53:14 -04:00
|
|
|
def sequence(start, stop, step=None):
|
|
|
|
"""
|
|
|
|
Generate a sequence of integers from `start` to `stop`, incrementing by `step`.
|
|
|
|
If `step` is not set, incrementing by 1 if `start` is less than or equal to `stop`,
|
|
|
|
otherwise -1.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.4.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2018-07-20 05:53:14 -04:00
|
|
|
>>> df1 = spark.createDataFrame([(-2, 2)], ('C1', 'C2'))
|
|
|
|
>>> df1.select(sequence('C1', 'C2').alias('r')).collect()
|
|
|
|
[Row(r=[-2, -1, 0, 1, 2])]
|
|
|
|
>>> df2 = spark.createDataFrame([(4, -4, -2)], ('C1', 'C2', 'C3'))
|
|
|
|
>>> df2.select(sequence('C1', 'C2', 'C3').alias('r')).collect()
|
|
|
|
[Row(r=[4, 2, 0, -2, -4])]
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
if step is None:
|
|
|
|
return Column(sc._jvm.functions.sequence(_to_java_column(start), _to_java_column(stop)))
|
|
|
|
else:
|
|
|
|
return Column(sc._jvm.functions.sequence(
|
|
|
|
_to_java_column(start), _to_java_column(stop), _to_java_column(step)))
|
|
|
|
|
|
|
|
|
[SPARK-32320][PYSPARK] Remove mutable default arguments
This is bad practice, and might lead to unexpected behaviour:
https://florimond.dev/blog/articles/2018/08/python-mutable-defaults-are-the-source-of-all-evil/
```
fokkodriesprongFan spark % grep -R "={}" python | grep def
python/pyspark/resource/profile.py: def __init__(self, _java_resource_profile=None, _exec_req={}, _task_req={}):
python/pyspark/sql/functions.py:def from_json(col, schema, options={}):
python/pyspark/sql/functions.py:def to_json(col, options={}):
python/pyspark/sql/functions.py:def schema_of_json(json, options={}):
python/pyspark/sql/functions.py:def schema_of_csv(csv, options={}):
python/pyspark/sql/functions.py:def to_csv(col, options={}):
python/pyspark/sql/functions.py:def from_csv(col, schema, options={}):
python/pyspark/sql/avro/functions.py:def from_avro(data, jsonFormatSchema, options={}):
```
```
fokkodriesprongFan spark % grep -R "=\[\]" python | grep def
python/pyspark/ml/tuning.py: def __init__(self, bestModel, avgMetrics=[], subModels=None):
python/pyspark/ml/tuning.py: def __init__(self, bestModel, validationMetrics=[], subModels=None):
```
### What changes were proposed in this pull request?
Removing the mutable default arguments.
### Why are the changes needed?
Removing the mutable default arguments, and changing the signature to `Optional[...]`.
### Does this PR introduce _any_ user-facing change?
No 👍
### How was this patch tested?
Using the Flake8 bugbear code analysis plugin.
Closes #29122 from Fokko/SPARK-32320.
Authored-by: Fokko Driesprong <fokko@apache.org>
Signed-off-by: Ruifeng Zheng <ruifengz@foxmail.com>
2020-12-07 20:35:36 -05:00
|
|
|
def from_csv(col, schema, options=None):
|
2018-10-16 21:32:05 -04:00
|
|
|
"""
|
|
|
|
Parses a column containing a CSV string to a row with the specified schema.
|
|
|
|
Returns `null`, in the case of an unparseable string.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 3.0.0
|
2018-10-16 21:32:05 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
string column in CSV format
|
2021-02-01 19:30:50 -05:00
|
|
|
schema :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
a string with schema in DDL format to use when parsing the CSV column.
|
|
|
|
options : dict, optional
|
|
|
|
options to control parsing. accepts the same options as the CSV datasource
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2018-10-31 21:14:16 -04:00
|
|
|
>>> data = [("1,2,3",)]
|
|
|
|
>>> df = spark.createDataFrame(data, ("value",))
|
|
|
|
>>> df.select(from_csv(df.value, "a INT, b INT, c INT").alias("csv")).collect()
|
|
|
|
[Row(csv=Row(a=1, b=2, c=3))]
|
|
|
|
>>> value = data[0][0]
|
|
|
|
>>> df.select(from_csv(df.value, schema_of_csv(value)).alias("csv")).collect()
|
|
|
|
[Row(csv=Row(_c0=1, _c1=2, _c2=3))]
|
2019-07-18 00:37:03 -04:00
|
|
|
>>> data = [(" abc",)]
|
|
|
|
>>> df = spark.createDataFrame(data, ("value",))
|
|
|
|
>>> options = {'ignoreLeadingWhiteSpace': True}
|
|
|
|
>>> df.select(from_csv(df.value, "s string", options).alias("csv")).collect()
|
2020-07-13 22:22:44 -04:00
|
|
|
[Row(csv=Row(s='abc'))]
|
2018-10-16 21:32:05 -04:00
|
|
|
"""
|
|
|
|
|
|
|
|
sc = SparkContext._active_spark_context
|
2020-07-13 22:22:44 -04:00
|
|
|
if isinstance(schema, str):
|
2018-10-16 21:32:05 -04:00
|
|
|
schema = _create_column_from_literal(schema)
|
|
|
|
elif isinstance(schema, Column):
|
|
|
|
schema = _to_java_column(schema)
|
|
|
|
else:
|
|
|
|
raise TypeError("schema argument should be a column or string")
|
|
|
|
|
2019-07-18 00:37:03 -04:00
|
|
|
jc = sc._jvm.functions.from_csv(_to_java_column(col), schema, _options_to_str(options))
|
2018-10-16 21:32:05 -04:00
|
|
|
return Column(jc)
|
|
|
|
|
|
|
|
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
def _unresolved_named_lambda_variable(*name_parts):
|
|
|
|
"""
|
|
|
|
Create `o.a.s.sql.expressions.UnresolvedNamedLambdaVariable`,
|
|
|
|
convert it to o.s.sql.Column and wrap in Python `Column`
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
name_parts : str
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
name_parts_seq = _to_seq(sc, name_parts)
|
|
|
|
expressions = sc._jvm.org.apache.spark.sql.catalyst.expressions
|
|
|
|
return Column(
|
|
|
|
sc._jvm.Column(
|
|
|
|
expressions.UnresolvedNamedLambdaVariable(name_parts_seq)
|
|
|
|
)
|
|
|
|
)
|
|
|
|
|
|
|
|
|
|
|
|
def _get_lambda_parameters(f):
|
|
|
|
import inspect
|
|
|
|
|
|
|
|
signature = inspect.signature(f)
|
|
|
|
parameters = signature.parameters.values()
|
|
|
|
|
|
|
|
# We should exclude functions that use
|
|
|
|
# variable args and keyword argnames
|
|
|
|
# as well as keyword only args
|
2020-11-27 11:22:45 -05:00
|
|
|
supported_parameter_types = {
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
inspect.Parameter.POSITIONAL_OR_KEYWORD,
|
|
|
|
inspect.Parameter.POSITIONAL_ONLY,
|
|
|
|
}
|
|
|
|
|
|
|
|
# Validate that
|
|
|
|
# function arity is between 1 and 3
|
|
|
|
if not (1 <= len(parameters) <= 3):
|
|
|
|
raise ValueError(
|
|
|
|
"f should take between 1 and 3 arguments, but provided function takes {}".format(
|
|
|
|
len(parameters)
|
|
|
|
)
|
|
|
|
)
|
|
|
|
|
|
|
|
# and all arguments can be used as positional
|
2020-11-27 11:22:45 -05:00
|
|
|
if not all(p.kind in supported_parameter_types for p in parameters):
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
raise ValueError(
|
|
|
|
"f should use only POSITIONAL or POSITIONAL OR KEYWORD arguments"
|
|
|
|
)
|
|
|
|
|
|
|
|
return parameters
|
|
|
|
|
|
|
|
|
|
|
|
def _create_lambda(f):
|
|
|
|
"""
|
|
|
|
Create `o.a.s.sql.expressions.LambdaFunction` corresponding
|
|
|
|
to transformation described by f
|
|
|
|
|
|
|
|
:param f: A Python of one of the following forms:
|
|
|
|
- (Column) -> Column: ...
|
|
|
|
- (Column, Column) -> Column: ...
|
|
|
|
- (Column, Column, Column) -> Column: ...
|
|
|
|
"""
|
2020-07-13 22:22:44 -04:00
|
|
|
parameters = _get_lambda_parameters(f)
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
expressions = sc._jvm.org.apache.spark.sql.catalyst.expressions
|
|
|
|
|
|
|
|
argnames = ["x", "y", "z"]
|
|
|
|
args = [
|
[SPARK-35382][PYTHON] Fix lambda variable name issues in nested DataFrame functions in Python APIs
### What changes were proposed in this pull request?
This PR fixes the same issue as #32424.
```py
from pyspark.sql.functions import flatten, struct, transform
df = spark.sql("SELECT array(1, 2, 3) as numbers, array('a', 'b', 'c') as letters")
df.select(flatten(
transform(
"numbers",
lambda number: transform(
"letters",
lambda letter: struct(number.alias("n"), letter.alias("l"))
)
)
).alias("zipped")).show(truncate=False)
```
**Before:**
```
+------------------------------------------------------------------------+
|zipped |
+------------------------------------------------------------------------+
|[{a, a}, {b, b}, {c, c}, {a, a}, {b, b}, {c, c}, {a, a}, {b, b}, {c, c}]|
+------------------------------------------------------------------------+
```
**After:**
```
+------------------------------------------------------------------------+
|zipped |
+------------------------------------------------------------------------+
|[{1, a}, {1, b}, {1, c}, {2, a}, {2, b}, {2, c}, {3, a}, {3, b}, {3, c}]|
+------------------------------------------------------------------------+
```
### Why are the changes needed?
To produce the correct results.
### Does this PR introduce _any_ user-facing change?
Yes, it fixes the results to be correct as mentioned above.
### How was this patch tested?
Added a unit test as well as manually.
Closes #32523 from ueshin/issues/SPARK-35382/nested_higher_order_functions.
Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-13 01:58:01 -04:00
|
|
|
_unresolved_named_lambda_variable(
|
|
|
|
expressions.UnresolvedNamedLambdaVariable.freshVarName(arg)
|
|
|
|
)
|
|
|
|
for arg in argnames[: len(parameters)]
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
]
|
|
|
|
|
|
|
|
result = f(*args)
|
|
|
|
|
|
|
|
if not isinstance(result, Column):
|
|
|
|
raise ValueError("f should return Column, got {}".format(type(result)))
|
|
|
|
|
|
|
|
jexpr = result._jc.expr()
|
|
|
|
jargs = _to_seq(sc, [arg._jc.expr() for arg in args])
|
|
|
|
|
|
|
|
return expressions.LambdaFunction(jexpr, jargs, False)
|
|
|
|
|
|
|
|
|
|
|
|
def _invoke_higher_order_function(name, cols, funs):
|
|
|
|
"""
|
|
|
|
Invokes expression identified by name,
|
|
|
|
(relative to ```org.apache.spark.sql.catalyst.expressions``)
|
|
|
|
and wraps the result with Column (first Scala one, then Python).
|
|
|
|
|
|
|
|
:param name: Name of the expression
|
|
|
|
:param cols: a list of columns
|
|
|
|
:param funs: a list of((*Column) -> Column functions.
|
|
|
|
|
|
|
|
:return: a Column
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
expressions = sc._jvm.org.apache.spark.sql.catalyst.expressions
|
|
|
|
expr = getattr(expressions, name)
|
|
|
|
|
|
|
|
jcols = [_to_java_column(col).expr() for col in cols]
|
|
|
|
jfuns = [_create_lambda(f) for f in funs]
|
|
|
|
|
|
|
|
return Column(sc._jvm.Column(expr(*jcols + jfuns)))
|
|
|
|
|
|
|
|
|
|
|
|
def transform(col, f):
|
|
|
|
"""
|
|
|
|
Returns an array of elements after applying a transformation to each element in the input array.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 3.1.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column or expression
|
|
|
|
f : function
|
|
|
|
a function that is applied to each element of the input array.
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
Can take one of the following forms:
|
|
|
|
|
|
|
|
- Unary ``(x: Column) -> Column: ...``
|
|
|
|
- Binary ``(x: Column, i: Column) -> Column...``, where the second argument is
|
|
|
|
a 0-based index of the element.
|
|
|
|
|
2021-02-01 19:30:50 -05:00
|
|
|
and can use methods of :class:`~pyspark.sql.Column`, functions defined in
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
:py:mod:`pyspark.sql.functions` and Scala ``UserDefinedFunctions``.
|
|
|
|
Python ``UserDefinedFunctions`` are not supported
|
|
|
|
(`SPARK-27052 <https://issues.apache.org/jira/browse/SPARK-27052>`__).
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Returns
|
|
|
|
-------
|
2021-02-01 19:30:50 -05:00
|
|
|
:class:`~pyspark.sql.Column`
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
>>> df = spark.createDataFrame([(1, [1, 2, 3, 4])], ("key", "values"))
|
|
|
|
>>> df.select(transform("values", lambda x: x * 2).alias("doubled")).show()
|
|
|
|
+------------+
|
|
|
|
| doubled|
|
|
|
|
+------------+
|
|
|
|
|[2, 4, 6, 8]|
|
|
|
|
+------------+
|
|
|
|
|
|
|
|
>>> def alternate(x, i):
|
|
|
|
... return when(i % 2 == 0, x).otherwise(-x)
|
|
|
|
>>> df.select(transform("values", alternate).alias("alternated")).show()
|
|
|
|
+--------------+
|
|
|
|
| alternated|
|
|
|
|
+--------------+
|
|
|
|
|[1, -2, 3, -4]|
|
|
|
|
+--------------+
|
|
|
|
"""
|
|
|
|
return _invoke_higher_order_function("ArrayTransform", [col], [f])
|
|
|
|
|
|
|
|
|
|
|
|
def exists(col, f):
|
|
|
|
"""
|
|
|
|
Returns whether a predicate holds for one or more elements in the array.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 3.1.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column or expression
|
|
|
|
f : function
|
|
|
|
``(x: Column) -> Column: ...`` returning the Boolean expression.
|
2021-02-01 19:30:50 -05:00
|
|
|
Can use methods of :class:`~pyspark.sql.Column`, functions defined in
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
:py:mod:`pyspark.sql.functions` and Scala ``UserDefinedFunctions``.
|
|
|
|
Python ``UserDefinedFunctions`` are not supported
|
|
|
|
(`SPARK-27052 <https://issues.apache.org/jira/browse/SPARK-27052>`__).
|
2021-02-01 19:30:50 -05:00
|
|
|
:return: a :class:`~pyspark.sql.Column`
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
>>> df = spark.createDataFrame([(1, [1, 2, 3, 4]), (2, [3, -1, 0])],("key", "values"))
|
|
|
|
>>> df.select(exists("values", lambda x: x < 0).alias("any_negative")).show()
|
|
|
|
+------------+
|
|
|
|
|any_negative|
|
|
|
|
+------------+
|
|
|
|
| false|
|
|
|
|
| true|
|
|
|
|
+------------+
|
|
|
|
"""
|
|
|
|
return _invoke_higher_order_function("ArrayExists", [col], [f])
|
|
|
|
|
|
|
|
|
|
|
|
def forall(col, f):
|
|
|
|
"""
|
|
|
|
Returns whether a predicate holds for every element in the array.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 3.1.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column or expression
|
|
|
|
f : function
|
|
|
|
``(x: Column) -> Column: ...`` returning the Boolean expression.
|
2021-02-01 19:30:50 -05:00
|
|
|
Can use methods of :class:`~pyspark.sql.Column`, functions defined in
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
:py:mod:`pyspark.sql.functions` and Scala ``UserDefinedFunctions``.
|
|
|
|
Python ``UserDefinedFunctions`` are not supported
|
|
|
|
(`SPARK-27052 <https://issues.apache.org/jira/browse/SPARK-27052>`__).
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Returns
|
|
|
|
-------
|
2021-02-01 19:30:50 -05:00
|
|
|
:class:`~pyspark.sql.Column`
|
2020-11-02 20:00:49 -05:00
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
>>> df = spark.createDataFrame(
|
|
|
|
... [(1, ["bar"]), (2, ["foo", "bar"]), (3, ["foobar", "foo"])],
|
|
|
|
... ("key", "values")
|
|
|
|
... )
|
|
|
|
>>> df.select(forall("values", lambda x: x.rlike("foo")).alias("all_foo")).show()
|
|
|
|
+-------+
|
|
|
|
|all_foo|
|
|
|
|
+-------+
|
|
|
|
| false|
|
|
|
|
| false|
|
|
|
|
| true|
|
|
|
|
+-------+
|
|
|
|
"""
|
|
|
|
return _invoke_higher_order_function("ArrayForAll", [col], [f])
|
|
|
|
|
|
|
|
|
|
|
|
def filter(col, f):
|
|
|
|
"""
|
|
|
|
Returns an array of elements for which a predicate holds in a given array.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 3.1.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column or expression
|
|
|
|
f : function
|
|
|
|
A function that returns the Boolean expression.
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
Can take one of the following forms:
|
|
|
|
|
|
|
|
- Unary ``(x: Column) -> Column: ...``
|
|
|
|
- Binary ``(x: Column, i: Column) -> Column...``, where the second argument is
|
|
|
|
a 0-based index of the element.
|
|
|
|
|
2021-02-01 19:30:50 -05:00
|
|
|
and can use methods of :class:`~pyspark.sql.Column`, functions defined in
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
:py:mod:`pyspark.sql.functions` and Scala ``UserDefinedFunctions``.
|
|
|
|
Python ``UserDefinedFunctions`` are not supported
|
|
|
|
(`SPARK-27052 <https://issues.apache.org/jira/browse/SPARK-27052>`__).
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Returns
|
|
|
|
-------
|
2021-02-01 19:30:50 -05:00
|
|
|
:class:`~pyspark.sql.Column`
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
>>> df = spark.createDataFrame(
|
|
|
|
... [(1, ["2018-09-20", "2019-02-03", "2019-07-01", "2020-06-01"])],
|
|
|
|
... ("key", "values")
|
|
|
|
... )
|
|
|
|
>>> def after_second_quarter(x):
|
|
|
|
... return month(to_date(x)) > 6
|
|
|
|
>>> df.select(
|
|
|
|
... filter("values", after_second_quarter).alias("after_second_quarter")
|
|
|
|
... ).show(truncate=False)
|
|
|
|
+------------------------+
|
|
|
|
|after_second_quarter |
|
|
|
|
+------------------------+
|
|
|
|
|[2018-09-20, 2019-07-01]|
|
|
|
|
+------------------------+
|
|
|
|
"""
|
|
|
|
return _invoke_higher_order_function("ArrayFilter", [col], [f])
|
|
|
|
|
|
|
|
|
2021-01-06 04:46:20 -05:00
|
|
|
def aggregate(col, initialValue, merge, finish=None):
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
"""
|
|
|
|
Applies a binary operator to an initial state and all elements in the array,
|
|
|
|
and reduces this to a single state. The final state is converted into the final result
|
|
|
|
by applying a finish function.
|
|
|
|
|
2021-02-01 19:30:50 -05:00
|
|
|
Both functions can use methods of :class:`~pyspark.sql.Column`, functions defined in
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
:py:mod:`pyspark.sql.functions` and Scala ``UserDefinedFunctions``.
|
|
|
|
Python ``UserDefinedFunctions`` are not supported
|
|
|
|
(`SPARK-27052 <https://issues.apache.org/jira/browse/SPARK-27052>`__).
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 3.1.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column or expression
|
2021-02-01 19:30:50 -05:00
|
|
|
initialValue : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
initial value. Name of column or expression
|
|
|
|
merge : function
|
|
|
|
a binary function ``(acc: Column, x: Column) -> Column...`` returning expression
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
of the same type as ``zero``
|
2020-11-02 20:00:49 -05:00
|
|
|
finish : function
|
|
|
|
an optional unary function ``(x: Column) -> Column: ...``
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
used to convert accumulated value.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Returns
|
|
|
|
-------
|
2021-02-01 19:30:50 -05:00
|
|
|
:class:`~pyspark.sql.Column`
|
2020-11-02 20:00:49 -05:00
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
>>> df = spark.createDataFrame([(1, [20.0, 4.0, 2.0, 6.0, 10.0])], ("id", "values"))
|
|
|
|
>>> df.select(aggregate("values", lit(0.0), lambda acc, x: acc + x).alias("sum")).show()
|
|
|
|
+----+
|
|
|
|
| sum|
|
|
|
|
+----+
|
|
|
|
|42.0|
|
|
|
|
+----+
|
|
|
|
|
|
|
|
>>> def merge(acc, x):
|
|
|
|
... count = acc.count + 1
|
|
|
|
... sum = acc.sum + x
|
|
|
|
... return struct(count.alias("count"), sum.alias("sum"))
|
|
|
|
>>> df.select(
|
|
|
|
... aggregate(
|
|
|
|
... "values",
|
|
|
|
... struct(lit(0).alias("count"), lit(0.0).alias("sum")),
|
|
|
|
... merge,
|
|
|
|
... lambda acc: acc.sum / acc.count,
|
|
|
|
... ).alias("mean")
|
|
|
|
... ).show()
|
|
|
|
+----+
|
|
|
|
|mean|
|
|
|
|
+----+
|
|
|
|
| 8.4|
|
|
|
|
+----+
|
|
|
|
"""
|
|
|
|
if finish is not None:
|
|
|
|
return _invoke_higher_order_function(
|
|
|
|
"ArrayAggregate",
|
2021-01-06 04:46:20 -05:00
|
|
|
[col, initialValue],
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
[merge, finish]
|
|
|
|
)
|
|
|
|
|
|
|
|
else:
|
|
|
|
return _invoke_higher_order_function(
|
|
|
|
"ArrayAggregate",
|
2021-01-06 04:46:20 -05:00
|
|
|
[col, initialValue],
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
[merge]
|
|
|
|
)
|
|
|
|
|
|
|
|
|
2021-01-06 04:46:20 -05:00
|
|
|
def zip_with(left, right, f):
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
"""
|
|
|
|
Merge two given arrays, element-wise, into a single array using a function.
|
|
|
|
If one array is shorter, nulls are appended at the end to match the length of the longer
|
|
|
|
array, before applying the function.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 3.1.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
left : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of the first column or expression
|
2021-02-01 19:30:50 -05:00
|
|
|
right : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of the second column or expression
|
|
|
|
f : function
|
|
|
|
a binary function ``(x1: Column, x2: Column) -> Column...``
|
2021-02-01 19:30:50 -05:00
|
|
|
Can use methods of :class:`~pyspark.sql.Column`, functions defined in
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
:py:mod:`pyspark.sql.functions` and Scala ``UserDefinedFunctions``.
|
|
|
|
Python ``UserDefinedFunctions`` are not supported
|
|
|
|
(`SPARK-27052 <https://issues.apache.org/jira/browse/SPARK-27052>`__).
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Returns
|
|
|
|
-------
|
2021-02-01 19:30:50 -05:00
|
|
|
:class:`~pyspark.sql.Column`
|
2020-11-02 20:00:49 -05:00
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
>>> df = spark.createDataFrame([(1, [1, 3, 5, 8], [0, 2, 4, 6])], ("id", "xs", "ys"))
|
|
|
|
>>> df.select(zip_with("xs", "ys", lambda x, y: x ** y).alias("powers")).show(truncate=False)
|
|
|
|
+---------------------------+
|
|
|
|
|powers |
|
|
|
|
+---------------------------+
|
|
|
|
|[1.0, 9.0, 625.0, 262144.0]|
|
|
|
|
+---------------------------+
|
|
|
|
|
|
|
|
>>> df = spark.createDataFrame([(1, ["foo", "bar"], [1, 2, 3])], ("id", "xs", "ys"))
|
|
|
|
>>> df.select(zip_with("xs", "ys", lambda x, y: concat_ws("_", x, y)).alias("xs_ys")).show()
|
|
|
|
+-----------------+
|
|
|
|
| xs_ys|
|
|
|
|
+-----------------+
|
|
|
|
|[foo_1, bar_2, 3]|
|
|
|
|
+-----------------+
|
|
|
|
"""
|
2021-01-06 04:46:20 -05:00
|
|
|
return _invoke_higher_order_function("ZipWith", [left, right], [f])
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
|
|
|
|
|
|
|
|
def transform_keys(col, f):
|
|
|
|
"""
|
|
|
|
Applies a function to every key-value pair in a map and returns
|
|
|
|
a map with the results of those applications as the new keys for the pairs.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 3.1.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column or expression
|
|
|
|
f : function
|
|
|
|
a binary function ``(k: Column, v: Column) -> Column...``
|
2021-02-01 19:30:50 -05:00
|
|
|
Can use methods of :class:`~pyspark.sql.Column`, functions defined in
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
:py:mod:`pyspark.sql.functions` and Scala ``UserDefinedFunctions``.
|
|
|
|
Python ``UserDefinedFunctions`` are not supported
|
|
|
|
(`SPARK-27052 <https://issues.apache.org/jira/browse/SPARK-27052>`__).
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Returns
|
|
|
|
-------
|
2021-02-01 19:30:50 -05:00
|
|
|
:class:`~pyspark.sql.Column`
|
2020-11-02 20:00:49 -05:00
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
>>> df = spark.createDataFrame([(1, {"foo": -2.0, "bar": 2.0})], ("id", "data"))
|
|
|
|
>>> df.select(transform_keys(
|
|
|
|
... "data", lambda k, _: upper(k)).alias("data_upper")
|
|
|
|
... ).show(truncate=False)
|
|
|
|
+-------------------------+
|
|
|
|
|data_upper |
|
|
|
|
+-------------------------+
|
2020-08-04 10:57:09 -04:00
|
|
|
|{BAR -> 2.0, FOO -> -2.0}|
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
+-------------------------+
|
|
|
|
"""
|
|
|
|
return _invoke_higher_order_function("TransformKeys", [col], [f])
|
|
|
|
|
|
|
|
|
|
|
|
def transform_values(col, f):
|
|
|
|
"""
|
|
|
|
Applies a function to every key-value pair in a map and returns
|
|
|
|
a map with the results of those applications as the new values for the pairs.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 3.1.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column or expression
|
|
|
|
f : function
|
|
|
|
a binary function ``(k: Column, v: Column) -> Column...``
|
2021-02-01 19:30:50 -05:00
|
|
|
Can use methods of :class:`~pyspark.sql.Column`, functions defined in
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
:py:mod:`pyspark.sql.functions` and Scala ``UserDefinedFunctions``.
|
|
|
|
Python ``UserDefinedFunctions`` are not supported
|
|
|
|
(`SPARK-27052 <https://issues.apache.org/jira/browse/SPARK-27052>`__).
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Returns
|
|
|
|
-------
|
2021-02-01 19:30:50 -05:00
|
|
|
:class:`~pyspark.sql.Column`
|
2020-11-02 20:00:49 -05:00
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
>>> df = spark.createDataFrame([(1, {"IT": 10.0, "SALES": 2.0, "OPS": 24.0})], ("id", "data"))
|
|
|
|
>>> df.select(transform_values(
|
|
|
|
... "data", lambda k, v: when(k.isin("IT", "OPS"), v + 10.0).otherwise(v)
|
|
|
|
... ).alias("new_data")).show(truncate=False)
|
|
|
|
+---------------------------------------+
|
|
|
|
|new_data |
|
|
|
|
+---------------------------------------+
|
2020-08-04 10:57:09 -04:00
|
|
|
|{OPS -> 34.0, IT -> 20.0, SALES -> 2.0}|
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
+---------------------------------------+
|
|
|
|
"""
|
|
|
|
return _invoke_higher_order_function("TransformValues", [col], [f])
|
|
|
|
|
|
|
|
|
|
|
|
def map_filter(col, f):
|
|
|
|
"""
|
|
|
|
Returns a map whose key-value pairs satisfy a predicate.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 3.1.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of column or expression
|
|
|
|
f : function
|
|
|
|
a binary function ``(k: Column, v: Column) -> Column...``
|
2021-02-01 19:30:50 -05:00
|
|
|
Can use methods of :class:`~pyspark.sql.Column`, functions defined in
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
:py:mod:`pyspark.sql.functions` and Scala ``UserDefinedFunctions``.
|
|
|
|
Python ``UserDefinedFunctions`` are not supported
|
|
|
|
(`SPARK-27052 <https://issues.apache.org/jira/browse/SPARK-27052>`__).
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Returns
|
|
|
|
-------
|
2021-02-01 19:30:50 -05:00
|
|
|
:class:`~pyspark.sql.Column`
|
2020-11-02 20:00:49 -05:00
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
>>> df = spark.createDataFrame([(1, {"foo": 42.0, "bar": 1.0, "baz": 32.0})], ("id", "data"))
|
|
|
|
>>> df.select(map_filter(
|
|
|
|
... "data", lambda _, v: v > 30.0).alias("data_filtered")
|
|
|
|
... ).show(truncate=False)
|
|
|
|
+--------------------------+
|
|
|
|
|data_filtered |
|
|
|
|
+--------------------------+
|
2020-08-04 10:57:09 -04:00
|
|
|
|{baz -> 32.0, foo -> 42.0}|
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
+--------------------------+
|
|
|
|
"""
|
|
|
|
return _invoke_higher_order_function("MapFilter", [col], [f])
|
|
|
|
|
|
|
|
|
|
|
|
def map_zip_with(col1, col2, f):
|
|
|
|
"""
|
|
|
|
Merge two given maps, key-wise into a single map using a function.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 3.1.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-02-01 19:30:50 -05:00
|
|
|
col1 : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of the first column or expression
|
2021-02-01 19:30:50 -05:00
|
|
|
col2 : :class:`~pyspark.sql.Column` or str
|
2020-11-02 20:00:49 -05:00
|
|
|
name of the second column or expression
|
|
|
|
f : function
|
|
|
|
a ternary function ``(k: Column, v1: Column, v2: Column) -> Column...``
|
2021-02-01 19:30:50 -05:00
|
|
|
Can use methods of :class:`~pyspark.sql.Column`, functions defined in
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
:py:mod:`pyspark.sql.functions` and Scala ``UserDefinedFunctions``.
|
|
|
|
Python ``UserDefinedFunctions`` are not supported
|
|
|
|
(`SPARK-27052 <https://issues.apache.org/jira/browse/SPARK-27052>`__).
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Returns
|
|
|
|
-------
|
2021-02-01 19:30:50 -05:00
|
|
|
:class:`~pyspark.sql.Column`
|
2020-11-02 20:00:49 -05:00
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
>>> df = spark.createDataFrame([
|
|
|
|
... (1, {"IT": 24.0, "SALES": 12.00}, {"IT": 2.0, "SALES": 1.4})],
|
|
|
|
... ("id", "base", "ratio")
|
|
|
|
... )
|
|
|
|
>>> df.select(map_zip_with(
|
|
|
|
... "base", "ratio", lambda k, v1, v2: round(v1 * v2, 2)).alias("updated_data")
|
|
|
|
... ).show(truncate=False)
|
|
|
|
+---------------------------+
|
|
|
|
|updated_data |
|
|
|
|
+---------------------------+
|
2020-08-04 10:57:09 -04:00
|
|
|
|{SALES -> 16.8, IT -> 48.0}|
|
[SPARK-30681][PYSPARK][SQL] Add higher order functions API to PySpark
### What changes were proposed in this pull request?
This PR add Python API for invoking following higher functions:
- `transform`
- `exists`
- `forall`
- `filter`
- `aggregate`
- `zip_with`
- `transform_keys`
- `transform_values`
- `map_filter`
- `map_zip_with`
to `pyspark.sql`. Each of these accepts plain Python functions of one of the following types
- `(Column) -> Column: ...`
- `(Column, Column) -> Column: ...`
- `(Column, Column, Column) -> Column: ...`
Internally this proposal piggbacks on objects supporting Scala implementation ([SPARK-27297](https://issues.apache.org/jira/browse/SPARK-27297)) by:
1. Creating required `UnresolvedNamedLambdaVariables` exposing these as PySpark `Columns`
2. Invoking Python function with these columns as arguments.
3. Using the result, and underlying JVM objects from 1., to create `expressions.LambdaFunction` which is passed to desired expression, and repacked as Python `Column`.
### Why are the changes needed?
Currently higher order functions are available only using SQL and Scala API and can use only SQL expressions
```python
df.selectExpr("transform(values, x -> x + 1)")
```
This works reasonably well for simple functions, but can get really ugly with complex functions (complex functions, casts), resulting objects are somewhat verbose and we don't get any IDE support. Additionally DSL used, though very simple, is not documented.
With changes propose here, above query could be rewritten as:
```python
df.select(transform("values", lambda x: x + 1))
```
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
- For positive cases this PR adds doctest strings covering possible usage patterns.
- For negative cases (unsupported function types) this PR adds unit tests.
### Notes
If approved, the same approach can be used in SparkR.
Closes #27406 from zero323/SPARK-30681.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-02-27 22:59:39 -05:00
|
|
|
+---------------------------+
|
|
|
|
"""
|
|
|
|
return _invoke_higher_order_function("MapZipWith", [col1, col2], [f])
|
|
|
|
|
|
|
|
|
2020-07-19 21:42:33 -04:00
|
|
|
# ---------------------- Partition transform functions --------------------------------
|
|
|
|
|
|
|
|
def years(col):
|
|
|
|
"""
|
|
|
|
Partition transform function: A transform for timestamps and dates
|
|
|
|
to partition data into years.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 3.1.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2020-07-19 21:42:33 -04:00
|
|
|
>>> df.writeTo("catalog.db.table").partitionedBy( # doctest: +SKIP
|
|
|
|
... years("ts")
|
|
|
|
... ).createOrReplace()
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Notes
|
|
|
|
-----
|
2020-11-27 11:22:45 -05:00
|
|
|
This function can be used only in combination with
|
2020-11-02 20:00:49 -05:00
|
|
|
:py:meth:`~pyspark.sql.readwriter.DataFrameWriterV2.partitionedBy`
|
|
|
|
method of the `DataFrameWriterV2`.
|
2020-07-19 21:42:33 -04:00
|
|
|
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.years(_to_java_column(col)))
|
|
|
|
|
|
|
|
|
|
|
|
def months(col):
|
|
|
|
"""
|
|
|
|
Partition transform function: A transform for timestamps and dates
|
|
|
|
to partition data into months.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 3.1.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2020-07-19 21:42:33 -04:00
|
|
|
>>> df.writeTo("catalog.db.table").partitionedBy(
|
|
|
|
... months("ts")
|
|
|
|
... ).createOrReplace() # doctest: +SKIP
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Notes
|
|
|
|
-----
|
2020-11-27 11:22:45 -05:00
|
|
|
This function can be used only in combination with
|
2020-11-02 20:00:49 -05:00
|
|
|
:py:meth:`~pyspark.sql.readwriter.DataFrameWriterV2.partitionedBy`
|
|
|
|
method of the `DataFrameWriterV2`.
|
2020-07-19 21:42:33 -04:00
|
|
|
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.months(_to_java_column(col)))
|
|
|
|
|
|
|
|
|
|
|
|
def days(col):
|
|
|
|
"""
|
|
|
|
Partition transform function: A transform for timestamps and dates
|
|
|
|
to partition data into days.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 3.1.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2020-07-19 21:42:33 -04:00
|
|
|
>>> df.writeTo("catalog.db.table").partitionedBy( # doctest: +SKIP
|
|
|
|
... days("ts")
|
|
|
|
... ).createOrReplace()
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Notes
|
|
|
|
-----
|
2020-11-27 11:22:45 -05:00
|
|
|
This function can be used only in combination with
|
2020-11-02 20:00:49 -05:00
|
|
|
:py:meth:`~pyspark.sql.readwriter.DataFrameWriterV2.partitionedBy`
|
|
|
|
method of the `DataFrameWriterV2`.
|
2020-07-19 21:42:33 -04:00
|
|
|
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.days(_to_java_column(col)))
|
|
|
|
|
|
|
|
|
|
|
|
def hours(col):
|
|
|
|
"""
|
|
|
|
Partition transform function: A transform for timestamps
|
|
|
|
to partition data into hours.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 3.1.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2020-07-19 21:42:33 -04:00
|
|
|
>>> df.writeTo("catalog.db.table").partitionedBy( # doctest: +SKIP
|
|
|
|
... hours("ts")
|
|
|
|
... ).createOrReplace()
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Notes
|
|
|
|
-----
|
2020-11-27 11:22:45 -05:00
|
|
|
This function can be used only in combination with
|
2020-11-02 20:00:49 -05:00
|
|
|
:py:meth:`~pyspark.sql.readwriter.DataFrameWriterV2.partitionedBy`
|
|
|
|
method of the `DataFrameWriterV2`.
|
2020-07-19 21:42:33 -04:00
|
|
|
|
|
|
|
"""
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
return Column(sc._jvm.functions.hours(_to_java_column(col)))
|
|
|
|
|
|
|
|
|
|
|
|
def bucket(numBuckets, col):
|
|
|
|
"""
|
|
|
|
Partition transform function: A transform for any type that partitions
|
|
|
|
by a hash of the input column.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 3.1.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2020-07-19 21:42:33 -04:00
|
|
|
>>> df.writeTo("catalog.db.table").partitionedBy( # doctest: +SKIP
|
|
|
|
... bucket(42, "ts")
|
|
|
|
... ).createOrReplace()
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This function can be used only in combination with
|
|
|
|
:py:meth:`~pyspark.sql.readwriter.DataFrameWriterV2.partitionedBy`
|
|
|
|
method of the `DataFrameWriterV2`.
|
2020-07-19 21:42:33 -04:00
|
|
|
|
|
|
|
"""
|
|
|
|
if not isinstance(numBuckets, (int, Column)):
|
|
|
|
raise TypeError(
|
[SPARK-32793][SQL] Add raise_error function, adds error message parameter to assert_true
## What changes were proposed in this pull request?
Adds a SQL function `raise_error` which underlies the refactored `assert_true` function. `assert_true` now also (optionally) accepts a custom error message field.
`raise_error` is exposed in SQL, Python, Scala, and R.
`assert_true` was previously only exposed in SQL; it is now also exposed in Python, Scala, and R.
### Why are the changes needed?
Improves usability of `assert_true` by clarifying error messaging, and adds the useful helper function `raise_error`.
### Does this PR introduce _any_ user-facing change?
Yes:
- Adds `raise_error` function to the SQL, Python, Scala, and R APIs.
- Adds `assert_true` function to the SQL, Python and R APIs.
### How was this patch tested?
Adds unit tests in SQL, Python, Scala, and R for `assert_true` and `raise_error`.
Closes #29947 from karenfeng/spark-32793.
Lead-authored-by: Karen Feng <karen.feng@databricks.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-10-07 23:05:39 -04:00
|
|
|
"numBuckets should be a Column or an int, got {}".format(type(numBuckets))
|
2020-07-19 21:42:33 -04:00
|
|
|
)
|
|
|
|
|
|
|
|
sc = SparkContext._active_spark_context
|
|
|
|
numBuckets = (
|
|
|
|
_create_column_from_literal(numBuckets)
|
|
|
|
if isinstance(numBuckets, int)
|
|
|
|
else _to_java_column(numBuckets)
|
|
|
|
)
|
|
|
|
return Column(sc._jvm.functions.bucket(numBuckets, _to_java_column(col)))
|
|
|
|
|
|
|
|
|
2015-08-04 22:25:24 -04:00
|
|
|
# ---------------------------- User Defined Function ----------------------------------
|
2015-07-31 19:05:26 -04:00
|
|
|
|
2017-02-15 13:16:34 -05:00
|
|
|
def udf(f=None, returnType=StringType()):
|
2017-10-10 18:32:01 -04:00
|
|
|
"""Creates a user defined function (UDF).
|
2016-11-22 06:40:18 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 1.3.0
|
2017-12-26 09:39:40 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
f : function
|
|
|
|
python function if used as a standalone function
|
|
|
|
returnType : :class:`pyspark.sql.types.DataType` or str
|
|
|
|
the return type of the user-defined function. The value can be either a
|
2018-01-18 08:33:04 -05:00
|
|
|
:class:`pyspark.sql.types.DataType` object or a DDL-formatted type string.
|
2016-07-28 17:57:15 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2015-02-17 13:22:48 -05:00
|
|
|
>>> from pyspark.sql.types import IntegerType
|
2015-02-14 02:03:22 -05:00
|
|
|
>>> slen = udf(lambda s: len(s), IntegerType())
|
2017-02-15 13:16:34 -05:00
|
|
|
>>> @udf
|
|
|
|
... def to_upper(s):
|
|
|
|
... if s is not None:
|
|
|
|
... return s.upper()
|
|
|
|
...
|
|
|
|
>>> @udf(returnType=IntegerType())
|
|
|
|
... def add_one(x):
|
|
|
|
... if x is not None:
|
|
|
|
... return x + 1
|
|
|
|
...
|
|
|
|
>>> df = spark.createDataFrame([(1, "John Doe", 21)], ("id", "name", "age"))
|
|
|
|
>>> df.select(slen("name").alias("slen(name)"), to_upper("name"), add_one("age")).show()
|
|
|
|
+----------+--------------+------------+
|
|
|
|
|slen(name)|to_upper(name)|add_one(age)|
|
|
|
|
+----------+--------------+------------+
|
|
|
|
| 8| JOHN DOE| 22|
|
|
|
|
+----------+--------------+------------+
|
2020-11-02 20:00:49 -05:00
|
|
|
|
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
The user-defined functions are considered deterministic by default. Due to
|
|
|
|
optimization, duplicate invocations may be eliminated or the function may even be invoked
|
|
|
|
more times than it is present in the query. If your function is not deterministic, call
|
|
|
|
`asNondeterministic` on the user defined function. E.g.:
|
|
|
|
|
|
|
|
>>> from pyspark.sql.types import IntegerType
|
|
|
|
>>> import random
|
|
|
|
>>> random_udf = udf(lambda: int(random.random() * 100), IntegerType()).asNondeterministic()
|
|
|
|
|
|
|
|
The user-defined functions do not support conditional expressions or short circuiting
|
|
|
|
in boolean expressions and it ends up with being executed all internally. If the functions
|
|
|
|
can fail on special rows, the workaround is to incorporate the condition into the functions.
|
|
|
|
|
|
|
|
The user-defined functions do not take keyword arguments on the calling side.
|
2017-02-15 13:16:34 -05:00
|
|
|
"""
|
[SPARK-25666][PYTHON] Internally document type conversion between Python data and SQL types in normal UDFs
### What changes were proposed in this pull request?
We are facing some problems about type conversions between Python data and SQL types in UDFs (Pandas UDFs as well).
It's even difficult to identify the problems (see https://github.com/apache/spark/pull/20163 and https://github.com/apache/spark/pull/22610).
This PR targets to internally document the type conversion table. Some of them looks buggy and we should fix them.
```python
import sys
import array
import datetime
from decimal import Decimal
from pyspark.sql import Row
from pyspark.sql.types import *
from pyspark.sql.functions import udf
if sys.version >= '3':
long = int
data = [
None,
True,
1,
long(1),
"a",
u"a",
datetime.date(1970, 1, 1),
datetime.datetime(1970, 1, 1, 0, 0),
1.0,
array.array("i", [1]),
[1],
(1,),
bytearray([65, 66, 67]),
Decimal(1),
{"a": 1},
Row(kwargs=1),
Row("namedtuple")(1),
]
types = [
BooleanType(),
ByteType(),
ShortType(),
IntegerType(),
LongType(),
StringType(),
DateType(),
TimestampType(),
FloatType(),
DoubleType(),
ArrayType(IntegerType()),
BinaryType(),
DecimalType(10, 0),
MapType(StringType(), IntegerType()),
StructType([StructField("_1", IntegerType())]),
]
df = spark.range(1)
results = []
count = 0
total = len(types) * len(data)
spark.sparkContext.setLogLevel("FATAL")
for t in types:
result = []
for v in data:
try:
row = df.select(udf(lambda: v, t)()).first()
ret_str = repr(row[0])
except Exception:
ret_str = "X"
result.append(ret_str)
progress = "SQL Type: [%s]\n Python Value: [%s(%s)]\n Result Python Value: [%s]" % (
t.simpleString(), str(v), type(v).__name__, ret_str)
count += 1
print("%s/%s:\n %s" % (count, total, progress))
results.append([t.simpleString()] + list(map(str, result)))
schema = ["SQL Type \\ Python Value(Type)"] + list(map(lambda v: "%s(%s)" % (str(v), type(v).__name__), data))
strings = spark.createDataFrame(results, schema=schema)._jdf.showString(20, 20, False)
print("\n".join(map(lambda line: " # %s # noqa" % line, strings.strip().split("\n"))))
```
This table was generated under Python 2 but the code above is Python 3 compatible as well.
## How was this patch tested?
Manually tested and lint check.
Closes #22655 from HyukjinKwon/SPARK-25666.
Authored-by: hyukjinkwon <gurwls223@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-08 03:47:15 -04:00
|
|
|
|
|
|
|
# The following table shows most of Python data and SQL type conversions in normal UDFs that
|
|
|
|
# are not yet visible to the user. Some of behaviors are buggy and might be changed in the near
|
|
|
|
# future. The table might have to be eventually documented externally.
|
[SPARK-28131][PYTHON] Update document type conversion between Python data and SQL types in normal UDFs (Python 3.7)
## What changes were proposed in this pull request?
This PR updates the chart generated at SPARK-25666. We deprecated Python 2. It's better to use Python 3.
We don't have to test `unicode` and `long` anymore in Python 3. So it was removed.
Use this code to generate the chart:
```python
import sys
import array
import datetime
from decimal import Decimal
from pyspark.sql import Row
from pyspark.sql.types import *
from pyspark.sql.functions import udf
data = [
None,
True,
1,
"a",
datetime.date(1970, 1, 1),
datetime.datetime(1970, 1, 1, 0, 0),
1.0,
array.array("i", [1]),
[1],
(1,),
bytearray([65, 66, 67]),
Decimal(1),
{"a": 1},
Row(kwargs=1),
Row("namedtuple")(1),
]
types = [
BooleanType(),
ByteType(),
ShortType(),
IntegerType(),
LongType(),
StringType(),
DateType(),
TimestampType(),
FloatType(),
DoubleType(),
ArrayType(IntegerType()),
BinaryType(),
DecimalType(10, 0),
MapType(StringType(), IntegerType()),
StructType([StructField("_1", IntegerType())]),
]
df = spark.range(1)
results = []
count = 0
total = len(types) * len(data)
spark.sparkContext.setLogLevel("FATAL")
for t in types:
result = []
for v in data:
try:
row = df.select(udf(lambda: v, t)()).first()
ret_str = repr(row[0])
except Exception:
ret_str = "X"
result.append(ret_str)
progress = "SQL Type: [%s]\n Python Value: [%s(%s)]\n Result Python Value: [%s]" % (
t.simpleString(), str(v), type(v).__name__, ret_str)
count += 1
print("%s/%s:\n %s" % (count, total, progress))
results.append([t.simpleString()] + list(map(str, result)))
schema = ["SQL Type \\ Python Value(Type)"] + list(map(lambda v: "%s(%s)" % (str(v), type(v).__name__), data))
strings = spark.createDataFrame(results, schema=schema)._jdf.showString(20, 20, False)
print("\n".join(map(lambda line: " # %s # noqa" % line, strings.strip().split("\n"))))
```
## How was this patch tested?
Manually.
Closes #24929 from HyukjinKwon/SPARK-28131.
Lead-authored-by: HyukjinKwon <gurwls223@apache.org>
Co-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-06-21 13:27:18 -04:00
|
|
|
# Please see SPARK-28131's PR to see the codes in order to generate the table below.
|
[SPARK-25666][PYTHON] Internally document type conversion between Python data and SQL types in normal UDFs
### What changes were proposed in this pull request?
We are facing some problems about type conversions between Python data and SQL types in UDFs (Pandas UDFs as well).
It's even difficult to identify the problems (see https://github.com/apache/spark/pull/20163 and https://github.com/apache/spark/pull/22610).
This PR targets to internally document the type conversion table. Some of them looks buggy and we should fix them.
```python
import sys
import array
import datetime
from decimal import Decimal
from pyspark.sql import Row
from pyspark.sql.types import *
from pyspark.sql.functions import udf
if sys.version >= '3':
long = int
data = [
None,
True,
1,
long(1),
"a",
u"a",
datetime.date(1970, 1, 1),
datetime.datetime(1970, 1, 1, 0, 0),
1.0,
array.array("i", [1]),
[1],
(1,),
bytearray([65, 66, 67]),
Decimal(1),
{"a": 1},
Row(kwargs=1),
Row("namedtuple")(1),
]
types = [
BooleanType(),
ByteType(),
ShortType(),
IntegerType(),
LongType(),
StringType(),
DateType(),
TimestampType(),
FloatType(),
DoubleType(),
ArrayType(IntegerType()),
BinaryType(),
DecimalType(10, 0),
MapType(StringType(), IntegerType()),
StructType([StructField("_1", IntegerType())]),
]
df = spark.range(1)
results = []
count = 0
total = len(types) * len(data)
spark.sparkContext.setLogLevel("FATAL")
for t in types:
result = []
for v in data:
try:
row = df.select(udf(lambda: v, t)()).first()
ret_str = repr(row[0])
except Exception:
ret_str = "X"
result.append(ret_str)
progress = "SQL Type: [%s]\n Python Value: [%s(%s)]\n Result Python Value: [%s]" % (
t.simpleString(), str(v), type(v).__name__, ret_str)
count += 1
print("%s/%s:\n %s" % (count, total, progress))
results.append([t.simpleString()] + list(map(str, result)))
schema = ["SQL Type \\ Python Value(Type)"] + list(map(lambda v: "%s(%s)" % (str(v), type(v).__name__), data))
strings = spark.createDataFrame(results, schema=schema)._jdf.showString(20, 20, False)
print("\n".join(map(lambda line: " # %s # noqa" % line, strings.strip().split("\n"))))
```
This table was generated under Python 2 but the code above is Python 3 compatible as well.
## How was this patch tested?
Manually tested and lint check.
Closes #22655 from HyukjinKwon/SPARK-25666.
Authored-by: hyukjinkwon <gurwls223@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-08 03:47:15 -04:00
|
|
|
#
|
[SPARK-28131][PYTHON] Update document type conversion between Python data and SQL types in normal UDFs (Python 3.7)
## What changes were proposed in this pull request?
This PR updates the chart generated at SPARK-25666. We deprecated Python 2. It's better to use Python 3.
We don't have to test `unicode` and `long` anymore in Python 3. So it was removed.
Use this code to generate the chart:
```python
import sys
import array
import datetime
from decimal import Decimal
from pyspark.sql import Row
from pyspark.sql.types import *
from pyspark.sql.functions import udf
data = [
None,
True,
1,
"a",
datetime.date(1970, 1, 1),
datetime.datetime(1970, 1, 1, 0, 0),
1.0,
array.array("i", [1]),
[1],
(1,),
bytearray([65, 66, 67]),
Decimal(1),
{"a": 1},
Row(kwargs=1),
Row("namedtuple")(1),
]
types = [
BooleanType(),
ByteType(),
ShortType(),
IntegerType(),
LongType(),
StringType(),
DateType(),
TimestampType(),
FloatType(),
DoubleType(),
ArrayType(IntegerType()),
BinaryType(),
DecimalType(10, 0),
MapType(StringType(), IntegerType()),
StructType([StructField("_1", IntegerType())]),
]
df = spark.range(1)
results = []
count = 0
total = len(types) * len(data)
spark.sparkContext.setLogLevel("FATAL")
for t in types:
result = []
for v in data:
try:
row = df.select(udf(lambda: v, t)()).first()
ret_str = repr(row[0])
except Exception:
ret_str = "X"
result.append(ret_str)
progress = "SQL Type: [%s]\n Python Value: [%s(%s)]\n Result Python Value: [%s]" % (
t.simpleString(), str(v), type(v).__name__, ret_str)
count += 1
print("%s/%s:\n %s" % (count, total, progress))
results.append([t.simpleString()] + list(map(str, result)))
schema = ["SQL Type \\ Python Value(Type)"] + list(map(lambda v: "%s(%s)" % (str(v), type(v).__name__), data))
strings = spark.createDataFrame(results, schema=schema)._jdf.showString(20, 20, False)
print("\n".join(map(lambda line: " # %s # noqa" % line, strings.strip().split("\n"))))
```
## How was this patch tested?
Manually.
Closes #24929 from HyukjinKwon/SPARK-28131.
Lead-authored-by: HyukjinKwon <gurwls223@apache.org>
Co-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-06-21 13:27:18 -04:00
|
|
|
# +-----------------------------+--------------+----------+------+---------------+--------------------+-----------------------------+----------+----------------------+---------+--------------------+----------------------------+------------+--------------+------------------+----------------------+ # noqa
|
|
|
|
# |SQL Type \ Python Value(Type)|None(NoneType)|True(bool)|1(int)| a(str)| 1970-01-01(date)|1970-01-01 00:00:00(datetime)|1.0(float)|array('i', [1])(array)|[1](list)| (1,)(tuple)|bytearray(b'ABC')(bytearray)| 1(Decimal)|{'a': 1}(dict)|Row(kwargs=1)(Row)|Row(namedtuple=1)(Row)| # noqa
|
|
|
|
# +-----------------------------+--------------+----------+------+---------------+--------------------+-----------------------------+----------+----------------------+---------+--------------------+----------------------------+------------+--------------+------------------+----------------------+ # noqa
|
|
|
|
# | boolean| None| True| None| None| None| None| None| None| None| None| None| None| None| X| X| # noqa
|
|
|
|
# | tinyint| None| None| 1| None| None| None| None| None| None| None| None| None| None| X| X| # noqa
|
|
|
|
# | smallint| None| None| 1| None| None| None| None| None| None| None| None| None| None| X| X| # noqa
|
|
|
|
# | int| None| None| 1| None| None| None| None| None| None| None| None| None| None| X| X| # noqa
|
|
|
|
# | bigint| None| None| 1| None| None| None| None| None| None| None| None| None| None| X| X| # noqa
|
|
|
|
# | string| None| 'true'| '1'| 'a'|'java.util.Gregor...| 'java.util.Gregor...| '1.0'| '[I@66cbb73a'| '[1]'|'[Ljava.lang.Obje...| '[B@5a51eb1a'| '1'| '{a=1}'| X| X| # noqa
|
|
|
|
# | date| None| X| X| X|datetime.date(197...| datetime.date(197...| X| X| X| X| X| X| X| X| X| # noqa
|
|
|
|
# | timestamp| None| X| X| X| X| datetime.datetime...| X| X| X| X| X| X| X| X| X| # noqa
|
|
|
|
# | float| None| None| None| None| None| None| 1.0| None| None| None| None| None| None| X| X| # noqa
|
|
|
|
# | double| None| None| None| None| None| None| 1.0| None| None| None| None| None| None| X| X| # noqa
|
|
|
|
# | array<int>| None| None| None| None| None| None| None| [1]| [1]| [1]| [65, 66, 67]| None| None| X| X| # noqa
|
|
|
|
# | binary| None| None| None|bytearray(b'a')| None| None| None| None| None| None| bytearray(b'ABC')| None| None| X| X| # noqa
|
|
|
|
# | decimal(10,0)| None| None| None| None| None| None| None| None| None| None| None|Decimal('1')| None| X| X| # noqa
|
|
|
|
# | map<string,int>| None| None| None| None| None| None| None| None| None| None| None| None| {'a': 1}| X| X| # noqa
|
|
|
|
# | struct<_1:int>| None| X| X| X| X| X| X| X|Row(_1=1)| Row(_1=1)| X| X| Row(_1=None)| Row(_1=1)| Row(_1=1)| # noqa
|
|
|
|
# +-----------------------------+--------------+----------+------+---------------+--------------------+-----------------------------+----------+----------------------+---------+--------------------+----------------------------+------------+--------------+------------------+----------------------+ # noqa
|
[SPARK-25666][PYTHON] Internally document type conversion between Python data and SQL types in normal UDFs
### What changes were proposed in this pull request?
We are facing some problems about type conversions between Python data and SQL types in UDFs (Pandas UDFs as well).
It's even difficult to identify the problems (see https://github.com/apache/spark/pull/20163 and https://github.com/apache/spark/pull/22610).
This PR targets to internally document the type conversion table. Some of them looks buggy and we should fix them.
```python
import sys
import array
import datetime
from decimal import Decimal
from pyspark.sql import Row
from pyspark.sql.types import *
from pyspark.sql.functions import udf
if sys.version >= '3':
long = int
data = [
None,
True,
1,
long(1),
"a",
u"a",
datetime.date(1970, 1, 1),
datetime.datetime(1970, 1, 1, 0, 0),
1.0,
array.array("i", [1]),
[1],
(1,),
bytearray([65, 66, 67]),
Decimal(1),
{"a": 1},
Row(kwargs=1),
Row("namedtuple")(1),
]
types = [
BooleanType(),
ByteType(),
ShortType(),
IntegerType(),
LongType(),
StringType(),
DateType(),
TimestampType(),
FloatType(),
DoubleType(),
ArrayType(IntegerType()),
BinaryType(),
DecimalType(10, 0),
MapType(StringType(), IntegerType()),
StructType([StructField("_1", IntegerType())]),
]
df = spark.range(1)
results = []
count = 0
total = len(types) * len(data)
spark.sparkContext.setLogLevel("FATAL")
for t in types:
result = []
for v in data:
try:
row = df.select(udf(lambda: v, t)()).first()
ret_str = repr(row[0])
except Exception:
ret_str = "X"
result.append(ret_str)
progress = "SQL Type: [%s]\n Python Value: [%s(%s)]\n Result Python Value: [%s]" % (
t.simpleString(), str(v), type(v).__name__, ret_str)
count += 1
print("%s/%s:\n %s" % (count, total, progress))
results.append([t.simpleString()] + list(map(str, result)))
schema = ["SQL Type \\ Python Value(Type)"] + list(map(lambda v: "%s(%s)" % (str(v), type(v).__name__), data))
strings = spark.createDataFrame(results, schema=schema)._jdf.showString(20, 20, False)
print("\n".join(map(lambda line: " # %s # noqa" % line, strings.strip().split("\n"))))
```
This table was generated under Python 2 but the code above is Python 3 compatible as well.
## How was this patch tested?
Manually tested and lint check.
Closes #22655 from HyukjinKwon/SPARK-25666.
Authored-by: hyukjinkwon <gurwls223@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-08 03:47:15 -04:00
|
|
|
#
|
|
|
|
# Note: DDL formatted string is used for 'SQL Type' for simplicity. This string can be
|
|
|
|
# used in `returnType`.
|
|
|
|
# Note: The values inside of the table are generated by `repr`.
|
|
|
|
# Note: 'X' means it throws an exception during the conversion.
|
[SPARK-28131][PYTHON] Update document type conversion between Python data and SQL types in normal UDFs (Python 3.7)
## What changes were proposed in this pull request?
This PR updates the chart generated at SPARK-25666. We deprecated Python 2. It's better to use Python 3.
We don't have to test `unicode` and `long` anymore in Python 3. So it was removed.
Use this code to generate the chart:
```python
import sys
import array
import datetime
from decimal import Decimal
from pyspark.sql import Row
from pyspark.sql.types import *
from pyspark.sql.functions import udf
data = [
None,
True,
1,
"a",
datetime.date(1970, 1, 1),
datetime.datetime(1970, 1, 1, 0, 0),
1.0,
array.array("i", [1]),
[1],
(1,),
bytearray([65, 66, 67]),
Decimal(1),
{"a": 1},
Row(kwargs=1),
Row("namedtuple")(1),
]
types = [
BooleanType(),
ByteType(),
ShortType(),
IntegerType(),
LongType(),
StringType(),
DateType(),
TimestampType(),
FloatType(),
DoubleType(),
ArrayType(IntegerType()),
BinaryType(),
DecimalType(10, 0),
MapType(StringType(), IntegerType()),
StructType([StructField("_1", IntegerType())]),
]
df = spark.range(1)
results = []
count = 0
total = len(types) * len(data)
spark.sparkContext.setLogLevel("FATAL")
for t in types:
result = []
for v in data:
try:
row = df.select(udf(lambda: v, t)()).first()
ret_str = repr(row[0])
except Exception:
ret_str = "X"
result.append(ret_str)
progress = "SQL Type: [%s]\n Python Value: [%s(%s)]\n Result Python Value: [%s]" % (
t.simpleString(), str(v), type(v).__name__, ret_str)
count += 1
print("%s/%s:\n %s" % (count, total, progress))
results.append([t.simpleString()] + list(map(str, result)))
schema = ["SQL Type \\ Python Value(Type)"] + list(map(lambda v: "%s(%s)" % (str(v), type(v).__name__), data))
strings = spark.createDataFrame(results, schema=schema)._jdf.showString(20, 20, False)
print("\n".join(map(lambda line: " # %s # noqa" % line, strings.strip().split("\n"))))
```
## How was this patch tested?
Manually.
Closes #24929 from HyukjinKwon/SPARK-28131.
Lead-authored-by: HyukjinKwon <gurwls223@apache.org>
Co-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-06-21 13:27:18 -04:00
|
|
|
# Note: Python 3.7.3 is used.
|
[SPARK-25666][PYTHON] Internally document type conversion between Python data and SQL types in normal UDFs
### What changes were proposed in this pull request?
We are facing some problems about type conversions between Python data and SQL types in UDFs (Pandas UDFs as well).
It's even difficult to identify the problems (see https://github.com/apache/spark/pull/20163 and https://github.com/apache/spark/pull/22610).
This PR targets to internally document the type conversion table. Some of them looks buggy and we should fix them.
```python
import sys
import array
import datetime
from decimal import Decimal
from pyspark.sql import Row
from pyspark.sql.types import *
from pyspark.sql.functions import udf
if sys.version >= '3':
long = int
data = [
None,
True,
1,
long(1),
"a",
u"a",
datetime.date(1970, 1, 1),
datetime.datetime(1970, 1, 1, 0, 0),
1.0,
array.array("i", [1]),
[1],
(1,),
bytearray([65, 66, 67]),
Decimal(1),
{"a": 1},
Row(kwargs=1),
Row("namedtuple")(1),
]
types = [
BooleanType(),
ByteType(),
ShortType(),
IntegerType(),
LongType(),
StringType(),
DateType(),
TimestampType(),
FloatType(),
DoubleType(),
ArrayType(IntegerType()),
BinaryType(),
DecimalType(10, 0),
MapType(StringType(), IntegerType()),
StructType([StructField("_1", IntegerType())]),
]
df = spark.range(1)
results = []
count = 0
total = len(types) * len(data)
spark.sparkContext.setLogLevel("FATAL")
for t in types:
result = []
for v in data:
try:
row = df.select(udf(lambda: v, t)()).first()
ret_str = repr(row[0])
except Exception:
ret_str = "X"
result.append(ret_str)
progress = "SQL Type: [%s]\n Python Value: [%s(%s)]\n Result Python Value: [%s]" % (
t.simpleString(), str(v), type(v).__name__, ret_str)
count += 1
print("%s/%s:\n %s" % (count, total, progress))
results.append([t.simpleString()] + list(map(str, result)))
schema = ["SQL Type \\ Python Value(Type)"] + list(map(lambda v: "%s(%s)" % (str(v), type(v).__name__), data))
strings = spark.createDataFrame(results, schema=schema)._jdf.showString(20, 20, False)
print("\n".join(map(lambda line: " # %s # noqa" % line, strings.strip().split("\n"))))
```
This table was generated under Python 2 but the code above is Python 3 compatible as well.
## How was this patch tested?
Manually tested and lint check.
Closes #22655 from HyukjinKwon/SPARK-25666.
Authored-by: hyukjinkwon <gurwls223@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-08 03:47:15 -04:00
|
|
|
|
2017-11-17 10:43:08 -05:00
|
|
|
# decorator @udf, @udf(), @udf(dataType())
|
|
|
|
if f is None or isinstance(f, (str, DataType)):
|
|
|
|
# If DataType has been passed as a positional argument
|
|
|
|
# for decorator use it as a returnType
|
|
|
|
return_type = f or returnType
|
|
|
|
return functools.partial(_create_udf, returnType=return_type,
|
|
|
|
evalType=PythonEvalType.SQL_BATCHED_UDF)
|
|
|
|
else:
|
|
|
|
return _create_udf(f=f, returnType=returnType,
|
|
|
|
evalType=PythonEvalType.SQL_BATCHED_UDF)
|
2017-02-15 13:16:34 -05:00
|
|
|
|
2017-09-22 04:17:41 -04:00
|
|
|
|
2015-02-14 02:03:22 -05:00
|
|
|
def _test():
|
|
|
|
import doctest
|
2016-05-23 21:14:48 -04:00
|
|
|
from pyspark.sql import Row, SparkSession
|
2015-02-17 13:22:48 -05:00
|
|
|
import pyspark.sql.functions
|
|
|
|
globs = pyspark.sql.functions.__dict__.copy()
|
2016-05-23 21:14:48 -04:00
|
|
|
spark = SparkSession.builder\
|
|
|
|
.master("local[4]")\
|
|
|
|
.appName("sql.functions tests")\
|
|
|
|
.getOrCreate()
|
|
|
|
sc = spark.sparkContext
|
2015-02-14 02:03:22 -05:00
|
|
|
globs['sc'] = sc
|
2016-05-23 21:14:48 -04:00
|
|
|
globs['spark'] = spark
|
2020-07-13 22:22:44 -04:00
|
|
|
globs['df'] = spark.createDataFrame([Row(age=2, name='Alice'), Row(age=5, name='Bob')])
|
2015-02-14 02:03:22 -05:00
|
|
|
(failure_count, test_count) = doctest.testmod(
|
2015-02-17 13:22:48 -05:00
|
|
|
pyspark.sql.functions, globs=globs,
|
2015-02-14 02:03:22 -05:00
|
|
|
optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE)
|
2016-05-23 21:14:48 -04:00
|
|
|
spark.stop()
|
2015-02-14 02:03:22 -05:00
|
|
|
if failure_count:
|
2018-03-08 06:38:34 -05:00
|
|
|
sys.exit(-1)
|
2015-02-14 02:03:22 -05:00
|
|
|
|
|
|
|
|
|
|
|
if __name__ == "__main__":
|
|
|
|
_test()
|